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/12/11 22:07:40 UTC

[01/11] incubator-quickstep git commit: Updates to compact key hash table

Repository: incubator-quickstep
Updated Branches:
  refs/heads/transitive-closure [created] 2aefd7bce


Updates to compact key hash table


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

Branch: refs/heads/transitive-closure
Commit: df20e4c09ec3a8490b1d1ddcd590d813a873181e
Parents: 3595bc1
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sun Nov 26 21:16:46 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Mon Nov 27 16:06:44 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/ExecutionGenerator.cpp          | 109 ++-------
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  39 +++-
 .../cost_model/StarSchemaSimpleCostModel.hpp    |   4 +-
 query_optimizer/rules/FuseAggregateJoin.cpp     |   6 -
 .../FinalizeAggregationOperator.cpp             |  29 +--
 .../FinalizeAggregationOperator.hpp             |   3 -
 .../InitializeAggregationOperator.cpp           |  24 +-
 .../InitializeAggregationOperator.hpp           |   7 +-
 storage/AggregationOperationState.cpp           | 134 ++++++-----
 storage/AggregationOperationState.hpp           |  38 +--
 storage/AggregationOperationState.proto         |   3 -
 storage/CMakeLists.txt                          |  24 ++
 storage/CollisionFreeVectorTable.cpp            |  58 ++++-
 storage/CollisionFreeVectorTable.hpp            |  42 ++--
 storage/CompactKeySeparateChainingHashTable.cpp | 195 ++++++++++++++++
 storage/CompactKeySeparateChainingHashTable.hpp | 234 +++++++++++++++++++
 storage/Flags.hpp                               |   1 -
 storage/HashTable.proto                         |  15 +-
 storage/HashTableBase.hpp                       |   1 +
 storage/HashTableFactory.hpp                    |  24 +-
 utility/CMakeLists.txt                          |  10 +
 utility/Range.hpp                               | 188 +++++++++++++++
 utility/ScopedArray.hpp                         |  78 +++++++
 24 files changed, 977 insertions(+), 290 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 5e0db44..011cecb 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -148,6 +148,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_relationaloperators_UpdateOperator
                       quickstep_relationaloperators_WindowAggregationOperator
                       quickstep_storage_AggregationOperationState_proto
+                      quickstep_storage_Flags
                       quickstep_storage_HashTableFactory
                       quickstep_storage_HashTable_proto
                       quickstep_storage_InsertDestination_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index b0d3c48..3ef74ee 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -132,6 +132,7 @@
 #include "relational_operators/UpdateOperator.hpp"
 #include "relational_operators/WindowAggregationOperator.hpp"
 #include "storage/AggregationOperationState.pb.h"
+#include "storage/Flags.hpp"
 #include "storage/HashTable.pb.h"
 #include "storage/HashTableFactory.hpp"
 #include "storage/InsertDestination.pb.h"
@@ -199,70 +200,6 @@ namespace S = ::quickstep::serialization;
 
 namespace {
 
-size_t CacheLineAlignedBytes(const size_t actual_bytes) {
-  return (actual_bytes + kCacheLineBytes - 1) / kCacheLineBytes * kCacheLineBytes;
-}
-
-size_t CalculateNumInitializationPartitionsForCollisionFreeVectorTable(const size_t memory_size) {
-  // At least 1 partition, at most (#workers * 2) partitions.
-  return std::max(1uL, std::min(memory_size / kCollisonFreeVectorInitBlobSize,
-                                static_cast<size_t>(2 * FLAGS_num_workers)));
-}
-
-void CalculateCollisionFreeAggregationInfo(
-    const size_t num_entries, const vector<pair<AggregationID, vector<const Type *>>> &group_by_aggrs_info,
-    S::CollisionFreeVectorInfo *collision_free_vector_info) {
-  size_t memory_size = CacheLineAlignedBytes(
-      BarrieredReadWriteConcurrentBitVector::BytesNeeded(num_entries));
-
-  for (std::size_t i = 0; i < group_by_aggrs_info.size(); ++i) {
-    const auto &group_by_aggr_info = group_by_aggrs_info[i];
-
-    size_t state_size = 0;
-    switch (group_by_aggr_info.first) {
-      case AggregationID::kCount: {
-        state_size = sizeof(atomic<size_t>);
-        break;
-      }
-      case AggregationID::kSum: {
-        const vector<const Type *> &argument_types = group_by_aggr_info.second;
-        DCHECK_EQ(1u, argument_types.size());
-        switch (argument_types.front()->getTypeID()) {
-          case TypeID::kInt:
-          case TypeID::kLong:
-            state_size = sizeof(atomic<std::int64_t>);
-            break;
-          case TypeID::kFloat:
-          case TypeID::kDouble:
-            state_size = sizeof(atomic<double>);
-            break;
-          default:
-            LOG(FATAL) << "No support by CollisionFreeVector";
-        }
-        break;
-      }
-      default:
-        LOG(FATAL) << "No support by CollisionFreeVector";
-    }
-
-    collision_free_vector_info->add_state_offsets(memory_size);
-    memory_size += CacheLineAlignedBytes(state_size * num_entries);
-  }
-
-  collision_free_vector_info->set_memory_size(memory_size);
-  collision_free_vector_info->set_num_init_partitions(
-      CalculateNumInitializationPartitionsForCollisionFreeVectorTable(memory_size));
-}
-
-size_t CalculateNumFinalizationPartitionsForCollisionFreeVectorTable(const size_t num_entries) {
-  // Set finalization segment size as 4096 entries.
-  constexpr size_t kFinalizeSegmentSize = 4uL * 1024L;
-
-  // At least 1 partition, at most (#workers * 2) partitions.
-  return std::max(1uL, std::min(num_entries / kFinalizeSegmentSize,
-                                static_cast<size_t>(2 * FLAGS_num_workers)));
-}
-
 bool CheckAggregatePartitioned(const std::size_t num_aggregate_functions,
                                const std::vector<bool> &is_distincts,
                                const std::vector<attribute_id> &group_by_attrs,
@@ -1800,8 +1737,6 @@ void ExecutionGenerator::convertAggregate(
       aggr_state_context_proto->mutable_aggregation_state();
   aggr_state_proto->set_relation_id(input_relation.getID());
 
-  bool use_parallel_initialization = false;
-
   std::vector<const Type*> group_by_types;
   std::vector<attribute_id> group_by_attrs;
   for (const E::NamedExpressionPtr &grouping_expression : physical_plan->grouping_expressions()) {
@@ -1865,28 +1800,30 @@ void ExecutionGenerator::convertAggregate(
     }
   }
 
+  bool use_parallel_initialization = false;
   bool aggr_state_is_partitioned = false;
   std::size_t aggr_state_num_partitions = 1u;
   if (!group_by_types.empty()) {
-    const std::size_t estimated_num_groups =
-        cost_model_for_aggregation_->estimateNumGroupsForAggregate(physical_plan);
-
     std::size_t max_num_groups;
     if (cost_model_for_aggregation_
             ->canUseCollisionFreeAggregation(physical_plan,
-                                             estimated_num_groups,
                                              &max_num_groups)) {
       // First option: use array-based aggregation if applicable.
       aggr_state_proto->set_hash_table_impl_type(
           serialization::HashTableImplType::COLLISION_FREE_VECTOR);
       aggr_state_proto->set_estimated_num_entries(max_num_groups);
       use_parallel_initialization = true;
-      aggr_state_num_partitions = CalculateNumFinalizationPartitionsForCollisionFreeVectorTable(max_num_groups);
-
-      DCHECK(!group_by_aggrs_info.empty());
-      CalculateCollisionFreeAggregationInfo(max_num_groups, group_by_aggrs_info,
-                                            aggr_state_proto->mutable_collision_free_vector_info());
+    } else if (cost_model_for_aggregation_
+                   ->canUseCompactKeySeparateChainingAggregation(physical_plan)) {
+      CHECK(aggregate_expressions.empty());
+      aggr_state_proto->set_hash_table_impl_type(
+          serialization::HashTableImplType::COMPACT_KEY_SEPARATE_CHAINING);
+      aggr_state_proto->set_estimated_num_entries(
+          cost_model_for_aggregation_->estimateCardinality(physical_plan->input()));
+      use_parallel_initialization = true;
     } else {
+      const std::size_t estimated_num_groups =
+          cost_model_for_aggregation_->estimateNumGroupsForAggregate(physical_plan);
       if (cost_model_for_aggregation_->canUseTwoPhaseCompactKeyAggregation(
               physical_plan, estimated_num_groups)) {
         // Second option: use thread-private compact-key aggregation if applicable.
@@ -1896,7 +1833,8 @@ void ExecutionGenerator::convertAggregate(
         // Otherwise, use SeparateChaining.
         aggr_state_proto->set_hash_table_impl_type(
             serialization::HashTableImplType::SEPARATE_CHAINING);
-        if (CheckAggregatePartitioned(aggregate_expressions.size(), is_distincts, group_by_attrs,
+        if (CheckAggregatePartitioned(aggregate_expressions.size(),
+                                      is_distincts, group_by_attrs,
                                       estimated_num_groups)) {
           aggr_state_is_partitioned = true;
           aggr_state_num_partitions = FLAGS_num_aggregation_partitions;
@@ -1931,15 +1869,12 @@ void ExecutionGenerator::convertAggregate(
   }
 
   if (use_parallel_initialization) {
-    DCHECK(aggr_state_proto->has_collision_free_vector_info());
-
     const QueryPlan::DAGNodeIndex initialize_aggregation_operator_index =
         execution_plan_->addRelationalOperator(
             new InitializeAggregationOperator(
                 query_handle_->query_id(),
                 aggr_state_index,
-                num_partitions,
-                aggr_state_proto->collision_free_vector_info().num_init_partitions()));
+                num_partitions));
 
     execution_plan_->addDirectDependency(aggregation_operator_index,
                                          initialize_aggregation_operator_index,
@@ -1961,7 +1896,6 @@ void ExecutionGenerator::convertAggregate(
                                           aggr_state_index,
                                           num_partitions,
                                           physical_plan->hasRepartition(),
-                                          aggr_state_num_partitions,
                                           *output_relation,
                                           insert_destination_index));
 
@@ -2032,10 +1966,7 @@ void ExecutionGenerator::convertCrossReferenceCoalesceAggregate(
 
   const size_t estimated_num_entries = physical_plan->group_by_key_value_range();
   aggr_state_proto->set_estimated_num_entries(estimated_num_entries);
-
-  const size_t aggr_state_num_partitions =
-      CalculateNumFinalizationPartitionsForCollisionFreeVectorTable(estimated_num_entries);
-  aggr_state_proto->set_num_partitions(aggr_state_num_partitions);
+  aggr_state_proto->set_num_partitions(1u);
 
   if (physical_plan->right_filter_predicate() != nullptr) {
     std::unique_ptr<const Predicate> predicate(
@@ -2071,16 +2002,12 @@ void ExecutionGenerator::convertCrossReferenceCoalesceAggregate(
     aggr_proto->set_is_distinct(false);
   }
 
-  CalculateCollisionFreeAggregationInfo(estimated_num_entries, group_by_aggrs_info,
-                                        aggr_state_proto->mutable_collision_free_vector_info());
-
   const QueryPlan::DAGNodeIndex initialize_aggregation_operator_index =
       execution_plan_->addRelationalOperator(
           new InitializeAggregationOperator(
               query_handle_->query_id(),
               aggr_state_index,
-              num_partitions,
-              aggr_state_proto->collision_free_vector_info().num_init_partitions()));
+              num_partitions));
 
   const QueryPlan::DAGNodeIndex build_aggregation_existence_map_operator_index =
       execution_plan_->addRelationalOperator(
@@ -2123,7 +2050,6 @@ void ExecutionGenerator::convertCrossReferenceCoalesceAggregate(
                                        build_aggregation_existence_map_operator_index,
                                        true /* is_pipeline_breaker */);
 
-
   // Create InsertDestination proto.
   const CatalogRelation *output_relation = nullptr;
   const QueryContext::insert_destination_id insert_destination_index =
@@ -2139,7 +2065,6 @@ void ExecutionGenerator::convertCrossReferenceCoalesceAggregate(
                                           aggr_state_index,
                                           num_partitions,
                                           physical_plan->hasRepartition(),
-                                          aggr_state_num_partitions,
                                           *output_relation,
                                           insert_destination_index));
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index e0e3dff..f7417b6 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -605,7 +605,6 @@ attribute_id StarSchemaSimpleCostModel::findCatalogRelationAttributeId(
 
 bool StarSchemaSimpleCostModel::canUseCollisionFreeAggregation(
     const P::AggregatePtr &aggregate,
-    const std::size_t estimated_num_groups,
     std::size_t *max_num_groups) {
 #ifdef QUICKSTEP_DISTRIBUTED
   // Currently we cannot do this fast path with the distributed setting. See
@@ -661,7 +660,7 @@ bool StarSchemaSimpleCostModel::canUseCollisionFreeAggregation(
   //    of hardcoding it as a gflag.
   if (min_cpp_value < 0 ||
       max_cpp_value >= FLAGS_collision_free_vector_table_max_size ||
-      max_cpp_value / static_cast<double>(estimated_num_groups) > 256.0) {
+      max_cpp_value / static_cast<double>(estimateNumGroupsForAggregate(aggregate)) > 256.0) {
     return false;
   }
 
@@ -767,6 +766,42 @@ bool StarSchemaSimpleCostModel::canUseTwoPhaseCompactKeyAggregation(
   return true;
 }
 
+bool StarSchemaSimpleCostModel::canUseCompactKeySeparateChainingAggregation(
+    const P::AggregatePtr &aggregate) {
+  P::TableReferencePtr table_reference;
+  if (!P::SomeTableReference::MatchesWithConditionalCast(aggregate->input(), &table_reference)) {
+    return false;
+  }
+
+  const auto &stat = table_reference->relation()->getStatistics();
+  if (!stat.isExact() || !stat.hasNumTuples()) {
+    return false;
+  }
+
+  if (stat.getNumTuples() <= 1000u) {
+    return false;
+  }
+
+  // Require fix-length non-nullable keys that can be packed into a 64-bit QWORD.
+  std::size_t total_key_size = 0;
+  for (const auto &key_expr : aggregate->grouping_expressions()) {
+    const Type &type = key_expr->getValueType();
+    if (type.isNullable() || type.isVariableLength()) {
+      return false;
+    }
+    total_key_size += type.maximumByteLength();
+  }
+  if (total_key_size > sizeof(std::uint64_t)) {
+    return false;
+  }
+
+  if (!aggregate->aggregate_expressions().empty()) {
+    return false;
+  }
+
+  return true;
+}
+
 }  // namespace cost
 }  // namespace optimizer
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index 99518cf..a056866 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -173,7 +173,6 @@ class StarSchemaSimpleCostModel : public CostModel {
    *        the collision-free aggregation fast path.
    *
    * @param aggregate The physical aggregate node to be checked.
-   * @param estimated_num_groups The estimated number of groups for the aggregate.
    * @param exact_num_groups If collision-free aggregation is applicable, the
    *        pointed content of this pointer will be set as the maximum possible
    *        number of groups that the collision-free hash table need to hold.
@@ -181,7 +180,6 @@ class StarSchemaSimpleCostModel : public CostModel {
    *         used to evaluate \p aggregate.
    */
   bool canUseCollisionFreeAggregation(const physical::AggregatePtr &aggregate,
-                                      const std::size_t estimated_num_groups,
                                       std::size_t *max_num_groups);
 
   /**
@@ -196,6 +194,8 @@ class StarSchemaSimpleCostModel : public CostModel {
   bool canUseTwoPhaseCompactKeyAggregation(const physical::AggregatePtr &aggregate,
                                            const std::size_t estimated_num_groups);
 
+  bool canUseCompactKeySeparateChainingAggregation(const physical::AggregatePtr &aggregate);
+
  private:
   std::size_t estimateCardinalityForAggregate(
       const physical::AggregatePtr &physical_plan);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/query_optimizer/rules/FuseAggregateJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/FuseAggregateJoin.cpp b/query_optimizer/rules/FuseAggregateJoin.cpp
index 6efc7e8..69e85af 100644
--- a/query_optimizer/rules/FuseAggregateJoin.cpp
+++ b/query_optimizer/rules/FuseAggregateJoin.cpp
@@ -109,23 +109,17 @@ P::PhysicalPtr FuseAggregateJoin::applyToNode(
 
   // Collision-free vector aggregation is applicable, and both the left and right
   // join attributes are range-bounded integer values.
-  const std::size_t estimated_num_groups =
-      cost_model_->estimateNumGroupsForAggregate(aggregate);
-
   std::size_t max_num_groups_left;
   if (!cost_model_->canUseCollisionFreeAggregation(aggregate,
-                                                   estimated_num_groups,
                                                    &max_num_groups_left)) {
     return node;
   }
-
   std::size_t max_num_groups_right;
   if (!cost_model_->canUseCollisionFreeAggregation(
            P::Aggregate::Create(hash_join->right(),
                                 E::ToNamedExpressions(hash_join->right_join_attributes()),
                                 aggregate->aggregate_expressions(),
                                 nullptr),
-           estimated_num_groups,
            &max_num_groups_right)) {
     return node;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 92fc7f6..cdba25a 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -50,7 +50,7 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
         query_context->getAggregationState(aggr_state_index_, part_id);
     DCHECK(agg_state != nullptr);
     for (std::size_t state_part_id = 0;
-         state_part_id < aggr_state_num_partitions_;
+         state_part_id < agg_state->getNumFinalizationPartitions();
          ++state_part_id) {
       container->addNormalWorkOrder(
           new FinalizeAggregationWorkOrder(
@@ -68,32 +68,7 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
 }
 
 bool FinalizeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
-  if (started_) {
-    return true;
-  }
-
-  for (partition_id part_id = 0; part_id < num_partitions_; ++part_id) {
-    for (std::size_t state_part_id = 0;
-         state_part_id < aggr_state_num_partitions_;
-         ++state_part_id) {
-      serialization::WorkOrder *proto = new serialization::WorkOrder;
-      proto->set_work_order_type(serialization::FINALIZE_AGGREGATION);
-      proto->set_query_id(query_id_);
-      proto->SetExtension(serialization::FinalizeAggregationWorkOrder::aggr_state_index,
-                          aggr_state_index_);
-      proto->SetExtension(serialization::FinalizeAggregationWorkOrder::partition_id,
-                          part_id);
-      proto->SetExtension(serialization::FinalizeAggregationWorkOrder::state_partition_id,
-                          state_part_id);
-      proto->SetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index,
-                          output_destination_index_);
-
-      container->addWorkOrderProto(proto, op_index_);
-    }
-  }
-
-  started_ = true;
-  return true;
+  LOG(FATAL) << "Not supported";
 }
 
 void FinalizeAggregationWorkOrder::execute() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index 5931ca2..087fad5 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -71,12 +71,10 @@ class FinalizeAggregationOperator : public RelationalOperator {
       const QueryContext::aggregation_state_id aggr_state_index,
       const std::size_t num_partitions,
       const bool has_repartition,
-      const std::size_t aggr_state_num_partitions,
       const CatalogRelation &output_relation,
       const QueryContext::insert_destination_id output_destination_index)
       : RelationalOperator(query_id, num_partitions, has_repartition, output_relation.getNumPartitions()),
         aggr_state_index_(aggr_state_index),
-        aggr_state_num_partitions_(aggr_state_num_partitions),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         started_(false) {
@@ -111,7 +109,6 @@ class FinalizeAggregationOperator : public RelationalOperator {
 
  private:
   const QueryContext::aggregation_state_id aggr_state_index_;
-  const std::size_t aggr_state_num_partitions_;
   const CatalogRelation &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;
   bool started_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/relational_operators/InitializeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeAggregationOperator.cpp b/relational_operators/InitializeAggregationOperator.cpp
index 89dfd7e..6bd9e8d 100644
--- a/relational_operators/InitializeAggregationOperator.cpp
+++ b/relational_operators/InitializeAggregationOperator.cpp
@@ -49,7 +49,7 @@ bool InitializeAggregationOperator::getAllWorkOrders(
     DCHECK(agg_state != nullptr);
 
     for (std::size_t state_part_id = 0;
-         state_part_id < aggr_state_num_init_partitions_;
+         state_part_id < agg_state->getNumInitializationPartitions();
          ++state_part_id) {
       container->addNormalWorkOrder(
           new InitializeAggregationWorkOrder(query_id_,
@@ -65,27 +65,7 @@ bool InitializeAggregationOperator::getAllWorkOrders(
 }
 
 bool InitializeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
-  if (started_) {
-    return true;
-  }
-
-  for (partition_id part_id = 0; part_id < num_partitions_; ++part_id) {
-    for (std::size_t state_part_id = 0;
-         state_part_id < aggr_state_num_init_partitions_;
-         ++state_part_id) {
-      serialization::WorkOrder *proto = new serialization::WorkOrder;
-      proto->set_work_order_type(serialization::INITIALIZE_AGGREGATION);
-      proto->set_query_id(query_id_);
-
-      proto->SetExtension(serialization::InitializeAggregationWorkOrder::aggr_state_index, aggr_state_index_);
-      proto->SetExtension(serialization::InitializeAggregationWorkOrder::partition_id, part_id);
-      proto->SetExtension(serialization::InitializeAggregationWorkOrder::state_partition_id, state_part_id);
-
-      container->addWorkOrderProto(proto, op_index_);
-    }
-  }
-  started_ = true;
-  return true;
+  LOG(FATAL) << "Not supported";
 }
 
 void InitializeAggregationWorkOrder::execute() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/relational_operators/InitializeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeAggregationOperator.hpp b/relational_operators/InitializeAggregationOperator.hpp
index cf9abe5..beedc14 100644
--- a/relational_operators/InitializeAggregationOperator.hpp
+++ b/relational_operators/InitializeAggregationOperator.hpp
@@ -58,16 +58,12 @@ class InitializeAggregationOperator : public RelationalOperator {
    * @param aggr_state_index The index of the AggregationOperationState in QueryContext.
    * @param num_partitions The number of partitions in 'input_relation'. If no
    *        partitions, it is one.
-   * @param aggr_state_num_init_partitions The number of partitions to be used
-   *        for initialize the aggregation state collision free vector table.
    **/
   InitializeAggregationOperator(const std::size_t query_id,
                                 const QueryContext::aggregation_state_id aggr_state_index,
-                                const std::size_t num_partitions,
-                                const std::size_t aggr_state_num_init_partitions)
+                                const std::size_t num_partitions)
       : RelationalOperator(query_id, num_partitions),
         aggr_state_index_(aggr_state_index),
-        aggr_state_num_init_partitions_(aggr_state_num_init_partitions),
         started_(false) {}
 
   ~InitializeAggregationOperator() override {}
@@ -90,7 +86,6 @@ class InitializeAggregationOperator : public RelationalOperator {
 
  private:
   const QueryContext::aggregation_state_id aggr_state_index_;
-  const std::size_t aggr_state_num_init_partitions_;
   bool started_;
 
   DISALLOW_COPY_AND_ASSIGN(InitializeAggregationOperator);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 73f1983..92798d8 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -83,14 +83,9 @@ AggregationOperationState::AggregationOperationState(
     const std::size_t num_partitions,
     const HashTableImplType hash_table_impl_type,
     const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
-    StorageManager *storage_manager,
-    const size_t collision_free_vector_memory_size,
-    const size_t collision_free_vector_num_init_partitions,
-    const vector<size_t> &collision_free_vector_state_offsets)
+    StorageManager *storage_manager)
     : input_relation_(input_relation),
-      is_aggregate_collision_free_(
-          group_by.empty() ? false
-                           : hash_table_impl_type == HashTableImplType::kCollisionFreeVector),
+      hash_table_impl_type_(hash_table_impl_type),
       is_aggregate_partitioned_(is_partitioned),
       predicate_(predicate),
       is_distinct_(std::move(is_distinct)),
@@ -209,18 +204,14 @@ AggregationOperationState::AggregationOperationState(
 
   if (!group_by_key_ids_.empty()) {
     // Aggregation with GROUP BY: create the hash table (pool).
-    if (is_aggregate_collision_free_) {
-      collision_free_hashtable_.reset(
+    if (useCollisionFreeVector() || useCompactKeySeparateChaining()) {
+      shared_hash_table_.reset(
           AggregationStateHashTableFactory::CreateResizable(
               hash_table_impl_type,
               group_by_types_,
               estimated_num_entries,
               group_by_handles,
-              storage_manager,
-              num_partitions,
-              collision_free_vector_memory_size,
-              collision_free_vector_num_init_partitions,
-              collision_free_vector_state_offsets));
+              storage_manager));
     } else if (is_aggregate_partitioned_) {
       if (all_distinct_) {
         DCHECK_EQ(1u, group_by_handles.size());
@@ -300,19 +291,6 @@ AggregationOperationState* AggregationOperationState::ReconstructFromProto(
         PredicateFactory::ReconstructFromProto(proto.predicate(), database));
   }
 
-  size_t collision_free_vector_memory_size = 0;
-  size_t collision_free_vector_num_init_partitions = 0;
-  vector<size_t> collision_free_vector_state_offsets;
-  if (proto.has_collision_free_vector_info()) {
-    const serialization::CollisionFreeVectorInfo &collision_free_vector_info =
-        proto.collision_free_vector_info();
-    collision_free_vector_memory_size = collision_free_vector_info.memory_size();
-    collision_free_vector_num_init_partitions = collision_free_vector_info.num_init_partitions();
-    for (int i = 0; i < collision_free_vector_info.state_offsets_size(); ++i) {
-      collision_free_vector_state_offsets.push_back(collision_free_vector_info.state_offsets(i));
-    }
-  }
-
   return new AggregationOperationState(
       database.getRelationSchemaById(proto.relation_id()),
       aggregate_functions,
@@ -325,10 +303,7 @@ AggregationOperationState* AggregationOperationState::ReconstructFromProto(
       proto.num_partitions(),
       HashTableImplTypeFromProto(proto.hash_table_impl_type()),
       distinctify_hash_table_impl_types,
-      storage_manager,
-      collision_free_vector_memory_size,
-      collision_free_vector_num_init_partitions,
-      collision_free_vector_state_offsets);
+      storage_manager);
 }
 
 bool AggregationOperationState::ProtoIsValid(
@@ -386,18 +361,6 @@ bool AggregationOperationState::ProtoIsValid(
             proto.hash_table_impl_type())) {
       return false;
     }
-
-    if (proto.hash_table_impl_type() == S::HashTableImplType::COLLISION_FREE_VECTOR) {
-      if (!proto.has_collision_free_vector_info()) {
-        return false;
-      }
-
-      const S::CollisionFreeVectorInfo &proto_collision_free_vector_info = proto.collision_free_vector_info();
-      if (!proto_collision_free_vector_info.IsInitialized() ||
-          proto_collision_free_vector_info.state_offsets_size() != group_by_expressions_size) {
-        return false;
-      }
-    }
   }
 
   if (proto.has_predicate()) {
@@ -411,22 +374,25 @@ bool AggregationOperationState::ProtoIsValid(
 
 CollisionFreeVectorTable* AggregationOperationState
     ::getCollisionFreeVectorTable() const {
-  return static_cast<CollisionFreeVectorTable *>(
-      collision_free_hashtable_.get());
+  return static_cast<CollisionFreeVectorTable *>(shared_hash_table_.get());
 }
 
 void AggregationOperationState::initialize(const std::size_t partition_id) {
-  if (is_aggregate_collision_free_) {
-    static_cast<CollisionFreeVectorTable *>(
-        collision_free_hashtable_.get())->initialize(partition_id);
+  if (useCollisionFreeVector()) {
+    static_cast<CollisionFreeVectorTable*>(
+        shared_hash_table_.get())->initialize(partition_id);
+  } else if (useCompactKeySeparateChaining()) {
+    static_cast<CompactKeySeparateChainingHashTable*>(
+        shared_hash_table_.get())->initialize(partition_id);
   } else {
     LOG(FATAL) << "AggregationOperationState::initialize() "
                << "is not supported by this aggregation";
   }
 }
 
-void AggregationOperationState::aggregateBlock(const block_id input_block,
-                                               LIPFilterAdaptiveProber *lip_filter_adaptive_prober) {
+void AggregationOperationState::aggregateBlock(
+    const block_id input_block,
+    LIPFilterAdaptiveProber *lip_filter_adaptive_prober) {
   BlockReference block(
       storage_manager_->getBlock(input_block, input_relation_));
   const auto &tuple_store = block->getTupleStorageSubBlock();
@@ -527,8 +493,8 @@ void AggregationOperationState::mergeGroupByHashTables(
 
 void AggregationOperationState::aggregateBlockHashTable(
     const ValueAccessorMultiplexer &accessor_mux) {
-  if (is_aggregate_collision_free_) {
-    aggregateBlockHashTableImplCollisionFree(accessor_mux);
+  if (useCollisionFreeVector() || useCompactKeySeparateChaining()) {
+    aggregateBlockHashTableImplSharedTable(accessor_mux);
   } else if (is_aggregate_partitioned_) {
     aggregateBlockHashTableImplPartitioned(accessor_mux);
   } else {
@@ -536,13 +502,13 @@ void AggregationOperationState::aggregateBlockHashTable(
   }
 }
 
-void AggregationOperationState::aggregateBlockHashTableImplCollisionFree(
+void AggregationOperationState::aggregateBlockHashTableImplSharedTable(
     const ValueAccessorMultiplexer &accessor_mux) {
-  DCHECK(collision_free_hashtable_ != nullptr);
+  DCHECK(shared_hash_table_ != nullptr);
 
-  collision_free_hashtable_->upsertValueAccessorCompositeKey(argument_ids_,
-                                                             group_by_key_ids_,
-                                                             accessor_mux);
+  shared_hash_table_->upsertValueAccessorCompositeKey(argument_ids_,
+                                                      group_by_key_ids_,
+                                                      accessor_mux);
 }
 
 void AggregationOperationState::aggregateBlockHashTableImplPartitioned(
@@ -672,8 +638,10 @@ void AggregationOperationState::finalizeSingleState(
 void AggregationOperationState::finalizeHashTable(
     const std::size_t partition_id,
     InsertDestination *output_destination) {
-  if (is_aggregate_collision_free_) {
+  if (useCollisionFreeVector()) {
     finalizeHashTableImplCollisionFree(partition_id, output_destination);
+  } else if (useCompactKeySeparateChaining()) {
+    finalizeHashTableImplCompactKeySeparateChaining(partition_id, output_destination);
   } else if (is_aggregate_partitioned_) {
     finalizeHashTableImplPartitioned(partition_id, output_destination);
   } else {
@@ -696,9 +664,8 @@ void AggregationOperationState::finalizeHashTable(
 void AggregationOperationState::finalizeHashTableImplCollisionFree(
     const std::size_t partition_id,
     InsertDestination *output_destination) {
-  std::vector<std::unique_ptr<ColumnVector>> final_values;
   CollisionFreeVectorTable *hash_table =
-      static_cast<CollisionFreeVectorTable *>(collision_free_hashtable_.get());
+      static_cast<CollisionFreeVectorTable *>(shared_hash_table_.get());
 
   const std::size_t max_length =
       hash_table->getNumTuplesInFinalizationPartition(partition_id);
@@ -727,6 +694,19 @@ void AggregationOperationState::finalizeHashTableImplCollisionFree(
   output_destination->bulkInsertTuples(&complete_result);
 }
 
+void AggregationOperationState::finalizeHashTableImplCompactKeySeparateChaining(
+    const std::size_t partition_id,
+    InsertDestination *output_destination) {
+  CompactKeySeparateChainingHashTable *hash_table =
+      static_cast<CompactKeySeparateChainingHashTable *>(shared_hash_table_.get());
+
+  ColumnVectorsValueAccessor complete_result;
+  hash_table->finalizeKeys(partition_id, &complete_result);
+
+  // Bulk-insert the complete result.
+  output_destination->bulkInsertTuples(&complete_result);
+}
+
 void AggregationOperationState::finalizeHashTableImplPartitioned(
     const std::size_t partition_id,
     InsertDestination *output_destination) {
@@ -950,8 +930,8 @@ void AggregationOperationState::finalizeHashTableImplThreadPrivateCompactKey(
 std::size_t AggregationOperationState::getMemoryConsumptionBytes() const {
   std::size_t memory = getMemoryConsumptionBytesHelper(distinctify_hashtables_);
   memory += getMemoryConsumptionBytesHelper(group_by_hashtables_);
-  if (collision_free_hashtable_ != nullptr) {
-    memory += collision_free_hashtable_->getMemoryConsumptionBytes();
+  if (shared_hash_table_ != nullptr) {
+    memory += shared_hash_table_->getMemoryConsumptionBytes();
   }
   if (group_by_hashtable_pool_ != nullptr) {
     memory += group_by_hashtable_pool_->getMemoryConsumptionPoolBytes();
@@ -974,4 +954,34 @@ std::size_t AggregationOperationState::getMemoryConsumptionBytesHelper(
   return memory;
 }
 
+std::size_t AggregationOperationState::getNumInitializationPartitions() const {
+  if (useCollisionFreeVector()) {
+    return static_cast<const CollisionFreeVectorTable*>(
+        shared_hash_table_.get())->getNumInitializationPartitions();
+  }
+  if (useCompactKeySeparateChaining()) {
+    return static_cast<const CompactKeySeparateChainingHashTable*>(
+        shared_hash_table_.get())->getNumInitializationPartitions();
+  }
+  return 0u;
+}
+
+std::size_t AggregationOperationState::getNumFinalizationPartitions() const {
+  if (group_by_key_ids_.empty()) {
+    return 1u;
+  }
+  if (useCollisionFreeVector()) {
+    return static_cast<const CollisionFreeVectorTable *>(
+        shared_hash_table_.get())->getNumFinalizationPartitions();
+  }
+  if (useCompactKeySeparateChaining()) {
+    return static_cast<const CompactKeySeparateChainingHashTable *>(
+        shared_hash_table_.get())->getNumFinalizationPartitions();
+  }
+  if (is_aggregate_partitioned_) {
+    return partitioned_group_by_hashtable_pool_->getNumPartitions();
+  }
+  return 1u;
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index c7680b0..ae14dd1 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -108,12 +108,6 @@ class AggregationOperationState {
    * @param storage_manager The StorageManager to use for allocating hash
    *        tables. Single aggregation state (when GROUP BY list is not
    *        specified) is not allocated using memory from storage manager.
-   * @param collision_free_vector_memory_size For CollisionFreeVectorTable,
-   *        the memory size.
-   * @param collision_free_vector_num_init_partitions For
-   *        CollisionFreeVectorTable, the number of partitions to initialize.
-   * @param collision_free_vector_state_offsets For CollisionFreeVectorTable,
-   *        the offsets for each state.
    */
   AggregationOperationState(
       const CatalogRelationSchema &input_relation,
@@ -127,10 +121,7 @@ class AggregationOperationState {
       const std::size_t num_partitions,
       const HashTableImplType hash_table_impl_type,
       const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
-      StorageManager *storage_manager,
-      const std::size_t collision_free_vector_memory_size = 0,
-      const std::size_t collision_free_vector_num_init_partitions = 0,
-      const std::vector<std::size_t> &collision_free_vector_state_offsets = std::vector<std::size_t>());
+      StorageManager *storage_manager);
 
   ~AggregationOperationState() {}
 
@@ -175,6 +166,14 @@ class AggregationOperationState {
   std::size_t getNumInitializationPartitions() const;
 
   /**
+   * @brief Get the number of partitions to be used for finalizing the
+   *        aggregation.
+   *
+   * @return The number of partitions to be used for finalizing the aggregation.
+   **/
+  std::size_t getNumFinalizationPartitions() const;
+
+  /**
    * @brief Initialize the specified partition of this aggregation.
    *
    * @param partition_id ID of the partition to be initialized.
@@ -219,6 +218,14 @@ class AggregationOperationState {
   std::size_t getMemoryConsumptionBytes() const;
 
  private:
+  bool useCollisionFreeVector() const {
+    return hash_table_impl_type_ == HashTableImplType::kCollisionFreeVector;
+  }
+
+  bool useCompactKeySeparateChaining() const {
+    return hash_table_impl_type_ == HashTableImplType::kCompactKeySeparateChaining;
+  }
+
   // Aggregate on input block.
   void aggregateBlockSingleState(const ValueAccessorMultiplexer &accessor_mux);
 
@@ -239,7 +246,7 @@ class AggregationOperationState {
                          InsertDestination *output_destination);
 
   // Specialized implementations for aggregateBlockHashTable.
-  void aggregateBlockHashTableImplCollisionFree(
+  void aggregateBlockHashTableImplSharedTable(
       const ValueAccessorMultiplexer &accessor_mux);
 
   void aggregateBlockHashTableImplPartitioned(
@@ -252,6 +259,9 @@ class AggregationOperationState {
   void finalizeHashTableImplCollisionFree(const std::size_t partition_id,
                                           InsertDestination *output_destination);
 
+  void finalizeHashTableImplCompactKeySeparateChaining(const std::size_t partition_id,
+                                                       InsertDestination *output_destination);
+
   void finalizeHashTableImplPartitioned(const std::size_t partition_id,
                                         InsertDestination *output_destination);
 
@@ -269,8 +279,8 @@ class AggregationOperationState {
   // filter predicate (if any), and the list of GROUP BY expressions (if any).
   const CatalogRelationSchema &input_relation_;
 
-  // Whether the aggregation is collision free or not.
-  const bool is_aggregate_collision_free_;
+  // Hash table implementation type.
+  const HashTableImplType hash_table_impl_type_;
 
   // Whether the aggregation is partitioned or not.
   const bool is_aggregate_partitioned_;
@@ -313,7 +323,7 @@ class AggregationOperationState {
 
   std::unique_ptr<PartitionedHashTablePool> partitioned_group_by_hashtable_pool_;
 
-  std::unique_ptr<AggregationStateHashTableBase> collision_free_hashtable_;
+  std::unique_ptr<AggregationStateHashTableBase> shared_hash_table_;
 
   StorageManager *storage_manager_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/AggregationOperationState.proto
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.proto b/storage/AggregationOperationState.proto
index d2305f1..1a8a302 100644
--- a/storage/AggregationOperationState.proto
+++ b/storage/AggregationOperationState.proto
@@ -45,7 +45,4 @@ message AggregationOperationState {
 
   optional bool is_partitioned = 8;
   optional uint64 num_partitions = 9 [default = 1];
-
-  // Required if 'hash_table_impl_type' is 'COLLISION_FREE_VECTOR'.
-  optional CollisionFreeVectorInfo collision_free_vector_info = 10;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index fb09e49..8ac7285 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -169,6 +169,9 @@ add_library(quickstep_storage_CollisionFreeVectorTable
             CollisionFreeVectorTable.cpp
             CollisionFreeVectorTable.hpp)
 add_library(quickstep_storage_ColumnStoreUtil ColumnStoreUtil.cpp ColumnStoreUtil.hpp)
+add_library(quickstep_storage_CompactKeySeparateChainingHashTable
+            CompactKeySeparateChainingHashTable.cpp
+            CompactKeySeparateChainingHashTable.hpp)
 add_library(quickstep_storage_CompressedBlockBuilder CompressedBlockBuilder.cpp CompressedBlockBuilder.hpp)
 add_library(quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
             CompressedColumnStoreTupleStorageSubBlock.cpp
@@ -474,6 +477,25 @@ target_link_libraries(quickstep_storage_ColumnStoreUtil
                       quickstep_types_operations_comparisons_ComparisonID
                       quickstep_types_operations_comparisons_ComparisonUtil
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_storage_CompactKeySeparateChainingHashTable
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_cli_Flags
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_StorageManager
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_CharType
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_PrimeNumber
+                      quickstep_utility_Range
+                      quickstep_utility_ScopedArray
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_CompressedBlockBuilder
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelationSchema
@@ -721,6 +743,7 @@ target_link_libraries(quickstep_storage_HashTable_proto
 target_link_libraries(quickstep_storage_HashTableFactory
                       glog
                       quickstep_storage_CollisionFreeVectorTable
+                      quickstep_storage_CompactKeySeparateChainingHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTable_proto
                       quickstep_storage_HashTableBase
@@ -1146,6 +1169,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_CSBTreeIndexSubBlock
                       quickstep_storage_CollisionFreeVectorTable
                       quickstep_storage_ColumnStoreUtil
+                      quickstep_storage_CompactKeySeparateChainingHashTable
                       quickstep_storage_CompressedBlockBuilder
                       quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
                       quickstep_storage_CompressedColumnStoreValueAccessor

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/CollisionFreeVectorTable.cpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeVectorTable.cpp b/storage/CollisionFreeVectorTable.cpp
index e803954..d836014 100644
--- a/storage/CollisionFreeVectorTable.cpp
+++ b/storage/CollisionFreeVectorTable.cpp
@@ -43,33 +43,66 @@ namespace quickstep {
 CollisionFreeVectorTable::CollisionFreeVectorTable(
     const Type *key_type,
     const std::size_t num_entries,
-    const std::size_t memory_size,
-    const std::size_t num_init_partitions,
-    const std::size_t num_finalize_partitions,
-    const std::vector<std::size_t> &state_offsets,
     const std::vector<AggregationHandle *> &handles,
     StorageManager *storage_manager)
     : key_type_(key_type),
       num_entries_(num_entries),
       num_handles_(handles.size()),
       handles_(handles),
-      memory_size_(memory_size),
-      num_init_partitions_(num_init_partitions),
-      num_finalize_partitions_(num_finalize_partitions),
+      num_finalize_partitions_(CalculateNumFinalizationPartitions(num_entries_)),
       storage_manager_(storage_manager) {
   DCHECK_GT(num_entries, 0u);
-  DCHECK_GT(num_finalize_partitions_, 0u);
-  DCHECK_EQ(num_handles_, state_offsets.size());
+
+  std::size_t required_memory = 0;
+  const std::size_t existence_map_offset = 0;
+  std::vector<std::size_t> state_offsets;
+
+  required_memory += CacheLineAlignedBytes(
+      BarrieredReadWriteConcurrentBitVector::BytesNeeded(num_entries));
+
+  for (std::size_t i = 0; i < num_handles_; ++i) {
+    const AggregationHandle *handle = handles_[i];
+    const std::vector<const Type *> argument_types = handle->getArgumentTypes();
+
+    std::size_t state_size = 0;
+    switch (handle->getAggregationID()) {
+      case AggregationID::kCount: {
+        state_size = sizeof(std::atomic<std::size_t>);
+        break;
+      }
+      case AggregationID::kSum: {
+        DCHECK_EQ(1u, argument_types.size());
+        switch (argument_types.front()->getTypeID()) {
+          case TypeID::kInt:  // Fall through
+          case TypeID::kLong:
+            state_size = sizeof(std::atomic<std::int64_t>);
+            break;
+          case TypeID::kFloat:  // Fall through
+          case TypeID::kDouble:
+            state_size = sizeof(std::atomic<double>);
+            break;
+          default:
+            LOG(FATAL) << "Not implemented";
+        }
+        break;
+      }
+      default:
+        LOG(FATAL) << "Not implemented";
+    }
+
+    state_offsets.emplace_back(required_memory);
+    required_memory += CacheLineAlignedBytes(state_size * num_entries);
+  }
 
   const std::size_t num_storage_slots =
-      storage_manager_->SlotsNeededForBytes(memory_size_);
+      storage_manager_->SlotsNeededForBytes(required_memory);
 
   const block_id blob_id = storage_manager_->createBlob(num_storage_slots);
   blob_ = storage_manager_->getBlobMutable(blob_id);
 
   void *memory_start = blob_->getMemoryMutable();
   existence_map_.reset(new BarrieredReadWriteConcurrentBitVector(
-      reinterpret_cast<char *>(memory_start),
+      reinterpret_cast<char *>(memory_start) + existence_map_offset,
       num_entries,
       false /* initialize */));
 
@@ -78,6 +111,9 @@ CollisionFreeVectorTable::CollisionFreeVectorTable(
     vec_tables_.emplace_back(
         reinterpret_cast<char *>(memory_start) + state_offsets.at(i));
   }
+
+  memory_size_ = required_memory;
+  num_init_partitions_ = CalculateNumInitializationPartitions(memory_size_);
 }
 
 CollisionFreeVectorTable::~CollisionFreeVectorTable() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/CollisionFreeVectorTable.hpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeVectorTable.hpp b/storage/CollisionFreeVectorTable.hpp
index 8e1342b..7a7d07e 100644
--- a/storage/CollisionFreeVectorTable.hpp
+++ b/storage/CollisionFreeVectorTable.hpp
@@ -58,12 +58,6 @@ class CollisionFreeVectorTable : public AggregationStateHashTableBase {
    *
    * @param key_type The group-by key type.
    * @param num_entries The estimated number of entries this table will hold.
-   * @param memory_size The memory size for this table.
-   * @param num_init_partitions The number of partitions to be used for
-   *        initializing the aggregation.
-   * @param num_finalize_partitions The number of partitions to be used for
-   *        finalizing the aggregation.
-   * @param state_offsets The offsets for each state in the table.
    * @param handles The aggregation handles.
    * @param storage_manager The StorageManager to use (a StorageBlob will be
    *        allocated to hold this table's contents).
@@ -71,10 +65,6 @@ class CollisionFreeVectorTable : public AggregationStateHashTableBase {
   CollisionFreeVectorTable(
       const Type *key_type,
       const std::size_t num_entries,
-      const std::size_t memory_size,
-      const std::size_t num_init_partitions,
-      const std::size_t num_finalize_partitions,
-      const std::vector<std::size_t> &state_offsets,
       const std::vector<AggregationHandle *> &handles,
       StorageManager *storage_manager);
 
@@ -188,6 +178,32 @@ class CollisionFreeVectorTable : public AggregationStateHashTableBase {
   }
 
  private:
+  inline static std::size_t CacheLineAlignedBytes(const std::size_t actual_bytes) {
+    return (actual_bytes + kCacheLineBytes - 1) / kCacheLineBytes * kCacheLineBytes;
+  }
+
+  inline static std::size_t CalculateNumInitializationPartitions(
+      const std::size_t memory_size) {
+    // Set initialization memory block size as 4MB.
+    constexpr std::size_t kInitBlockSize = 4uL * 1024u * 1024u;
+
+    // At least 1 partition, at most 80 partitions.
+    // TODO(jianqiao): set the upbound as (# of workers * 2) instead of the
+    // hardcoded 80.
+    return std::max(1uL, std::min(memory_size / kInitBlockSize, 80uL));
+  }
+
+  inline static std::size_t CalculateNumFinalizationPartitions(
+      const std::size_t num_entries) {
+    // Set finalization segment size as 4096 entries.
+    constexpr std::size_t kFinalizeSegmentSize = 4uL * 1024L;
+
+    // At least 1 partition, at most 80 partitions.
+    // TODO(jianqiao): set the upbound as (# of workers * 2) instead of the
+    // hardcoded 80.
+    return std::max(1uL, std::min(num_entries / kFinalizeSegmentSize, 80uL));
+  }
+
   inline std::size_t calculatePartitionLength() const {
     const std::size_t partition_length =
         (num_entries_ + num_finalize_partitions_ - 1) / num_finalize_partitions_;
@@ -325,9 +341,9 @@ class CollisionFreeVectorTable : public AggregationStateHashTableBase {
   std::unique_ptr<BarrieredReadWriteConcurrentBitVector> existence_map_;
   std::vector<void *> vec_tables_;
 
-  const std::size_t memory_size_;
-  const std::size_t num_init_partitions_;
-  const std::size_t num_finalize_partitions_;
+  std::size_t memory_size_;
+  std::size_t num_init_partitions_;
+  std::size_t num_finalize_partitions_;
 
   StorageManager *storage_manager_;
   MutableBlobReference blob_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/CompactKeySeparateChainingHashTable.cpp
----------------------------------------------------------------------
diff --git a/storage/CompactKeySeparateChainingHashTable.cpp b/storage/CompactKeySeparateChainingHashTable.cpp
new file mode 100644
index 0000000..fd7a69f
--- /dev/null
+++ b/storage/CompactKeySeparateChainingHashTable.cpp
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "storage/CompactKeySeparateChainingHashTable.hpp"
+
+#include <cstddef>
+#include <memory>
+#include <vector>
+#include <type_traits>
+
+#include "cli/Flags.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/CharType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/PrimeNumber.hpp"
+#include "utility/Range.hpp"
+#include "utility/ScopedArray.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class AggregationHandle;
+
+namespace {
+
+template <typename Functor>
+inline auto InvokeOnKeySize(const std::size_t key_size,
+                            const Functor &functor) {
+  switch (key_size) {
+    case 0u:
+      break;
+    case 1u:
+      return functor(std::integral_constant<std::size_t, 1u>());
+    case 2u:
+      return functor(std::integral_constant<std::size_t, 2u>());
+    case 3u:
+      return functor(std::integral_constant<std::size_t, 3u>());
+    case 4u:
+      return functor(std::integral_constant<std::size_t, 4u>());
+    case 5u:
+      return functor(std::integral_constant<std::size_t, 5u>());
+    case 6u:
+      return functor(std::integral_constant<std::size_t, 6u>());
+    case 7u:
+      return functor(std::integral_constant<std::size_t, 7u>());
+    case 8u:
+      return functor(std::integral_constant<std::size_t, 8u>());
+    default:
+      break;
+  }
+  LOG(FATAL) << "Not supported";
+}
+
+}  // namespace
+
+CompactKeySeparateChainingHashTable::CompactKeySeparateChainingHashTable(
+    const std::vector<const Type*> &key_types,
+    const std::size_t num_entries,
+    const std::vector<AggregationHandle *> &handles,
+    StorageManager *storage_manager)
+    : key_types_(key_types),
+      buckets_allocated_(0) {
+  DCHECK_GT(num_entries, 0u);
+  if (!handles.empty()) {
+    LOG(FATAL) << "CompactKeySeparateChainingHashTable do not "
+               << "support aggregate functions yet";
+  }
+
+  num_slots_ = get_next_prime_number(num_entries * kHashTableLoadFactor);
+  num_key_buckets_ = num_slots_ / kHashTableLoadFactor;
+
+  // TODO: use storage manager for allocating memory.
+  slots_.reset(num_slots_);
+  key_buckets_.reset(num_key_buckets_);
+
+  std::size_t total_key_size = 0;
+  for (const Type *key_type : key_types) {
+    CHECK(!key_type->isNullable()) << "Not supported";
+    CHECK(!key_type->isVariableLength()) << "Not supported";
+
+    const std::size_t key_size = key_type->maximumByteLength();
+    key_sizes_.emplace_back(key_size);
+    total_key_size += key_size;
+  }
+
+  if (total_key_size > sizeof(KeyCode)) {
+    LOG(FATAL) << "Total key size exceeded KeyCode size";
+  }
+
+  slots_init_splitter_ = std::make_unique<RangeSplitter>(
+      RangeSplitter::CreateWithPartitionLength(
+          0, num_slots_, kInitMinPartitionLength, FLAGS_num_workers));
+
+  key_buckets_init_splitter_ = std::make_unique<RangeSplitter>(
+      RangeSplitter::CreateWithNumPartitions(
+          0, num_key_buckets_, slots_init_splitter_->getNumPartitions()));
+}
+
+bool CompactKeySeparateChainingHashTable::upsertValueAccessorCompositeKey(
+    const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_attr_ids,
+    const ValueAccessorMultiplexer &accessor_mux) {
+  DCHECK(argument_ids.empty());
+  DCHECK_EQ(key_sizes_.size(), key_attr_ids.size());
+
+  const std::size_t num_tuples =
+      accessor_mux.getValueAccessorBySource(
+          key_attr_ids.front().source)->getNumTuplesVirtual();
+
+  ScopedArray<KeyCode> key_codes(num_tuples, true);
+
+  std::size_t offset = 0;
+  for (std::size_t i = 0; i < key_sizes_.size(); ++i) {
+    const MultiSourceAttributeId &key_attr = key_attr_ids[i];
+
+    InvokeOnAnyValueAccessor(
+        accessor_mux.getValueAccessorBySource(key_attr.source),
+        [&](auto *accessor) -> void {
+
+      InvokeOnKeySize(
+          key_sizes_[i],
+          [&](auto key_size) -> void {
+        this->constructCompactKeyCodeComponent(num_tuples,
+                                               offset,
+                                               decltype(key_size)::value,
+                                               accessor,
+                                               key_attr.attr_id,
+                                               key_codes.get());
+      });
+    });
+
+    offset += key_sizes_[i];
+  }
+
+  for (std::size_t i = 0; i < num_tuples; ++i) {
+    locateBucketInternal(key_codes[i]);
+  }
+
+  return true;
+}
+
+void CompactKeySeparateChainingHashTable::finalizeKeys(
+    const std::size_t partition_id,
+    ColumnVectorsValueAccessor *output) const {
+  DCHECK(final_splitter_ != nullptr);
+  const Range range = final_splitter_->getPartition(partition_id);
+
+  std::size_t offset = 0;
+  for (std::size_t i = 0; i < key_sizes_.size(); ++i) {
+    std::unique_ptr<NativeColumnVector> cv =
+        std::make_unique<NativeColumnVector>(*key_types_[i], range.size());
+
+    InvokeOnKeySize(
+        key_sizes_[i],
+        [&](auto key_size) -> void {
+      const char *key_ptr =
+          reinterpret_cast<const char*>(&key_buckets_[range.begin()].key_code) + offset;
+      for (std::size_t j = 0; j < range.size(); ++j) {
+        std::memcpy(cv->getPtrForDirectWrite(),
+                    key_ptr,
+                    decltype(key_size)::value);
+        key_ptr += kKeyBucketDataSize;
+      }
+    });
+
+    output->addColumn(ColumnVectorPtr(cv.release()));
+    offset += key_sizes_[i];
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/CompactKeySeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/CompactKeySeparateChainingHashTable.hpp b/storage/CompactKeySeparateChainingHashTable.hpp
new file mode 100644
index 0000000..0d057e4
--- /dev/null
+++ b/storage/CompactKeySeparateChainingHashTable.hpp
@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_STORAGE_COMPACT_KEY_SEPARATE_CHAINING_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_COMPACT_KEY_SEPARATE_CHAINING_HASH_TABLE_HPP_
+
+#include <atomic>
+#include <cstdint>
+#include <memory>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "cli/Flags.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/StorageConstants.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "utility/Macros.hpp"
+#include "utility/Range.hpp"
+#include "utility/ScopedArray.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class AggregationHandle;
+class ColumnVectorsValueAccessor;
+class StorageManager;
+
+class CompactKeySeparateChainingHashTable : public AggregationStateHashTableBase {
+ public:
+  CompactKeySeparateChainingHashTable(
+      const std::vector<const Type*> &key_types,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle *> &handles,
+      StorageManager *storage_manager);
+
+  bool upsertValueAccessorCompositeKey(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_attr_ids,
+      const ValueAccessorMultiplexer &accessor_mux) override;
+
+  void destroyPayload() override {}
+
+  HashTableImplType getImplType() const override {
+    return HashTableImplType::kCompactKeySeparateChaining;
+  }
+
+  std::size_t getMemoryConsumptionBytes() const override {
+    return kSlotDataSize * num_slots_ + kKeyBucketDataSize * num_key_buckets_;
+  }
+
+  inline std::size_t getNumInitializationPartitions() const {
+    return slots_init_splitter_->getNumPartitions();
+  }
+
+  inline std::size_t getNumFinalizationPartitions() const {
+    if (final_splitter_ == nullptr) {
+      final_splitter_ = std::make_unique<RangeSplitter>(
+          RangeSplitter::CreateWithPartitionLength(
+              0, buckets_allocated_.load(std::memory_order_relaxed),
+              kFinalMinPartitionLength, FLAGS_num_workers * 2));
+    }
+    return final_splitter_->getNumPartitions();
+  }
+
+  void initialize(const std::size_t partition_id) {
+    const Range slots_range = slots_init_splitter_->getPartition(partition_id);
+    std::memset(slots_.get() + slots_range.begin(),
+                0,
+                slots_range.size() * kSlotDataSize);
+
+    const Range key_buckets_range =
+        key_buckets_init_splitter_->getPartition(partition_id);
+    std::memset(key_buckets_.get() + key_buckets_range.begin(),
+                0,
+                key_buckets_range.size() * kKeyBucketDataSize);
+  }
+
+  void finalizeKeys(const std::size_t partition_id,
+                    ColumnVectorsValueAccessor *output) const;
+
+ private:
+  inline static std::size_t CacheLineAlignedBytes(const std::size_t actual_bytes) {
+    return (actual_bytes + kCacheLineBytes - 1) / kCacheLineBytes * kCacheLineBytes;
+  }
+
+  inline static std::size_t CalculateNumInitializationPartitions(
+      const std::size_t memory_size) {
+    // Set initialization memory block size as 4MB.
+    constexpr std::size_t kInitBlockSize = 4uL * 1024u * 1024u;
+
+    // At least 1 partition, at most 80 partitions.
+    return std::max(1uL, std::min(memory_size / kInitBlockSize, 80uL));
+  }
+
+  inline static std::size_t CalculateNumFinalizationPartitions(
+      const std::size_t num_entries) {
+    // Set finalization segment size as 4096 entries.
+    constexpr std::size_t kFinalizeSegmentSize = 4uL * 1024L;
+
+    // At least 1 partition, at most 80 partitions.
+    return std::max(1uL, std::min(num_entries / kFinalizeSegmentSize, 80uL));
+  }
+
+  using KeyCode = std::uint64_t;
+  using BucketIndex = std::uint32_t;
+
+  inline BucketIndex locateBucketInternal(const KeyCode key_code);
+
+  template <typename ValueAccessorT>
+  inline void constructCompactKeyCodeComponent(const std::size_t num_tuples,
+                                               const std::size_t offset,
+                                               const std::size_t key_size,
+                                               ValueAccessorT *accessor,
+                                               const attribute_id attr,
+                                               KeyCode *key_codes);
+
+  static constexpr std::size_t kInitMinPartitionLength = 1024uL * 256uL;
+  static constexpr std::size_t kFinalMinPartitionLength = 1024uL * 4uL;
+
+  struct KeyBucket {
+    KeyCode key_code;
+    std::atomic<BucketIndex> next;
+  };
+
+  static constexpr std::size_t kSlotDataSize = sizeof(std::atomic<BucketIndex>);
+  static constexpr std::size_t kKeyBucketDataSize = sizeof(KeyBucket);
+  static constexpr BucketIndex kExclusiveState = std::numeric_limits<BucketIndex>::max();
+
+  const std::vector<const Type*> key_types_;
+  std::vector<std::size_t> key_sizes_;
+
+  ScopedArray<std::atomic<BucketIndex>> slots_;
+  ScopedArray<KeyBucket> key_buckets_;
+
+  std::size_t num_slots_;
+  std::size_t num_key_buckets_;
+  std::atomic<std::size_t> buckets_allocated_;
+
+  std::unique_ptr<RangeSplitter> slots_init_splitter_;
+  std::unique_ptr<RangeSplitter> key_buckets_init_splitter_;
+  mutable std::unique_ptr<RangeSplitter> final_splitter_;
+
+  DISALLOW_COPY_AND_ASSIGN(CompactKeySeparateChainingHashTable);
+};
+
+// ----------------------------------------------------------------------------
+// Implementations of class methods follow.
+
+inline CompactKeySeparateChainingHashTable::BucketIndex
+    CompactKeySeparateChainingHashTable::locateBucketInternal(const KeyCode key_code) {
+  std::atomic<BucketIndex> *pending_chain = &slots_[key_code % num_slots_];
+
+  for (;;) {
+    BucketIndex existing_chain = 0;
+
+    // Check if current node is the end of the chain.
+    if (pending_chain->compare_exchange_strong(existing_chain,
+                                               kExclusiveState,
+                                               std::memory_order_acq_rel)) {
+      const BucketIndex bucket_index =
+          buckets_allocated_.fetch_add(1, std::memory_order_relaxed);
+
+      // TODO(jianqiao): Resize.
+      if (bucket_index > num_key_buckets_) {
+        LOG(FATAL) << "Need resize, not handled";
+      }
+
+      // Store key code into key bucket.
+      key_buckets_[bucket_index].key_code = key_code;
+
+      // Update the chaing pointer to point to the new node.
+      pending_chain->store(bucket_index + 1, std::memory_order_release);
+
+      return bucket_index;
+    }
+
+    // Spin until the pointer is available.
+    while (existing_chain == kExclusiveState) {
+      existing_chain = pending_chain->load(std::memory_order_acquire);
+    }
+
+    if (existing_chain == 0) {
+      // Other thread had to roll back, so try again.
+      continue;
+    }
+
+    const BucketIndex bucket_index = existing_chain - 1;
+    KeyBucket &key_bucket = key_buckets_[bucket_index];
+    if (key_bucket.key_code == key_code) {
+      return bucket_index;
+    } else {
+      pending_chain = &key_bucket.next;
+    }
+  }
+}
+
+template <typename ValueAccessorT>
+inline void CompactKeySeparateChainingHashTable
+    ::constructCompactKeyCodeComponent(const std::size_t num_tuples,
+                                       const std::size_t offset,
+                                       const std::size_t key_size,
+                                       ValueAccessorT *accessor,
+                                       const attribute_id attr,
+                                       KeyCode *key_codes) {
+  accessor->beginIteration();
+  for (std::size_t i = 0; i < num_tuples; ++i) {
+    accessor->next();
+    std::memcpy(reinterpret_cast<char*>(key_codes + i) + offset,
+                accessor->template getUntypedValue<false>(attr),
+                key_size);
+  }
+}
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_COMPACT_KEY_SEPARATE_CHAINING_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/Flags.hpp
----------------------------------------------------------------------
diff --git a/storage/Flags.hpp b/storage/Flags.hpp
index 1d5527c..87f7da4 100644
--- a/storage/Flags.hpp
+++ b/storage/Flags.hpp
@@ -41,7 +41,6 @@ DECLARE_bool(use_hdfs);
 DECLARE_string(hdfs_namenode_host);
 DECLARE_int32(hdfs_namenode_port);
 DECLARE_int32(hdfs_num_replications);
-
 #endif
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/HashTable.proto
----------------------------------------------------------------------
diff --git a/storage/HashTable.proto b/storage/HashTable.proto
index d489b9f..40c8e32 100644
--- a/storage/HashTable.proto
+++ b/storage/HashTable.proto
@@ -23,16 +23,11 @@ import "types/Type.proto";
 
 enum HashTableImplType {
   COLLISION_FREE_VECTOR = 0;
-  LINEAR_OPEN_ADDRESSING = 1;
-  SEPARATE_CHAINING = 2;
-  SIMPLE_SCALAR_SEPARATE_CHAINING = 3;
-  THREAD_PRIVATE_COMPACT_KEY = 4;
-}
-
-message CollisionFreeVectorInfo {
-  required uint64 memory_size = 1;
-  required uint64 num_init_partitions = 2;
-  repeated uint64 state_offsets = 3;
+  COMPACT_KEY_SEPARATE_CHAINING = 1;
+  LINEAR_OPEN_ADDRESSING = 2;
+  SEPARATE_CHAINING = 3;
+  SIMPLE_SCALAR_SEPARATE_CHAINING = 4;
+  THREAD_PRIVATE_COMPACT_KEY = 5;
 }
 
 // NOTE(chasseur): This proto describes the run-time parameters for a resizable

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index 4d9310c..2e89554 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -42,6 +42,7 @@ class ValueAccessor;
  **/
 enum class HashTableImplType {
   kCollisionFreeVector,
+  kCompactKeySeparateChaining,
   kLinearOpenAddressing,
   kSeparateChaining,
   kSimpleScalarSeparateChaining,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index 732920f..df71100 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -25,6 +25,7 @@
 #include <vector>
 
 #include "storage/CollisionFreeVectorTable.hpp"
+#include "storage/CompactKeySeparateChainingHashTable.hpp"
 #include "storage/HashTable.hpp"
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTable.pb.h"
@@ -118,6 +119,8 @@ inline HashTableImplType HashTableImplTypeFromProto(
   switch (proto_type) {
     case serialization::HashTableImplType::COLLISION_FREE_VECTOR:
       return HashTableImplType::kCollisionFreeVector;
+    case serialization::HashTableImplType::COMPACT_KEY_SEPARATE_CHAINING:
+      return HashTableImplType::kCompactKeySeparateChaining;
     case serialization::HashTableImplType::LINEAR_OPEN_ADDRESSING:
       return HashTableImplType::kLinearOpenAddressing;
     case serialization::HashTableImplType::SEPARATE_CHAINING:
@@ -356,14 +359,6 @@ class AggregationStateHashTableFactory {
    * @param storage_manager The StorageManager to use (a StorageBlob will be
    *        allocated to hold the hash table's contents). Forwarded as-is to the
    *        hash table constructor.
-   * @param num_partitions The number of partitions of this aggregation state
-   *        hash table.
-   * @param collision_free_vector_memory_size For CollisionFreeVectorTable,
-   *        the memory size.
-   * @param collision_free_vector_num_init_partitions For
-   *        CollisionFreeVectorTable, the number of partitions to initialize.
-   * @param collision_free_vector_state_offsets For CollisionFreeVectorTable,
-   *        the offsets for each state.
    * @return A new aggregation state hash table.
    **/
   static AggregationStateHashTableBase* CreateResizable(
@@ -371,18 +366,15 @@ class AggregationStateHashTableFactory {
       const std::vector<const Type*> &key_types,
       const std::size_t num_entries,
       const std::vector<AggregationHandle *> &handles,
-      StorageManager *storage_manager,
-      const std::size_t num_partitions = 1u,
-      const std::size_t collision_free_vector_memory_size = 0,
-      const std::size_t collision_free_vector_num_init_partitions = 0,
-      const std::vector<std::size_t> &collision_free_vector_state_offsets = std::vector<std::size_t>()) {
+      StorageManager *storage_manager) {
     switch (hash_table_type) {
       case HashTableImplType::kCollisionFreeVector:
         DCHECK_EQ(1u, key_types.size());
         return new CollisionFreeVectorTable(
-            key_types.front(), num_entries, collision_free_vector_memory_size,
-            collision_free_vector_num_init_partitions, num_partitions,
-            collision_free_vector_state_offsets, handles, storage_manager);
+            key_types.front(), num_entries, handles, storage_manager);
+      case HashTableImplType::kCompactKeySeparateChaining:
+        return new CompactKeySeparateChainingHashTable(
+            key_types, num_entries, handles, storage_manager);
       case HashTableImplType::kSeparateChaining:
         return new PackedPayloadHashTable(
             key_types, num_entries, handles, storage_manager);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index c5c9dd8..c78ec59 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -194,6 +194,8 @@ add_library(quickstep_utility_PrimeNumber PrimeNumber.cpp PrimeNumber.hpp)
 add_library(quickstep_utility_PtrList ../empty_src.cpp PtrList.hpp)
 add_library(quickstep_utility_PtrMap ../empty_src.cpp PtrMap.hpp)
 add_library(quickstep_utility_PtrVector ../empty_src.cpp PtrVector.hpp)
+add_library(quickstep_utility_Range ../empty_src.cpp Range.hpp)
+add_library(quickstep_utility_ScopedArray ../empty_src.cpp ScopedArray.hpp)
 add_library(quickstep_utility_ScopedBuffer ../empty_src.cpp ScopedBuffer.hpp)
 add_library(quickstep_utility_ScopedDeleter ../empty_src.cpp ScopedDeleter.hpp)
 add_library(quickstep_utility_ScopedReassignment ../empty_src.cpp ScopedReassignment.hpp)
@@ -308,6 +310,12 @@ target_link_libraries(quickstep_utility_PtrMap
 target_link_libraries(quickstep_utility_PtrVector
                       glog
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_Range
+                      glog
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_ScopedArray
+                      quickstep_utility_ScopedBuffer
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_ScopedBuffer
                       glog
                       quickstep_utility_Alignment
@@ -380,6 +388,8 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_PtrList
                       quickstep_utility_PtrMap
                       quickstep_utility_PtrVector
+                      quickstep_utility_Range
+                      quickstep_utility_ScopedArray
                       quickstep_utility_ScopedBuffer
                       quickstep_utility_ScopedDeleter
                       quickstep_utility_ScopedReassignment

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/utility/Range.hpp
----------------------------------------------------------------------
diff --git a/utility/Range.hpp b/utility/Range.hpp
new file mode 100644
index 0000000..4fb55be
--- /dev/null
+++ b/utility/Range.hpp
@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_RANGE_HPP_
+#define QUICKSTEP_UTILITY_RANGE_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <limits>
+
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class Range {
+ public:
+  Range(const std::size_t begin, const std::size_t end)
+      : begin_(begin), end_(end) {
+    DCHECK_LE(begin_, end_);
+  }
+
+  Range(const Range &range)
+      : begin_(range.begin_), end_(range.end_) {
+    DCHECK_LE(begin_, end_);
+  }
+
+  inline std::size_t begin() const {
+    return begin_;
+  }
+
+  inline std::size_t end() const {
+    return end_;
+  }
+
+  inline std::size_t size() const {
+    return end_ - begin_;
+  }
+
+ private:
+  const std::size_t begin_;
+  const std::size_t end_;
+};
+
+class RangeSplitter {
+ public:
+  static RangeSplitter CreateWithPartitionLength(
+      const std::size_t begin,
+      const std::size_t end,
+      const std::size_t min_partition_length,
+      const std::size_t max_num_partitions = kMaxNumPartitions) {
+    DCHECK_LE(begin, end);
+    DCHECK_GT(min_partition_length, 0u);
+    DCHECK_GT(max_num_partitions, 0u);
+
+    const std::size_t range_length = end - begin;
+    const std::size_t est_num_partitions = range_length / min_partition_length;
+
+    const std::size_t num_partitions =
+        std::max(1uL, std::min(est_num_partitions, max_num_partitions));
+    const std::size_t partition_length = range_length / num_partitions;
+    return RangeSplitter(begin, end, num_partitions, partition_length);
+  }
+
+  static RangeSplitter CreateWithPartitionLength(
+      const Range &range,
+      const std::size_t min_partition_length,
+      const std::size_t max_num_partitions = kMaxNumPartitions) {
+    return CreateWithPartitionLength(
+        range.begin(), range.end(), min_partition_length, max_num_partitions);
+  }
+
+  static RangeSplitter CreateWithMinMaxPartitionLength(
+      const std::size_t begin,
+      const std::size_t end,
+      const std::size_t min_partition_length,
+      const std::size_t max_partition_length,
+      const std::size_t ept_num_partitions) {
+    DCHECK_LE(begin, end);
+    DCHECK_LE(min_partition_length, max_partition_length);
+    DCHECK_GT(min_partition_length, 0u);
+    DCHECK_GT(max_partition_length, 0u);
+
+    const std::size_t range_length = end - begin;
+    const std::size_t ept_partition_length = range_length / ept_num_partitions;
+
+    std::size_t partition_length;
+    if (ept_partition_length < min_partition_length) {
+      partition_length = min_partition_length;
+    } else if (ept_partition_length > max_partition_length) {
+      partition_length = max_partition_length;
+    } else {
+      partition_length = ept_partition_length;
+    }
+
+    const std::size_t num_partitions =
+        std::max(1uL, range_length / partition_length);
+    return RangeSplitter(begin, end, num_partitions, partition_length);
+  }
+
+  static RangeSplitter CreateWithMinMaxPartitionLength(
+      const Range &range,
+      const std::size_t min_partition_length,
+      const std::size_t max_partition_length,
+      const std::size_t ept_num_partitions) {
+    return CreateWithMinMaxPartitionLength(
+        range.begin(), range.end(),
+        min_partition_length, max_partition_length,
+        ept_num_partitions);
+  }
+
+  static RangeSplitter CreateWithNumPartitions(
+      const std::size_t begin,
+      const std::size_t end,
+      const std::size_t num_partitions) {
+    DCHECK_LE(begin, end);
+    DCHECK_GT(num_partitions, 0u);
+
+    const std::size_t partition_length = (end - begin) / num_partitions;
+    return RangeSplitter(begin, end, num_partitions, partition_length);
+  }
+
+  static RangeSplitter CreateWithNumPartitions(
+      const Range &range,
+      const std::size_t num_partitions) {
+    return CreateWithNumPartitions(range.begin(), range.end(), num_partitions);
+  }
+
+  RangeSplitter(const RangeSplitter &other)
+      : begin_(other.begin_), end_(other.end_),
+        num_partitions_(other.num_partitions_),
+        partition_length_(other.partition_length_) {}
+
+  inline std::size_t getNumPartitions() const {
+    return num_partitions_;
+  }
+
+  inline Range getPartition(const std::size_t partition_id) const {
+    DCHECK_LT(partition_id, num_partitions_);
+    const std::size_t begin = begin_ + partition_length_ * partition_id;
+    const std::size_t end =
+        partition_id == num_partitions_ - 1
+            ? end_
+            : begin + partition_length_;
+    return Range(begin, end);
+  }
+
+ private:
+  RangeSplitter(const std::size_t begin,
+                const std::size_t end,
+                const std::size_t num_partitions,
+                const std::size_t partition_length)
+      : begin_(begin),
+        end_(end),
+        num_partitions_(num_partitions),
+        partition_length_(partition_length) {
+    DCHECK_LE(num_partitions_ * partition_length_, end_);
+  }
+
+  static constexpr std::size_t kMaxNumPartitions =
+      std::numeric_limits<std::size_t>::max();
+
+  const std::size_t begin_;
+  const std::size_t end_;
+  const std::size_t num_partitions_;
+  const std::size_t partition_length_;
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_RANGE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/df20e4c0/utility/ScopedArray.hpp
----------------------------------------------------------------------
diff --git a/utility/ScopedArray.hpp b/utility/ScopedArray.hpp
new file mode 100644
index 0000000..c464ba9
--- /dev/null
+++ b/utility/ScopedArray.hpp
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_SCOPED_ARRAY_HPP_
+#define QUICKSTEP_UTILITY_SCOPED_ARRAY_HPP_
+
+#include <cstddef>
+#include <utility>
+
+#include "utility/ScopedBuffer.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+template <typename T>
+class ScopedArray {
+ public:
+  explicit ScopedArray(const std::size_t length, const bool initialize = false)
+      : buffer_(length * sizeof(T), initialize) {}
+
+  explicit ScopedArray(T *data = nullptr)
+      : buffer_(data) {}
+
+  explicit ScopedArray(ScopedArray &&orig)
+      : buffer_(std::move(orig.buffer_)) {}
+
+  inline void reset(const std::size_t length, const bool initialize = false) {
+    buffer_.reset(length * sizeof(T), initialize);
+  }
+
+  inline void reset(T *data = nullptr) {
+    buffer_.reset(data);
+  }
+
+  inline ScopedArray& operator=(ScopedArray &&rhs) {
+    buffer_ = std::move(rhs.buffer_);
+    return *this;
+  }
+
+  inline T* get() const {
+    return static_cast<T*>(buffer_.get());
+  }
+
+  inline T* release() {
+    return static_cast<T*>(buffer_.release());
+  }
+
+  inline T* operator->() const {
+    return get();
+  }
+
+  inline T& operator[](const std::size_t index) const {
+    return get()[index];
+  }
+
+ private:
+  ScopedBuffer buffer_;
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_SCOPED_ARRAY_HPP_


[10/11] incubator-quickstep git commit: Initialize updates for transitive closure

Posted by ji...@apache.org.
Initialize updates for transitive closure


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

Branch: refs/heads/transitive-closure
Commit: 734ddc1ec10459e60e7f59edbf296f7988070e05
Parents: d725bca
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sun Dec 10 21:00:16 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Sun Dec 10 21:00:16 2017 -0600

----------------------------------------------------------------------
 parser/CMakeLists.txt                           |    8 +
 parser/ParseTableReference.hpp                  |    3 +-
 parser/ParseTransitiveClosureTableReference.cpp |   48 +
 parser/ParseTransitiveClosureTableReference.hpp |   83 +
 parser/SqlLexer.lpp                             |    6 +-
 parser/SqlParser.ypp                            |    8 +-
 parser/preprocessed/SqlLexer_gen.cpp            | 1380 +++++----
 parser/preprocessed/SqlLexer_gen.hpp            |    2 +-
 parser/preprocessed/SqlParser_gen.cpp           | 2909 +++++++++---------
 parser/preprocessed/SqlParser_gen.hpp           |  201 +-
 query_execution/CMakeLists.txt                  |    1 +
 query_execution/QueryContext.cpp                |    6 +
 query_execution/QueryContext.hpp                |   12 +
 query_execution/QueryContext.proto              |   13 +-
 query_optimizer/CMakeLists.txt                  |    4 +
 query_optimizer/ExecutionGenerator.cpp          |  144 +-
 query_optimizer/ExecutionGenerator.hpp          |    3 +
 query_optimizer/PhysicalGenerator.cpp           |    2 +-
 query_optimizer/cost_model/CMakeLists.txt       |    2 +
 query_optimizer/cost_model/SimpleCostModel.cpp  |   10 +
 query_optimizer/cost_model/SimpleCostModel.hpp  |    4 +
 .../cost_model/StarSchemaSimpleCostModel.cpp    |   26 +
 .../cost_model/StarSchemaSimpleCostModel.hpp    |    4 +
 query_optimizer/logical/CMakeLists.txt          |    9 +
 query_optimizer/logical/LogicalType.hpp         |    1 +
 query_optimizer/logical/PatternMatcher.hpp      |    2 +
 query_optimizer/logical/TransitiveClosure.cpp   |   80 +
 query_optimizer/logical/TransitiveClosure.hpp   |  134 +
 query_optimizer/physical/CMakeLists.txt         |    9 +
 query_optimizer/physical/PatternMatcher.hpp     |    2 +
 query_optimizer/physical/PhysicalType.hpp       |    1 +
 query_optimizer/physical/TransitiveClosure.cpp  |   81 +
 query_optimizer/physical/TransitiveClosure.hpp  |  141 +
 query_optimizer/resolver/CMakeLists.txt         |    2 +
 query_optimizer/resolver/Resolver.cpp           |   45 +
 query_optimizer/resolver/Resolver.hpp           |    6 +-
 query_optimizer/strategy/CMakeLists.txt         |    5 +
 query_optimizer/strategy/OneToOne.cpp           |   41 +
 .../BuildTransitiveClosureOperator.cpp          |  136 +
 .../BuildTransitiveClosureOperator.hpp          |  180 ++
 relational_operators/CMakeLists.txt             |   59 +
 .../InitializeTransitiveClosureOperator.cpp     |   87 +
 .../InitializeTransitiveClosureOperator.hpp     |  109 +
 relational_operators/RelationalOperator.hpp     |    3 +
 .../TransitiveClosureOperator.cpp               |    0
 .../TransitiveClosureOperator.hpp               |   57 +
 storage/AggregationOperationState.hpp           |    6 +-
 storage/CMakeLists.txt                          |    5 +
 storage/TransitiveClosureState.cpp              |    0
 storage/TransitiveClosureState.hpp              |   82 +
 .../BarrieredReadWriteConcurrentBitVector.hpp   |    2 +
 utility/BlockIDStream.hpp                       |   83 +
 utility/CMakeLists.txt                          |    6 +
 utility/TransitiveClosureContext.cpp            |    0
 utility/TransitiveClosureContext.hpp            |    1 +
 55 files changed, 4003 insertions(+), 2251 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index d4aaab4..7004e88 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -114,6 +114,9 @@ add_library(quickstep_parser_ParseString ParseString.cpp ParseString.hpp)
 add_library(quickstep_parser_ParseSubqueryExpression ParseSubqueryExpression.cpp ParseSubqueryExpression.hpp)
 add_library(quickstep_parser_ParseSubqueryTableReference ParseSubqueryTableReference.cpp ParseSubqueryTableReference.hpp)
 add_library(quickstep_parser_ParseTableReference ParseTableReference.cpp ParseTableReference.hpp)
+add_library(quickstep_parser_ParseTransitiveClosureTableReference
+            ParseTransitiveClosureTableReference.cpp
+            ParseTransitiveClosureTableReference.hpp)
 add_library(quickstep_parser_ParseTreeNode ../empty_src.cpp ParseTreeNode.hpp)
 add_library(quickstep_parser_ParseWindow ../empty_src.cpp ParseWindow.hpp)
 add_library(quickstep_parser_ParserUtil ParserUtil.cpp ParserUtil.hpp)
@@ -311,6 +314,9 @@ target_link_libraries(quickstep_parser_ParseTableReference
                       quickstep_parser_ParseTreeNode
                       quickstep_utility_Macros
                       quickstep_utility_PtrList)
+target_link_libraries(quickstep_parser_ParseTransitiveClosureTableReference
+                      quickstep_parser_ParseTableReference
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_parser_ParseTreeNode
                       quickstep_utility_Macros
                       quickstep_utility_TreeStringSerializable)
@@ -361,6 +367,7 @@ target_link_libraries(quickstep_parser_SqlParser
                       quickstep_parser_ParseSubqueryExpression
                       quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseTableReference
+                      quickstep_parser_ParseTransitiveClosureTableReference
                       quickstep_parser_ParseWindow
                       quickstep_parser_ParserUtil
                       quickstep_storage_StorageBlockInfo
@@ -442,6 +449,7 @@ target_link_libraries(quickstep_parser
                       quickstep_parser_ParseSubqueryExpression
                       quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseTableReference
+                      quickstep_parser_ParseTransitiveClosureTableReference
                       quickstep_parser_ParseTreeNode
                       quickstep_parser_ParseWindow
                       quickstep_parser_SqlLexer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/ParseTableReference.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseTableReference.hpp b/parser/ParseTableReference.hpp
index 9ef71e4..4f3aeb1 100644
--- a/parser/ParseTableReference.hpp
+++ b/parser/ParseTableReference.hpp
@@ -101,7 +101,8 @@ class ParseTableReference : public ParseTreeNode {
     kGeneratorTableReference,
     kJoinedTableReference,
     kSimpleTableReference,
-    kSubqueryTableReference
+    kSubqueryTableReference,
+    kTransitiveClosureTableReference
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/ParseTransitiveClosureTableReference.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseTransitiveClosureTableReference.cpp b/parser/ParseTransitiveClosureTableReference.cpp
new file mode 100644
index 0000000..9182928
--- /dev/null
+++ b/parser/ParseTransitiveClosureTableReference.cpp
@@ -0,0 +1,48 @@
+/**
+ * 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 "parser/ParseTransitiveClosureTableReference.hpp"
+
+#include <string>
+#include <vector>
+
+namespace quickstep {
+
+void ParseTransitiveClosureTableReference::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<const ParseTreeNode*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const {
+  ParseTableReference::getFieldStringItems(inline_field_names,
+                                           inline_field_values,
+                                           non_container_child_field_names,
+                                           non_container_child_fields,
+                                           container_child_field_names,
+                                           container_child_fields);
+
+  non_container_child_field_names->push_back("start_table");
+  non_container_child_fields->push_back(start_table_.get());
+
+  non_container_child_field_names->push_back("edge_table");
+  non_container_child_fields->push_back(edge_table_.get());
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/ParseTransitiveClosureTableReference.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseTransitiveClosureTableReference.hpp b/parser/ParseTransitiveClosureTableReference.hpp
new file mode 100644
index 0000000..3655865
--- /dev/null
+++ b/parser/ParseTransitiveClosureTableReference.hpp
@@ -0,0 +1,83 @@
+/**
+ * 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_PARSER_PARSE_TRANSITIVE_CLOSURE_TABLE_REFERENCE_HPP_
+#define QUICKSTEP_PARSER_PARSE_TRANSITIVE_CLOSURE_TABLE_REFERENCE_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parser/ParseTableReference.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class ParseTreeNode;
+
+/** \addtogroup Parser
+ *  @{
+ */
+
+class ParseTransitiveClosureTableReference : public ParseTableReference {
+ public:
+  ParseTransitiveClosureTableReference(const int line_number,
+                            const int column_number,
+                            ParseTableReference *start_table,
+                            ParseTableReference *edge_table)
+    : ParseTableReference(line_number, column_number),
+      start_table_(start_table),
+      edge_table_(edge_table) {
+  }
+
+  TableReferenceType getTableReferenceType() const override {
+    return kTransitiveClosureTableReference;
+  }
+
+  std::string getName() const override { return "TransitiveClosure"; }
+
+  const ParseTableReference* start_table() const {
+    return start_table_.get();
+  }
+
+  const ParseTableReference* edge_table() const {
+    return edge_table_.get();
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<const ParseTreeNode*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
+
+ private:
+  std::unique_ptr<ParseTableReference> start_table_;
+  std::unique_ptr<ParseTableReference> edge_table_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParseTransitiveClosureTableReference);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_PARSER_PARSE_TRANSITIVE_CLOSURE_TABLE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index d818d0b..534aaf4 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -89,6 +89,7 @@ class ParseSubqueryExpression;
 class ParseSubqueryTableReference;
 class ParseTableReference;
 class ParseTableReferenceSignature;
+class ParseTransitiveClosureTableReference;
 class ParseWindow;
 class Type;
 class UnaryOperation;
@@ -183,16 +184,17 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "blockproperties"  return TOKEN_BLOCKPROPERTIES;
   "blocksample"      return TOKEN_BLOCKSAMPLE;
   "bloomfilter"      return TOKEN_BLOOM_FILTER;
-  "case"             return TOKEN_CASE;
-  "csbtree"          return TOKEN_CSB_TREE;
   "by"               return TOKEN_BY;
+  "case"             return TOKEN_CASE;
   "char"             return TOKEN_CHARACTER;
   "character"        return TOKEN_CHARACTER;
   "check"            return TOKEN_CHECK;
   "column"           return TOKEN_COLUMN;
+  "connect"          return TOKEN_CONNECT;
   "constraint"       return TOKEN_CONSTRAINT;
   "copy"             return TOKEN_COPY;
   "create"           return TOKEN_CREATE;
+  "csbtree"          return TOKEN_CSB_TREE;
   "current"          return TOKEN_CURRENT;
   "date"             return TOKEN_DATE;
   "datetime"         return TOKEN_DATETIME;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index ba69b3d..103a019 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -97,6 +97,7 @@ typedef struct YYLTYPE {
 #include "parser/ParseSubqueryExpression.hpp"
 #include "parser/ParseSubqueryTableReference.hpp"
 #include "parser/ParseTableReference.hpp"
+#include "parser/ParseTransitiveClosureTableReference.hpp"
 #include "parser/ParseWindow.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/Type.hpp"
@@ -259,15 +260,16 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_BLOCKPROPERTIES;
 %token TOKEN_BLOCKSAMPLE;
 %token TOKEN_BLOOM_FILTER;
-%token TOKEN_CSB_TREE;
 %token TOKEN_BY;
 %token TOKEN_CASE;
 %token TOKEN_CHARACTER;
 %token TOKEN_CHECK;
 %token TOKEN_COLUMN;
+%token TOKEN_CONNECT;
 %token TOKEN_CONSTRAINT;
 %token TOKEN_COPY;
 %token TOKEN_CREATE;
+%token TOKEN_CSB_TREE;
 %token TOKEN_CURRENT;
 %token TOKEN_DATE;
 %token TOKEN_DATETIME;
@@ -1361,6 +1363,10 @@ joined_table_reference:
   joined_table_reference join_type TOKEN_JOIN table_reference TOKEN_ON or_expression {
     $$ = new quickstep::ParseJoinedTableReference(@3.first_line, @3.first_column, $2, $1, $4, $6);
   }
+  | '(' table_reference TOKEN_CONNECT TOKEN_BY table_reference ')' table_reference_signature {
+    $$ = new quickstep::ParseTransitiveClosureTableReference(@3.first_line, @3.first_column, $2, $5);
+    $$->set_table_reference_signature($7);
+  }
   | table_reference {
     $$ = $1;
   };


[02/11] incubator-quickstep git commit: Updates to save blocks and analyze

Posted by ji...@apache.org.
Updates to save blocks and analyze


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

Branch: refs/heads/transitive-closure
Commit: b45cdbc63be488aad3211b5146dfe6f8fbfdc024
Parents: df20e4c
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Nov 27 16:21:03 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Mon Nov 27 16:21:03 2017 -0600

----------------------------------------------------------------------
 catalog/CatalogRelationStatistics.hpp |   9 ++
 cli/CommandExecutor.cpp               | 249 ++++++++++++++++++++++++-----
 cli/Constants.hpp                     |   2 +
 storage/StorageManager.cpp            |  12 +-
 4 files changed, 222 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b45cdbc6/catalog/CatalogRelationStatistics.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.hpp b/catalog/CatalogRelationStatistics.hpp
index df95231..55fc747 100644
--- a/catalog/CatalogRelationStatistics.hpp
+++ b/catalog/CatalogRelationStatistics.hpp
@@ -68,6 +68,15 @@ class CatalogRelationStatistics {
   serialization::CatalogRelationStatistics getProto() const;
 
   /**
+   * @brief Clear all statistics.
+   */
+  void clear() {
+    num_tuples_ = kNullValue;
+    column_stats_.clear();
+    is_exact_ = true;
+  }
+
+  /**
    * @brief Check whether the statistics are exact for the relation.
    *
    * return True if the statistics are exact for the relation, false otherwise.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b45cdbc6/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 6a84672..7976d7d 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -201,9 +201,28 @@ void ExecuteAnalyze(const PtrVector<ParseString> &arguments,
     CatalogRelationStatistics *mutable_stat =
         mutable_relation->getStatisticsMutable();
 
+    mutable_stat->clear();
+
     const std::string rel_name = EscapeQuotes(relation.getName(), '"');
 
-    // Get the number of distinct values for each column.
+    // Get the number of tuples for the relation.
+    std::string query_string = "SELECT COUNT(*) FROM \"";
+    query_string.append(rel_name);
+    query_string.append("\";");
+
+    TypedValue num_tuples =
+        ExecuteQueryForSingleResult(main_thread_client_id,
+                                    foreman_client_id,
+                                    query_string,
+                                    bus,
+                                    storage_manager,
+                                    query_processor,
+                                    parser_wrapper.get());
+
+    DCHECK_EQ(TypeID::kLong, num_tuples.getTypeID());
+    mutable_stat->setNumTuples(num_tuples.getLiteral<std::int64_t>());
+
+    // Get the min/max values for each column.
     for (const CatalogAttribute &attribute : relation) {
       const std::string attr_name = EscapeQuotes(attribute.getName(), '"');
       const Type &attr_type = attribute.getType();
@@ -211,24 +230,15 @@ void ExecuteAnalyze(const PtrVector<ParseString> &arguments,
           AggregateFunctionMin::Instance().canApplyToTypes({&attr_type});
       bool is_max_applicable =
           AggregateFunctionMax::Instance().canApplyToTypes({&attr_type});
+      if (!is_min_applicable || !is_max_applicable) {
+        continue;
+      }
 
-      // NOTE(jianqiao): Note that the relation name and the attribute names may
-      // contain non-letter characters, e.g. CREATE TABLE "with space"("1" int).
-      // So here we need to format the names with double quotes (").
-      std::string query_string = "SELECT COUNT(DISTINCT \"";
+      std::string query_string = "SELECT MIN(\"";
       query_string.append(attr_name);
-      query_string.append("\")");
-      if (is_min_applicable) {
-        query_string.append(", MIN(\"");
-        query_string.append(attr_name);
-        query_string.append("\")");
-      }
-      if (is_max_applicable) {
-        query_string.append(", MAX(\"");
-        query_string.append(attr_name);
-        query_string.append("\")");
-      }
-      query_string.append(" FROM \"");
+      query_string.append("\"), MAX(\"");
+      query_string.append(attr_name);
+      query_string.append("\") FROM \"");
       query_string.append(rel_name);
       query_string.append("\";");
 
@@ -240,42 +250,181 @@ void ExecuteAnalyze(const PtrVector<ParseString> &arguments,
                                    storage_manager,
                                    query_processor,
                                    parser_wrapper.get());
-
-      auto results_it = results.begin();
-      DCHECK_EQ(TypeID::kLong, results_it->getTypeID());
+      DCHECK_EQ(2u, results.size());
 
       const attribute_id attr_id = attribute.getID();
-      mutable_stat->setNumDistinctValues(attr_id,
-                                         results_it->getLiteral<std::int64_t>());
-      if (is_min_applicable) {
-        ++results_it;
-        mutable_stat->setMinValue(attr_id, *results_it);
+      mutable_stat->setMinValue(attr_id, results[0]);
+      mutable_stat->setMaxValue(attr_id, results[1]);
+    }
+
+    // Get the number of distinct values for each column.
+    for (const CatalogAttribute &attribute : relation) {
+      const std::string attr_name = EscapeQuotes(attribute.getName(), '"');
+
+      std::string query_string = "SELECT COUNT(*) FROM (SELECT \"";
+      query_string.append(attr_name);
+      query_string.append("\" FROM \"");
+      query_string.append(rel_name);
+      query_string.append("\" GROUP BY \"");
+      query_string.append(attr_name);
+      query_string.append("\") t;");
+
+      TypedValue num_distinct_values =
+          ExecuteQueryForSingleResult(main_thread_client_id,
+                                      foreman_client_id,
+                                      query_string,
+                                      bus,
+                                      storage_manager,
+                                      query_processor,
+                                      parser_wrapper.get());
+
+      DCHECK_EQ(TypeID::kLong, num_distinct_values.getTypeID());
+      mutable_stat->setNumDistinctValues(
+          attribute.getID(), num_distinct_values.getLiteral<std::int64_t>());
+    }
+
+    fprintf(out, "done\n");
+    fflush(out);
+  }
+  query_processor->markCatalogAltered();
+  query_processor->saveCatalog();
+}
+
+void ExecuteAnalyzeRange(const PtrVector<ParseString> &arguments,
+                         const tmb::client_id main_thread_client_id,
+                         const tmb::client_id foreman_client_id,
+                         MessageBus *bus,
+                         StorageManager *storage_manager,
+                         QueryProcessor *query_processor,
+                         FILE *out) {
+  const CatalogDatabase &database = *query_processor->getDefaultDatabase();
+
+  std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
+  std::vector<std::reference_wrapper<const CatalogRelation>> relations;
+  if (arguments.empty()) {
+    relations.insert(relations.begin(), database.begin(), database.end());
+  } else {
+    for (const auto &rel_name : arguments) {
+      const CatalogRelation *rel = database.getRelationByName(rel_name.value());
+      if (rel == nullptr) {
+        THROW_SQL_ERROR_AT(&rel_name) << "Table does not exist";
+      } else {
+        relations.emplace_back(*rel);
+      }
+    }
+  }
+
+  // Analyze each relation in the database.
+  for (const CatalogRelation &relation : relations) {
+    fprintf(out, "Analyzing %s ... ", relation.getName().c_str());
+    fflush(out);
+
+    CatalogRelation *mutable_relation =
+        query_processor->getDefaultDatabase()->getRelationByIdMutable(relation.getID());
+    CatalogRelationStatistics *mutable_stat =
+        mutable_relation->getStatisticsMutable();
+
+    if (!mutable_stat->isExact()) {
+      mutable_stat->clear();
+
+      const std::string rel_name = EscapeQuotes(relation.getName(), '"');
+
+      for (const CatalogAttribute &attribute : relation) {
+        const std::string attr_name = EscapeQuotes(attribute.getName(), '"');
+        const Type &attr_type = attribute.getType();
+        bool is_min_applicable =
+            AggregateFunctionMin::Instance().canApplyToTypes({&attr_type});
+        bool is_max_applicable =
+            AggregateFunctionMax::Instance().canApplyToTypes({&attr_type});
+        if (!is_min_applicable || !is_max_applicable) {
+          continue;
+        }
+
+        std::string query_string = "SELECT MIN(\"";
+        query_string.append(attr_name);
+        query_string.append("\"), MAX(\"");
+        query_string.append(attr_name);
+        query_string.append("\") FROM \"");
+        query_string.append(rel_name);
+        query_string.append("\";");
+
+        std::vector<TypedValue> results =
+            ExecuteQueryForSingleRow(main_thread_client_id,
+                                     foreman_client_id,
+                                     query_string,
+                                     bus,
+                                     storage_manager,
+                                     query_processor,
+                                     parser_wrapper.get());
+        DCHECK_EQ(2u, results.size());
+
+        const attribute_id attr_id = attribute.getID();
+        mutable_stat->setMinValue(attr_id, results[0]);
+        mutable_stat->setMaxValue(attr_id, results[1]);
       }
-      if (is_max_applicable) {
-        ++results_it;
-        mutable_stat->setMaxValue(attr_id, *results_it);
+    }
+    fprintf(out, "done\n");
+    fflush(out);
+  }
+  query_processor->markCatalogAltered();
+  query_processor->saveCatalog();
+}
+
+void ExecuteAnalyzeCount(const PtrVector<ParseString> &arguments,
+                         const tmb::client_id main_thread_client_id,
+                         const tmb::client_id foreman_client_id,
+                         MessageBus *bus,
+                         StorageManager *storage_manager,
+                         QueryProcessor *query_processor,
+                         FILE *out) {
+  const CatalogDatabase &database = *query_processor->getDefaultDatabase();
+
+  std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
+  std::vector<std::reference_wrapper<const CatalogRelation>> relations;
+  if (arguments.empty()) {
+    relations.insert(relations.begin(), database.begin(), database.end());
+  } else {
+    for (const auto &rel_name : arguments) {
+      const CatalogRelation *rel = database.getRelationByName(rel_name.value());
+      if (rel == nullptr) {
+        THROW_SQL_ERROR_AT(&rel_name) << "Table does not exist";
+      } else {
+        relations.emplace_back(*rel);
       }
     }
+  }
 
-    // Get the number of tuples for the relation.
-    std::string query_string = "SELECT COUNT(*) FROM \"";
-    query_string.append(rel_name);
-    query_string.append("\";");
+  // Analyze each relation in the database.
+  for (const CatalogRelation &relation : relations) {
+    fprintf(out, "Analyzing %s ... ", relation.getName().c_str());
+    fflush(out);
 
-    TypedValue num_tuples =
-        ExecuteQueryForSingleResult(main_thread_client_id,
-                                    foreman_client_id,
-                                    query_string,
-                                    bus,
-                                    storage_manager,
-                                    query_processor,
-                                    parser_wrapper.get());
+    CatalogRelation *mutable_relation =
+        query_processor->getDefaultDatabase()->getRelationByIdMutable(relation.getID());
+    CatalogRelationStatistics *mutable_stat =
+        mutable_relation->getStatisticsMutable();
 
-    DCHECK_EQ(TypeID::kLong, num_tuples.getTypeID());
-    mutable_stat->setNumTuples(num_tuples.getLiteral<std::int64_t>());
+    if (!mutable_stat->isExact()) {
+      mutable_stat->clear();
 
-    mutable_stat->setExactness(true);
+      // Get the number of tuples for the relation.
+      std::string query_string = "SELECT COUNT(*) FROM \"";
+      query_string.append(EscapeQuotes(relation.getName(), '"'));
+      query_string.append("\";");
 
+      TypedValue num_tuples =
+          ExecuteQueryForSingleResult(main_thread_client_id,
+                                      foreman_client_id,
+                                      query_string,
+                                      bus,
+                                      storage_manager,
+                                      query_processor,
+                                      parser_wrapper.get());
+
+      DCHECK_EQ(TypeID::kLong, num_tuples.getTypeID());
+      mutable_stat->setNumTuples(num_tuples.getLiteral<std::int64_t>());
+
+    }
     fprintf(out, "done\n");
     fflush(out);
   }
@@ -314,6 +463,20 @@ void executeCommand(const ParseStatement &statement,
                    bus,
                    storage_manager,
                    query_processor, out);
+  } else if (command_str == kAnalyzeRangeCommand) {
+    ExecuteAnalyzeRange(arguments,
+                        main_thread_client_id,
+                        foreman_client_id,
+                        bus,
+                        storage_manager,
+                        query_processor, out);
+  } else if (command_str == kAnalyzeCountCommand) {
+    ExecuteAnalyzeCount(arguments,
+                        main_thread_client_id,
+                        foreman_client_id,
+                        bus,
+                        storage_manager,
+                        query_processor, out);
   } else {
     THROW_SQL_ERROR_AT(command.command()) << "Invalid Command";
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b45cdbc6/cli/Constants.hpp
----------------------------------------------------------------------
diff --git a/cli/Constants.hpp b/cli/Constants.hpp
index 0b4a37b..8934b03 100644
--- a/cli/Constants.hpp
+++ b/cli/Constants.hpp
@@ -30,6 +30,8 @@ namespace cli {
 constexpr char kDescribeDatabaseCommand[] = "\\dt";
 constexpr char kDescribeTableCommand[] = "\\d";
 constexpr char kAnalyzeCommand[] = "\\analyze";
+constexpr char kAnalyzeRangeCommand[] = "\\analyzerange";
+constexpr char kAnalyzeCountCommand[] = "\\analyzecount";
 
 /** @} */
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b45cdbc6/storage/StorageManager.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.cpp b/storage/StorageManager.cpp
index 5924607..78db0cd 100644
--- a/storage/StorageManager.cpp
+++ b/storage/StorageManager.cpp
@@ -271,9 +271,7 @@ StorageManager::~StorageManager() {
        it != blocks_.end();
        ++it) {
     if (it->second.block->isDirty()) {
-      LOG(WARNING) << (it->second.block->isBlob() ? "Blob " : "Block ")
-                   << "with ID " << BlockIdUtil::ToString(it->first)
-                   << " is dirty during StorageManager shutdown";
+      saveBlockOrBlob(it->first, true);
     }
     delete it->second.block;
     deallocateSlots(it->second.block_memory, it->second.block_memory_size);
@@ -393,6 +391,10 @@ bool StorageManager::saveBlockOrBlob(const block_id block, const bool force) {
   // particular entry in 'blocks_' for the specified 'block'. If and when we
   // switch blocks_ to something with more fine-grained locking, this should
   // be revisited.
+  if (!force) {
+    return true;
+  }
+
   SpinSharedMutexSharedLock<false> read_lock(blocks_shared_mutex_);
 
   std::unordered_map<block_id, BlockHandle>::iterator block_it = blocks_.find(block);
@@ -400,10 +402,6 @@ bool StorageManager::saveBlockOrBlob(const block_id block, const bool force) {
     return false;
   }
 
-  if (!(force || block_it->second.block->isDirty())) {
-    return true;
-  }
-
   bool res = file_manager_->writeBlockOrBlob(block,
                                              block_it->second.block_memory,
                                              kSlotSizeBytes * (block_it->second.block_memory_size));


[04/11] incubator-quickstep git commit: Updates to fix the union all + reorder column problem.

Posted by ji...@apache.org.
Updates to fix the union all + reorder column problem.


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

Branch: refs/heads/transitive-closure
Commit: a08044e42418a93091d7440a04cc944d4bfede8a
Parents: a57638e
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Fri Dec 1 02:15:05 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Fri Dec 1 02:16:10 2017 -0600

----------------------------------------------------------------------
 cli/QuickstepCli.cpp                     | 20 +++++++++++++++++++-
 query_optimizer/resolver/Resolver.cpp    |  2 +-
 query_optimizer/rules/ReorderColumns.cpp |  6 +++++-
 storage/StorageManager.cpp               | 11 +++++++----
 4 files changed, 32 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a08044e4/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index e8ca56c..15d8ede 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -23,6 +23,7 @@
 #include <cstddef>
 #include <cstdio>
 #include <exception>
+#include <fstream>
 #include <memory>
 #include <string>
 #include <utility>
@@ -154,6 +155,8 @@ DEFINE_string(mode, "local",
 DECLARE_bool(profile_and_report_workorder_perf);
 DECLARE_bool(visualize_execution_dag);
 
+DEFINE_string(execution_dag_log, "", "The log file for writing execution dag to.");
+
 }  // namespace quickstep
 
 int main(int argc, char* argv[]) {
@@ -299,6 +302,13 @@ int main(int argc, char* argv[]) {
   std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
   std::chrono::time_point<std::chrono::steady_clock> start, end;
 
+  std::unique_ptr<std::ofstream> execution_dag_log_out;
+  const bool has_execution_dag_log = !quickstep::FLAGS_execution_dag_log.empty();
+  if (has_execution_dag_log) {
+    execution_dag_log_out = std::make_unique<std::ofstream>(
+        quickstep::FLAGS_execution_dag_log, std::ios::out | std::ios::app);
+  }
+
 #ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
   bool started_profiling = false;
 #endif
@@ -353,6 +363,9 @@ int main(int argc, char* argv[]) {
         }
         // Here the statement is presumed to be a query.
         const std::size_t query_id = query_processor->query_id();
+        if (has_execution_dag_log) {
+          *execution_dag_log_out << "#####QUERY ID " << query_id << "#####" << "\n";
+        }
         const CatalogRelation *query_result_relation = nullptr;
         std::unique_ptr<quickstep::ExecutionDAGVisualizer> dag_visualizer;
 
@@ -418,7 +431,12 @@ int main(int argc, char* argv[]) {
             if (profiling_stats) {
               dag_visualizer->bindProfilingStats(*profiling_stats);
             }
-            std::cerr << "\n" << dag_visualizer->toDOT() << "\n";
+            if (has_execution_dag_log) {
+              CHECK(execution_dag_log_out != nullptr);
+              *execution_dag_log_out << "\n" << dag_visualizer->toDOT() << "\n";
+            } else {
+              std::cerr << "\n" << dag_visualizer->toDOT() << "\n";
+            }
           }
         } catch (const std::exception &e) {
           fprintf(io_handle->err(), "QUERY EXECUTION ERROR: %s\n", e.what());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a08044e4/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 0b6dc22..dc4bc2e 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -1586,7 +1586,7 @@ L::LogicalPtr Resolver::resolveSetOperations(
     possible_attributes.push_back(possible_attribute);
   }
 
-  for (std::size_t opid = 0; opid < operation_attributes.size(); ++opid) {
+  for (std::size_t opid = 0; opid < attribute_matrix.size(); ++opid) {
     // Generate a cast operation if needed.
     std::vector<E::NamedExpressionPtr> cast_expressions;
     for (std::size_t aid = 0; aid < operation_attributes.size(); ++aid) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a08044e4/query_optimizer/rules/ReorderColumns.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReorderColumns.cpp b/query_optimizer/rules/ReorderColumns.cpp
index 4783a8d..d12e8c7 100644
--- a/query_optimizer/rules/ReorderColumns.cpp
+++ b/query_optimizer/rules/ReorderColumns.cpp
@@ -61,7 +61,11 @@ P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr &input,
   if (skip_transform) {
     std::vector<P::PhysicalPtr> new_children;
     for (const P::PhysicalPtr &child : input->children()) {
-      new_children.emplace_back(applyInternal(child, lock_ordering && is_not_transformable));
+      bool child_lock_ordering = lock_ordering && is_not_transformable;
+      if (child->getPhysicalType() == P::PhysicalType::kUnionAll) {
+        child_lock_ordering = true;
+      }
+      new_children.emplace_back(applyInternal(child, child_lock_ordering));
     }
 
     if (new_children != input->children()) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a08044e4/storage/StorageManager.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.cpp b/storage/StorageManager.cpp
index 78db0cd..3bd76f7 100644
--- a/storage/StorageManager.cpp
+++ b/storage/StorageManager.cpp
@@ -135,6 +135,8 @@ DEFINE_int32(block_domain, 1,
 static const volatile bool block_domain_dummy
     = gflags::RegisterFlagValidator(&FLAGS_block_domain, &ValidateBlockDomain);
 
+DEFINE_bool(force_save_dirty_blocks, false, "Force save dirty blocks.");
+
 /**
  * @brief Set or validate the buffer pool slots. When automatically picking a
  *        default value, check if the system is "small" or "large." Set the
@@ -391,10 +393,6 @@ bool StorageManager::saveBlockOrBlob(const block_id block, const bool force) {
   // particular entry in 'blocks_' for the specified 'block'. If and when we
   // switch blocks_ to something with more fine-grained locking, this should
   // be revisited.
-  if (!force) {
-    return true;
-  }
-
   SpinSharedMutexSharedLock<false> read_lock(blocks_shared_mutex_);
 
   std::unordered_map<block_id, BlockHandle>::iterator block_it = blocks_.find(block);
@@ -402,6 +400,11 @@ bool StorageManager::saveBlockOrBlob(const block_id block, const bool force) {
     return false;
   }
 
+  if (!force && !(FLAGS_force_save_dirty_blocks && block_it->second.block->isDirty())) {
+    return true;
+  }
+
+
   bool res = file_manager_->writeBlockOrBlob(block,
                                              block_it->second.block_memory,
                                              kSlotSizeBytes * (block_it->second.block_memory_size));


[05/11] incubator-quickstep git commit: Updates to cost model

Posted by ji...@apache.org.
Updates to cost model


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

Branch: refs/heads/transitive-closure
Commit: d725bcafec86b7637ce08a24c309b2ea1981a618
Parents: a08044e
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Fri Dec 1 17:20:25 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Fri Dec 1 17:20:25 2017 -0600

----------------------------------------------------------------------
 query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d725bcaf/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index f7417b6..729a563 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -486,7 +486,7 @@ std::size_t StarSchemaSimpleCostModel::getNumDistinctValues(
       return stat.getNumDistinctValues(rel_attr_id);
     }
   }
-  return estimateCardinalityForTableReference(table_reference);
+  return estimateCardinalityForTableReference(table_reference) * 0.5;
 }
 
 bool StarSchemaSimpleCostModel::impliesUniqueAttributes(
@@ -520,7 +520,7 @@ bool StarSchemaSimpleCostModel::impliesUniqueAttributes(
           std::static_pointer_cast<const P::TableReference>(physical_plan);
       const CatalogRelationStatistics &stat =
           table_reference->relation()->getStatistics();
-      if (stat.hasNumTuples()) {
+      if (stat.isExact() && stat.hasNumTuples()) {
         const std::size_t num_tuples = stat.getNumTuples();
         for (const auto &attr : attributes) {
           const attribute_id rel_attr_id =


[08/11] incubator-quickstep git commit: Initialize updates for transitive closure

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 9b77875..b1b4547 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -133,6 +133,7 @@ typedef struct YYLTYPE {
 #include "parser/ParseSubqueryExpression.hpp"
 #include "parser/ParseSubqueryTableReference.hpp"
 #include "parser/ParseTableReference.hpp"
+#include "parser/ParseTransitiveClosureTableReference.hpp"
 #include "parser/ParseWindow.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/Type.hpp"
@@ -153,7 +154,7 @@ typedef struct YYLTYPE {
 // Needed for Bison 2.6 and higher, which do not automatically provide this typedef.
 typedef void* yyscan_t;
 
-#line 157 "SqlParser_gen.cpp" /* yacc.c:339  */
+#line 158 "SqlParser_gen.cpp" /* yacc.c:339  */
 
 # ifndef YY_NULLPTR
 #  if defined __cplusplus && 201103L <= __cplusplus
@@ -221,107 +222,108 @@ extern int quickstep_yydebug;
     TOKEN_BLOCKPROPERTIES = 288,
     TOKEN_BLOCKSAMPLE = 289,
     TOKEN_BLOOM_FILTER = 290,
-    TOKEN_CSB_TREE = 291,
-    TOKEN_BY = 292,
-    TOKEN_CASE = 293,
-    TOKEN_CHARACTER = 294,
-    TOKEN_CHECK = 295,
-    TOKEN_COLUMN = 296,
+    TOKEN_BY = 291,
+    TOKEN_CASE = 292,
+    TOKEN_CHARACTER = 293,
+    TOKEN_CHECK = 294,
+    TOKEN_COLUMN = 295,
+    TOKEN_CONNECT = 296,
     TOKEN_CONSTRAINT = 297,
     TOKEN_COPY = 298,
     TOKEN_CREATE = 299,
-    TOKEN_CURRENT = 300,
-    TOKEN_DATE = 301,
-    TOKEN_DATETIME = 302,
-    TOKEN_DAY = 303,
-    TOKEN_DECIMAL = 304,
-    TOKEN_DEFAULT = 305,
-    TOKEN_DELETE = 306,
-    TOKEN_DESC = 307,
-    TOKEN_DISTINCT = 308,
-    TOKEN_DOUBLE = 309,
-    TOKEN_DROP = 310,
-    TOKEN_ELSE = 311,
-    TOKEN_END = 312,
-    TOKEN_EXISTS = 313,
-    TOKEN_EXTRACT = 314,
-    TOKEN_FALSE = 315,
-    TOKEN_FIRST = 316,
-    TOKEN_FLOAT = 317,
-    TOKEN_FOLLOWING = 318,
-    TOKEN_FOR = 319,
-    TOKEN_FOREIGN = 320,
-    TOKEN_FROM = 321,
-    TOKEN_FULL = 322,
-    TOKEN_GROUP = 323,
-    TOKEN_HASH = 324,
-    TOKEN_HAVING = 325,
-    TOKEN_HOUR = 326,
-    TOKEN_IN = 327,
-    TOKEN_INDEX = 328,
-    TOKEN_INNER = 329,
-    TOKEN_INSERT = 330,
-    TOKEN_INTEGER = 331,
-    TOKEN_INTERVAL = 332,
-    TOKEN_INTO = 333,
-    TOKEN_JOIN = 334,
-    TOKEN_KEY = 335,
-    TOKEN_LAST = 336,
-    TOKEN_LEFT = 337,
-    TOKEN_LIMIT = 338,
-    TOKEN_LONG = 339,
-    TOKEN_MINUTE = 340,
-    TOKEN_MONTH = 341,
-    TOKEN_NULL = 342,
-    TOKEN_NULLS = 343,
-    TOKEN_OFF = 344,
-    TOKEN_ON = 345,
-    TOKEN_ORDER = 346,
-    TOKEN_OUTER = 347,
-    TOKEN_OVER = 348,
-    TOKEN_PARTITION = 349,
-    TOKEN_PARTITIONS = 350,
-    TOKEN_PERCENT = 351,
-    TOKEN_PRECEDING = 352,
-    TOKEN_PRIMARY = 353,
-    TOKEN_PRIORITY = 354,
-    TOKEN_QUIT = 355,
-    TOKEN_RANGE = 356,
-    TOKEN_REAL = 357,
-    TOKEN_REFERENCES = 358,
-    TOKEN_RIGHT = 359,
-    TOKEN_ROW = 360,
-    TOKEN_ROW_DELIMITER = 361,
-    TOKEN_ROWS = 362,
-    TOKEN_SECOND = 363,
-    TOKEN_SELECT = 364,
-    TOKEN_SET = 365,
-    TOKEN_SMA = 366,
-    TOKEN_SMALLINT = 367,
-    TOKEN_STDERR = 368,
-    TOKEN_STDOUT = 369,
-    TOKEN_SUBSTRING = 370,
-    TOKEN_TABLE = 371,
-    TOKEN_THEN = 372,
-    TOKEN_TIME = 373,
-    TOKEN_TIMESTAMP = 374,
-    TOKEN_TO = 375,
-    TOKEN_TRUE = 376,
-    TOKEN_TUPLESAMPLE = 377,
-    TOKEN_UNBOUNDED = 378,
-    TOKEN_UNIQUE = 379,
-    TOKEN_UPDATE = 380,
-    TOKEN_USING = 381,
-    TOKEN_VALUES = 382,
-    TOKEN_VARCHAR = 383,
-    TOKEN_WHEN = 384,
-    TOKEN_WHERE = 385,
-    TOKEN_WINDOW = 386,
-    TOKEN_WITH = 387,
-    TOKEN_YEAR = 388,
-    TOKEN_YEARMONTH = 389,
-    TOKEN_EOF = 390,
-    TOKEN_LEX_ERROR = 391
+    TOKEN_CSB_TREE = 300,
+    TOKEN_CURRENT = 301,
+    TOKEN_DATE = 302,
+    TOKEN_DATETIME = 303,
+    TOKEN_DAY = 304,
+    TOKEN_DECIMAL = 305,
+    TOKEN_DEFAULT = 306,
+    TOKEN_DELETE = 307,
+    TOKEN_DESC = 308,
+    TOKEN_DISTINCT = 309,
+    TOKEN_DOUBLE = 310,
+    TOKEN_DROP = 311,
+    TOKEN_ELSE = 312,
+    TOKEN_END = 313,
+    TOKEN_EXISTS = 314,
+    TOKEN_EXTRACT = 315,
+    TOKEN_FALSE = 316,
+    TOKEN_FIRST = 317,
+    TOKEN_FLOAT = 318,
+    TOKEN_FOLLOWING = 319,
+    TOKEN_FOR = 320,
+    TOKEN_FOREIGN = 321,
+    TOKEN_FROM = 322,
+    TOKEN_FULL = 323,
+    TOKEN_GROUP = 324,
+    TOKEN_HASH = 325,
+    TOKEN_HAVING = 326,
+    TOKEN_HOUR = 327,
+    TOKEN_IN = 328,
+    TOKEN_INDEX = 329,
+    TOKEN_INNER = 330,
+    TOKEN_INSERT = 331,
+    TOKEN_INTEGER = 332,
+    TOKEN_INTERVAL = 333,
+    TOKEN_INTO = 334,
+    TOKEN_JOIN = 335,
+    TOKEN_KEY = 336,
+    TOKEN_LAST = 337,
+    TOKEN_LEFT = 338,
+    TOKEN_LIMIT = 339,
+    TOKEN_LONG = 340,
+    TOKEN_MINUTE = 341,
+    TOKEN_MONTH = 342,
+    TOKEN_NULL = 343,
+    TOKEN_NULLS = 344,
+    TOKEN_OFF = 345,
+    TOKEN_ON = 346,
+    TOKEN_ORDER = 347,
+    TOKEN_OUTER = 348,
+    TOKEN_OVER = 349,
+    TOKEN_PARTITION = 350,
+    TOKEN_PARTITIONS = 351,
+    TOKEN_PERCENT = 352,
+    TOKEN_PRECEDING = 353,
+    TOKEN_PRIMARY = 354,
+    TOKEN_PRIORITY = 355,
+    TOKEN_QUIT = 356,
+    TOKEN_RANGE = 357,
+    TOKEN_REAL = 358,
+    TOKEN_REFERENCES = 359,
+    TOKEN_RIGHT = 360,
+    TOKEN_ROW = 361,
+    TOKEN_ROW_DELIMITER = 362,
+    TOKEN_ROWS = 363,
+    TOKEN_SECOND = 364,
+    TOKEN_SELECT = 365,
+    TOKEN_SET = 366,
+    TOKEN_SMA = 367,
+    TOKEN_SMALLINT = 368,
+    TOKEN_STDERR = 369,
+    TOKEN_STDOUT = 370,
+    TOKEN_SUBSTRING = 371,
+    TOKEN_TABLE = 372,
+    TOKEN_THEN = 373,
+    TOKEN_TIME = 374,
+    TOKEN_TIMESTAMP = 375,
+    TOKEN_TO = 376,
+    TOKEN_TRUE = 377,
+    TOKEN_TUPLESAMPLE = 378,
+    TOKEN_UNBOUNDED = 379,
+    TOKEN_UNIQUE = 380,
+    TOKEN_UPDATE = 381,
+    TOKEN_USING = 382,
+    TOKEN_VALUES = 383,
+    TOKEN_VARCHAR = 384,
+    TOKEN_WHEN = 385,
+    TOKEN_WHERE = 386,
+    TOKEN_WINDOW = 387,
+    TOKEN_WITH = 388,
+    TOKEN_YEAR = 389,
+    TOKEN_YEARMONTH = 390,
+    TOKEN_EOF = 391,
+    TOKEN_LEX_ERROR = 392
   };
 #endif
 
@@ -330,7 +332,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 121 "../SqlParser.ypp" /* yacc.c:355  */
+#line 122 "../SqlParser.ypp" /* yacc.c:355  */
 
   quickstep::ParseString *string_value_;
 
@@ -432,7 +434,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 436 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 438 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -461,13 +463,13 @@ int quickstep_yyparse (yyscan_t yyscanner, quickstep::ParseStatement **parsedSta
 #endif /* !YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED  */
 
 /* Copy the second part of user declarations.  */
-#line 223 "../SqlParser.ypp" /* yacc.c:358  */
+#line 224 "../SqlParser.ypp" /* yacc.c:358  */
 
 /* This header needs YYSTYPE, which is defined by the %union directive above */
 #include "SqlLexer_gen.hpp"
 void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string &feature);
 
-#line 471 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 473 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -711,21 +713,21 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  50
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1327
+#define YYLAST   1296
 
 /* YYNTOKENS -- Number of terminals.  */
-#define YYNTOKENS  148
+#define YYNTOKENS  149
 /* YYNNTS -- Number of nonterminals.  */
 #define YYNNTS  111
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  300
+#define YYNRULES  301
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  555
+#define YYNSTATES  562
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
 #define YYUNDEFTOK  2
-#define YYMAXUTOK   391
+#define YYMAXUTOK   392
 
 #define YYTRANSLATE(YYX)                                                \
   ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK)
@@ -735,11 +737,11 @@ union yyalloc
 static const yytype_uint8 yytranslate[] =
 {
        0,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-     143,     2,     2,     2,     2,     2,     2,     2,     2,     2,
+     144,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,   147,     2,     2,
-     144,   145,    23,    21,   146,    22,    27,    24,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,     2,     2,   142,
+       2,     2,     2,     2,     2,     2,     2,   148,     2,     2,
+     145,   146,    23,    21,   147,    22,    27,    24,     2,     2,
+       2,     2,     2,     2,     2,     2,     2,     2,     2,   143,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
@@ -773,44 +775,44 @@ static const yytype_uint8 yytranslate[] =
      110,   111,   112,   113,   114,   115,   116,   117,   118,   119,
      120,   121,   122,   123,   124,   125,   126,   127,   128,   129,
      130,   131,   132,   133,   134,   135,   136,   137,   138,   139,
-     140,   141
+     140,   141,   142
 };
 
 #if YYDEBUG
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   644,   644,   648,   652,   656,   660,   663,   670,   673,
-     676,   679,   682,   685,   688,   691,   694,   697,   703,   709,
-     716,   722,   729,   738,   743,   752,   757,   762,   766,   772,
-     777,   780,   783,   788,   791,   794,   797,   800,   803,   806,
-     809,   812,   815,   827,   830,   833,   851,   871,   874,   877,
-     882,   887,   893,   899,   908,   912,   918,   921,   926,   931,
-     936,   943,   950,   954,   960,   963,   968,   971,   976,   979,
-     984,   987,  1006,  1009,  1014,  1018,  1024,  1027,  1030,  1033,
-    1038,  1041,  1044,  1051,  1056,  1067,  1072,  1077,  1081,  1085,
-    1091,  1094,  1100,  1108,  1111,  1114,  1120,  1125,  1130,  1134,
-    1140,  1144,  1147,  1152,  1155,  1160,  1165,  1170,  1174,  1180,
-    1189,  1192,  1197,  1200,  1219,  1224,  1228,  1234,  1240,  1249,
-    1254,  1262,  1268,  1274,  1277,  1280,  1285,  1288,  1293,  1297,
-    1303,  1306,  1309,  1314,  1319,  1324,  1327,  1330,  1335,  1338,
-    1341,  1344,  1347,  1350,  1353,  1356,  1361,  1364,  1369,  1373,
-    1377,  1380,  1384,  1387,  1392,  1395,  1400,  1403,  1408,  1412,
-    1418,  1421,  1426,  1429,  1434,  1437,  1442,  1445,  1464,  1467,
-    1472,  1476,  1482,  1488,  1493,  1496,  1501,  1504,  1509,  1512,
-    1517,  1520,  1525,  1526,  1529,  1534,  1535,  1538,  1543,  1547,
-    1553,  1560,  1563,  1566,  1571,  1574,  1577,  1583,  1586,  1591,
-    1596,  1605,  1610,  1619,  1624,  1627,  1632,  1635,  1640,  1646,
-    1652,  1655,  1658,  1661,  1664,  1667,  1673,  1682,  1685,  1690,
-    1693,  1698,  1701,  1706,  1709,  1712,  1715,  1719,  1723,  1726,
-    1729,  1732,  1735,  1740,  1744,  1748,  1751,  1756,  1761,  1765,
-    1771,  1774,  1779,  1783,  1789,  1794,  1798,  1804,  1809,  1812,
-    1817,  1821,  1827,  1830,  1833,  1836,  1848,  1852,  1871,  1884,
-    1899,  1902,  1905,  1908,  1911,  1914,  1919,  1923,  1929,  1933,
-    1939,  1942,  1947,  1951,  1958,  1961,  1964,  1967,  1970,  1973,
-    1976,  1979,  1982,  1985,  1990,  2001,  2004,  2009,  2012,  2015,
-    2021,  2025,  2031,  2034,  2042,  2045,  2048,  2051,  2057,  2062,
-    2067
+       0,   646,   646,   650,   654,   658,   662,   665,   672,   675,
+     678,   681,   684,   687,   690,   693,   696,   699,   705,   711,
+     718,   724,   731,   740,   745,   754,   759,   764,   768,   774,
+     779,   782,   785,   790,   793,   796,   799,   802,   805,   808,
+     811,   814,   817,   829,   832,   835,   853,   873,   876,   879,
+     884,   889,   895,   901,   910,   914,   920,   923,   928,   933,
+     938,   945,   952,   956,   962,   965,   970,   973,   978,   981,
+     986,   989,  1008,  1011,  1016,  1020,  1026,  1029,  1032,  1035,
+    1040,  1043,  1046,  1053,  1058,  1069,  1074,  1079,  1083,  1087,
+    1093,  1096,  1102,  1110,  1113,  1116,  1122,  1127,  1132,  1136,
+    1142,  1146,  1149,  1154,  1157,  1162,  1167,  1172,  1176,  1182,
+    1191,  1194,  1199,  1202,  1221,  1226,  1230,  1236,  1242,  1251,
+    1256,  1264,  1270,  1276,  1279,  1282,  1287,  1290,  1295,  1299,
+    1305,  1308,  1311,  1316,  1321,  1326,  1329,  1332,  1337,  1340,
+    1343,  1346,  1349,  1352,  1355,  1358,  1363,  1366,  1370,  1375,
+    1379,  1383,  1386,  1390,  1393,  1398,  1401,  1406,  1409,  1414,
+    1418,  1424,  1427,  1432,  1435,  1440,  1443,  1448,  1451,  1470,
+    1473,  1478,  1482,  1488,  1494,  1499,  1502,  1507,  1510,  1515,
+    1518,  1523,  1526,  1531,  1532,  1535,  1540,  1541,  1544,  1549,
+    1553,  1559,  1566,  1569,  1572,  1577,  1580,  1583,  1589,  1592,
+    1597,  1602,  1611,  1616,  1625,  1630,  1633,  1638,  1641,  1646,
+    1652,  1658,  1661,  1664,  1667,  1670,  1673,  1679,  1688,  1691,
+    1696,  1699,  1704,  1707,  1712,  1715,  1718,  1721,  1725,  1729,
+    1732,  1735,  1738,  1741,  1746,  1750,  1754,  1757,  1762,  1767,
+    1771,  1777,  1780,  1785,  1789,  1795,  1800,  1804,  1810,  1815,
+    1818,  1823,  1827,  1833,  1836,  1839,  1842,  1854,  1858,  1877,
+    1890,  1905,  1908,  1911,  1914,  1917,  1920,  1925,  1929,  1935,
+    1939,  1945,  1948,  1953,  1957,  1964,  1967,  1970,  1973,  1976,
+    1979,  1982,  1985,  1988,  1991,  1996,  2007,  2010,  2015,  2018,
+    2021,  2027,  2031,  2037,  2040,  2048,  2051,  2054,  2057,  2063,
+    2068,  2073
 };
 #endif
 
@@ -828,20 +830,21 @@ static const char *const yytname[] =
   "TOKEN_ALL", "TOKEN_UNION", "TOKEN_INTERSECT", "TOKEN_ADD",
   "TOKEN_ALTER", "TOKEN_AS", "TOKEN_ASC", "TOKEN_BIGINT", "TOKEN_BIT",
   "TOKEN_BITWEAVING", "TOKEN_BLOCKPROPERTIES", "TOKEN_BLOCKSAMPLE",
-  "TOKEN_BLOOM_FILTER", "TOKEN_CSB_TREE", "TOKEN_BY", "TOKEN_CASE",
-  "TOKEN_CHARACTER", "TOKEN_CHECK", "TOKEN_COLUMN", "TOKEN_CONSTRAINT",
-  "TOKEN_COPY", "TOKEN_CREATE", "TOKEN_CURRENT", "TOKEN_DATE",
-  "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL", "TOKEN_DEFAULT",
-  "TOKEN_DELETE", "TOKEN_DESC", "TOKEN_DISTINCT", "TOKEN_DOUBLE",
-  "TOKEN_DROP", "TOKEN_ELSE", "TOKEN_END", "TOKEN_EXISTS", "TOKEN_EXTRACT",
-  "TOKEN_FALSE", "TOKEN_FIRST", "TOKEN_FLOAT", "TOKEN_FOLLOWING",
-  "TOKEN_FOR", "TOKEN_FOREIGN", "TOKEN_FROM", "TOKEN_FULL", "TOKEN_GROUP",
-  "TOKEN_HASH", "TOKEN_HAVING", "TOKEN_HOUR", "TOKEN_IN", "TOKEN_INDEX",
-  "TOKEN_INNER", "TOKEN_INSERT", "TOKEN_INTEGER", "TOKEN_INTERVAL",
-  "TOKEN_INTO", "TOKEN_JOIN", "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT",
-  "TOKEN_LIMIT", "TOKEN_LONG", "TOKEN_MINUTE", "TOKEN_MONTH", "TOKEN_NULL",
-  "TOKEN_NULLS", "TOKEN_OFF", "TOKEN_ON", "TOKEN_ORDER", "TOKEN_OUTER",
-  "TOKEN_OVER", "TOKEN_PARTITION", "TOKEN_PARTITIONS", "TOKEN_PERCENT",
+  "TOKEN_BLOOM_FILTER", "TOKEN_BY", "TOKEN_CASE", "TOKEN_CHARACTER",
+  "TOKEN_CHECK", "TOKEN_COLUMN", "TOKEN_CONNECT", "TOKEN_CONSTRAINT",
+  "TOKEN_COPY", "TOKEN_CREATE", "TOKEN_CSB_TREE", "TOKEN_CURRENT",
+  "TOKEN_DATE", "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL",
+  "TOKEN_DEFAULT", "TOKEN_DELETE", "TOKEN_DESC", "TOKEN_DISTINCT",
+  "TOKEN_DOUBLE", "TOKEN_DROP", "TOKEN_ELSE", "TOKEN_END", "TOKEN_EXISTS",
+  "TOKEN_EXTRACT", "TOKEN_FALSE", "TOKEN_FIRST", "TOKEN_FLOAT",
+  "TOKEN_FOLLOWING", "TOKEN_FOR", "TOKEN_FOREIGN", "TOKEN_FROM",
+  "TOKEN_FULL", "TOKEN_GROUP", "TOKEN_HASH", "TOKEN_HAVING", "TOKEN_HOUR",
+  "TOKEN_IN", "TOKEN_INDEX", "TOKEN_INNER", "TOKEN_INSERT",
+  "TOKEN_INTEGER", "TOKEN_INTERVAL", "TOKEN_INTO", "TOKEN_JOIN",
+  "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT", "TOKEN_LIMIT", "TOKEN_LONG",
+  "TOKEN_MINUTE", "TOKEN_MONTH", "TOKEN_NULL", "TOKEN_NULLS", "TOKEN_OFF",
+  "TOKEN_ON", "TOKEN_ORDER", "TOKEN_OUTER", "TOKEN_OVER",
+  "TOKEN_PARTITION", "TOKEN_PARTITIONS", "TOKEN_PERCENT",
   "TOKEN_PRECEDING", "TOKEN_PRIMARY", "TOKEN_PRIORITY", "TOKEN_QUIT",
   "TOKEN_RANGE", "TOKEN_REAL", "TOKEN_REFERENCES", "TOKEN_RIGHT",
   "TOKEN_ROW", "TOKEN_ROW_DELIMITER", "TOKEN_ROWS", "TOKEN_SECOND",
@@ -912,14 +915,14 @@ static const yytype_uint16 yytoknum[] =
      360,   361,   362,   363,   364,   365,   366,   367,   368,   369,
      370,   371,   372,   373,   374,   375,   376,   377,   378,   379,
      380,   381,   382,   383,   384,   385,   386,   387,   388,   389,
-     390,   391,    59,    10,    40,    41,    44,    37
+     390,   391,   392,    59,    10,    40,    41,    44,    37
 };
 # endif
 
-#define YYPACT_NINF -410
+#define YYPACT_NINF -302
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-410)))
+  (!!((Yystate) == (-302)))
 
 #define YYTABLE_NINF -139
 
@@ -930,62 +933,63 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-     405,  -410,  -410,   -52,    43,   -13,    34,   -38,   116,  -410,
-      48,   231,   231,  -410,   115,   119,  -410,  -410,  -410,  -410,
-    -410,  -410,  -410,  -410,  -410,  -410,   166,    46,    49,  -410,
-     -36,   205,   231,  -410,  -410,    38,   132,   231,   231,   231,
-     231,   231,  -410,  -410,   652,    97,    72,  -410,   203,    95,
-    -410,  -410,  -410,   130,   175,    46,    48,   158,  -410,   130,
-    -410,  -410,  -410,    25,    61,   128,   270,   128,   184,   142,
-     149,  -410,   106,  -410,  -410,   309,   313,  -410,  -410,  -410,
-     743,   153,  -410,   218,  -410,  -410,   189,  -410,  -410,   331,
-    -410,  -410,  -410,  -410,   199,  -410,  -410,   207,   272,   837,
-     347,   305,   211,  -410,  -410,   328,    19,  -410,  -410,   262,
-    -410,  -410,  -410,  -410,  -410,  1019,     5,   231,   231,   220,
-     231,    38,   231,  -410,   130,   373,  -410,   148,   243,  -410,
-    -410,  -410,   244,  -410,   128,  -410,   231,   231,   561,  -410,
-    -410,   245,   231,  -410,  -410,  -410,   561,    52,    17,  -410,
-     400,  -410,   140,   140,  1110,   402,  -410,     1,    29,  -410,
-      20,   149,  1110,  -410,  -410,   231,  1110,  -410,  -410,  -410,
-    -410,  1110,    21,   313,  -410,   231,   323,   -64,  -410,   416,
-    -410,   130,  -410,   167,  -410,   128,   130,    49,  -410,   231,
-     152,   231,   231,   231,  -410,   269,  -410,   169,  1177,   928,
-     220,   470,   426,   433,  -410,  -410,   407,   428,  1188,   303,
-     173,    23,  1110,    64,  -410,  1110,  -410,   388,   306,  -410,
-    -410,  -410,  -410,  -410,  -410,   381,  -410,    66,   310,  -410,
-    -410,     8,   209,   215,  -410,   311,   209,     9,   385,  -410,
-    -410,    19,  -410,   360,  -410,  -410,   315,  1110,  -410,   301,
-     180,   231,  -410,  1110,  -410,   231,  -410,  -410,  -410,   320,
-     384,   386,   329,  -410,  -410,  -410,   196,  -410,  -410,  -410,
-    -410,  -410,    16,   231,   341,   152,   231,  -410,   174,  -410,
-    -410,     2,    81,   561,   561,   206,  -410,  -410,  -410,  -410,
-    -410,  -410,  -410,  -410,  1110,   334,  1110,    58,  -410,   210,
-     335,   349,  1110,    68,  -410,   420,   301,  -410,  -410,  1110,
-     476,  -410,   162,   231,  -410,  -410,   389,  -410,   390,   391,
-     406,    20,  -410,   486,   487,   209,   453,   421,   455,   353,
-     403,  -410,   223,  -410,  1110,  -410,   301,  -410,   561,   357,
-     358,   231,  -410,   231,  -410,  -410,  -410,  -410,  -410,  -410,
-    -410,   231,  -410,  -410,  -410,   225,   477,    86,  -410,   361,
-     366,  -410,   417,   364,  1188,  -410,   430,   231,  -410,  -410,
-     174,  -410,  -410,   433,  -410,  -410,  -410,  1110,   367,   284,
-     837,  -410,   301,   424,  -410,  -410,  1188,  1188,   374,   301,
-    1110,  -410,    33,   -16,  -410,  -410,  -410,  -410,  -410,    20,
-     215,   419,   422,  -410,  1110,   561,   429,  1110,  -410,   475,
-      18,  -410,   301,    31,   231,   231,   227,  -410,   233,  -410,
-     231,  -410,  -410,  -410,  -410,   382,   152,   489,   432,  -410,
-     561,  -410,  -410,   392,  -410,   289,   837,  -410,  1110,   238,
-    -410,  -410,   240,  1188,   301,  -410,   521,  -410,   437,  -410,
-    -410,   393,   426,   498,   456,   393,  1110,  -410,  -410,  -410,
-     522,  -410,   246,   252,  -410,  -410,  -410,   231,  -410,  -410,
-     399,   504,  -410,    32,   231,  1110,   254,   301,  -410,  -410,
-     257,   404,   561,  1110,   540,   412,   408,  -410,   274,    11,
-     442,  -410,   286,   231,   127,  -410,   410,   301,  -410,  -410,
-    -410,   426,   408,  -410,   231,  -410,   412,  -410,  1110,  -410,
-    -410,   460,   454,   447,   458,   549,   231,  -410,   290,  -410,
-    -410,   425,  -410,   528,  -410,  -410,    -6,  -410,  -410,  -410,
-    -410,    12,   431,  -410,   231,   434,  -410,  -410,   495,   462,
-     496,  -410,   231,   293,   360,  -410,  -410,  -410,   299,   473,
-     435,  -410,   563,  -410,  -410
+     276,  -302,  -302,   -27,    51,    13,    30,   -17,    56,  -302,
+      65,   213,   213,  -302,   157,   168,  -302,  -302,  -302,  -302,
+    -302,  -302,  -302,  -302,  -302,  -302,   186,     3,   101,  -302,
+     118,   215,   213,  -302,  -302,    17,   134,   213,   213,   213,
+     213,   213,  -302,  -302,   686,   117,   107,  -302,   238,   103,
+    -302,  -302,  -302,   159,   207,     3,    65,   190,  -302,   159,
+    -302,  -302,  -302,    60,    78,   165,   312,   165,   234,   175,
+     211,  -302,    98,  -302,  -302,   319,   328,  -302,  -302,  -302,
+     753,   196,  -302,   268,  -302,  -302,   209,  -302,  -302,   357,
+    -302,  -302,  -302,  -302,   221,  -302,  -302,   226,   291,   847,
+     376,   315,   249,  -302,  -302,   310,    21,  -302,  -302,   299,
+    -302,  -302,  -302,  -302,  -302,  1008,    29,   213,   213,   261,
+     213,    17,   213,  -302,   159,   402,  -302,   274,   289,  -302,
+    -302,  -302,   266,  -302,   165,  -302,   213,   213,   592,  -302,
+    -302,   282,   213,  -302,  -302,  -302,   592,    32,    28,  -302,
+     415,  -302,    12,    12,  1075,   426,  -302,     2,     8,  -302,
+      34,   211,  1075,  -302,  -302,   213,  1075,  -302,  -302,  -302,
+    -302,  1075,    45,   328,  -302,   213,   430,   106,  -302,   428,
+    -302,   159,  -302,   138,  -302,   165,   159,   101,  -302,   213,
+     131,   213,   213,   213,  -302,   298,  -302,   166,   320,   914,
+     261,   525,   440,   441,  -302,  -302,  1218,   435,  1134,   309,
+     199,    76,  1075,    47,  -302,  1075,  -302,   394,   313,  -302,
+    -302,  -302,  -302,  -302,  -302,   386,  -302,   218,   316,  -302,
+    -302,    19,    92,   156,  -302,   314,    92,    81,   389,  -302,
+    -302,    21,  -302,   364,  -302,  -302,   322,  1075,  -302,   337,
+     214,   213,  -302,  1075,  -302,   213,  -302,  -302,  -302,   324,
+     384,   385,   331,  -302,  -302,  -302,   222,  -302,  -302,  -302,
+    -302,  -302,    15,   213,   345,   131,   213,  -302,   193,  -302,
+    -302,     1,    58,   592,   592,   205,  -302,  -302,  -302,  -302,
+    -302,  -302,  -302,  -302,  1075,   333,  1075,    87,  -302,   230,
+     334,   347,  1075,    42,  -302,   418,   337,  -302,  -302,  1075,
+     477,  -302,   219,   442,   213,  -302,  -302,   388,  -302,   393,
+     395,   407,    34,  -302,   480,   489,    92,   456,   423,   459,
+     355,   406,  -302,   232,  -302,  1075,  -302,   337,  -302,   592,
+     359,   360,   213,  -302,   213,  -302,  -302,  -302,  -302,  -302,
+    -302,  -302,   213,  -302,  -302,  -302,   237,   481,   170,  -302,
+     362,   369,  -302,   417,   366,  1134,  -302,   429,   213,  -302,
+    -302,   193,  -302,  -302,   441,  -302,  -302,  -302,  1075,   371,
+     272,   847,  -302,   337,   421,  -302,  -302,  1134,  1134,   372,
+     337,  1075,  -302,    26,    49,  -302,   478,  -302,  -302,  -302,
+    -302,    46,   156,   419,   420,  -302,  1075,   592,   427,  1075,
+    -302,   484,   -25,  -302,   337,     6,   213,   213,   239,  -302,
+     243,  -302,   213,  -302,  -302,  -302,  -302,   381,   131,   490,
+     433,  -302,   592,  -302,  -302,   382,  -302,   306,   847,  -302,
+    1075,   246,  -302,  -302,   248,  1134,   337,  -302,   527,  -302,
+      46,    19,   443,  -302,  -302,   396,   440,   495,   451,   396,
+    1075,  -302,  -302,  -302,   522,  -302,   258,   269,  -302,  -302,
+    -302,   213,  -302,  -302,   397,   503,  -302,     9,   213,  1075,
+     273,   337,  -302,  -302,   277,   403,   404,   592,  1075,   545,
+     416,   409,  -302,   224,    55,   448,  -302,   279,   213,    10,
+    -302,   412,   337,  -302,  -302,  -302,    92,   440,   409,  -302,
+     213,  -302,   416,  -302,  1075,  -302,  -302,   465,   460,   454,
+     463,   553,   213,  -302,   283,  -302,  -302,   424,  -302,  -302,
+     538,  -302,  -302,    40,  -302,  -302,  -302,  -302,    71,   434,
+    -302,   213,   436,  -302,  -302,   504,   461,   505,  -302,   213,
+     285,   364,  -302,  -302,  -302,   295,   482,   438,  -302,   575,
+    -302,  -302
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -993,95 +997,96 @@ static const yytype_int16 yypact[] =
      means the default is an error.  */
 static const yytype_uint16 yydefact[] =
 {
-       0,     6,   300,     0,     0,     0,     0,     0,     0,    18,
+       0,     6,   301,     0,     0,     0,     0,     0,     0,    18,
      123,     0,     0,     7,     0,     0,    15,     8,    10,    11,
       13,    14,     9,    17,    12,    16,     0,   112,   119,   121,
-       0,   298,     0,   292,   293,     0,     0,     0,     0,     0,
-       0,     0,   124,   125,     0,     0,   114,   115,     0,   156,
+       0,   299,     0,   293,   294,     0,     0,     0,     0,     0,
+       0,     0,   124,   125,     0,     0,   114,   115,     0,   157,
        1,     3,     2,     0,     0,   112,   123,     0,   110,     0,
-       5,     4,   299,     0,     0,   103,     0,   103,     0,     0,
-     197,    25,     0,   256,   253,     0,   284,   126,    40,    29,
+       5,     4,   300,     0,     0,   103,     0,   103,     0,     0,
+     198,    25,     0,   257,   254,     0,   285,   126,    40,    29,
        0,     0,    30,    31,    34,    36,     0,    37,    39,     0,
-      41,   252,    35,    38,     0,    32,    33,     0,     0,     0,
-       0,     0,   127,   128,   232,   132,   218,   220,   222,   225,
-     228,   229,   230,   224,   223,     0,   270,     0,     0,     0,
+      41,   253,    35,    38,     0,    32,    33,     0,     0,     0,
+       0,     0,   127,   128,   233,   132,   219,   221,   223,   226,
+     229,   230,   231,   225,   224,     0,   271,     0,     0,     0,
        0,     0,     0,   111,     0,     0,   120,     0,     0,   100,
      102,   101,     0,    98,   103,    97,     0,     0,     0,   106,
-     198,     0,     0,    94,   254,   255,     0,     0,   248,   245,
-       0,    43,     0,   257,     0,     0,    44,     0,     0,   259,
-       0,   197,     0,   285,   286,     0,     0,   131,   288,   289,
-     287,     0,     0,     0,   221,     0,     0,   197,   108,     0,
-     116,     0,   117,     0,   290,   103,     0,   118,   113,     0,
+     199,     0,     0,    94,   255,   256,     0,     0,   249,   246,
+       0,    43,     0,   258,     0,     0,    44,     0,     0,   260,
+       0,   198,     0,   286,   287,     0,     0,   131,   289,   290,
+     288,     0,     0,     0,   222,     0,     0,   198,   108,     0,
+     116,     0,   117,     0,   291,   103,     0,   118,   113,     0,
        0,     0,     0,     0,    96,    66,    27,     0,     0,     0,
-       0,     0,   199,   201,   203,   205,     0,   223,     0,    93,
-       0,     0,     0,   248,   242,     0,   246,     0,     0,   262,
-     263,   264,   261,   265,   260,     0,   258,     0,     0,   134,
-     231,     0,     0,   158,   147,   133,   152,   135,   160,   129,
-     130,   217,   219,   174,   226,   271,     0,     0,   233,   250,
-       0,     0,   105,     0,   157,     0,    99,    95,    19,     0,
+       0,     0,   200,   202,   204,   206,     0,   224,     0,    93,
+       0,     0,     0,   249,   243,     0,   247,     0,     0,   263,
+     264,   265,   262,   266,   261,     0,   259,     0,     0,   134,
+     232,     0,     0,   159,   148,   133,   153,   135,   161,   129,
+     130,   218,   220,   175,   227,   272,     0,     0,   234,   251,
+       0,     0,   105,     0,   158,     0,    99,    95,    19,     0,
        0,     0,     0,    20,    21,    22,     0,    74,    76,    77,
-      78,    79,     0,     0,     0,    64,     0,    42,    56,   204,
-     212,     0,     0,     0,     0,     0,   274,   276,   277,   278,
-     279,   275,   280,   282,     0,     0,     0,     0,   266,     0,
-       0,     0,     0,     0,   243,     0,   249,   241,    45,     0,
-       0,    46,   138,     0,   148,   154,   144,   139,   140,   142,
-       0,     0,   151,     0,     0,   150,     0,   162,     0,     0,
-     176,   234,     0,   235,     0,   107,   109,   291,     0,     0,
-       0,     0,   104,     0,    81,    84,    82,   296,   297,   295,
-     294,     0,    80,    85,   272,     0,   270,     0,    63,    65,
-      68,    28,     0,     0,     0,    47,     0,     0,    49,    55,
-      57,    26,   211,   200,   202,   281,   283,     0,     0,     0,
-       0,   213,   210,     0,   209,   268,     0,     0,     0,   247,
-       0,   240,     0,     0,   153,   155,   145,   141,   143,     0,
-     159,     0,     0,   149,     0,     0,   164,     0,   227,     0,
-     178,   236,   251,     0,     0,     0,     0,    75,     0,    67,
-       0,    86,    87,    88,    89,    90,     0,     0,    70,    48,
-       0,    51,    50,     0,    54,     0,     0,   215,     0,     0,
-     208,   267,     0,     0,   244,   237,     0,   238,     0,   136,
-     137,   161,   163,     0,   166,   175,     0,   181,   180,   173,
-       0,    61,     0,     0,    58,    83,   273,     0,    24,    62,
-       0,     0,    23,     0,     0,     0,     0,   206,   214,   269,
-       0,     0,     0,     0,     0,   168,   177,   188,   191,     0,
-       0,    59,     0,     0,     0,    52,     0,   207,   216,    92,
-     239,   146,   165,   167,     0,   122,   169,   170,     0,   192,
-     193,   194,     0,     0,     0,     0,     0,    91,     0,    72,
-      73,     0,    53,     0,   171,   189,     0,   190,   182,   184,
-     183,     0,     0,    69,     0,     0,   195,   196,     0,     0,
-       0,   179,     0,     0,   174,   185,   187,   186,     0,     0,
-       0,    60,     0,   172,    71
+      78,    79,     0,     0,     0,    64,     0,    42,    56,   205,
+     213,     0,     0,     0,     0,     0,   275,   277,   278,   279,
+     280,   276,   281,   283,     0,     0,     0,     0,   267,     0,
+       0,     0,     0,     0,   244,     0,   250,   242,    45,     0,
+       0,    46,   138,   148,     0,   149,   155,   144,   139,   140,
+     142,     0,     0,   152,     0,     0,   151,     0,   163,     0,
+       0,   177,   235,     0,   236,     0,   107,   109,   292,     0,
+       0,     0,     0,   104,     0,    81,    84,    82,   297,   298,
+     296,   295,     0,    80,    85,   273,     0,   271,     0,    63,
+      65,    68,    28,     0,     0,     0,    47,     0,     0,    49,
+      55,    57,    26,   212,   201,   203,   282,   284,     0,     0,
+       0,     0,   214,   211,     0,   210,   269,     0,     0,     0,
+     248,     0,   241,     0,     0,   154,     0,   156,   145,   141,
+     143,     0,   160,     0,     0,   150,     0,     0,   165,     0,
+     228,     0,   179,   237,   252,     0,     0,     0,     0,    75,
+       0,    67,     0,    86,    87,    88,    89,    90,     0,     0,
+      70,    48,     0,    51,    50,     0,    54,     0,     0,   216,
+       0,     0,   209,   268,     0,     0,   245,   238,     0,   239,
+       0,     0,     0,   136,   137,   162,   164,     0,   167,   176,
+       0,   182,   181,   174,     0,    61,     0,     0,    58,    83,
+     274,     0,    24,    62,     0,     0,    23,     0,     0,     0,
+       0,   207,   215,   270,     0,     0,     0,     0,     0,     0,
+     169,   178,   189,   192,     0,     0,    59,     0,     0,     0,
+      52,     0,   208,   217,    92,   240,     0,   146,   166,   168,
+       0,   122,   170,   171,     0,   193,   194,   195,     0,     0,
+       0,     0,     0,    91,     0,    72,    73,     0,    53,   147,
+       0,   172,   190,     0,   191,   183,   185,   184,     0,     0,
+      69,     0,     0,   196,   197,     0,     0,     0,   180,     0,
+       0,   175,   186,   188,   187,     0,     0,     0,    60,     0,
+     173,    71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -410,  -410,  -410,  -410,  -410,  -410,  -410,  -410,  -148,  -410,
-     376,   216,  -410,  -410,  -264,  -410,  -410,  -410,  -410,  -410,
-    -410,  -409,   236,  -410,  -410,  -410,  -410,  -410,  -410,  -410,
-    -410,    85,    -3,  -410,  -410,  -410,   325,  -410,   526,  -410,
-    -410,   467,   217,   464,   -43,   533,  -410,  -410,   436,  -410,
-    -114,  -410,  -410,  -186,   192,  -184,    -9,  -410,  -410,  -410,
-    -410,  -410,  -410,  -410,    89,    55,  -410,  -410,  -410,  -410,
-    -410,  -410,   109,    92,  -410,  -410,    79,  -410,  -142,   318,
-     322,   409,   -42,   441,   439,   501,  -154,  -410,  -410,  -410,
-    -410,   398,  -410,   469,   413,  -232,  -195,   449,  -324,  -410,
-    -129,  -410,  -410,  -410,  -410,  -410,  -141,    -4,  -410,  -410,
-    -410
+    -302,  -302,  -302,  -302,  -302,  -302,  -302,  -302,  -147,  -302,
+     390,   216,  -302,  -302,  -269,  -302,  -302,  -302,  -302,  -302,
+    -302,  -246,   242,  -302,  -302,  -302,  -302,  -302,  -302,  -302,
+    -302,    25,   -40,  -302,  -302,  -302,   340,  -302,   537,  -302,
+    -302,   476,   212,   471,   -33,   544,  -302,  -302,   439,  -302,
+    -101,  -302,  -302,  -144,  -220,  -226,   -10,  -302,  -302,  -302,
+    -302,  -302,  -302,  -302,    91,    53,  -302,  -302,  -302,  -302,
+    -302,  -302,   121,    96,  -302,  -302,   -49,  -302,  -143,   323,
+     321,   413,   -39,   445,   446,   501,  -145,  -302,  -302,  -302,
+    -302,   408,  -302,   472,   410,  -223,  -195,   466,  -301,  -302,
+    -134,  -302,  -302,  -302,  -302,  -302,  -141,    -4,  -302,  -302,
+    -302
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
       -1,    14,    15,    16,    17,    18,    19,    20,   196,   197,
-     100,   369,   370,   371,   263,   359,   360,   274,   428,   472,
-     521,   266,   267,   268,   269,   270,   271,   425,   468,    21,
+     100,   370,   371,   372,   263,   360,   361,   274,   430,   476,
+     527,   266,   267,   268,   269,   270,   271,   427,   472,    21,
       22,    65,   133,    23,    24,   177,   178,    25,    58,    26,
       46,    47,   157,    28,    29,    44,   101,   102,   103,   161,
-     104,   325,   320,   233,   234,   314,   315,   235,   327,   406,
-     454,   485,   505,   506,   507,   329,   330,   410,   459,   460,
-     515,   541,   486,   487,   511,   527,   139,   140,   202,   203,
+     104,   326,   321,   312,   234,   315,   316,   235,   328,   408,
+     458,   490,   511,   512,   513,   330,   331,   412,   463,   464,
+     521,   548,   491,   492,   517,   534,   139,   140,   202,   203,
      204,   205,   206,   106,   107,   108,   109,   110,   111,   112,
      213,   214,   148,   149,   217,   250,   113,   225,   299,   209,
-     114,   355,   296,   115,   166,   171,   183,   116,   353,    30,
+     114,   356,   296,   115,   166,   171,   183,   116,   354,    30,
       31
 };
 
@@ -1090,374 +1095,369 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      36,   210,   105,    48,   211,   182,   236,    45,    49,   207,
-     283,   358,    33,   298,    34,   332,   126,   207,   512,   538,
-      33,   344,    34,   345,    33,    33,    34,    34,    63,   143,
-      56,   283,   175,    68,    69,    70,    71,    72,   147,   283,
-     283,   258,   168,   169,   346,   312,   232,    33,   323,    34,
-     163,   164,   322,   446,   163,   164,   127,   158,   492,   281,
-     536,   513,   539,   442,   135,    37,   129,    56,   383,    32,
-     207,   138,   207,   163,   164,    56,    42,   236,   215,    59,
-     537,   347,   251,    40,   518,   128,   280,   163,   164,   163,
-     164,   285,   286,   287,   288,   289,   290,   291,   292,   293,
-     294,   167,   163,   164,    60,    39,    43,    61,    38,    48,
-     348,   349,   227,   179,    49,    50,   184,   232,   186,   480,
-     105,    67,    10,   421,   457,   215,   422,   423,   361,   447,
-     458,   194,   195,   198,   249,   400,   324,   310,   184,   514,
-     540,   403,   350,   257,   354,   302,   229,   372,   439,   176,
-     384,   146,   231,   176,   207,   207,   237,    10,   295,   282,
-     351,   240,   469,    64,   231,   243,   170,   236,   244,   431,
-     303,   245,   451,   306,   230,   455,   461,   495,   445,   130,
-     131,   381,   256,    57,   362,   198,   212,   264,   265,   272,
-     390,   441,   298,   219,   189,   190,   413,   259,   212,    41,
-     416,   519,   424,    66,   476,   249,   185,   232,    62,   207,
-     418,   336,   117,    33,    53,    34,   220,    27,   118,   363,
-      10,    35,   260,   375,   376,   377,   230,   237,    49,   364,
-     221,   222,    49,   520,   316,    33,   119,    34,   141,   120,
-     238,   317,   313,    55,    10,   236,    54,   179,   298,   318,
-     142,   337,   379,   223,   382,   261,   252,    64,   122,    51,
-     389,    52,   125,   452,   437,   132,   365,   392,   352,   356,
-     121,   319,   198,   462,   463,   134,   207,   366,   224,   136,
-      10,   262,   367,   378,   138,   232,   137,   316,   473,   191,
-     192,   466,   412,   438,   317,   163,   164,   150,   475,  -138,
-     151,   207,   318,   368,   395,   163,   164,   394,   509,    49,
-     163,   164,   254,   255,   275,   276,   144,   237,   301,   255,
-     145,    49,   163,   164,   319,   333,   334,    33,    73,    34,
-      74,   510,    33,   152,    34,   435,   153,   184,   249,   272,
-     501,   342,   343,   154,    75,    76,   246,   184,   444,   163,
-     164,   155,   159,   207,   156,   385,   386,   162,    78,    79,
-     172,   165,   249,   433,   181,   249,    80,    81,   411,   334,
-     419,   420,   464,   255,    82,    83,   160,    84,   465,   255,
-     188,   247,    85,   478,   334,   479,   386,    86,   193,   208,
-      87,   490,   255,   543,   249,   237,   477,   491,   255,   498,
-     334,   548,   499,   386,    88,    89,     1,   218,     2,   228,
-     184,   184,    90,   273,   488,    91,   356,   285,   286,   287,
-     288,   289,   290,   291,   292,   293,   294,   253,   163,   164,
-      92,   517,   343,   497,   283,   533,   343,     3,   549,   255,
-      93,   488,   284,    94,   551,   255,    95,    96,   297,   300,
-     307,   308,   309,     4,     5,   311,    97,   321,   326,   328,
-     331,     6,    98,   272,   338,     7,   488,    99,   248,   339,
-     496,   340,   357,   341,    33,    73,    34,    74,   380,   387,
-     199,   388,   391,   393,   295,     8,   396,   397,   398,   272,
-     399,    75,    76,   401,   402,   404,   405,   407,   408,   409,
-     523,   414,   415,   427,   175,    78,    79,   426,   430,   429,
-       9,   436,   532,    80,    81,   432,   440,   456,   443,    10,
-     449,    82,    83,   450,    84,   453,   467,   470,   481,    85,
-     184,   471,   482,   200,    86,    11,   474,    87,   184,   334,
-     483,   489,    12,   493,   484,    13,   494,   503,   504,   500,
-     516,    88,    89,   526,   508,   522,   528,   529,   531,    90,
-     530,   535,    91,   545,   547,    33,    73,    34,    74,   534,
-     554,   199,   546,   552,   278,   542,   335,    92,   544,   417,
-     553,   123,    75,    76,    10,   180,   434,    93,   187,   124,
-      94,   448,   502,    95,    96,   524,    78,    79,   239,   550,
-     525,   373,   226,    97,    80,    81,   374,   241,   279,    98,
-     242,   304,    82,    83,   201,    84,   174,   216,     0,     0,
-      85,     0,     0,     0,   200,    86,   305,     0,    87,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,    88,    89,     0,     0,     0,     0,     0,     0,
-      90,     0,     0,    91,     0,     0,    33,    73,    34,    74,
-       0,     0,     0,     0,     0,     0,     0,     0,    92,     0,
-       0,     0,     0,    75,    76,    77,     0,     0,    93,     0,
-       0,    94,     0,     0,    95,    96,     0,    78,    79,     0,
-       0,     0,     0,     0,    97,    80,    81,     0,     0,     0,
-      98,     0,     0,    82,    83,   201,    84,     0,     0,     0,
-       0,    85,     0,     0,     0,     0,    86,     0,     0,    87,
-       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,    88,    89,     0,     0,     0,     0,     0,
-       0,    90,     0,     0,    91,     0,     0,    33,    73,    34,
-      74,     0,     0,     0,     0,     0,     0,     0,     0,    92,
-       0,     0,     0,     0,    75,    76,     0,     0,     0,    93,
-       0,     0,    94,     0,     0,    95,    96,     0,    78,    79,
-       0,     0,     0,     0,     0,    97,    80,    81,     0,     0,
-       0,    98,     0,     0,    82,    83,    99,    84,     0,     0,
-       0,     0,    85,     0,     0,     0,     0,    86,     0,     0,
-      87,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    88,    89,     0,     0,     0,     0,
+      36,   210,    48,   211,   207,   105,   359,    45,    49,   283,
+     323,   313,   207,   298,   283,   236,   233,   283,   182,    33,
+     345,    34,   346,    33,   333,    34,   126,   135,    63,   163,
+     164,    56,    56,    68,    69,    70,    71,    72,    33,   143,
+      34,   147,   258,   347,   168,   169,    56,   163,   164,    33,
+      33,    34,    34,   163,   164,    33,   175,    34,   281,   232,
+     158,    67,   518,   163,   164,   207,   219,   207,   285,   286,
+     287,   288,   289,   290,   291,   292,   293,   294,   545,   163,
+     164,   348,   461,   129,   283,   525,   236,   444,   462,   220,
+     215,   127,    37,    42,   194,    32,    33,   384,    34,   280,
+     405,   167,    39,   221,   222,    40,   519,   543,    48,   215,
+     349,   350,   238,   179,    49,   227,   184,   526,   186,   448,
+     324,   128,   546,   105,    43,   314,   223,   544,   252,   362,
+     232,    59,   195,   198,    10,    38,   295,   249,   184,   355,
+      41,    57,   351,    64,   484,   256,   185,   373,   229,   207,
+     207,   224,   465,   257,   230,   500,   237,    50,   441,   473,
+     352,   240,   282,   146,   231,   391,    10,   212,   244,   170,
+     433,   245,   447,   303,   176,   259,   306,   236,   402,   231,
+     385,   452,   212,   455,   520,   198,   459,   264,   265,   272,
+     243,   451,   443,   298,   382,   449,   415,   130,   131,   302,
+     547,   418,   260,   363,   230,   207,    66,   423,   249,   325,
+     424,   420,    27,    10,   337,   480,    35,    33,    62,    34,
+     425,   232,   376,   377,   378,   497,   176,   237,    49,   317,
+     486,   141,    49,   117,    53,   261,   318,   364,    55,   163,
+     164,  -138,   138,   142,   319,   163,   164,   179,   120,   365,
+     298,   338,   524,   251,   118,   380,   236,   383,   515,    60,
+      64,   262,    61,   390,   456,   121,   320,    54,   353,   357,
+     393,   119,   198,   207,    10,   466,   467,     1,   439,     2,
+     529,   440,   516,   379,   254,   255,   366,   426,   470,   477,
+     310,   122,   317,   163,   164,   125,   414,   367,   207,   318,
+     232,    10,   368,   132,   397,   236,   236,   319,     3,    51,
+      49,    52,   275,   276,    33,   479,    34,   134,   237,   189,
+     137,   190,    49,   369,     4,     5,   144,   163,   164,   320,
+     136,   163,   164,     6,   191,   145,   192,     7,   184,   437,
+     272,   150,   249,   165,   507,   301,   255,   138,   184,   232,
+     232,   151,   446,   207,   152,    78,    79,     8,   163,   164,
+     334,   335,   153,    81,   435,   395,   154,   249,   343,   344,
+     249,   155,    82,    83,   156,    84,   386,   387,   413,   335,
+      85,   159,     9,   421,   422,   468,   255,   160,    87,   469,
+     255,    10,   482,   335,   483,   387,   162,   237,   172,   249,
+     550,   481,    88,   277,   495,   255,   181,    11,   555,   188,
+      90,   193,   184,   184,    12,   496,   255,    13,   357,   503,
+     335,   493,   218,   504,   387,   523,   344,   208,    92,   540,
+     344,   556,   255,   228,    33,    73,    34,    74,    93,   253,
+     502,   558,   255,   273,    95,    96,   237,   237,   283,   493,
+     284,    75,    76,   246,    97,   297,   300,   307,   309,   308,
+      98,   322,   311,   327,   329,    78,    79,   272,   332,   339,
+     340,   341,    80,    81,   501,   493,   342,   358,   381,   388,
+     389,   392,    82,    83,   394,    84,   398,   403,   396,   247,
+      85,   399,   401,   400,   272,    86,   404,   406,    87,   407,
+     409,   410,    49,   411,   416,   417,   530,   429,   175,   428,
+     431,   432,    88,    89,   442,   434,   438,   445,   539,   450,
+      90,   453,   454,    91,   457,   460,   471,   478,   474,    33,
+      73,    34,    74,   475,   485,   199,   488,   184,    92,   487,
+     489,   494,   498,   335,   499,   184,    75,    76,    93,   505,
+     506,    94,   509,   510,    95,    96,   514,   522,   528,   533,
+      78,    79,   538,   535,    97,   536,   537,    80,    81,   541,
+      98,   542,   553,   552,   554,    99,   248,    82,    83,   549,
+      84,   551,   561,   559,   560,    85,   419,   436,   278,   200,
+      86,   336,   123,    87,   180,   187,    33,    73,    34,    74,
+     124,   239,   199,   531,   557,   375,   374,    88,    89,   508,
+     532,   241,   279,    75,    76,    90,   174,   242,    91,   226,
+     216,   304,     0,   305,     0,     0,     0,    78,    79,     0,
+       0,     0,     0,    92,    80,    81,     0,     0,     0,     0,
+      10,     0,     0,    93,    82,    83,    94,    84,     0,    95,
+      96,     0,    85,     0,     0,     0,   200,    86,     0,    97,
+      87,     0,     0,     0,     0,    98,     0,     0,     0,     0,
+     201,     0,     0,     0,    88,    89,     0,     0,     0,     0,
        0,     0,    90,     0,     0,    91,     0,     0,     0,     0,
-       0,    33,    73,    34,    74,     0,     0,     0,     0,     0,
-      92,     0,     0,     0,     0,     0,     0,     0,    75,    76,
+      33,    73,    34,    74,     0,     0,     0,     0,     0,     0,
+      92,     0,     0,     0,     0,     0,     0,    75,    76,    77,
       93,     0,     0,    94,     0,     0,    95,    96,     0,     0,
-       0,     0,    78,    79,     0,     0,    97,   146,     0,     0,
-      80,    81,    98,     0,     0,     0,     0,    99,    82,    83,
+       0,    78,    79,     0,     0,     0,    97,     0,    80,    81,
+       0,     0,    98,     0,     0,     0,     0,   201,    82,    83,
        0,    84,     0,     0,     0,     0,    85,     0,     0,     0,
-       0,    86,     0,     0,    87,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    88,    89,
-       0,     0,     0,     0,     0,     0,    90,     0,     0,    91,
+       0,    86,     0,     0,    87,     0,     0,    33,    73,    34,
+      74,     0,     0,     0,     0,     0,     0,     0,    88,    89,
+       0,     0,     0,     0,    75,    76,    90,     0,     0,    91,
+       0,     0,     0,     0,     0,     0,     0,     0,    78,    79,
+       0,     0,     0,     0,    92,    80,    81,     0,     0,     0,
+       0,     0,     0,     0,    93,    82,    83,    94,    84,     0,
+      95,    96,     0,    85,     0,     0,     0,     0,    86,     0,
+      97,    87,     0,     0,     0,     0,    98,     0,     0,     0,
+       0,    99,     0,     0,     0,    88,    89,     0,     0,     0,
+       0,     0,     0,    90,     0,     0,    91,     0,     0,     0,
+       0,    33,    73,    34,    74,     0,     0,     0,     0,     0,
+       0,    92,     0,     0,     0,     0,     0,     0,    75,    76,
+       0,    93,     0,     0,    94,     0,     0,    95,    96,     0,
+       0,     0,    78,    79,     0,     0,     0,    97,   146,    80,
+      81,     0,     0,    98,     0,     0,     0,     0,    99,    82,
+      83,     0,    84,     0,     0,     0,     0,    85,     0,     0,
+       0,     0,    86,     0,     0,    87,     0,     0,    33,    73,
+      34,    74,     0,     0,     0,     0,     0,     0,     0,    88,
+      89,     0,     0,     0,     0,    75,    76,    90,     0,     0,
+      91,     0,     0,     0,     0,     0,     0,     0,     0,    78,
+      79,     0,     0,     0,     0,    92,    80,    81,     0,     0,
+       0,     0,    10,     0,     0,    93,    82,    83,    94,    84,
+       0,    95,    96,     0,    85,     0,     0,     0,   200,    86,
+       0,    97,    87,     0,     0,     0,     0,    98,     0,     0,
+       0,     0,    99,     0,     0,     0,    88,    89,     0,     0,
+       0,     0,     0,     0,    90,     0,     0,    91,     0,     0,
        0,     0,    33,    73,    34,    74,     0,     0,     0,     0,
-       0,     0,     0,     0,    92,     0,     0,     0,     0,    75,
-      76,    10,     0,     0,    93,     0,     0,    94,     0,     0,
-      95,    96,     0,    78,    79,     0,     0,     0,     0,     0,
-      97,    80,    81,     0,     0,     0,    98,     0,     0,    82,
-      83,    99,    84,     0,     0,     0,     0,    85,     0,     0,
-       0,   200,    86,     0,     0,    87,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,     0,    88,
-      89,     0,     0,     0,     0,     0,     0,    90,     0,     0,
-      91,     0,     0,    33,    73,    34,    74,     0,     0,     0,
-       0,     0,     0,     0,     0,    92,     0,     0,     0,     0,
-      75,   173,     0,     0,     0,    93,     0,     0,    94,     0,
-       0,    95,    96,     0,    78,    79,     0,     0,     0,     0,
-       0,    97,    80,    81,     0,     0,     0,    98,     0,     0,
-      82,    83,   201,    84,     0,     0,     0,     0,    85,     0,
-       0,     0,     0,    86,     0,     0,    87,     0,     0,     0,
+       0,     0,    92,     0,     0,     0,     0,     0,     0,    75,
+     173,     0,    93,     0,     0,    94,     0,     0,    95,    96,
+       0,     0,     0,    78,    79,     0,     0,     0,    97,     0,
+      80,    81,     0,     0,    98,     0,     0,     0,     0,   201,
+      82,    83,     0,    84,     0,     0,     0,     0,    85,     0,
+       0,     0,     0,    86,     0,     0,    87,     0,     0,    33,
+      73,    34,    74,     0,     0,     0,     0,     0,     0,     0,
+      88,    89,     0,     0,     0,     0,    75,    76,    90,     0,
+       0,    91,     0,     0,     0,     0,     0,     0,     0,     0,
+      78,    79,     0,     0,     0,     0,    92,    80,    81,     0,
+       0,     0,     0,     0,     0,     0,    93,    82,    83,    94,
+      84,     0,    95,    96,     0,    85,     0,     0,     0,    73,
+      86,    74,    97,    87,     0,     0,     0,     0,    98,     0,
+       0,     0,     0,    99,     0,    75,   173,    88,    89,     0,
+       0,     0,     0,     0,     0,    90,     0,     0,    91,    78,
+      79,     0,     0,     0,     0,     0,     0,    81,     0,     0,
+       0,     0,     0,    92,     0,     0,    82,    83,     0,    84,
+       0,     0,     0,    93,    85,     0,    94,     0,     0,    95,
+      96,     0,    87,     0,     0,     0,     0,     0,     0,    97,
+       0,     0,     0,     0,     0,    98,    88,    89,     0,     0,
+      99,     0,     0,     0,    90,     0,     0,    91,   285,   286,
+     287,   288,   289,   290,   291,   292,   293,   294,     0,   163,
+     164,     0,    92,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,    93,     0,     0,     0,     0,     0,    95,    96,
+       0,     0,     0,     0,     0,     0,     0,     0,    97,     0,
+       0,     0,     0,     0,    98,     0,     0,     0,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-      88,    89,     0,     0,     0,     0,     0,     0,    90,     0,
-       0,    91,     0,     0,    33,    73,    34,    74,     0,     0,
-       0,     0,     0,     0,     0,     0,    92,     0,     0,     0,
-       0,    75,    76,     0,     0,     0,    93,     0,     0,    94,
-       0,     0,    95,    96,     0,    78,    79,     0,     0,     0,
-       0,     0,    97,    80,    81,     0,     0,     0,    98,     0,
-       0,    82,    83,    99,    84,     0,     0,     0,     0,    85,
-       0,     0,     0,     0,    86,     0,     0,    87,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    88,    89,    73,     0,    74,     0,     0,     0,    90,
-       0,     0,    91,     0,     0,     0,     0,     0,     0,    75,
-     173,     0,    78,    79,     0,     0,     0,    92,     0,     0,
-       0,    81,     0,    78,    79,     0,     0,    93,    82,    83,
-      94,    84,    81,    95,    96,     0,    85,     0,     0,    82,
-      83,     0,    84,    97,    87,     0,     0,    85,     0,    98,
-       0,     0,     0,     0,    99,    87,     0,     0,    88,   277,
-       0,     0,     0,     0,     0,     0,    90,     0,     0,    88,
-      89,     0,     0,     0,     0,     0,     0,    90,     0,     0,
-      91,     0,     0,     0,    92,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    93,    92,     0,     0,     0,     0,
-      95,    96,     0,     0,     0,    93,     0,     0,     0,     0,
-      97,    95,    96,     0,     0,     0,    98,     0,     0,     0,
-       0,    97,     0,     0,     0,     0,     0,    98
+       0,     0,     0,     0,     0,     0,   295
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   142,    44,    12,   146,   119,   160,    11,    12,   138,
-       8,   275,     4,   208,     6,   247,    59,   146,     7,     7,
-       4,     5,     6,     7,     4,     4,     6,     6,    32,    72,
-      29,     8,    27,    37,    38,    39,    40,    41,    80,     8,
-       8,   189,    23,    24,    28,   231,   160,     4,    39,     6,
-      21,    22,   236,    69,    21,    22,    31,    99,   467,   201,
-      66,    50,    50,   387,    67,    78,     5,    29,    10,   121,
-     199,   135,   201,    21,    22,    29,    28,   231,    61,    30,
-      86,    65,   146,   121,   493,    60,   200,    21,    22,    21,
-      22,    10,    11,    12,    13,    14,    15,    16,    17,    18,
-      19,   105,    21,    22,   140,    71,    58,   143,   121,   118,
-      94,    95,   154,   117,   118,     0,   120,   231,   122,   443,
-     162,    36,   114,    37,   106,    61,    40,    41,   276,   145,
-     112,   134,   136,   137,   176,   321,   127,    71,   142,   128,
-     128,   325,   126,   186,   273,   122,   145,   145,   380,   144,
-      92,   134,   144,   144,   283,   284,   160,   114,    77,   201,
-     144,   165,   426,   125,   144,   144,   147,   321,   172,   364,
-     212,   175,   404,   215,   145,   407,   145,   145,   145,   118,
-     119,   295,   185,   137,    10,   189,   134,   191,   192,   193,
-     122,   386,   387,    53,    46,    47,   338,    45,   134,    83,
-     341,    74,   116,    71,   436,   247,   121,   321,     3,   338,
-     351,   253,   115,     4,    48,     6,    76,     0,   146,    45,
-     114,     4,    70,    17,    18,    19,   145,   231,   232,    55,
-      90,    91,   236,   106,    72,     4,    33,     6,   132,   144,
-     161,    79,    33,    26,   114,   399,    80,   251,   443,    87,
-     144,   255,   294,   113,   296,   103,   177,   125,    83,   140,
-     302,   142,   104,   405,   378,   137,    92,   309,   272,   273,
-      53,   109,   276,   414,   415,     5,   405,   103,   138,    95,
-     114,   129,   108,    77,   135,   399,   144,    72,   430,    46,
-      47,   420,   334,     9,    79,    21,    22,   144,     9,    84,
-      82,   430,    87,   129,   313,    21,    22,   145,    34,   313,
-      21,    22,   145,   146,   145,   146,     7,   321,   145,   146,
-       7,   325,    21,    22,   109,   145,   146,     4,     5,     6,
-       7,    57,     4,   144,     6,   377,     5,   341,   380,   343,
-     482,   145,   146,   144,    21,    22,    23,   351,   390,    21,
-      22,   144,     5,   482,    82,   145,   146,   146,    35,    36,
-      98,    33,   404,   367,   144,   407,    43,    44,   145,   146,
-     145,   146,   145,   146,    51,    52,    71,    54,   145,   146,
-       7,    58,    59,   145,   146,   145,   146,    64,   144,   144,
-      67,   145,   146,   534,   436,   399,   438,   145,   146,   145,
-     146,   542,   145,   146,    81,    82,     1,     7,     3,     7,
-     414,   415,    89,   144,   456,    92,   420,    10,    11,    12,
-      13,    14,    15,    16,    17,    18,    19,    11,    21,    22,
-     107,   145,   146,   475,     8,   145,   146,    32,   145,   146,
-     117,   483,     9,   120,   145,   146,   123,   124,    20,   146,
-      62,   145,    71,    48,    49,   145,   133,   146,    73,    99,
-     145,    56,   139,   467,   144,    60,   508,   144,   145,    85,
-     474,    85,   131,   144,     4,     5,     6,     7,   144,   144,
-      10,   132,    62,     7,    77,    80,    97,    97,    97,   493,
-      84,    21,    22,     7,     7,    42,    75,    42,   145,    96,
-     504,   144,   144,   137,    27,    35,    36,   146,   144,    92,
-     105,   144,   516,    43,    44,    85,    92,    42,   144,   114,
-     101,    51,    52,   101,    54,    96,   144,    38,     7,    59,
-     534,    99,    95,    63,    64,   130,   144,    67,   542,   146,
-      42,    19,   137,   144,    88,   140,    42,     7,   136,   145,
-     108,    81,    82,    93,   146,   145,   102,   110,     9,    89,
-     102,    33,    92,    68,    68,     4,     5,     6,     7,   144,
-       7,    10,   110,   100,   198,   144,   251,   107,   144,   343,
-     145,    55,    21,    22,   114,   118,   370,   117,   124,    56,
-     120,   399,   483,   123,   124,   506,    35,    36,   162,   544,
-     508,   283,   153,   133,    43,    44,   284,   166,   199,   139,
-     171,   213,    51,    52,   144,    54,   115,   148,    -1,    -1,
-      59,    -1,    -1,    -1,    63,    64,   213,    -1,    67,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,
-      89,    -1,    -1,    92,    -1,    -1,     4,     5,     6,     7,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,
-      -1,    -1,    -1,    21,    22,    23,    -1,    -1,   117,    -1,
-      -1,   120,    -1,    -1,   123,   124,    -1,    35,    36,    -1,
-      -1,    -1,    -1,    -1,   133,    43,    44,    -1,    -1,    -1,
-     139,    -1,    -1,    51,    52,   144,    54,    -1,    -1,    -1,
-      -1,    59,    -1,    -1,    -1,    -1,    64,    -1,    -1,    67,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,
-      -1,    89,    -1,    -1,    92,    -1,    -1,     4,     5,     6,
-       7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,
-      -1,    -1,    -1,    -1,    21,    22,    -1,    -1,    -1,   117,
-      -1,    -1,   120,    -1,    -1,   123,   124,    -1,    35,    36,
-      -1,    -1,    -1,    -1,    -1,   133,    43,    44,    -1,    -1,
-      -1,   139,    -1,    -1,    51,    52,   144,    54,    -1,    -1,
-      -1,    -1,    59,    -1,    -1,    -1,    -1,    64,    -1,    -1,
-      67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,
-      -1,    -1,    89,    -1,    -1,    92,    -1,    -1,    -1,    -1,
+       4,   142,    12,   146,   138,    44,   275,    11,    12,     8,
+     236,   231,   146,   208,     8,   160,   160,     8,   119,     4,
+       5,     6,     7,     4,   247,     6,    59,    67,    32,    21,
+      22,    29,    29,    37,    38,    39,    40,    41,     4,    72,
+       6,    80,   189,    28,    23,    24,    29,    21,    22,     4,
+       4,     6,     6,    21,    22,     4,    27,     6,   201,   160,
+      99,    36,     7,    21,    22,   199,    54,   201,    10,    11,
+      12,    13,    14,    15,    16,    17,    18,    19,     7,    21,
+      22,    66,   107,     5,     8,    75,   231,   388,   113,    77,
+      62,    31,    79,    28,   134,   122,     4,    10,     6,   200,
+     326,   105,    72,    91,    92,   122,    51,    67,   118,    62,
+      95,    96,   161,   117,   118,   154,   120,   107,   122,    70,
+      39,    61,    51,   162,    59,    33,   114,    87,   177,   276,
+     231,    30,   136,   137,   115,   122,    78,   176,   142,   273,
+      84,   138,   127,   126,   445,   185,   121,   146,   146,   283,
+     284,   139,   146,   186,   146,   146,   160,     0,   381,   428,
+     145,   165,   201,   135,   145,   123,   115,   135,   172,   148,
+     365,   175,   146,   212,   145,    44,   215,   322,   322,   145,
+      93,   401,   135,   406,   129,   189,   409,   191,   192,   193,
+     145,   145,   387,   388,   295,   146,   339,   119,   120,   123,
+     129,   342,    71,    10,   146,   339,    72,    37,   247,   128,
+      40,   352,     0,   115,   253,   438,     4,     4,     3,     6,
+      50,   322,    17,    18,    19,   471,   145,   231,   232,    73,
+     450,   133,   236,   116,    48,   104,    80,    44,    26,    21,
+      22,    85,   136,   145,    88,    21,    22,   251,   145,    56,
+     445,   255,   498,   147,   147,   294,   401,   296,    34,   141,
+     126,   130,   144,   302,   407,    53,   110,    81,   272,   273,
+     309,    33,   276,   407,   115,   416,   417,     1,   379,     3,
+     506,     9,    58,    78,   146,   147,    93,   117,   422,   432,
+      72,    84,    73,    21,    22,   105,   335,   104,   432,    80,
+     401,   115,   109,   138,   314,   450,   451,    88,    32,   141,
+     314,   143,   146,   147,     4,     9,     6,     5,   322,    45,
+     145,    47,   326,   130,    48,    49,     7,    21,    22,   110,
+      96,    21,    22,    57,    45,     7,    47,    61,   342,   378,
+     344,   145,   381,    33,   487,   146,   147,   136,   352,   450,
+     451,    83,   391,   487,   145,    35,    36,    81,    21,    22,
+     146,   147,     5,    43,   368,   146,   145,   406,   146,   147,
+     409,   145,    52,    53,    83,    55,   146,   147,   146,   147,
+      60,     5,   106,   146,   147,   146,   147,    72,    68,   146,
+     147,   115,   146,   147,   146,   147,   147,   401,    99,   438,
+     541,   440,    82,    83,   146,   147,   145,   131,   549,     7,
+      90,   145,   416,   417,   138,   146,   147,   141,   422,   146,
+     147,   460,     7,   146,   147,   146,   147,   145,   108,   146,
+     147,   146,   147,     7,     4,     5,     6,     7,   118,    11,
+     479,   146,   147,   145,   124,   125,   450,   451,     8,   488,
+       9,    21,    22,    23,   134,    20,   147,    63,    72,   146,
+     140,   147,   146,    74,   100,    35,    36,   471,   146,   145,
+      86,    86,    42,    43,   478,   514,   145,   132,   145,   145,
+     133,    63,    52,    53,     7,    55,    98,     7,    46,    59,
+      60,    98,    85,    98,   498,    65,     7,    41,    68,    76,
+      41,   146,   506,    97,   145,   145,   510,   138,    27,   147,
+      93,   145,    82,    83,    93,    86,   145,   145,   522,    41,
+      90,   102,   102,    93,    97,    41,   145,   145,    38,     4,
+       5,     6,     7,   100,     7,    10,    41,   541,   108,    96,
+      89,    19,   145,   147,    41,   549,    21,    22,   118,   146,
+     146,   121,     7,   137,   124,   125,   147,   109,   146,    94,
+      35,    36,     9,   103,   134,   111,   103,    42,    43,   145,
+     140,    33,   111,    69,    69,   145,   146,    52,    53,   145,
+      55,   145,     7,   101,   146,    60,   344,   371,   198,    64,
+      65,   251,    55,    68,   118,   124,     4,     5,     6,     7,
+      56,   162,    10,   512,   551,   284,   283,    82,    83,   488,
+     514,   166,   199,    21,    22,    90,   115,   171,    93,   153,
+     148,   213,    -1,   213,    -1,    -1,    -1,    35,    36,    -1,
+      -1,    -1,    -1,   108,    42,    43,    -1,    -1,    -1,    -1,
+     115,    -1,    -1,   118,    52,    53,   121,    55,    -1,   124,
+     125,    -1,    60,    -1,    -1,    -1,    64,    65,    -1,   134,
+      68,    -1,    -1,    -1,    -1,   140,    -1,    -1,    -1,    -1,
+     145,    -1,    -1,    -1,    82,    83,    -1,    -1,    -1,    -1,
+      -1,    -1,    90,    -1,    -1,    93,    -1,    -1,    -1,    -1,
+       4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,    -1,
+     108,    -1,    -1,    -1,    -1,    -1,    -1,    21,    22,    23,
+     118,    -1,    -1,   121,    -1,    -1,   124,   125,    -1,    -1,
+      -1,    35,    36,    -1,    -1,    -1,   134,    -1,    42,    43,
+      -1,    -1,   140,    -1,    -1,    -1,    -1,   145,    52,    53,
+      -1,    55,    -1,    -1,    -1,    -1,    60,    -1,    -1,    -1,
+      -1,    65,    -1,    -1,    68,    -1,    -1,     4,     5,     6,
+       7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    82,    83,
+      -1,    -1,    -1,    -1,    21,    22,    90,    -1,    -1,    93,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    35,    36,
+      -1,    -1,    -1,    -1,   108,    42,    43,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,   118,    52,    53,   121,    55,    -1,
+     124,   125,    -1,    60,    -1,    -1,    -1,    -1,    65,    -1,
+     134,    68,    -1,    -1,    -1,    -1,   140,    -1,    -1,    -1,
+      -1,   145,    -1,    -1,    -1,    82,    83,    -1,    -1,    -1,
+      -1,    -1,    -1,    90,    -1,    -1,    93,    -1,    -1,    -1,
       -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
-     107,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    21,    22,
-     117,    -1,    -1,   120,    -1,    -1,   123,   124,    -1,    -1,
-      -1,    -1,    35,    36,    -1,    -1,   133,   134,    -1,    -1,
-      43,    44,   139,    -1,    -1,    -1,    -1,   144,    51,    52,
-      -1,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,
-      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
+      -1,   108,    -1,    -1,    -1,    -1,    -1,    -1,    21,    22,
+      -1,   118,    -1,    -1,   121,    -1,    -1,   124,   125,    -1,
+      -1,    -1,    35,    36,    -1,    -1,    -1,   134,   135,    42,
+      43,    -1,    -1,   140,    -1,    -1,    -1,    -1,   145,    52,
+      53,    -1,    55,    -1,    -1,    -1,    -1,    60,    -1,    -1,
+      -1,    -1,    65,    -1,    -1,    68,    -1,    -1,     4,     5,
+       6,     7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    82,
+      83,    -1,    -1,    -1,    -1,    21,    22,    90,    -1,    -1,
+      93,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    35,
+      36,    -1,    -1,    -1,    -1,   108,    42,    43,    -1,    -1,
+      -1,    -1,   115,    -1,    -1,   118,    52,    53,   121,    55,
+      -1,   124,   125,    -1,    60,    -1,    -1,    -1,    64,    65,
+      -1,   134,    68,    -1,    -1,    -1,    -1,   140,    -1,    -1,
+      -1,    -1,   145,    -1,    -1,    -1,    82,    83,    -1,    -1,
+      -1,    -1,    -1,    -1,    90,    -1,    -1,    93,    -1,    -1,
       -1,    -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    21,
-      22,   114,    -1,    -1,   117,    -1,    -1,   120,    -1,    -1,
-     123,   124,    -1,    35,    36,    -1,    -1,    -1,    -1,    -1,
-     133,    43,    44,    -1,    -1,    -1,   139,    -1,    -1,    51,
-      52,   144,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,
-      -1,    63,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,
-      82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,
-      92,    -1,    -1,     4,     5,     6,     7,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,
-      21,    22,    -1,    -1,    -1,   117,    -1,    -1,   120,    -1,
-      -1,   123,   124,    -1,    35,    36,    -1,    -1,    -1,    -1,
-      -1,   133,    43,    44,    -1,    -1,    -1,   139,    -1,    -1,
-      51,    52,   144,    54,    -1,    -1,    -1,    -1,    59,    -1,
-      -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,
-      -1,    92,    -1,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,
-      -1,    21,    22,    -1,    -1,    -1,   117,    -1,    -1,   120,
-      -1,    -1,   123,   124,    -1,    35,    36,    -1,    -1,    -1,
-      -1,    -1,   133,    43,    44,    -1,    -1,    -1,   139,    -1,
-      -1,    51,    52,   144,    54,    -1,    -1,    -1,    -1,    59,
-      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
+      -1,    -1,   108,    -1,    -1,    -1,    -1,    -1,    -1,    21,
+      22,    -1,   118,    -1,    -1,   121,    -1,    -1,   124,   125,
+      -1,    -1,    -1,    35,    36,    -1,    -1,    -1,   134,    -1,
+      42,    43,    -1,    -1,   140,    -1,    -1,    -1,    -1,   145,
+      52,    53,    -1,    55,    -1,    -1,    -1,    -1,    60,    -1,
+      -1,    -1,    -1,    65,    -1,    -1,    68,    -1,    -1,     4,
+       5,     6,     7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      82,    83,    -1,    -1,    -1,    -1,    21,    22,    90,    -1,
+      -1,    93,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      35,    36,    -1,    -1,    -1,    -1,   108,    42,    43,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,   118,    52,    53,   121,
+      55,    -1,   124,   125,    -1,    60,    -1,    -1,    -1,     5,
+      65,     7,   134,    68,    -1,    -1,    -1,    -1,   140,    -1,
+      -1,    -1,    -1,   145,    -1,    21,    22,    82,    83,    -1,
+      -1,    -1,    -1,    -1,    -1,    90,    -1,    -1,    93,    35,
+      36,    -1,    -1,    -1,    -1,    -1,    -1,    43,    -1,    -1,
+      -1,    -1,    -1,   108,    -1,    -1,    52,    53,    -1,    55,
+      -1,    -1,    -1,   118,    60,    -1,   121,    -1,    -1,   124,
+     125,    -1,    68,    -1,    -1,    -1,    -1,    -1,    -1,   134,
+      -1,    -1,    -1,    -1,    -1,   140,    82,    83,    -1,    -1,
+     145,    -1,    -1,    -1,    90,    -1,    -1,    93,    10,    11,
+      12,    13,    14,    15,    16,    17,    18,    19,    -1,    21,
+      22,    -1,   108,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,   118,    -1,    -1,    -1,    -1,    -1,   124,   125,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   134,    -1,
+      -1,    -1,    -1,    -1,   140,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    81,    82,     5,    -1,     7,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,    21,
-      22,    -1,    35,    36,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    44,    -1,    35,    36,    -1,    -1,   117,    51,    52,
-     120,    54,    44,   123,   124,    -1,    59,    -1,    -1,    51,
-      52,    -1,    54,   133,    67,    -1,    -1,    59,    -1,   139,
-      -1,    -1,    -1,    -1,   144,    67,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    81,
-      82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,
-      92,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,   117,   107,    -1,    -1,    -1,    -1,
-     123,   124,    -1,    -1,    -1,   117,    -1,    -1,    -1,    -1,
-     133,   123,   124,    -1,    -1,    -1,   139,    -1,    -1,    -1,
-      -1,   133,    -1,    -1,    -1,    -1,    -1,   139
+      -1,    -1,    -1,    -1,    -1,    -1,    78
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
      symbol of state STATE-NUM.  */
 static const yytype_uint16 yystos[] =
 {
-       0,     1,     3,    32,    48,    49,    56,    60,    80,   105,
-     114,   130,   137,   140,   149,   150,   151,   152,   153,   154,
-     155,   177,   178,   181,   182,   185,   187,   190,   191,   192,
-     257,   258,   121,     4,     6,   190,   255,    78,   121,    71,
-     121,    83,    28,    58,   193,   255,   188,   189,   204,   255,
-       0,   140,   142,    48,    80,   190,    29,   137,   186,    30,
-     140,   143,     3,   255,   125,   179,    71,   179,   255,   255,
-     255,   255,   255,     5,     7,    21,    22,    23,    35,    36,
-      43,    44,    51,    52,    54,    59,    64,    67,    81,    82,
-      89,    92,   107,   117,   120,   123,   124,   133,   139,   144,
-     158,   194,   195,   196,   198,   230,   231,   232,   233,   234,
-     235,   236,   237,   244,   248,   251,   255,   115,   146,    33,
-     144,   190,    83,   186,   193,   104,   192,    31,    60,     5,
-     118,   119,   137,   180,     5,   180,    95,   144,   135,   224,
-     225,   132,   144,   192,     7,     7,   134,   230,   240,   241,
-     144,    82,   144,     5,   144,   144,    82,   190,   230,     5,
-      71,   197,   146,    21,    22,    33,   252,   255,    23,    24,
-     147,   253,    98,    22,   233,    27,   144,   183,   184,   255,
-     189,   144,   198,   254,   255,   179,   255,   191,     7,    46,
-      47,    46,    47,   144,   180,   255,   156,   157,   255,    10,
-      63,   144,   226,   227,   228,   229,   230,   248,   144,   247,
-     254,   226,   134,   238,   239,    61,   241,   242,     7,    53,
-      76,    90,    91,   113,   138,   245,   245,   230,     7,   145,
-     145,   144,   198,   201,   202,   205,   234,   255,   224,   196,
-     255,   231,   232,   144,   255,   255,    23,    58,   145,   230,
-     243,   146,   224,    11,   145,   146,   180,   192,   156,    45,
-      70,   103,   129,   162,   255,   255,   169,   170,   171,   172,
-     173,   174,   255,   144,   165,   145,   146,    82,   158,   229,
-     198,   226,   230,     8,     9,    10,    11,    12,    13,    14,
-      15,    16,    17,    18,    19,    77,   250,    20,   244,   246,
-     146,   145,   122,   230,   239,   242,   230,    62,   145,    71,
-      71,   145,   201,    33,   203,   204,    72,    79,    87,   109,
-     200,   146,   203,    39,   127,   199,    73,   206,    99,   213,
-     214,   145,   243,   145,   146,   184,   230,   255,   144,    85,
-      85,   144,   145,   146,     5,     7,    28,    65,    94,    95,
-     126,   144,   255,   256,   248,   249,   255,   131,   162,   163,
-     164,   156,    10,    45,    55,    92,   103,   108,   129,   159,
-     160,   161,   145,   227,   228,    17,    18,    19,    77,   230,
-     144,   198,   230,    10,    92,   145,   146,   144,   132,   230,
-     122,    62,   230,     7,   145,   204,    97,    97,    97,    84,
-     201,     7,     7,   203,    42,    75,   207,    42,   145,    96,
-     215,   145,   230,   226,   144,   144,   254,   170,   254,   145,
-     146,    37,    40,    41,   116,   175,   146,   137,   166,    92,
-     144,   244,    85,   255,   159,   230,   144,   198,     9,   243,
-      92,   244,   246,   144,   230,   145,    69,   145,   202,   101,
-     101,   243,   226,    96,   208,   243,    42,   106,   112,   216,
-     217,   145,   254,   254,   145,   145,   248,   144,   176,   162,
-      38,    99,   167,   226,   144,     9,   243,   230,   145,   145,
-     246,     7,    95,    42,    88,   209,   220,   221,   230,    19,
-     145,   145,   169,   144,    42,   145,   255,   230,   145,   145,
-     145,   226,   220,     7,   136,   210,   211,   212,   146,    34,
-      57,   222,     7,    50,   128,   218,   108,   145,   169,    74,
-     106,   168,   145,   255,   212,   221,    93,   223,   102,   110,
-     102,     9,   255,   145,   144,    33,    66,    86,     7,    50,
-     128,   219,   144,   254,   144,    68,   110,    68,   254,   145,
-     213,   145,   100,   145,     7
+       0,     1,     3,    32,    48,    49,    57,    61,    81,   106,
+     115,   131,   138,   141,   150,   151,   152,   153,   154,   155,
+     156,   178,   179,   182,   183,   186,   188,   191,   192,   193,
+     258,   259,   122,     4,     6,   191,   256,    79,   122,    72,
+     122,    84,    28,    59,   194,   256,   189,   190,   205,   256,
+       0,   141,   143,    48,    81,   191,    29,   138,   187,    30,
+     141,   144,     3,   256,   126,   180,    72,   180,   256,   256,
+     256,   256,   256,     5,     7,    21,    22,    23,    35,    36,
+      42,    43,    52,    53,    55,    60,    65,    68,    82,    83,
+      90,    93,   108,   118,   121,   124,   125,   134,   140,   145,
+     159,   195,   196,   197,   199,   231,   232,   233,   234,   235,
+     236,   237,   238,   245,   249,   252,   256,   116,   147,    33,
+     145,   191,    84,   187,   194,   105,   193,    31,    61,     5,
+     119,   120,   138,   181,     5,   181,    96,   145,   136,   225,
+     226,   133,   145,   193,     7,     7,   135,   231,   241,   242,
+     145,    83,   145,     5,   145,   145,    83,   191,   231,     5,
+      72,   198,   147,    21,    22,    33,   253,   256,    23,    24,
+     148,   254,    99,    22,   234,    27,   145,   184,   185,   256,
+     190,   145,   199,   255,   256,   180,   256,   192,     7,    45,
+      47,    45,    47,   145,   181,   256,   157,   158,   256,    10,
+      64,   145,   227,   228,   229,   230,   231,   249,   145,   248,
+     255,   227,   135,   239,   240,    62,   242,   243,     7,    54,
+      77,    91,    92,   114,   139,   246,   246,   231,     7,   146,
+     146,   145,   199,   202,   203,   206,   235,   256,   225,   197,
+     256,   232,   233,   145,   256,   256,    23,    59,   146,   231,
+     244,   147,   225,    11,   146,   147,   181,   193,   157,    44,
+      71,   104,   130,   163,   256,   256,   170,   171,   172,   173,
+     174,   175,   256,   145,   166,   146,   147,    83,   159,   230,
+     199,   227,   231,     8,     9,    10,    11,    12,    13,    14,
+      15,    16,    17,    18,    19,    78,   251,    20,   245,   247,
+     147,   146,   123,   231,   240,   243,   231,    63,   146,    72,
+      72,   146,   202,   203,    33,   204,   205,    73,    80,    88,
+     110,   201,   147,   204,    39,   128,   200,    74,   207,   100,
+     214,   215,   146,   244,   146,   147,   185,   231,   256,   145,
+      86,    86,   145,   146,   147,     5,     7,    28,    66,    95,
+      96,   127,   145,   256,   257,   249,   250,   256,   132,   163,
+     164,   165,   157,    10,    44,    56,    93,   104,   109,   130,
+     160,   161,   162,   146,   228,   229,    17,    18,    19,    78,
+     231,   145,   199,   231,    10,    93,   146,   147,   145,   133,
+     231,   123,    63,   231,     7,   146,    46,   205,    98,    98,
+      98,    85,   202,     7,     7,   204,    41,    76,   208,    41,
+     146,    97,   216,   146,   231,   227,   145,   145,   255,   171,
+     255,   146,   147,    37,    40,    50,   117,   176,   147,   138,
+     167,    93,   145,   245,    86,   256,   160,   231,   145,   199,
+       9,   244,    93,   245,   247,   145,   231,   146,    70,   146,
+      41,   145,   203,   102,   102,   244,   227,    97,   209,   244,
+      41,   107,   113,   217,   218,   146,   255,   255,   146,   146,
+     249,   145,   177,   163,    38,   100,   168,   227,   145,     9,
+     244,   231,   146,   146,   247,     7,   203,    96,    41,    89,
+     210,   221,   222,   231,    19,   146,   146,   170,   145,    41,
+     146,   256,   231,   146,   146,   146,   146,   227,   221,     7,
+     137,   211,   212,   213,   147,    34,    58,   223,     7,    51,
+     129,   219,   109,   146,   170,    75,   107,   169,   146,   204,
+     256,   213,   222,    94,   224,   103,   111,   103,     9,   256,
+     146,   145,    33,    67,    87,     7,    51,   129,   220,   145,
+     255,   145,    69,   111,    69,   255,   146,   214,   146,   101,
+     146,     7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
 static const yytype_uint16 yyr1[] =
 {
-       0,   148,   149,   149,   149,   149,   149,   149,   150,   150,
-     150,   150,   150,   150,   150,   150,   150,   150,   151,   152,
-     152,   152,   152,   153,   154,   155,   156,   157,   157,   158,
-     158,   158,   158,   158,   158,   158,   158,   158,   158,   158,
-     158,   158,   158,   158,   158,   158,   158,   159,   159,   159,
-     159,   159,   159,   159,   160,   160,   161,   161,   162,   162,
-     162,   162,   163,   163,   164,   164,   165,   165,   166,   166,
-     167,   167,   168,   168,   169,   169,   170,   170,   170,   170,
-     171,   171,   171,   172,   173,   174,   175,   175,   175,   175,
-     176,   176,   177,   177,   177,   177,   178,   178,   178,   178,
-     179,   179,   179,   180,   180,   181,   182,   183,   183,   184,
-     185,   185,   186,   186,   187,   188,   188,   189,   190,   190,
-     191,   191,   192,   193,   193,   193,   194,   194,   195,   195,
-     196,   196,   196,   197,   198,   199,   199,   199,   200,   200,
-     200,   200,   200,   200,   200,   200,   201,   201,   202,   202,
-     202,   202,   202,   202,   203,   203,   204,   204,   205,   205,
-     206,   206,   207,   207,   208,   208,   209,   209,   210,   210,
-     211,   211,   212,   213,   214,   214,   215,   215,   216,   216,
-     217,   217,   218,   218,   218,   219,   219,   219,   220,   220,
-     221,   222,   222,   222,   223,   223,   223,   224,   224,   225,
-     226,   226,   227,   227,   228,   228,   229,   229,   229,   229,
-     229,   229,   229,   229,   229,   229,   229,   230,   230,   231,
-     231,   232,   232,   233,   233,   233,   233,   233,   233,   233,
-     233,   233,   233,   234,   234,   234,   234,   235,   236,   236,
-     237,   237,   238,   238,   239,   240,   240,   241,   242,   242,
-     243,   243,   244,   244,   244,   244,   244,   244,   244,   244,
-     245,   245,   245,   245,   245,   245,   246,   246,   247,   247,
-     248,   248,   249,   249,   250,   250,   250,   250,   250,   250,
-     250,   250,   250,   250,   251,   252,   252,   253,   253,   253,
-     254,   254,   255,   255,   256,   256,   256,   256,   257,   258,
-     258
+       0,   149,   150,   150,   150,   150,   150,   150,   151,   151,
+     151,   151,   151,   151,   151,   151,   151,   151,   152,   153,
+     153,   153,   153,   154,   155,   156,   157,   158,   158,   159,
+     159,   159,   159,   159,   159,   159,   159,   159,   159,   159,
+     159,   159,   159,   159,   159,   159,   159,   160,   160,   160,
+     160,   160,   160,   160,   161,   161,   162,   162,   163,   163,
+     163,   163,   164,   164,   165,   165,   166,   166,   167,   167,
+     168,   168,   169,   169,   170,   170,   171,   171,   171,   171,
+     172,   172,   172,   173,   174,   175,   176,   176,   176,   176,
+     177,   177,   178,   178,   178,   178,   179,   179,   179,   179,
+     180,   180,   180,   181,   181,   182,   183,   184,   184,   185,
+     186,   186,   187,   187,   188,   189,   189,   190,   191,   191,
+     192,   192,   193,   194,   194,   194,   195,   195,   196,   196,
+     197,   197,   197,   198,   199,   200,   200,   200,   201,   201,
+     201,   201,   201,   201,   201,   201,   202,   202,   202,   203,
+     203,   203,   203,   203,   203,   204,   204,   205,   205,   206,
+     206,   207,   207,   208,   208,   209,   209,   210,   210,   211,
+     211,   212,   212,   213,   214,   215,   215,   216,   216,   217,
+     217,   218,   218,   219,   219,   219,   220,   220,   220,   221,
+     221,   222,   223,   223,   223,   224,   224,   224,   225,   225,
+     226,   227,   227,   228,   228,   229,   229,   230,   230,   230,
+     230,   230,   230,   230,   230,   230,   230,   230,   231,   231,
+     232,   232,   233,   233,   234,   234,   234,   234,   234,   234,
+     234,   234,   234,   234,   235,   235,   235,   235,   236,   237,
+     237,   238,   238,   239,   239,   240,   241,   241,   242,   243,
+     243,   244,   244,   245,   245,   245,   245,   245,   245,   245,
+     245,   246,   246,   246,   246,   246,   246,   247,   247,   248,
+     248,   249,   249,   250,   250,   251,   251,   251,   251,   251,
+     251,   251,   251,   251,   251,   252,   253,   253,   254,   254,
+     254,   255,   255,   256,   256,   257,   257,   257,   257,   258,
+     259,   259
 };
 
   /* YYR2[YYN] -- Number of symbols on the right hand side of rule YYN.  */
@@ -1477,23 +1477,23 @@ static const yytype_uint8 yyr2[] =
        2,     3,     0,     3,     2,     1,     3,     3,     4,     1,
        3,     1,    10,     0,     1,     1,     1,     1,     1,     3,
        3,     2,     1,     2,     3,     0,     3,     3,     0,     1,
-       1,     2,     1,     2,     1,     2,     6,     1,     2,     3,
-       2,     2,     1,     3,     1,     2,     1,     4,     1,     3,
-       0,     3,     0,     2,     0,     3,     0,     2,     0,     1,
-       1,     2,     6,     3,     0,     3,     0,     3,     0,     5,
-       1,     1,     2,     2,     2,     2,     2,     2,     1,     3,
-       3,     0,     1,     1,     0,     2,     2,     0,     1,     2,
-       3,     1,     3,     1,     2,     1,     5,     6,     4,     3,
-       3,     3,     2,     3,     5,     4,     6,     3,     1,     3,
-       1,     2,     1,     1,     1,     1,     3,     5,     1,     1,
-       1,     3,     1,     3,     4,     4,     5,     6,     6,     8,
-       5,     4,     1,     2,     4,     1,     2,     4,     0,     2,
-       1,     3,     1,     1,     2,     2,     1,     2,     3,     2,
-       1,     1,     1,     1,     1,     1,     1,     3,     3,     5,
-       1,     3,     1,     3,     1,     1,     1,     1,     1,     1,
-       1,     2,     1,     2,     1

<TRUNCATED>


[06/11] incubator-quickstep git commit: Initialize updates for transitive closure

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 7b9ed96..6cc7f08 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -38,6 +38,9 @@ add_library(quickstep_relationaloperators_BuildAggregationExistenceMapOperator
             BuildAggregationExistenceMapOperator.hpp)
 add_library(quickstep_relationaloperators_BuildHashOperator BuildHashOperator.cpp BuildHashOperator.hpp)
 add_library(quickstep_relationaloperators_BuildLIPFilterOperator BuildLIPFilterOperator.cpp BuildLIPFilterOperator.hpp)
+add_library(quickstep_relationaloperators_BuildTransitiveClosureOperator
+            BuildTransitiveClosureOperator.cpp
+            BuildTransitiveClosureOperator.hpp)
 add_library(quickstep_relationaloperators_CreateIndexOperator ../empty_src.cpp CreateIndexOperator.hpp)
 add_library(quickstep_relationaloperators_CreateTableOperator ../empty_src.cpp CreateTableOperator.hpp)
 add_library(quickstep_relationaloperators_DestroyAggregationStateOperator
@@ -53,6 +56,9 @@ add_library(quickstep_relationaloperators_HashJoinOperator HashJoinOperator.cpp
 add_library(quickstep_relationaloperators_InitializeAggregationOperator
             InitializeAggregationOperator.cpp
             InitializeAggregationOperator.hpp)
+add_library(quickstep_relationaloperators_InitializeTransitiveClosureOperator
+            InitializeTransitiveClosureOperator.cpp
+            InitializeTransitiveClosureOperator.hpp)
 add_library(quickstep_relationaloperators_InsertOperator InsertOperator.cpp InsertOperator.hpp)
 add_library(quickstep_relationaloperators_NestedLoopsJoinOperator
             NestedLoopsJoinOperator.cpp
@@ -73,6 +79,9 @@ add_library(quickstep_relationaloperators_SortRunGenerationOperator SortRunGener
             SortRunGenerationOperator.hpp)
 add_library(quickstep_relationaloperators_TableExportOperator TableExportOperator.cpp TableExportOperator.hpp)
 add_library(quickstep_relationaloperators_TableGeneratorOperator TableGeneratorOperator.cpp TableGeneratorOperator.hpp)
+add_library(quickstep_relationaloperators_TransitiveClosureOperator
+            TransitiveClosureOperator.cpp
+            TransitiveClosureOperator.hpp)
 add_library(quickstep_relationaloperators_TextScanOperator TextScanOperator.cpp TextScanOperator.hpp)
 add_library(quickstep_relationaloperators_UnionAllOperator UnionAllOperator.cpp UnionAllOperator.hpp)
 add_library(quickstep_relationaloperators_UpdateOperator UpdateOperator.cpp UpdateOperator.hpp)
@@ -171,6 +180,28 @@ target_link_libraries(quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_utility_lipfilter_LIPFilterBuilder
                       quickstep_utility_lipfilter_LIPFilterUtil
                       tmb)
+target_link_libraries(quickstep_relationaloperators_BuildTransitiveClosureOperator
+                      glog
+                      quickstep_catalog_CatalogRelation
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_cli_Flags
+                      quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
+                      quickstep_queryexecution_WorkOrdersContainer
+                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
+                      quickstep_storage_StorageBlock
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageManager
+                      quickstep_storage_TransitiveClosureState
+                      quickstep_storage_TupleStorageSubBlock
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_utility_BlockIDStream
+                      quickstep_utility_Macros
+                      quickstep_utility_Range
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_CreateIndexOperator
                       glog
                       quickstep_catalog_CatalogRelation
@@ -304,6 +335,20 @@ target_link_libraries(quickstep_relationaloperators_InitializeAggregationOperato
                       quickstep_storage_AggregationOperationState
                       quickstep_utility_Macros
                       tmb)
+target_link_libraries(quickstep_relationaloperators_InitializeTransitiveClosureOperator
+                      glog
+                      quickstep_cli_Flags
+                      quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
+                      quickstep_queryexecution_WorkOrdersContainer
+                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_TransitiveClosureState
+                      quickstep_utility_Macros
+                      quickstep_utility_Range
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_InsertOperator
                       glog
                       quickstep_catalog_CatalogRelation
@@ -537,6 +582,17 @@ if (QUICKSTEP_HAVE_FILE_MANAGER_HDFS)
   target_link_libraries(quickstep_relationaloperators_TextScanOperator
                         ${LIBHDFS3_LIBRARIES})
 endif(QUICKSTEP_HAVE_FILE_MANAGER_HDFS)
+target_link_libraries(quickstep_relationaloperators_TransitiveClosureOperator
+                      glog
+                      quickstep_catalog_CatalogRelation
+                      quickstep_queryexecution_QueryContext
+                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_TransitiveClosureState
+                      quickstep_utility_Macros
+                      quickstep_utility_Range
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_UnionAllOperator
                       glog
                       quickstep_catalog_CatalogRelation
@@ -639,6 +695,7 @@ target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_BuildAggregationExistenceMapOperator
                       quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_relationaloperators_BuildHashOperator
+                      quickstep_relationaloperators_BuildTransitiveClosureOperator
                       quickstep_relationaloperators_CreateIndexOperator
                       quickstep_relationaloperators_CreateTableOperator
                       quickstep_relationaloperators_DeleteOperator
@@ -648,6 +705,7 @@ target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_relationaloperators_HashJoinOperator
                       quickstep_relationaloperators_InitializeAggregationOperator
+                      quickstep_relationaloperators_InitializeTransitiveClosureOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_relationaloperators_RebuildWorkOrder
@@ -662,6 +720,7 @@ target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_TableExportOperator
                       quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_relationaloperators_TextScanOperator
+                      quickstep_relationaloperators_TransitiveClosureOperator
                       quickstep_relationaloperators_UnionAllOperator
                       quickstep_relationaloperators_UpdateOperator
                       quickstep_relationaloperators_WindowAggregationOperator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/relational_operators/InitializeTransitiveClosureOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeTransitiveClosureOperator.cpp b/relational_operators/InitializeTransitiveClosureOperator.cpp
new file mode 100644
index 0000000..a6ffe6f
--- /dev/null
+++ b/relational_operators/InitializeTransitiveClosureOperator.cpp
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "relational_operators/InitializeTransitiveClosureOperator.hpp"
+
+#include <algorithm>
+#include <cstddef>
+
+#include "cli/Flags.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
+#include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
+#include "storage/TransitiveClosureState.hpp"
+#include "utility/Range.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+bool InitializeTransitiveClosureOperator::getAllWorkOrders(
+    WorkOrdersContainer *container,
+    QueryContext *query_context,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
+  if (started_) {
+    return true;
+  }
+
+  TransitiveClosureState *state =
+      query_context->getTransitiveClosureState(transitive_closure_context_index_);
+
+  constexpr std::size_t kMinBatchSize = 1024ul * 1024ul * 4ul;
+  const std::size_t range = state->range();
+  const std::size_t num_batches =
+      std::max(1ul, std::min(range * range / kMinBatchSize,
+                             static_cast<std::size_t>(FLAGS_num_workers)));
+
+  const RangeSplitter splitter =
+      RangeSplitter::CreateWithNumPartitions(0, range, num_batches);
+
+  for (std::size_t i = 0; i < splitter.getNumPartitions(); ++i) {
+    container->addNormalWorkOrder(
+        new InitializeTransitiveClosureWorkOrder(query_id_,
+                                                 splitter.getPartition(i),
+                                                 state),
+        op_index_);
+  }
+
+  started_ = true;
+  return true;
+}
+
+bool InitializeTransitiveClosureOperator::getAllWorkOrderProtos(
+    WorkOrderProtosContainer *container)  {
+  LOG(FATAL) << "Not supported";
+}
+
+void InitializeTransitiveClosureWorkOrder::execute() {
+  if (range_.begin() == 0) {
+    state_->initializeStart();
+  }
+  for (std::size_t i = range_.begin(); i < range_.end(); ++i) {
+    state_->initializeEdgeComponent(i);
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/relational_operators/InitializeTransitiveClosureOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeTransitiveClosureOperator.hpp b/relational_operators/InitializeTransitiveClosureOperator.hpp
new file mode 100644
index 0000000..1694620
--- /dev/null
+++ b/relational_operators/InitializeTransitiveClosureOperator.hpp
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_TRANSITIVE_CLOSURE_OPERATOR_HPP_
+#define QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_TRANSITIVE_CLOSURE_OPERATOR_HPP_
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+#include "query_execution/QueryContext.hpp"
+#include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/TransitiveClosureState.hpp"
+#include "utility/Macros.hpp"
+#include "utility/Range.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb { class MessageBus; }
+
+namespace quickstep {
+
+class StorageManager;
+class WorkOrderProtosContainer;
+class WorkOrdersContainer;
+
+/** \addtogroup RelationalOperators
+ *  @{
+ */
+
+class InitializeTransitiveClosureOperator : public RelationalOperator {
+ public:
+  InitializeTransitiveClosureOperator(const std::size_t query_id,
+                                      const std::size_t transitive_closure_context_index)
+      : RelationalOperator(query_id, 1u),
+        transitive_closure_context_index_(transitive_closure_context_index),
+        started_(false) {
+  }
+
+  ~InitializeTransitiveClosureOperator() override {}
+
+  OperatorType getOperatorType() const override {
+    return kInitializeTransitiveClosure;
+  }
+
+  std::string getName() const override {
+    return "InitializeTransitiveClosureOperator";
+  }
+
+  bool getAllWorkOrders(WorkOrdersContainer *container,
+                        QueryContext *query_context,
+                        StorageManager *storage_manager,
+                        const tmb::client_id scheduler_client_id,
+                        tmb::MessageBus *bus) override;
+
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
+ private:
+  const std::size_t transitive_closure_context_index_;
+  bool started_;
+
+  DISALLOW_COPY_AND_ASSIGN(InitializeTransitiveClosureOperator);
+};
+
+class InitializeTransitiveClosureWorkOrder : public WorkOrder {
+ public:
+  InitializeTransitiveClosureWorkOrder(const std::size_t query_id,
+                                       const Range &range,
+                                       TransitiveClosureState *state)
+      : WorkOrder(query_id, 1u),
+        range_(range),
+        state_(state) {}
+
+  ~InitializeTransitiveClosureWorkOrder() override {}
+
+  void execute() override;
+
+ private:
+  const Range range_;
+  TransitiveClosureState *state_;
+
+  DISALLOW_COPY_AND_ASSIGN(InitializeTransitiveClosureWorkOrder);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_TRANSITIVE_CLOSURE_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index 8eb59f0..d0c7c50 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -67,6 +67,7 @@ class RelationalOperator {
     kBuildAggregationExistenceMap,
     kBuildHash,
     kBuildLIPFilter,
+    kBuildTransitiveClosure,
     kCreateIndex,
     kCreateTable,
     kDelete,
@@ -75,6 +76,7 @@ class RelationalOperator {
     kDropTable,
     kFinalizeAggregation,
     kInitializeAggregation,
+    kInitializeTransitiveClosure,
     kInnerJoin,
     kInsert,
     kLeftAntiJoin,
@@ -89,6 +91,7 @@ class RelationalOperator {
     kTableExport,
     kTableGenerator,
     kTextScan,
+    kTransitiveClosure,
     kUnionAll,
     kUpdate,
     kWindowAggregation,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/relational_operators/TransitiveClosureOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TransitiveClosureOperator.cpp b/relational_operators/TransitiveClosureOperator.cpp
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/relational_operators/TransitiveClosureOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TransitiveClosureOperator.hpp b/relational_operators/TransitiveClosureOperator.hpp
new file mode 100644
index 0000000..f983a4a
--- /dev/null
+++ b/relational_operators/TransitiveClosureOperator.hpp
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_RELATIONAL_OPERATORS_TRANSITIVE_CLOSURE_OPERATOR_HPP_
+#define QUICKSTEP_RELATIONAL_OPERATORS_TRANSITIVE_CLOSURE_OPERATOR_HPP_
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+#include "catalog/CatalogRelation.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/TransitiveClosureState.hpp"
+#include "utility/Macros.hpp"
+#include "utility/Range.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb { class MessageBus; }
+
+namespace quickstep {
+
+class StorageManager;
+class WorkOrderProtosContainer;
+class WorkOrdersContainer;
+
+/** \addtogroup RelationalOperators
+ *  @{
+ */
+
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_TRANSITIVE_CLOSURE_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index ae14dd1..ee782cc 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -17,8 +17,8 @@
  * under the License.
  **/
 
-#ifndef QUICKSTEP_RELATIONAL_OPERATORS_AGGREGATION_OPERATION_HPP_
-#define QUICKSTEP_RELATIONAL_OPERATORS_AGGREGATION_OPERATION_HPP_
+#ifndef QUICKSTEP_STORAGE_AGGREGATION_OPERATION_STATE_HPP_
+#define QUICKSTEP_STORAGE_AGGREGATION_OPERATION_STATE_HPP_
 
 #include <cstddef>
 #include <memory>
@@ -334,4 +334,4 @@ class AggregationOperationState {
 
 }  // namespace quickstep
 
-#endif  // QUICKSTEP_RELATIONAL_OPERATORS_AGGREGATION_OPERATION_HPP_
+#endif  // QUICKSTEP_STORAGE_AGGREGATION_OPERATION_STATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 8ac7285..4699244 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -256,6 +256,7 @@ add_library(quickstep_storage_SubBlocksReference ../empty_src.cpp SubBlocksRefer
 add_library(quickstep_storage_ThreadPrivateCompactKeyHashTable
             ThreadPrivateCompactKeyHashTable.cpp
             ThreadPrivateCompactKeyHashTable.hpp)
+add_library(quickstep_storage_TransitiveClosureState TransitiveClosureState.cpp TransitiveClosureState.hpp)
 add_library(quickstep_storage_TupleIdSequence ../empty_src.cpp TupleIdSequence.hpp)
 add_library(quickstep_storage_TupleReference ../empty_src.cpp TupleReference.hpp)
 add_library(quickstep_storage_TupleStorageSubBlock TupleStorageSubBlock.cpp TupleStorageSubBlock.hpp)
@@ -1087,6 +1088,9 @@ target_link_libraries(quickstep_storage_ThreadPrivateCompactKeyHashTable
                       quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_utility_Macros
                       quickstep_utility_ScopedBuffer)
+target_link_libraries(quickstep_storage_TransitiveClosureState
+                      quickstep_utility_BarrieredReadWriteConcurrentBitVector
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_TupleIdSequence
                       glog
                       quickstep_storage_StorageBlockInfo
@@ -1215,6 +1219,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_SubBlockTypeRegistryMacros
                       quickstep_storage_SubBlocksReference
                       quickstep_storage_ThreadPrivateCompactKeyHashTable
+                      quickstep_storage_TransitiveClosureState
                       quickstep_storage_TupleIdSequence
                       quickstep_storage_TupleReference
                       quickstep_storage_TupleStorageSubBlock

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/storage/TransitiveClosureState.cpp
----------------------------------------------------------------------
diff --git a/storage/TransitiveClosureState.cpp b/storage/TransitiveClosureState.cpp
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/storage/TransitiveClosureState.hpp
----------------------------------------------------------------------
diff --git a/storage/TransitiveClosureState.hpp b/storage/TransitiveClosureState.hpp
new file mode 100644
index 0000000..852972a
--- /dev/null
+++ b/storage/TransitiveClosureState.hpp
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_STORAGE_TRANSITIVE_CLOSURE_STATE_HPP_
+#define QUICKSTEP_STORAGE_TRANSITIVE_CLOSURE_STATE_HPP_
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <vector>
+
+#include "utility/BarrieredReadWriteConcurrentBitVector.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+class TransitiveClosureState {
+ public:
+  explicit TransitiveClosureState(const int range)
+      : range_(range),
+        edges_(range_) {
+  }
+
+  int range() const {
+    return range_;
+  }
+
+  void initializeStart() {
+    DCHECK(starts_ == nullptr);
+    starts_ = std::make_unique<BarrieredReadWriteConcurrentBitVector>(range_);
+  }
+
+  void initializeEdgeComponent(const int source) {
+    DCHECK(edges_[source] == nullptr);
+    edges_[source] = std::make_unique<BarrieredReadWriteConcurrentBitVector>(range_);
+  }
+
+  inline void addStart(const int value) {
+    if (value < range_) {
+      starts_->setBit(value);
+    }
+  }
+
+  inline void addEdge(const int source, const int destination) {
+    DCHECK_LT(source, range_);
+    DCHECK_LT(destination, range_);
+    edges_[source]->setBit(destination);
+  }
+
+ private:
+  const int range_;
+  std::unique_ptr<BarrieredReadWriteConcurrentBitVector> starts_;
+  std::vector<std::unique_ptr<BarrieredReadWriteConcurrentBitVector>> edges_;
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_TRANSITIVE_CLOSURE_STATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/utility/BarrieredReadWriteConcurrentBitVector.hpp
----------------------------------------------------------------------
diff --git a/utility/BarrieredReadWriteConcurrentBitVector.hpp b/utility/BarrieredReadWriteConcurrentBitVector.hpp
index 0086c7f..1dcb58e 100644
--- a/utility/BarrieredReadWriteConcurrentBitVector.hpp
+++ b/utility/BarrieredReadWriteConcurrentBitVector.hpp
@@ -131,6 +131,7 @@ class BarrieredReadWriteConcurrentBitVector {
    * @return The value of the bit at bit_num.
    **/
   inline bool getBit(const std::size_t bit_num) const {
+    DCHECK_LT(bit_num, num_bits_);
     const std::size_t data_value =
         data_array_[bit_num >> kHigherOrderShift].load(std::memory_order_relaxed);
     return (data_value << (bit_num & kLowerOrderMask)) & kTopBit;
@@ -143,6 +144,7 @@ class BarrieredReadWriteConcurrentBitVector {
    * @param value The new value to set for the bit at bit_num.
    **/
   inline void setBit(const std::size_t bit_num) const {
+    DCHECK_LT(bit_num, num_bits_);
     data_array_[bit_num >> kHigherOrderShift].fetch_or(
         kTopBit >> (bit_num & kLowerOrderMask), std::memory_order_relaxed);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/utility/BlockIDStream.hpp
----------------------------------------------------------------------
diff --git a/utility/BlockIDStream.hpp b/utility/BlockIDStream.hpp
new file mode 100644
index 0000000..20205cc
--- /dev/null
+++ b/utility/BlockIDStream.hpp
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_BLOCK_ID_STREAM_HPP_
+#define QUICKSTEP_UTILITY_BLOCK_ID_STREAM_HPP_
+
+#include <cstddef>
+#include <vector>
+
+#include "catalog/CatalogRelation.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+class BlockIDStream {
+ public:
+  BlockIDStream(const CatalogRelation &relation,
+                const bool relation_is_stored,
+                const bool &done_feeding_blocks)
+      : relation_is_stored_(relation_is_stored),
+        done_feeding_blocks_(done_feeding_blocks),
+        current_position_(0) {
+    if (relation_is_stored_) {
+      block_ids_ = relation.getBlocksSnapshot();
+    }
+  }
+
+  inline void append(const block_id block) {
+    block_ids_.emplace_back(block);
+  }
+
+  inline bool hasNext() const {
+    return current_position_ < block_ids_.size();
+  }
+
+  inline block_id getNext() {
+    DCHECK(current_position_ < block_ids_.size());
+    return block_ids_[current_position_++];
+  }
+
+  bool isEndOfStream() const {
+    return (!hasNext()) && (relation_is_stored_ || done_feeding_blocks_);
+  }
+
+ private:
+  const bool relation_is_stored_;
+  const bool &done_feeding_blocks_;
+
+  std::vector<block_id> block_ids_;
+  std::size_t current_position_;
+
+  DISALLOW_COPY_AND_ASSIGN(BlockIDStream);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_BLOCK_ID_STREAM_HPP_
+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index c78ec59..2193315 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -177,6 +177,7 @@ add_library(quickstep_utility_CompositeHash ../empty_src.cpp CompositeHash.hpp)
 add_library(quickstep_utility_BarrieredReadWriteConcurrentBitVector
             ../empty_src.cpp
             BarrieredReadWriteConcurrentBitVector.hpp)
+add_library(quickstep_utility_BlockIDStream ../empty_src.cpp BlockIDStream.hpp)
 add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
 add_library(quickstep_utility_DisjointTreeForest ../empty_src.cpp DisjointTreeForest.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
@@ -253,6 +254,10 @@ target_link_libraries(quickstep_utility_CompositeHash
 target_link_libraries(quickstep_utility_BarrieredReadWriteConcurrentBitVector
                       quickstep_utility_BitManipulation
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_BlockIDStream
+                      quickstep_catalog_CatalogRelation
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_BulkIoConfiguration
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_DAG
@@ -365,6 +370,7 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_BarrieredReadWriteConcurrentBitVector
                       quickstep_utility_BitManipulation
                       quickstep_utility_BitVector
+                      quickstep_utility_BlockIDStream
                       quickstep_utility_BloomFilter
                       quickstep_utility_BloomFilter_proto
                       quickstep_utility_BulkIoConfiguration

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/utility/TransitiveClosureContext.cpp
----------------------------------------------------------------------
diff --git a/utility/TransitiveClosureContext.cpp b/utility/TransitiveClosureContext.cpp
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/utility/TransitiveClosureContext.hpp
----------------------------------------------------------------------
diff --git a/utility/TransitiveClosureContext.hpp b/utility/TransitiveClosureContext.hpp
new file mode 100644
index 0000000..8b13789
--- /dev/null
+++ b/utility/TransitiveClosureContext.hpp
@@ -0,0 +1 @@
+


[11/11] incubator-quickstep git commit: Updates to transitive closure

Posted by ji...@apache.org.
Updates to transitive closure


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

Branch: refs/heads/transitive-closure
Commit: 2aefd7bce5bad9bb6063b4fd71ec37876d58662d
Parents: 734ddc1
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Dec 11 14:45:08 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Mon Dec 11 16:07:23 2017 -0600

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp          |  19 ++-
 query_optimizer/PhysicalGenerator.cpp           |   2 +-
 .../BuildTransitiveClosureOperator.cpp          |   2 -
 relational_operators/CMakeLists.txt             |   5 +
 .../InitializeTransitiveClosureOperator.cpp     |   6 +-
 .../TransitiveClosureOperator.cpp               | 158 +++++++++++++++++++
 .../TransitiveClosureOperator.hpp               |  86 +++++++++-
 storage/TransitiveClosureState.hpp              |   8 +
 types/containers/ColumnVector.hpp               |   7 +
 .../BarrieredReadWriteConcurrentBitVector.hpp   |   7 +
 utility/BitVector.hpp                           |  28 +++-
 11 files changed, 314 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 8f29271..648b937 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -2488,18 +2488,23 @@ void ExecutionGenerator::convertTransitiveClosure(
                                  &output_relation,
                                  insert_destination_proto);
 
+  const QueryPlan::DAGNodeIndex tc_operator_index =
+      execution_plan_->addRelationalOperator(
+          new TransitiveClosureOperator(query_handle_->query_id(),
+                                        transitive_closure_state_index,
+                                        *output_relation,
+                                        insert_destination_index));
+  insert_destination_proto->set_relational_op_index(tc_operator_index);
 
-  (void)insert_destination_index;
+  execution_plan_->addDirectDependency(tc_operator_index,
+                                       build_tc_operator_index,
+                                       true /* is_pipeline_breaker */);
 
-  // TODO: fix
-  insert_destination_proto->set_relational_op_index(build_tc_operator_index /* FIX */);
   physical_to_output_relation_map_.emplace(
       std::piecewise_construct,
       std::forward_as_tuple(physical_plan),
-      std::forward_as_tuple(build_tc_operator_index /* FIX */, output_relation));
-
-  temporary_relation_info_vec_.emplace_back(build_tc_operator_index /* FIX */,
-                                            output_relation);
+      std::forward_as_tuple(tc_operator_index, output_relation));
+  temporary_relation_info_vec_.emplace_back(tc_operator_index, output_relation);
 }
 
 }  // namespace optimizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index b7b0db0..865cd11 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -194,7 +194,7 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
              << physical_plan_->toString();
   }
 
-  std::cerr << "Optimized physical plan:\n" << physical_plan_->toString();
+  DVLOG(4) << "Optimized physical plan:\n" << physical_plan_->toString();
 
   if (FLAGS_visualize_plan) {
     quickstep::PlanVisualizer plan_visualizer;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/relational_operators/BuildTransitiveClosureOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildTransitiveClosureOperator.cpp b/relational_operators/BuildTransitiveClosureOperator.cpp
index e151756..919a974 100644
--- a/relational_operators/BuildTransitiveClosureOperator.cpp
+++ b/relational_operators/BuildTransitiveClosureOperator.cpp
@@ -107,7 +107,6 @@ void BuildTransitiveClosureWorkOrder::execute() {
 }
 
 void BuildTransitiveClosureWorkOrder::buildStartRelation(ValueAccessor *accessor) {
-  std::cout << "BuildStartRelation: " << block_ << "\n";
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {
@@ -119,7 +118,6 @@ void BuildTransitiveClosureWorkOrder::buildStartRelation(ValueAccessor *accessor
 }
 
 void BuildTransitiveClosureWorkOrder::buildEdgeRelation(ValueAccessor *accessor) {
-  std::cout << "BuildEdgeRelation: " << block_ << "\n";
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 6cc7f08..e85eb4e 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -585,11 +585,16 @@ endif(QUICKSTEP_HAVE_FILE_MANAGER_HDFS)
 target_link_libraries(quickstep_relationaloperators_TransitiveClosureOperator
                       glog
                       quickstep_catalog_CatalogRelation
+                      quickstep_cli_Flags
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
+                      quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_TransitiveClosureState
+                      quickstep_utility_BitVector
                       quickstep_utility_Macros
                       quickstep_utility_Range
                       tmb)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/relational_operators/InitializeTransitiveClosureOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeTransitiveClosureOperator.cpp b/relational_operators/InitializeTransitiveClosureOperator.cpp
index a6ffe6f..ff21cf9 100644
--- a/relational_operators/InitializeTransitiveClosureOperator.cpp
+++ b/relational_operators/InitializeTransitiveClosureOperator.cpp
@@ -45,10 +45,15 @@ bool InitializeTransitiveClosureOperator::getAllWorkOrders(
   if (started_) {
     return true;
   }
+  started_ = true;
 
   TransitiveClosureState *state =
       query_context->getTransitiveClosureState(transitive_closure_context_index_);
 
+  if (state->range() == 0) {
+    return true;
+  }
+
   constexpr std::size_t kMinBatchSize = 1024ul * 1024ul * 4ul;
   const std::size_t range = state->range();
   const std::size_t num_batches =
@@ -66,7 +71,6 @@ bool InitializeTransitiveClosureOperator::getAllWorkOrders(
         op_index_);
   }
 
-  started_ = true;
   return true;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/relational_operators/TransitiveClosureOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TransitiveClosureOperator.cpp b/relational_operators/TransitiveClosureOperator.cpp
index e69de29..2d2776a 100644
--- a/relational_operators/TransitiveClosureOperator.cpp
+++ b/relational_operators/TransitiveClosureOperator.cpp
@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "relational_operators/TransitiveClosureOperator.hpp"
+
+#include <algorithm>
+#include <cstddef>
+
+#include "cli/Flags.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
+#include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
+#include "storage/TransitiveClosureState.hpp"
+#include "types/IntType.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/BitVector.hpp"
+#include "utility/Range.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+bool TransitiveClosureOperator::getAllWorkOrders(
+    WorkOrdersContainer *container,
+    QueryContext *query_context,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
+  if (started_) {
+    return true;
+  }
+  started_ = true;
+
+  TransitiveClosureState *state =
+      query_context->getTransitiveClosureState(transitive_closure_context_index_);
+
+  if (state->range() == 0) {
+    return true;
+  }
+
+  InsertDestination *output_destination =
+      query_context->getInsertDestination(output_destination_index_);
+
+  const std::size_t num_batches = std::min(state->range(), FLAGS_num_workers * 2);
+  const RangeSplitter splitter =
+      RangeSplitter::CreateWithNumPartitions(0, state->range(), num_batches);
+
+  for (std::size_t i = 0; i < splitter.getNumPartitions(); ++i) {
+    container->addNormalWorkOrder(
+        new TransitiveClosureWorkOrder(query_id_,
+                                       splitter.getPartition(i),
+                                       state,
+                                       output_destination),
+        op_index_);
+  }
+
+  return true;
+}
+
+bool TransitiveClosureOperator::getAllWorkOrderProtos(
+    WorkOrderProtosContainer *container)  {
+  LOG(FATAL) << "Not supported";
+}
+
+void TransitiveClosureWorkOrder::execute() {
+  std::vector<int> delta;
+  delta.reserve(range_);
+  BitVector<false> next(range_, false);
+  BitVector<false> result(range_, false);
+
+  const int kBulkInsertBatchSize = std::max(0x10000, state_->range());
+
+  std::shared_ptr<NativeColumnVector> src_cv =
+      std::make_shared<NativeColumnVector>(IntType::InstanceNonNullable(),
+                                           kBulkInsertBatchSize);
+  std::shared_ptr<NativeColumnVector> dst_cv =
+      std::make_shared<NativeColumnVector>(IntType::InstanceNonNullable(),
+                                           kBulkInsertBatchSize);
+
+  int total = 0;
+  for (int src = interval_.begin(); src < interval_.end(); ++src) {
+    if (state_->hasStart(src)) {
+      // Evaluate single source transitive closure.
+      evaluateSingleSource(src, &delta, &next, &result);
+
+      const int num_values = result.onesCount();
+      if (total + num_values > kBulkInsertBatchSize) {
+        bulkInsert(src_cv, dst_cv);
+        src_cv->clear();
+        dst_cv->clear();
+        total = 0;
+      }
+
+      std::size_t dst = -1;
+      for (int i = 0; i < num_values; ++i) {
+        *static_cast<int*>(src_cv->getPtrForDirectWrite()) = src;
+        dst = result.firstOne(dst + 1);
+        *static_cast<int*>(dst_cv->getPtrForDirectWrite()) = dst;
+      }
+
+      total += num_values;
+    }
+  }
+  if (total > 0) {
+    bulkInsert(src_cv, dst_cv);
+  }
+}
+
+void TransitiveClosureWorkOrder::evaluateSingleSource(
+    const int start,
+    std::vector<int> *delta,
+    BitVector<false> *next,
+    BitVector<false> *result) const {
+  delta->clear();
+  delta->emplace_back(start);
+  result->clear();
+
+  while (!delta->empty()) {
+    next->clear();
+    for (const int source : *delta) {
+      next->unionWith(state_->getEdgeData(source));
+    }
+    delta->clear();
+    next->subtractTo(*result, delta);
+    result->unionWith(*next);
+  }
+}
+
+void TransitiveClosureWorkOrder::bulkInsert(const ColumnVectorPtr &src_cv,
+                                            const ColumnVectorPtr &dst_cv) {
+  ColumnVectorsValueAccessor columns;
+  columns.addColumn(src_cv);
+  columns.addColumn(dst_cv);
+  output_destination_->bulkInsertTuples(&columns);
+}
+
+}  // namespace quickstep
+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/relational_operators/TransitiveClosureOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TransitiveClosureOperator.hpp b/relational_operators/TransitiveClosureOperator.hpp
index f983a4a..d4ac13c 100644
--- a/relational_operators/TransitiveClosureOperator.hpp
+++ b/relational_operators/TransitiveClosureOperator.hpp
@@ -30,6 +30,8 @@
 #include "relational_operators/WorkOrder.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/TransitiveClosureState.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "utility/BitVector.hpp"
 #include "utility/Macros.hpp"
 #include "utility/Range.hpp"
 
@@ -41,6 +43,8 @@ namespace tmb { class MessageBus; }
 
 namespace quickstep {
 
+class NativeColumnVector;
+class InsertDestination;
 class StorageManager;
 class WorkOrderProtosContainer;
 class WorkOrdersContainer;
@@ -49,9 +53,89 @@ class WorkOrdersContainer;
  *  @{
  */
 
+class TransitiveClosureOperator : public RelationalOperator {
+ public:
+  TransitiveClosureOperator(const std::size_t query_id,
+                            const std::size_t transitive_closure_context_index,
+                            const CatalogRelation &output_relation,
+                            const QueryContext::insert_destination_id output_destination_index)
+      : RelationalOperator(query_id, 1u),
+        transitive_closure_context_index_(transitive_closure_context_index),
+        output_relation_(output_relation),
+        output_destination_index_(output_destination_index),
+        started_(false) {
+  }
+
+  ~TransitiveClosureOperator() override {}
+
+  OperatorType getOperatorType() const override {
+    return kTransitiveClosure;
+  }
+
+  std::string getName() const override {
+    return "TransitiveClosureOperator";
+  }
+
+  bool getAllWorkOrders(WorkOrdersContainer *container,
+                        QueryContext *query_context,
+                        StorageManager *storage_manager,
+                        const tmb::client_id scheduler_client_id,
+                        tmb::MessageBus *bus) override;
+
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
+  QueryContext::insert_destination_id getInsertDestinationID() const override {
+    return output_destination_index_;
+  }
+
+  const relation_id getOutputRelationID() const override {
+    return output_relation_.getID();
+  }
+
+ private:
+  const std::size_t transitive_closure_context_index_;
+  const CatalogRelation &output_relation_;
+  const QueryContext::insert_destination_id output_destination_index_;
+
+  bool started_;
+
+  DISALLOW_COPY_AND_ASSIGN(TransitiveClosureOperator);
+};
+
+class TransitiveClosureWorkOrder : public WorkOrder {
+ public:
+  TransitiveClosureWorkOrder(const std::size_t query_id,
+                             const Range &interval,
+                             TransitiveClosureState *state,
+                             InsertDestination *output_destination)
+      : WorkOrder(query_id, 1u),
+        interval_(interval),
+        range_(state->range()),
+        state_(state),
+        output_destination_(output_destination) {}
+
+  ~TransitiveClosureWorkOrder() override {}
+
+  void execute() override;
+
+ private:
+  void evaluateSingleSource(const int start,
+                            std::vector<int> *delta,
+                            BitVector<false> *next,
+                            BitVector<false> *result) const;
+
+  void bulkInsert(const ColumnVectorPtr &src_cv, const ColumnVectorPtr &dst_cv);
+
+  const Range interval_;
+  const int range_;
+  TransitiveClosureState *state_;
+  InsertDestination *output_destination_;
+
+  DISALLOW_COPY_AND_ASSIGN(TransitiveClosureWorkOrder);
+};
 
 /** @} */
 
 }  // namespace quickstep
 
-#endif  // QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_TRANSITIVE_CLOSURE_OPERATOR_HPP_
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_TRANSITIVE_CLOSURE_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/storage/TransitiveClosureState.hpp
----------------------------------------------------------------------
diff --git a/storage/TransitiveClosureState.hpp b/storage/TransitiveClosureState.hpp
index 852972a..7cc06a8 100644
--- a/storage/TransitiveClosureState.hpp
+++ b/storage/TransitiveClosureState.hpp
@@ -69,6 +69,14 @@ class TransitiveClosureState {
     edges_[source]->setBit(destination);
   }
 
+  inline bool hasStart(const int value) {
+    return starts_->getBit(value);
+  }
+
+  inline const void* getEdgeData(const int source) const {
+    return edges_[source]->getData();
+  }
+
  private:
   const int range_;
   std::unique_ptr<BarrieredReadWriteConcurrentBitVector> starts_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index 5ef9871..029a409 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -172,6 +172,13 @@ class NativeColumnVector : public ColumnVector {
     return true;
   }
 
+  void clear() {
+    actual_length_ = 0;
+    if (null_bitmap_ != nullptr) {
+      null_bitmap_->clear();
+    }
+  }
+
   /**
    * @brief Determine if this NativeColumnVector's Type is nullable.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/utility/BarrieredReadWriteConcurrentBitVector.hpp
----------------------------------------------------------------------
diff --git a/utility/BarrieredReadWriteConcurrentBitVector.hpp b/utility/BarrieredReadWriteConcurrentBitVector.hpp
index 1dcb58e..b52aa9f 100644
--- a/utility/BarrieredReadWriteConcurrentBitVector.hpp
+++ b/utility/BarrieredReadWriteConcurrentBitVector.hpp
@@ -118,6 +118,13 @@ class BarrieredReadWriteConcurrentBitVector {
   }
 
   /**
+   * @return The underlying bytes of this bit vector.
+   **/
+  inline const void *getData() const {
+    return data_array_;
+  }
+
+  /**
    * @brief Clear this bit vector, setting all bits to zero.
    **/
   inline void clear() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2aefd7bc/utility/BitVector.hpp
----------------------------------------------------------------------
diff --git a/utility/BitVector.hpp b/utility/BitVector.hpp
index 4472407..e245dce 100644
--- a/utility/BitVector.hpp
+++ b/utility/BitVector.hpp
@@ -25,6 +25,7 @@
 #include <cstdlib>
 #include <cstring>
 #include <limits>
+#include <vector>
 
 #include "utility/BitManipulation.hpp"
 #include "utility/Macros.hpp"
@@ -75,7 +76,7 @@ class BitVector {
    *
    * @param num_bits The length of the BitVector in bits.
    **/
-  explicit BitVector(const std::size_t num_bits)
+  explicit BitVector(const std::size_t num_bits, const bool initialize = true)
       : owned_(true),
         short_version_(enable_short_version && (num_bits < 33)),
         // NOTE(chasseur): If 'num_bits' is 0, we put 'this' in 'data_array_'
@@ -86,7 +87,9 @@ class BitVector {
                                                             : this)),
         num_bits_(num_bits),
         data_array_size_((num_bits >> kHigherOrderShift) + (num_bits & kLowerOrderMask ? 1 : 0)) {
-    clear();
+    if (initialize) {
+      clear();
+    }
   }
 
   /**
@@ -855,6 +858,27 @@ class BitVector {
     return num_bits_;
   }
 
+  inline void unionWith(const void *other) {
+    DCHECK(!enable_short_version);
+    const std::size_t *other_data_array = static_cast<const std::size_t*>(other);
+    for (std::size_t array_idx = 0; array_idx < data_array_size_; ++array_idx) {
+      data_array_[array_idx] |= other_data_array[array_idx];
+    }
+  }
+
+  inline void subtractTo(const BitVector &other, std::vector<int> *output) {
+    DCHECK(!enable_short_version);
+    for (std::size_t array_idx = 0; array_idx < data_array_size_; ++array_idx) {
+      const std::size_t base = array_idx << kHigherOrderShift;
+      std::size_t value = data_array_[array_idx] & ~other.data_array_[array_idx];
+      while (value != 0) {
+        const std::size_t offset = leading_zero_count<std::size_t>(value);
+        value ^= TopBit<std::size_t>() >> offset;
+        output->emplace_back(base + offset);
+      }
+    }
+  }
+
  private:
   // This works as long as the bit-width of size_t is power of 2:
   static const std::size_t kLowerOrderMask = (sizeof(std::size_t) << 3) - 1;


[09/11] incubator-quickstep git commit: Initialize updates for transitive closure

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index 4800cde..6847a14 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -592,8 +592,8 @@ static void yynoreturn yy_fatal_error ( const char* msg , yyscan_t yyscanner );
 	yyg->yy_hold_char = *yy_cp; \
 	*yy_cp = '\0'; \
 	yyg->yy_c_buf_p = yy_cp;
-#define YY_NUM_RULES 164
-#define YY_END_OF_BUFFER 165
+#define YY_NUM_RULES 165
+#define YY_END_OF_BUFFER 166
 /* This struct is not used in this scanner,
    but its presence is necessary. */
 struct yy_trans_info
@@ -601,72 +601,74 @@ struct yy_trans_info
 	flex_int32_t yy_verify;
 	flex_int32_t yy_nxt;
 	};
-static const flex_int16_t yy_accept[589] =
+static const flex_int16_t yy_accept[593] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
-        0,    0,  165,    2,    2,  163,  163,  162,  161,  163,
-      140,  136,  139,  136,  136,  159,  132,  129,  133,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  137,    4,    5,    5,    3,  155,
-      155,  152,  156,  156,  150,  157,  157,  154,    1,  162,
-      130,  160,  159,  159,  159,    0,  134,  131,  135,  158,
-      158,  158,  158,   10,  158,  158,  158,   22,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  138,
-
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,   58,   67,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,   81,   82,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  113,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,    4,    5,    3,  155,  151,  156,
-      149,  149,  141,  143,  144,  145,  146,  147,  148,  149,
-      157,  153,  160,  159,    0,  159,    6,    7,  158,    9,
-       11,  158,  158,   15,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,   33,  158,  158,  158,  158,
-
-      158,  158,  158,  158,   43,  158,  158,  158,  158,  158,
-      158,   50,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,   62,  158,   69,  158,  158,  158,  158,  158,  158,
-      158,   77,  158,   80,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,   98,  158,  158,
-      103,  104,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  141,
-      143,  142,  158,  158,  158,  158,  158,  158,  158,   20,
-       23,  158,  158,  158,   28,  158,  158,  158,   31,  158,
-      158,  158,   37,  158,  158,   41,   42,  158,  158,  158,
-
-      158,  158,  158,  158,   52,   53,  158,   55,  158,   57,
-      158,  158,  158,  158,   66,   68,   70,   71,   72,  158,
-       74,  158,  158,   78,  158,  158,   85,  158,  158,  158,
-      158,  158,   92,  158,   94,  158,  158,  158,  100,  158,
-      158,  158,  158,  158,  158,  158,  158,  110,  111,  114,
-      158,  158,  158,  158,  158,  158,  158,  158,  123,  158,
-      158,  126,  127,  141,  142,    8,  158,  158,  158,  158,
-      158,  158,  158,   25,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,   46,   47,
-       48,  158,  158,   54,  158,   59,   60,  158,  158,  158,
-
-       73,  158,   76,   79,   83,   84,  158,  158,  158,  158,
-      158,   93,  158,  158,   97,  158,  158,  158,  158,  158,
-      158,  158,  109,  158,  158,  158,  117,  158,  158,  120,
-      158,  158,  124,  158,  158,  158,  158,   14,  158,  158,
-      158,  158,  158,   26,  158,   29,  158,  158,  158,  158,
-      158,   36,  158,  158,   40,   44,  158,  158,  158,   56,
-       61,  158,  158,  158,   75,  158,  158,  158,  158,  158,
-      158,   96,  158,  101,  102,  158,  106,  107,  158,  158,
-      158,  158,  118,  119,  121,  158,  125,  158,  158,   13,
-      158,  158,  158,  158,  158,  158,   21,   30,  158,   34,
-
-       35,  158,  158,   45,  158,   51,   63,  158,  158,  158,
-       88,  158,   90,  158,  158,  158,  158,  158,  158,  158,
-      158,  122,  158,  158,  158,  158,  158,  158,  158,  158,
-       32,  158,   39,  158,  158,   65,  158,  158,   91,  158,
-      158,  105,  158,  158,  158,  158,  158,   12,  158,  158,
-      158,  158,   24,  158,  158,   49,   64,   86,   89,  158,
-      158,  108,  112,  158,  116,  128,   16,  158,  158,  158,
-       27,   38,   87,   95,  158,  158,  158,   18,   19,  158,
-      115,  158,  158,  158,   99,  158,   17,    0
+        0,    0,  166,    2,    2,  164,  164,  163,  162,  164,
+      141,  137,  140,  137,  137,  160,  133,  130,  134,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  138,    4,    5,    5,    3,  156,
+      156,  153,  157,  157,  151,  158,  158,  155,    1,  163,
+      131,  161,  160,  160,  160,    0,  135,  132,  136,  159,
+      159,  159,  159,   10,  159,  159,  159,   20,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  139,
+
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,   59,   68,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,   82,   83,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  114,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,    4,    5,    3,  156,  152,  157,
+      150,  150,  142,  144,  145,  146,  147,  148,  149,  150,
+      158,  154,  161,  160,    0,  160,    6,    7,  159,    9,
+       11,  159,  159,   15,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,   34,  159,  159,  159,  159,
+
+      159,  159,  159,  159,   44,  159,  159,  159,  159,  159,
+      159,   51,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,   63,  159,   70,  159,  159,  159,  159,  159,  159,
+      159,   78,  159,   81,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,   99,  159,  159,
+      104,  105,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  142,
+      144,  143,  159,  159,  159,  159,  159,  159,  159,   21,
+       22,  159,  159,  159,  159,   28,  159,  159,  159,   32,
+      159,  159,  159,   38,  159,  159,   42,   43,  159,  159,
+
+      159,  159,  159,  159,  159,   53,   54,  159,   56,  159,
+       58,  159,  159,  159,  159,   67,   69,   71,   72,   73,
+      159,   75,  159,  159,   79,  159,  159,   86,  159,  159,
+      159,  159,  159,   93,  159,   95,  159,  159,  159,  101,
+      159,  159,  159,  159,  159,  159,  159,  159,  111,  112,
+      115,  159,  159,  159,  159,  159,  159,  159,  159,  124,
+      159,  159,  127,  128,  142,  143,    8,  159,  159,  159,
+      159,  159,  159,  159,   24,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  159,
+       47,   48,   49,  159,  159,   55,  159,   60,   61,  159,
+
+      159,  159,   74,  159,   77,   80,   84,   85,  159,  159,
+      159,  159,  159,   94,  159,  159,   98,  159,  159,  159,
+      159,  159,  159,  159,  110,  159,  159,  159,  118,  159,
+      159,  121,  159,  159,  125,  159,  159,  159,  159,   14,
+      159,  159,  159,  159,  159,   25,  159,  159,   29,  159,
+      159,  159,  159,  159,   37,  159,  159,   41,   45,  159,
+      159,  159,   57,   62,  159,  159,  159,   76,  159,  159,
+      159,  159,  159,  159,   97,  159,  102,  103,  159,  107,
+      108,  159,  159,  159,  159,  119,  120,  122,  159,  126,
+      159,  159,   13,  159,  159,  159,  159,  159,   26,  159,
+
+       30,   31,  159,   35,   36,  159,  159,   46,  159,   52,
+       64,  159,  159,  159,   89,  159,   91,  159,  159,  159,
+      159,  159,  159,  159,  159,  123,  159,  159,  159,  159,
+      159,  159,  159,  159,   33,  159,   40,  159,  159,   66,
+      159,  159,   92,  159,  159,  106,  159,  159,  159,  159,
+      159,   12,  159,  159,  159,  159,   23,  159,  159,   50,
+       65,   87,   90,  159,  159,  109,  113,  159,  117,  129,
+       16,  159,  159,  159,   27,   39,   88,   96,  159,  159,
+      159,   18,   19,  159,  116,  159,  159,  159,  100,  159,
+       17,    0
+
     } ;
 
 static const YY_CHAR yy_ec[256] =
@@ -713,155 +715,155 @@ static const YY_CHAR yy_meta[72] =
         8
     } ;
 
-static const flex_int16_t yy_base[604] =
+static const flex_int16_t yy_base[608] =
     {   0,
         0,    1,   46,    0,  117,  162,    2,    3,  127,  128,
-        6,   10,  147, 1316, 1316,    0, 1316,   13, 1316,  130,
-     1316, 1316, 1316,  129,    6,  129,    4, 1316,   28,  124,
+        6,   10,  147, 1320, 1320,    0, 1320,   13, 1320,  130,
+     1320, 1320, 1320,  129,    6,  129,    4, 1320,   28,  124,
       159,  213,  165,  167,  263,   92,  158,  163,   96,  107,
       214,  160,  186,  219,  221,  155,  281,  274,  325,  257,
-      186,  209,    0,  219, 1316,   27,    4,   19,    0,    0,
+      186,  209,    0,  219, 1320,   27,    4,   19,    0,    0,
         0,   17,    0,    0,  389,    0,    0,    8,    0,   22,
-     1316,    0,  293,  325,  343,   18, 1316, 1316, 1316,    0,
+     1320,    0,  293,  325,  343,   18, 1320, 1320, 1320,    0,
       223,  265,  234,  242,  260,  292,  288,    0,  299,  330,
-      337,  324,  334,  324,  325,  380,  325,  331,  346, 1316,
+      337,  324,  334,  324,  325,  380,  325,  331,  346, 1320,
 
       348,  364,  378,  376,  371,  378,  382,  386,  390,  389,
       386,  385,  435,    0,  402,  389,  400,  435,  433,  431,
       433,  436,  431,  440,  447,    0,  452,  437,  453,  441,
       442,  456,  453,  449,  465,  457,  444,  494,  468,  495,
       500,  501,  499,  492,    0,  486,  492,  507,  506,  502,
-      500,  508,  501,  516,    0,   29,    0,    0, 1316,    0,
-     1316, 1316,   22,   24, 1316, 1316, 1316, 1316, 1316,    0,
-        0, 1316,    0,  524,   26,   28,    0,    0,  517,    0,
-      518,  501,  516,  504,  545,  525,  531,  552,  536,  542,
-      537,  562,  544,  547,  561,    0,  558,  567,  564,  567,
-
-      551,  570,  557,  569,    0,  556,  558,  560,  561,  580,
-      570,  578,  572,  575,  567,  598,  598,  595,  610,  613,
-      614,  615,  607,    0,  602,  603,  619,  616,  619,  606,
-      608,    0,  617,    0,  626,  627,  615,  614,  634,  635,
-      626,  620,  636,  633,  641,  659,  657,  652,  658,  671,
-        0,  665,  673,  660,  668,  668,  678,  679,  673,  671,
-      672,  689,  677,  671,  692,  683,  692,  690,  681,   30,
-      125,    0,  685,  690,  705,  708,  718,  718,  718,    0,
-      733,  724,  723,  717,    0,  718,  722,  736,  722,  730,
-      723,  725,  741,  738,  736,    0,    0,  729,  749,  748,
-
-      734,  735,  741,  748,    0,    0,  743,    0,  747,    0,
-      738,  750,  762,  774,    0,    0,    0,    0,    0,  767,
-        0,  769,  785,  775,  777,  778,    0,  789,  794,  795,
-      800,  784,    0,  798,    0,  786,  781,  786,    0,  803,
-      794,  808,  800,  795,  793,  795,  812,    0,  800,    0,
-      815,  805,  824,  818,  825,  840,  845,  843,    0,  847,
-      838,    0,  841,  131, 1316,    0,  852,  852,  838,  858,
-      844,  855,  859,    0,  850,  847,  861,  864,  856,  862,
-      871,  861,  870,  863,  864,  879,  877,  894,    0,    0,
-        0,  880,  898,    0,  901,    0,    0,  889,  905,  892,
-
-        0,  907,    0,    0,    0,    0,  895,  902,  913,  900,
-      910,    0,  915,  905,    0,  917,  919,  904,  918,  910,
-      909,  912,    0,  911,  914,  921,    0,  931,  937,    0,
-      935,  954,    0,  938,  948,  957,  953,    0,  946,  951,
-      969,  963,  953,    0,  973,    0,  970,  956,  964,  966,
-      959,    0,  976,  978,    0,    0,  962,  976,  972,    0,
-        0,  969,  983,  988,    0,  982,  973,  985,  975,  992,
-      999,    0, 1007,    0,    0, 1007,    0,    0, 1015, 1024,
-     1025, 1023,    0,    0,    0, 1010,    0, 1016, 1017,    0,
-     1023, 1018, 1021, 1023, 1031, 1028,    0,    0, 1033,    0,
-
-        0, 1030, 1020,    0, 1029,    0,    0, 1041, 1033, 1031,
-        0, 1033,    0, 1024, 1048, 1043, 1038, 1056, 1058, 1064,
-     1074,    0, 1062, 1076, 1070, 1069, 1070, 1068, 1071, 1076,
-        0, 1077,    0, 1085, 1073,    0, 1080, 1088,    0, 1091,
-     1084,    0, 1091, 1085, 1086, 1099, 1096,    0, 1098, 1102,
-     1097, 1105,    0, 1096, 1121,    0,    0, 1110,    0, 1116,
-     1128,    0,    0, 1128,    0,    0,    0, 1123, 1137, 1125,
-        0,    0,    0,    0, 1124, 1141, 1127,    0,    0, 1143,
-        0, 1140, 1132, 1146,    0, 1133,    0, 1316, 1198, 1208,
-     1218, 1228, 1238, 1242, 1245, 1251, 1261, 1271, 1281, 1291,
-
-     1301, 1306, 1308
+      500,  508,  501,  516,    0,   29,    0,    0, 1320,    0,
+     1320, 1320,   22,   24, 1320, 1320, 1320, 1320, 1320,    0,
+        0, 1320,    0,  524,   26,   28,    0,    0,  517,    0,
+      518,  501,  516,  504,  545,  525,  531,  552,  536,  547,
+      537,  562,  544,  547,  562,    0,  559,  568,  565,  568,
+
+      552,  571,  558,  570,    0,  557,  561,  561,  562,  581,
+      571,  580,  574,  576,  585,  599,  604,  597,  613,  614,
+      615,  616,  608,    0,  603,  604,  620,  617,  620,  607,
+      609,    0,  618,    0,  627,  628,  616,  617,  635,  636,
+      628,  620,  638,  634,  659,  660,  663,  654,  661,  672,
+        0,  666,  674,  661,  669,  668,  679,  680,  674,  672,
+      673,  690,  678,  674,  693,  683,  694,  691,  685,   30,
+      125,    0,  686,  698,  717,  709,  724,  720,  721,    0,
+      734,  725,  724,  733,  719,    0,  720,  724,  738,  724,
+      732,  725,  727,  743,  740,  738,    0,    0,  733,  753,
+
+      750,  736,  737,  744,  752,    0,    0,  746,    0,  756,
+        0,  756,  763,  769,  782,    0,    0,    0,    0,    0,
+      772,    0,  773,  786,  776,  778,  779,    0,  789,  796,
+      797,  802,  786,    0,  800,    0,  788,  783,  788,    0,
+      807,  798,  810,  802,  797,  796,  799,  815,    0,  809,
+        0,  833,  818,  831,  826,  830,  844,  847,  845,    0,
+      849,  840,    0,  843,  131, 1320,    0,  854,  854,  840,
+      860,  846,  857,  861,    0,  852,  866,  852,  866,  867,
+      859,  866,  876,  866,  881,  882,  883,  898,  886,  905,
+        0,    0,    0,  886,  903,    0,  904,    0,    0,  892,
+
+      908,  895,    0,  911,    0,    0,    0,    0,  898,  905,
+      916,  903,  913,    0,  918,  908,    0,  922,  924,  909,
+      921,  913,  913,  917,    0,  916,  925,  940,    0,  950,
+      956,    0,  944,  965,    0,  944,  953,  960,  956,    0,
+      949,  954,  973,  966,  956,    0,  957,  977,    0,  974,
+      960,  968,  970,  965,    0,  982,  984,    0,    0,  968,
+      980,  977,    0,    0,  975,  989, 1001,    0, 1003,  993,
+     1010,  996, 1004, 1011,    0, 1014,    0,    0, 1013,    0,
+        0, 1019, 1028, 1029, 1028,    0,    0,    0, 1015,    0,
+     1020, 1021,    0, 1027, 1022, 1025, 1027, 1035,    0, 1034,
+
+        0,    0, 1039,    0,    0, 1036, 1026,    0, 1033,    0,
+        0, 1046, 1039, 1037,    0, 1046,    0, 1045, 1068, 1068,
+     1059, 1068, 1070, 1071, 1080,    0, 1066, 1080, 1074, 1074,
+     1075, 1072, 1075, 1080,    0, 1081,    0, 1089, 1077,    0,
+     1084, 1094,    0, 1097, 1090,    0, 1097, 1089, 1091, 1105,
+     1102,    0, 1111, 1123, 1117, 1130,    0, 1117, 1133,    0,
+        0, 1122,    0, 1123, 1134,    0,    0, 1132,    0,    0,
+        0, 1127, 1141, 1130,    0,    0,    0,    0, 1129, 1145,
+     1131,    0,    0, 1147,    0, 1144, 1136, 1150,    0, 1137,
+        0, 1320, 1202, 1212, 1222, 1232, 1242, 1246, 1249, 1255,
+
+     1265, 1275, 1285, 1295, 1305, 1310, 1312
     } ;
 
-static const flex_int16_t yy_def[604] =
+static const flex_int16_t yy_def[608] =
     {   0,
-      589,  589,  588,    3,  590,  590,  591,  591,  592,  592,
-      593,  593,  588,  588,  588,  594,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  588,  588,  588,  588,  596,  597,
-      597,  588,  598,  598,  599,  600,  600,  588,  594,  588,
-      588,  601,  588,  588,  588,  588,  588,  588,  588,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  588,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  588,  588,  596,  597,  588,  598,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  602,
-      600,  588,  601,  588,  588,  588,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  588,
-      588,  603,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  588,  588,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,    0,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-
-      588,  588,  588
+      593,  593,  592,    3,  594,  594,  595,  595,  596,  596,
+      597,  597,  592,  592,  592,  598,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  592,  592,  592,  592,  600,  601,
+      601,  592,  602,  602,  603,  604,  604,  592,  598,  592,
+      592,  605,  592,  592,  592,  592,  592,  592,  592,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  592,
+
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  592,  592,  600,  601,  592,  602,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  606,
+      604,  592,  605,  592,  592,  592,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  592,
+      592,  607,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  592,  592,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,    0,  592,  592,  592,  592,  592,  592,  592,  592,
+
+      592,  592,  592,  592,  592,  592,  592
     } ;
 
-static const flex_int16_t yy_nxt[1388] =
+static const flex_int16_t yy_nxt[1392] =
     {   0,
-      588,  155,   15,   15,   61,   61,  156,  156,   67,   62,
+      592,  155,   15,   15,   61,   61,  156,  156,   67,   62,
        62,   68,   67,  172,   70,   68,   70,   73,   73,   77,
        78,  156,  156,   70,  159,   70,  175,  175,  155,  176,
       176,  156,  156,  270,  271,  271,  271,  176,  176,  176,
-      176,  364,  271,   79,   16,   16,   17,   18,   19,   18,
+      176,  365,  271,   79,   16,   16,   17,   18,   19,   18,
        20,   21,   22,   23,   22,   24,   25,   26,   26,   17,
        27,   28,   29,   30,   31,   32,   33,   34,   35,   36,
        37,   38,   39,   40,   41,   42,   43,   44,   45,   46,
@@ -872,45 +874,45 @@ static const flex_int16_t yy_nxt[1388] =
        48,   49,   50,   51,   52,   53,   54,   17,   56,   57,
        58,   17,   17,   17,   17,   17,  110,  115,  116,   64,
        64,   17,   17,   17,   62,   62,  271,  271,   72,   74,
-       75,   75,  271,  271,   81,   71,  588,  588,  588,  588,
-       76,  588,   82,  588,   83,  110,  115,  116,  588,   84,
+       75,   75,  271,  271,   81,   71,  592,  592,  592,  592,
+       76,  592,   82,  592,   83,  110,  115,  116,  592,   84,
        17,   17,   17,   56,   57,   58,   17,   17,   17,   17,
        17,   65,   65,   81,  100,  111,   17,   17,   17,   76,
        85,   82,   95,   83,   86,  121,   96,   87,   84,  112,
-       97,  122,  133,  113,  588,  101,   98,  102,  114,   99,
+       97,  122,  133,  113,  592,  101,   98,  102,  114,   99,
 
-       88,  588,  588,  151,  111,   17,   17,  103,  588,   85,
-      588,   95,  588,   86,  121,   96,   87,  123,  112,   97,
+       88,  592,  592,  151,  111,   17,   17,  103,  592,   85,
+      592,   95,  592,   86,  121,   96,   87,  123,  112,   97,
       122,  133,  113,  124,  101,   98,  102,  114,   99,   88,
        89,  117,  151,  152,  153,  118,  103,   90,  130,  119,
       154,  125,  131,  177,   91,  120,  123,   92,   93,  126,
-       94,  588,  124,  127,  180,  132,  128,  129,  588,   89,
-      117,  181,  152,  153,  118,  588,   90,  130,  119,  154,
-      125,  131,  177,   91,  120,  588,   92,   93,  126,   94,
-      104,  588,  127,  180,  132,  128,  129,  148,  105,  149,
-      181,  106,  150,  178,  107,  138,  182,  108,  134,  588,
-
-      109,  179,  135,  139,   73,   73,  136,  588,  588,  104,
-      140,  141,  137,  588,   76,  183,  148,  105,  149,  185,
+       94,  592,  124,  127,  180,  132,  128,  129,  592,   89,
+      117,  181,  152,  153,  118,  592,   90,  130,  119,  154,
+      125,  131,  177,   91,  120,  592,   92,   93,  126,   94,
+      104,  592,  127,  180,  132,  128,  129,  148,  105,  149,
+      181,  106,  150,  178,  107,  138,  182,  108,  134,  592,
+
+      109,  179,  135,  139,   73,   73,  136,  592,  592,  104,
+      140,  141,  137,  592,   76,  183,  148,  105,  149,  185,
       106,  150,  178,  107,  138,  182,  108,  134,  184,  109,
-      179,  135,  139,  588,  186,  136,  174,  174,  588,  140,
+      179,  135,  139,  592,  186,  136,  174,  174,  592,  140,
       141,  137,  142,   76,  183,  192,   76,  187,  185,  143,
       144,  188,  193,   74,   75,   75,  145,  184,  194,  146,
       201,  195,  147,  186,   76,  189,  196,  190,  202,  191,
-      588,  142,  588,  588,  192,   76,  187,  203,  143,  144,
-      188,  193,  588,  204,  205,  145,  588,  194,  146,  201,
+      592,  142,  592,  592,  192,   76,  187,  203,  143,  144,
+      188,  193,  592,  204,  205,  145,  592,  194,  146,  201,
       195,  147,  162,   76,  189,  196,  190,  202,  191,  197,
 
       163,  164,  198,  206,  208,  209,  203,  165,  199,  210,
       211,  166,  204,  205,  207,  200,  212,  213,  214,  167,
-      215,  216,  218,  168,  217,  169,  588,  223,  197,  170,
+      215,  216,  218,  168,  217,  169,  592,  223,  197,  170,
       224,  198,  206,  208,  209,  225,  165,  199,  210,  211,
-      166,  588,  588,  207,  200,  212,  213,  214,  167,  215,
+      166,  592,  592,  207,  200,  212,  213,  214,  167,  215,
       216,  218,  168,  217,  169,  219,  223,  226,  170,  224,
       227,  229,  228,  230,  225,  220,  231,  232,  233,  234,
       221,  222,  235,  236,  237,  238,  239,  240,  242,  243,
       247,  241,  244,  248,  219,  252,  226,  245,  246,  227,
-      229,  228,  230,  588,  220,  231,  232,  233,  234,  221,
+      229,  228,  230,  592,  220,  231,  232,  233,  234,  221,
 
       222,  235,  236,  237,  238,  239,  240,  242,  243,  247,
       241,  244,  248,  249,  252,  253,  245,  246,  254,  255,
@@ -919,99 +921,101 @@ static const flex_int16_t yy_nxt[1388] =
       275,  276,  249,  277,  253,   76,  280,  254,  255,  256,
       257,  250,  258,  259,  260,  262,  263,  264,  266,  251,
       267,  261,  269,  265,  278,  281,  268,  273,  274,  275,
-      276,  282,  277,  283,   76,  280,  279,  284,  285,  286,
-      287,  288,  289,  290,  291,  292,  293,  294,  295,  296,
+      276,  282,  277,  283,   76,  280,  279,  284,  286,  287,
+      288,  289,  285,  290,  291,  292,  293,  294,  295,  296,
       297,  298,  299,  278,  281,  300,  301,  302,  303,  304,
 
-      282,  305,  283,  306,  307,  279,  284,  285,  286,  287,
-      288,  289,  290,  291,  292,  293,  294,  295,  296,  297,
+      282,  305,  283,  306,  307,  279,  284,  286,  287,  288,
+      289,  285,  290,  291,  292,  293,  294,  295,  296,  297,
       298,  299,  308,  309,  300,  301,  302,  303,  304,  310,
-      305,  311,  306,  307,  312,  313,  314,  316,  317,  318,
-      319,  320,  321,  322,  323,  324,  315,  325,  326,  327,
-      328,  308,  309,  329,  330,  331,  333,  332,  310,  334,
-      311,  335,  336,  312,  313,  314,  316,  317,  318,  319,
-      320,  321,  322,  323,  324,  315,  325,  326,  327,  328,
-      337,  338,  329,  330,  331,  333,  332,  339,  334,  341,
-      335,  336,  342,  343,  344,  346,  347,  340,  348,  349,
-
-      350,  351,  352,  353,  345,  354,  355,  356,  357,  337,
-      338,  358,  361,  359,  362,  363,  339,  360,  341,  366,
-      367,  342,  343,  344,  346,  347,  368,  348,  349,  350,
-      351,  352,  353,  345,  354,  355,  356,  357,  369,  370,
-      358,  361,  359,  362,  363,  371,  360,  372,  366,  367,
-      373,  374,  375,  376,  377,  368,  378,  379,  380,  381,
+      305,  311,  306,  307,  312,  313,  314,  315,  317,  318,
+      319,  320,  321,  322,  323,  324,  325,  316,  326,  327,
+      328,  308,  309,  329,  330,  331,  334,  332,  310,  333,
+      311,  335,  336,  312,  313,  314,  315,  317,  318,  319,
+      320,  321,  322,  323,  324,  325,  316,  326,  327,  328,
+      337,  338,  329,  330,  331,  334,  332,  339,  333,  340,
+      335,  336,  342,  343,  344,  345,  347,  348,  349,  341,
+
+      350,  351,  352,  353,  354,  346,  355,  356,  357,  337,
+      338,  358,  359,  360,  362,  363,  339,  361,  340,  364,
+      367,  342,  343,  344,  345,  347,  348,  349,  368,  350,
+      351,  352,  353,  354,  346,  355,  356,  357,  369,  370,
+      358,  359,  360,  362,  363,  371,  361,  372,  364,  367,
+      373,  374,  375,  376,  377,  378,  379,  368,  380,  381,
       382,  383,  384,  385,  386,  387,  388,  369,  370,  389,
       390,  391,  392,  393,  371,  394,  372,  395,  396,  373,
-      374,  375,  376,  377,  397,  378,  379,  380,  381,  382,
+      374,  375,  376,  377,  378,  379,  397,  380,  381,  382,
       383,  384,  385,  386,  387,  388,  398,  399,  389,  390,
 
-      391,  392,  393,  401,  394,  402,  395,  396,  400,  403,
-      404,  405,  406,  397,  407,  408,  409,  410,  411,  412,
+      391,  392,  393,  400,  394,  401,  395,  396,  403,  404,
+      405,  406,  407,  408,  409,  397,  402,  410,  411,  412,
       413,  414,  415,  416,  417,  398,  399,  418,  419,  420,
-      421,  422,  401,  423,  402,  424,  425,  400,  403,  404,
-      405,  406,  426,  407,  408,  409,  410,  411,  412,  413,
+      421,  422,  400,  423,  401,  424,  425,  403,  404,  405,
+      406,  407,  408,  409,  426,  402,  410,  411,  412,  413,
       414,  415,  416,  417,  427,  428,  418,  419,  420,  421,
       422,  429,  423,  430,  424,  425,  431,  432,  433,  434,
-      435,  426,  436,  437,  438,  439,  440,  442,  443,  441,
-      444,  445,  446,  427,  428,  447,  448,  449,  450,  451,
+      435,  436,  437,  426,  438,  439,  440,  441,  442,  444,
+      445,  443,  446,  427,  428,  447,  448,  449,  450,  451,
       429,  452,  430,  453,  454,  431,  432,  433,  434,  435,
 
-      455,  436,  437,  438,  439,  440,  442,  443,  441,  444,
-      445,  446,  456,  457,  447,  448,  449,  450,  451,  458,
-      452,  459,  453,  454,  460,  461,  462,  463,  465,  455,
-      464,  466,  467,  468,  469,  470,  471,  472,  473,  474,
+      436,  437,  455,  438,  439,  440,  441,  442,  444,  445,
+      443,  446,  456,  457,  447,  448,  449,  450,  451,  458,
+      452,  459,  453,  454,  460,  461,  462,  463,  464,  465,
+      466,  455,  468,  467,  469,  470,  471,  472,  473,  474,
       475,  456,  457,  476,  477,  478,  479,  480,  458,  481,
-      459,  482,  483,  460,  461,  462,  463,  465,  484,  464,
-      466,  467,  468,  469,  470,  471,  472,  473,  474,  475,
+      459,  482,  483,  460,  461,  462,  463,  464,  465,  466,
+      484,  468,  467,  469,  470,  471,  472,  473,  474,  475,
       485,  486,  476,  477,  478,  479,  480,  487,  481,  488,
-      482,  483,  489,  490,  491,  492,  493,  484,  494,  495,
+      482,  483,  489,  490,  491,  492,  493,  494,  495,  484,
       496,  497,  498,  499,  500,  501,  502,  503,  504,  485,
 
       486,  505,  506,  507,  508,  509,  487,  510,  488,  511,
-      512,  489,  490,  491,  492,  493,  513,  494,  495,  496,
+      512,  489,  490,  491,  492,  493,  494,  495,  513,  496,
       497,  498,  499,  500,  501,  502,  503,  504,  514,  515,
       505,  506,  507,  508,  509,  516,  510,  517,  511,  512,
-      518,  519,  520,  521,  522,  513,  523,  524,  525,  526,
+      518,  519,  520,  521,  522,  523,  524,  513,  525,  526,
       527,  528,  529,  530,  531,  532,  533,  514,  515,  534,
       535,  536,  537,  538,  516,  539,  517,  540,  541,  518,
-      519,  520,  521,  522,  542,  523,  524,  525,  526,  527,
+      519,  520,  521,  522,  523,  524,  542,  525,  526,  527,
       528,  529,  530,  531,  532,  533,  543,  544,  534,  535,
       536,  537,  538,  545,  539,  546,  540,  541,  547,  548,
 
-      549,  550,  551,  542,  552,  553,  554,  555,  556,  557,
+      549,  550,  551,  552,  553,  542,  554,  555,  556,  557,
       558,  559,  560,  561,  562,  543,  544,  563,  564,  565,
       566,  567,  545,  568,  546,  569,  570,  547,  548,  549,
-      550,  551,  571,  552,  553,  554,  555,  556,  557,  558,
+      550,  551,  552,  553,  571,  554,  555,  556,  557,  558,
       559,  560,  561,  562,  572,  573,  563,  564,  565,  566,
       567,  574,  568,  575,  569,  570,  576,  577,  578,  579,
-      580,  571,  581,  582,  583,  584,  585,  586,  587,  588,
-      588,  588,  588,  572,  573,  588,  588,  588,  588,  588,
-      574,  588,  575,  588,  588,  576,  577,  578,  579,  580,
-      588,  581,  582,  583,  584,  585,  586,  587,   14,   14,
-
-       14,   14,   14,   14,   14,   14,   14,   14,   59,   59,
-       59,   59,   59,   59,   59,   59,   59,   59,   60,   60,
-       60,   60,   60,   60,   60,   60,   60,   60,   63,   63,
-       63,   63,   63,   63,   63,   63,   63,   63,   66,   66,
-       66,   66,   66,   66,   66,   66,   66,   66,   69,   69,
-       80,   80,   80,  588,   80,  157,  157,  157,  157,  588,
-      157,  158,  158,  158,  588,  158,  158,  158,  158,  158,
-      158,  160,  160,  160,  588,  160,  160,  160,  160,  588,
-      160,  161,  161,  161,  161,  161,  161,  161,  161,  161,
-      161,  171,  171,  588,  171,  171,  171,  171,  171,  171,
-
-      171,  173,  588,  173,  173,  173,  173,  173,  173,  173,
-      173,  272,  272,  365,  365,   13,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588
+      580,  581,  582,  571,  583,  584,  585,  586,  587,  588,
+      589,  590,  591,  572,  573,  592,  592,  592,  592,  592,
+      574,  592,  575,  592,  592,  576,  577,  578,  579,  580,
+      581,  582,  592,  583,  584,  585,  586,  587,  588,  589,
+
+      590,  591,   14,   14,   14,   14,   14,   14,   14,   14,
+       14,   14,   59,   59,   59,   59,   59,   59,   59,   59,
+       59,   59,   60,   60,   60,   60,   60,   60,   60,   60,
+       60,   60,   63,   63,   63,   63,   63,   63,   63,   63,
+       63,   63,   66,   66,   66,   66,   66,   66,   66,   66,
+       66,   66,   69,   69,   80,   80,   80,  592,   80,  157,
+      157,  157,  157,  592,  157,  158,  158,  158,  592,  158,
+      158,  158,  158,  158,  158,  160,  160,  160,  592,  160,
+      160,  160,  160,  592,  160,  161,  161,  161,  161,  161,
+      161,  161,  161,  161,  161,  171,  171,  592,  171,  171,
+
+      171,  171,  171,  171,  171,  173,  592,  173,  173,  173,
+      173,  173,  173,  173,  173,  272,  272,  366,  366,   13,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592
+
     } ;
 
-static const flex_int16_t yy_chk[1388] =
+static const flex_int16_t yy_chk[1392] =
     {   0,
         0,  155,    1,    2,    7,    8,   57,   57,   11,    7,
         8,   11,   12,   68,   18,   12,   18,   25,   25,   27,
@@ -1028,7 +1032,7 @@ static const flex_int16_t yy_chk[1388] =
         3,    3,    3,    3,    3,    3,    3,    5,    5,    5,
         5,    5,    5,    5,    5,    5,   36,   39,   40,    9,
        10,    5,    5,    5,    9,   10,  271,  271,   24,   26,
-       26,   26,  364,  364,   30,   20,   13,    0,    0,    0,
+       26,   26,  365,  365,   30,   20,   13,    0,    0,    0,
        26,    0,   30,    0,   30,   36,   39,   40,    0,   30,
         5,    5,    6,    6,    6,    6,    6,    6,    6,    6,
         6,    9,   10,   30,   34,   37,    6,    6,    6,   26,
@@ -1076,99 +1080,101 @@ static const flex_int16_t yy_chk[1388] =
       144,  138,  146,  147,  148,  149,  150,  151,  152,  138,
       153,  148,  154,  151,  185,  187,  153,  179,  181,  182,
       183,  188,  184,  189,  174,  186,  185,  190,  191,  192,
-      193,  194,  195,  197,  198,  199,  200,  201,  202,  203,
-      204,  206,  207,  185,  187,  208,  209,  210,  211,  212,
-
-      188,  213,  189,  214,  215,  185,  190,  191,  192,  193,
-      194,  195,  197,  198,  199,  200,  201,  202,  203,  204,
-      206,  207,  216,  217,  208,  209,  210,  211,  212,  218,
-      213,  219,  214,  215,  220,  221,  222,  223,  225,  226,
+      193,  194,  190,  195,  197,  198,  199,  200,  201,  202,
+      203,  204,  206,  185,  187,  207,  208,  209,  210,  211,
+
+      188,  212,  189,  213,  214,  185,  190,  191,  192,  193,
+      194,  190,  195,  197,  198,  199,  200,  201,  202,  203,
+      204,  206,  215,  216,  207,  208,  209,  210,  211,  217,
+      212,  218,  213,  214,  219,  220,  221,  222,  223,  225,
+      226,  227,  228,  229,  230,  231,  233,  222,  235,  236,
+      237,  215,  216,  238,  239,  240,  242,  241,  217,  241,
+      218,  243,  244,  219,  220,  221,  222,  223,  225,  226,
       227,  228,  229,  230,  231,  233,  222,  235,  236,  237,
-      238,  216,  217,  239,  240,  241,  242,  241,  218,  243,
-      219,  244,  245,  220,  221,  222,  223,  225,  226,  227,
-      228,  229,  230,  231,  233,  222,  235,  236,  237,  238,
-      246,  247,  239,  240,  241,  242,  241,  248,  243,  249,
-      244,  245,  250,  252,  253,  254,  255,  248,  256,  257,
-
-      258,  259,  260,  261,  253,  261,  262,  263,  264,  246,
-      247,  265,  267,  266,  268,  269,  248,  266,  249,  273,
-      274,  250,  252,  253,  254,  255,  275,  256,  257,  258,
-      259,  260,  261,  253,  261,  262,  263,  264,  276,  277,
-      265,  267,  266,  268,  269,  278,  266,  279,  273,  274,
-      281,  282,  283,  284,  286,  275,  287,  288,  289,  290,
-      291,  292,  293,  294,  295,  298,  299,  276,  277,  300,
-      301,  302,  303,  304,  278,  307,  279,  309,  311,  281,
-      282,  283,  284,  286,  312,  287,  288,  289,  290,  291,
-      292,  293,  294,  295,  298,  299,  313,  314,  300,  301,
-
-      302,  303,  304,  320,  307,  322,  309,  311,  314,  323,
-      324,  325,  326,  312,  328,  329,  330,  331,  332,  334,
-      336,  337,  338,  340,  341,  313,  314,  342,  343,  344,
-      345,  346,  320,  347,  322,  349,  351,  314,  323,  324,
-      325,  326,  352,  328,  329,  330,  331,  332,  334,  336,
-      337,  338,  340,  341,  353,  354,  342,  343,  344,  345,
-      346,  355,  347,  356,  349,  351,  357,  358,  360,  361,
-      363,  352,  367,  368,  369,  370,  371,  372,  373,  371,
-      375,  376,  377,  353,  354,  378,  379,  380,  381,  382,
-      355,  383,  356,  384,  385,  357,  358,  360,  361,  363,
-
-      386,  367,  368,  369,  370,  371,  372,  373,  371,  375,
-      376,  377,  387,  388,  378,  379,  380,  381,  382,  392,
-      383,  393,  384,  385,  395,  398,  399,  400,  402,  386,
-      400,  407,  408,  409,  410,  411,  413,  414,  416,  417,
-      418,  387,  388,  419,  420,  421,  422,  424,  392,  425,
-      393,  426,  428,  395,  398,  399,  400,  402,  429,  400,
-      407,  408,  409,  410,  411,  413,  414,  416,  417,  418,
-      431,  432,  419,  420,  421,  422,  424,  434,  425,  435,
-      426,  428,  436,  437,  439,  440,  441,  429,  442,  443,
-      445,  447,  448,  449,  450,  451,  453,  454,  457,  431,
-
-      432,  458,  459,  462,  463,  464,  434,  466,  435,  467,
-      468,  436,  437,  439,  440,  441,  469,  442,  443,  445,
-      447,  448,  449,  450,  451,  453,  454,  457,  470,  471,
-      458,  459,  462,  463,  464,  473,  466,  476,  467,  468,
-      479,  480,  481,  482,  486,  469,  488,  489,  491,  492,
-      493,  494,  495,  496,  499,  502,  503,  470,  471,  505,
-      508,  509,  510,  512,  473,  514,  476,  515,  516,  479,
-      480,  481,  482,  486,  517,  488,  489,  491,  492,  493,
-      494,  495,  496,  499,  502,  503,  518,  519,  505,  508,
-      509,  510,  512,  520,  514,  521,  515,  516,  523,  524,
-
-      525,  526,  527,  517,  528,  529,  530,  532,  534,  535,
-      537,  538,  540,  541,  543,  518,  519,  544,  545,  546,
-      547,  549,  520,  550,  521,  551,  552,  523,  524,  525,
-      526,  527,  554,  528,  529,  530,  532,  534,  535,  537,
-      538,  540,  541,  543,  555,  558,  544,  545,  546,  547,
-      549,  560,  550,  561,  551,  552,  564,  568,  569,  570,
-      575,  554,  576,  577,  580,  582,  583,  584,  586,    0,
-        0,    0,    0,  555,  558,    0,    0,    0,    0,    0,
-      560,    0,  561,    0,    0,  564,  568,  569,  570,  575,
-        0,  576,  577,  580,  582,  583,  584,  586,  589,  589,
-
-      589,  589,  589,  589,  589,  589,  589,  589,  590,  590,
-      590,  590,  590,  590,  590,  590,  590,  590,  591,  591,
-      591,  591,  591,  591,  591,  591,  591,  591,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  593,  593,
-      593,  593,  593,  593,  593,  593,  593,  593,  594,  594,
-      595,  595,  595,    0,  595,  596,  596,  596,  596,    0,
-      596,  597,  597,  597,    0,  597,  597,  597,  597,  597,
-      597,  598,  598,  598,    0,  598,  598,  598,  598,    0,
-      598,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  600,  600,    0,  600,  600,  600,  600,  600,  600,
-
-      600,  601,    0,  601,  601,  601,  601,  601,  601,  601,
-      601,  602,  602,  603,  603,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588
+      245,  246,  238,  239,  240,  242,  241,  247,  241,  248,
+      243,  244,  249,  250,  252,  253,  254,  255,  256,  248,
+
+      257,  258,  259,  260,  261,  253,  261,  262,  263,  245,
+      246,  264,  265,  266,  267,  268,  247,  266,  248,  269,
+      273,  249,  250,  252,  253,  254,  255,  256,  274,  257,
+      258,  259,  260,  261,  253,  261,  262,  263,  275,  276,
+      264,  265,  266,  267,  268,  277,  266,  278,  269,  273,
+      279,  281,  282,  283,  284,  285,  287,  274,  288,  289,
+      290,  291,  292,  293,  294,  295,  296,  275,  276,  299,
+      300,  301,  302,  303,  277,  304,  278,  305,  308,  279,
+      281,  282,  283,  284,  285,  287,  310,  288,  289,  290,
+      291,  292,  293,  294,  295,  296,  312,  313,  299,  300,
+
+      301,  302,  303,  314,  304,  315,  305,  308,  321,  323,
+      324,  325,  326,  327,  329,  310,  315,  330,  331,  332,
+      333,  335,  337,  338,  339,  312,  313,  341,  342,  343,
+      344,  345,  314,  346,  315,  347,  348,  321,  323,  324,
+      325,  326,  327,  329,  350,  315,  330,  331,  332,  333,
+      335,  337,  338,  339,  352,  353,  341,  342,  343,  344,
+      345,  354,  346,  355,  347,  348,  356,  357,  358,  359,
+      361,  362,  364,  350,  368,  369,  370,  371,  372,  373,
+      374,  372,  376,  352,  353,  377,  378,  379,  380,  381,
+      354,  382,  355,  383,  384,  356,  357,  358,  359,  361,
+
+      362,  364,  385,  368,  369,  370,  371,  372,  373,  374,
+      372,  376,  386,  387,  377,  378,  379,  380,  381,  388,
+      382,  389,  383,  384,  390,  394,  395,  397,  400,  401,
+      402,  385,  404,  402,  409,  410,  411,  412,  413,  415,
+      416,  386,  387,  418,  419,  420,  421,  422,  388,  423,
+      389,  424,  426,  390,  394,  395,  397,  400,  401,  402,
+      427,  404,  402,  409,  410,  411,  412,  413,  415,  416,
+      428,  430,  418,  419,  420,  421,  422,  431,  423,  433,
+      424,  426,  434,  436,  437,  438,  439,  441,  442,  427,
+      443,  444,  445,  447,  448,  450,  451,  452,  453,  428,
+
+      430,  454,  456,  457,  460,  461,  431,  462,  433,  465,
+      466,  434,  436,  437,  438,  439,  441,  442,  467,  443,
+      444,  445,  447,  448,  450,  451,  452,  453,  469,  470,
+      454,  456,  457,  460,  461,  471,  462,  472,  465,  466,
+      473,  474,  476,  479,  482,  483,  484,  467,  485,  489,
+      491,  492,  494,  495,  496,  497,  498,  469,  470,  500,
+      503,  506,  507,  509,  471,  512,  472,  513,  514,  473,
+      474,  476,  479,  482,  483,  484,  516,  485,  489,  491,
+      492,  494,  495,  496,  497,  498,  518,  519,  500,  503,
+      506,  507,  509,  520,  512,  521,  513,  514,  522,  523,
+
+      524,  525,  527,  528,  529,  516,  530,  531,  532,  533,
+      534,  536,  538,  539,  541,  518,  519,  542,  544,  545,
+      547,  548,  520,  549,  521,  550,  551,  522,  523,  524,
+      525,  527,  528,  529,  553,  530,  531,  532,  533,  534,
+      536,  538,  539,  541,  554,  555,  542,  544,  545,  547,
+      548,  556,  549,  558,  550,  551,  559,  562,  564,  565,
+      568,  572,  573,  553,  574,  579,  580,  581,  584,  586,
+      587,  588,  590,  554,  555,    0,    0,    0,    0,    0,
+      556,    0,  558,    0,    0,  559,  562,  564,  565,  568,
+      572,  573,    0,  574,  579,  580,  581,  584,  586,  587,
+
+      588,  590,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  595,  595,  595,  595,  595,  595,  595,  595,
+      595,  595,  596,  596,  596,  596,  596,  596,  596,  596,
+      596,  596,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  598,  598,  599,  599,  599,    0,  599,  600,
+      600,  600,  600,    0,  600,  601,  601,  601,    0,  601,
+      601,  601,  601,  601,  601,  602,  602,  602,    0,  602,
+      602,  602,  602,    0,  602,  603,  603,  603,  603,  603,
+      603,  603,  603,  603,  603,  604,  604,    0,  604,  604,
+
+      604,  604,  604,  604,  604,  605,    0,  605,  605,  605,
+      605,  605,  605,  605,  605,  606,  606,  607,  607,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      592
+
     } ;
 
 /* Table of booleans, true if rule could match eol. */
-static const flex_int32_t yy_rule_can_match_eol[165] =
+static const flex_int32_t yy_rule_can_match_eol[166] =
     {   0,
 0, 1, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
@@ -1177,8 +1183,8 @@ static const flex_int32_t yy_rule_can_match_eol[165] =
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
-    0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 1, 1, 0, 0, 
-    0, 1, 0, 0, 0,     };
+    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 1, 1, 0, 
+    0, 0, 1, 0, 0, 0,     };
 
 /* The intent behind this definition is that it'll catch
  * any uses of REJECT which flex missed.
@@ -1272,6 +1278,7 @@ class ParseSubqueryExpression;
 class ParseSubqueryTableReference;
 class ParseTableReference;
 class ParseTableReferenceSignature;
+class ParseTransitiveClosureTableReference;
 class ParseWindow;
 class Type;
 class UnaryOperation;
@@ -1289,14 +1296,14 @@ class UnaryOperation;
     yycolumn += yyleng;                                   \
   }
 
-#line 1292 "SqlLexer_gen.cpp"
+#line 1299 "SqlLexer_gen.cpp"
 /* FIXME(chasseur, qzeng): Add support for hexadecimal literals. */
 /**
  * These patterns are based on the SQL-2011 standard for syntax of numeric
  * literals (Part 2, Section 5.3 of the standard).
  **/
 
-#line 1299 "SqlLexer_gen.cpp"
+#line 1306 "SqlLexer_gen.cpp"
 
 #define INITIAL 0
 #define CONDITION_SQL 1
@@ -1583,10 +1590,10 @@ YY_DECL
 		}
 
 	{
-#line 132 "../SqlLexer.lpp"
+#line 133 "../SqlLexer.lpp"
 
 
-#line 1589 "SqlLexer_gen.cpp"
+#line 1596 "SqlLexer_gen.cpp"
 
 	while ( /*CONSTCOND*/1 )		/* loops until end-of-file is reached */
 		{
@@ -1613,13 +1620,13 @@ yy_match:
 			while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state )
 				{
 				yy_current_state = (int) yy_def[yy_current_state];
-				if ( yy_current_state >= 589 )
+				if ( yy_current_state >= 593 )
 					yy_c = yy_meta[yy_c];
 				}
 			yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
 			++yy_cp;
 			}
-		while ( yy_current_state != 588 );
+		while ( yy_current_state != 592 );
 		yy_cp = yyg->yy_last_accepting_cpos;
 		yy_current_state = yyg->yy_last_accepting_state;
 
@@ -1653,7 +1660,7 @@ do_action:	/* This label is used only to access EOF actions. */
 
 case 1:
 YY_RULE_SETUP
-#line 135 "../SqlLexer.lpp"
+#line 136 "../SqlLexer.lpp"
 {
     /* A forward slash character represents a system command. */
     BEGIN(CONDITION_COMMAND);
@@ -1665,7 +1672,7 @@ YY_RULE_SETUP
 case 2:
 /* rule 2 can match eol */
 YY_RULE_SETUP
-#line 143 "../SqlLexer.lpp"
+#line 144 "../SqlLexer.lpp"
 {
     /* This is a SQL command. Place the char back and process normally. */
     yyless(0);
@@ -1677,7 +1684,7 @@ YY_RULE_SETUP
 
 case 3:
 YY_RULE_SETUP
-#line 152 "../SqlLexer.lpp"
+#line 153 "../SqlLexer.lpp"
 {
     /* This is a command argument. */
     yylval->string_value_ = new quickstep::ParseString(
@@ -1687,7 +1694,7 @@ YY_RULE_SETUP
 	YY_BREAK
 case 4:
 YY_RULE_SETUP
-#line 159 "../SqlLexer.lpp"
+#line 160 "../SqlLexer.lpp"
 {
     /* Ignore whitespace. */
   }
@@ -1695,7 +1702,7 @@ YY_RULE_SETUP
 case 5:
 /* rule 5 can match eol */
 YY_RULE_SETUP
-#line 163 "../SqlLexer.lpp"
+#line 164 "../SqlLexer.lpp"
 {
     /* Newline reverts the lexer to the initial state. */
     yycolumn = 0;
@@ -1707,687 +1714,692 @@ YY_RULE_SETUP
 
 case 6:
 YY_RULE_SETUP
-#line 172 "../SqlLexer.lpp"
+#line 173 "../SqlLexer.lpp"
 return TOKEN_ADD;
 	YY_BREAK
 case 7:
 YY_RULE_SETUP
-#line 173 "../SqlLexer.lpp"
+#line 174 "../SqlLexer.lpp"
 return TOKEN_ALL;
 	YY_BREAK
 case 8:
 YY_RULE_SETUP
-#line 174 "../SqlLexer.lpp"
+#line 175 "../SqlLexer.lpp"
 return TOKEN_ALTER;
 	YY_BREAK
 case 9:
 YY_RULE_SETUP
-#line 175 "../SqlLexer.lpp"
+#line 176 "../SqlLexer.lpp"
 return TOKEN_AND;
 	YY_BREAK
 case 10:
 YY_RULE_SETUP
-#line 176 "../SqlLexer.lpp"
+#line 177 "../SqlLexer.lpp"
 return TOKEN_AS;
 	YY_BREAK
 case 11:
 YY_RULE_SETUP
-#line 177 "../SqlLexer.lpp"
+#line 178 "../SqlLexer.lpp"
 return TOKEN_ASC;
 	YY_BREAK
 case 12:
 YY_RULE_SETUP
-#line 178 "../SqlLexer.lpp"
+#line 179 "../SqlLexer.lpp"
 return TOKEN_ASC;
 	YY_BREAK
 case 13:
 YY_RULE_SETUP
-#line 179 "../SqlLexer.lpp"
+#line 180 "../SqlLexer.lpp"
 return TOKEN_BETWEEN;
 	YY_BREAK
 case 14:
 YY_RULE_SETUP
-#line 180 "../SqlLexer.lpp"
+#line 181 "../SqlLexer.lpp"
 return TOKEN_BIGINT;
 	YY_BREAK
 case 15:
 YY_RULE_SETUP
-#line 181 "../SqlLexer.lpp"
+#line 182 "../SqlLexer.lpp"
 return TOKEN_BIT;
 	YY_BREAK
 case 16:
 YY_RULE_SETUP
-#line 182 "../SqlLexer.lpp"
+#line 183 "../SqlLexer.lpp"
 return TOKEN_BITWEAVING;
 	YY_BREAK
 case 17:
 YY_RULE_SETUP
-#line 183 "../SqlLexer.lpp"
+#line 184 "../SqlLexer.lpp"
 return TOKEN_BLOCKPROPERTIES;
 	YY_BREAK
 case 18:
 YY_RULE_SETUP
-#line 184 "../SqlLexer.lpp"
+#line 185 "../SqlLexer.lpp"
 return TOKEN_BLOCKSAMPLE;
 	YY_BREAK
 case 19:
 YY_RULE_SETUP
-#line 185 "../SqlLexer.lpp"
+#line 186 "../SqlLexer.lpp"
 return TOKEN_BLOOM_FILTER;
 	YY_BREAK
 case 20:
 YY_RULE_SETUP
-#line 186 "../SqlLexer.lpp"
-return TOKEN_CASE;
-	YY_BREAK
-case 21:
-YY_RULE_SETUP
 #line 187 "../SqlLexer.lpp"
-return TOKEN_CSB_TREE;
+return TOKEN_BY;
 	YY_BREAK
-case 22:
+case 21:
 YY_RULE_SETUP
 #line 188 "../SqlLexer.lpp"
-return TOKEN_BY;
+return TOKEN_CASE;
 	YY_BREAK
-case 23:
+case 22:
 YY_RULE_SETUP
 #line 189 "../SqlLexer.lpp"
 return TOKEN_CHARACTER;
 	YY_BREAK
-case 24:
+case 23:
 YY_RULE_SETUP
 #line 190 "../SqlLexer.lpp"
 return TOKEN_CHARACTER;
 	YY_BREAK
-case 25:
+case 24:
 YY_RULE_SETUP
 #line 191 "../SqlLexer.lpp"
 return TOKEN_CHECK;
 	YY_BREAK
-case 26:
+case 25:
 YY_RULE_SETUP
 #line 192 "../SqlLexer.lpp"
 return TOKEN_COLUMN;
 	YY_BREAK
-case 27:
+case 26:
 YY_RULE_SETUP
 #line 193 "../SqlLexer.lpp"
+return TOKEN_CONNECT;
+	YY_BREAK
+case 27:
+YY_RULE_SETUP
+#line 194 "../SqlLexer.lpp"
 return TOKEN_CONSTRAINT;
 	YY_BREAK
 case 28:
 YY_RULE_SETUP
-#line 194 "../SqlLexer.lpp"
+#line 195 "../SqlLexer.lpp"
 return TOKEN_COPY;
 	YY_BREAK
 case 29:
 YY_RULE_SETUP
-#line 195 "../SqlLexer.lpp"
+#line 196 "../SqlLexer.lpp"
 return TOKEN_CREATE;
 	YY_BREAK
 case 30:
 YY_RULE_SETUP
-#line 196 "../SqlLexer.lpp"
-return TOKEN_CURRENT;
+#line 197 "../SqlLexer.lpp"
+return TOKEN_CSB_TREE;
 	YY_BREAK
 case 31:
 YY_RULE_SETUP
-#line 197 "../SqlLexer.lpp"
-return TOKEN_DATE;
+#line 198 "../SqlLexer.lpp"
+return TOKEN_CURRENT;
 	YY_BREAK
 case 32:
 YY_RULE_SETUP
-#line 198 "../SqlLexer.lpp"
-return TOKEN_DATETIME;
+#line 199 "../SqlLexer.lpp"
+return TOKEN_DATE;
 	YY_BREAK
 case 33:
 YY_RULE_SETUP
-#line 199 "../SqlLexer.lpp"
-return TOKEN_DAY;
+#line 200 "../SqlLexer.lpp"
+return TOKEN_DATETIME;
 	YY_BREAK
 case 34:
 YY_RULE_SETUP
-#line 200 "../SqlLexer.lpp"
-return TOKEN_DECIMAL;
+#line 201 "../SqlLexer.lpp"
+return TOKEN_DAY;
 	YY_BREAK
 case 35:
 YY_RULE_SETUP
-#line 201 "../SqlLexer.lpp"
-return TOKEN_DEFAULT;
+#line 202 "../SqlLexer.lpp"
+return TOKEN_DECIMAL;
 	YY_BREAK
 case 36:
 YY_RULE_SETUP
-#line 202 "../SqlLexer.lpp"
-return TOKEN_DELETE;
+#line 203 "../SqlLexer.lpp"
+return TOKEN_DEFAULT;
 	YY_BREAK
 case 37:
 YY_RULE_SETUP
-#line 203 "../SqlLexer.lpp"
-return TOKEN_DESC;
+#line 204 "../SqlLexer.lpp"
+return TOKEN_DELETE;
 	YY_BREAK
 case 38:
 YY_RULE_SETUP
-#line 204 "../SqlLexer.lpp"
+#line 205 "../SqlLexer.lpp"
 return TOKEN_DESC;
 	YY_BREAK
 case 39:
 YY_RULE_SETUP
-#line 205 "../SqlLexer.lpp"
-return TOKEN_DISTINCT;
+#line 206 "../SqlLexer.lpp"
+return TOKEN_DESC;
 	YY_BREAK
 case 40:
 YY_RULE_SETUP
-#line 206 "../SqlLexer.lpp"
-return TOKEN_DOUBLE;
+#line 207 "../SqlLexer.lpp"
+return TOKEN_DISTINCT;
 	YY_BREAK
 case 41:
 YY_RULE_SETUP
-#line 207 "../SqlLexer.lpp"
-return TOKEN_DROP;
+#line 208 "../SqlLexer.lpp"
+return TOKEN_DOUBLE;
 	YY_BREAK
 case 42:
 YY_RULE_SETUP
-#line 208 "../SqlLexer.lpp"
-return TOKEN_ELSE;
+#line 209 "../SqlLexer.lpp"
+return TOKEN_DROP;
 	YY_BREAK
 case 43:
 YY_RULE_SETUP
-#line 209 "../SqlLexer.lpp"
-return TOKEN_END;
+#line 210 "../SqlLexer.lpp"
+return TOKEN_ELSE;
 	YY_BREAK
 case 44:
 YY_RULE_SETUP
-#line 210 "../SqlLexer.lpp"
-return TOKEN_EXISTS;
+#line 211 "../SqlLexer.lpp"
+return TOKEN_END;
 	YY_BREAK
 case 45:
 YY_RULE_SETUP
-#line 211 "../SqlLexer.lpp"
-return TOKEN_EXTRACT;
+#line 212 "../SqlLexer.lpp"
+return TOKEN_EXISTS;
 	YY_BREAK
 case 46:
 YY_RULE_SETUP
-#line 212 "../SqlLexer.lpp"
-return TOKEN_FALSE;
+#line 213 "../SqlLexer.lpp"
+return TOKEN_EXTRACT;
 	YY_BREAK
 case 47:
 YY_RULE_SETUP
-#line 213 "../SqlLexer.lpp"
-return TOKEN_FIRST;
+#line 214 "../SqlLexer.lpp"
+return TOKEN_FALSE;
 	YY_BREAK
 case 48:
 YY_RULE_SETUP
-#line 214 "../SqlLexer.lpp"
-return TOKEN_FLOAT;
+#line 215 "../SqlLexer.lpp"
+return TOKEN_FIRST;
 	YY_BREAK
 case 49:
 YY_RULE_SETUP
-#line 215 "../SqlLexer.lpp"
-return TOKEN_FOLLOWING;
+#line 216 "../SqlLexer.lpp"
+return TOKEN_FLOAT;
 	YY_BREAK
 case 50:
 YY_RULE_SETUP
-#line 216 "../SqlLexer.lpp"
-return TOKEN_FOR;
+#line 217 "../SqlLexer.lpp"
+return TOKEN_FOLLOWING;
 	YY_BREAK
 case 51:
 YY_RULE_SETUP
-#line 217 "../SqlLexer.lpp"
-return TOKEN_FOREIGN;
+#line 218 "../SqlLexer.lpp"
+return TOKEN_FOR;
 	YY_BREAK
 case 52:
 YY_RULE_SETUP
-#line 218 "../SqlLexer.lpp"
-return TOKEN_FROM;
+#line 219 "../SqlLexer.lpp"
+return TOKEN_FOREIGN;
 	YY_BREAK
 case 53:
 YY_RULE_SETUP
-#line 219 "../SqlLexer.lpp"
-return TOKEN_FULL;
+#line 220 "../SqlLexer.lpp"
+return TOKEN_FROM;
 	YY_BREAK
 case 54:
 YY_RULE_SETUP
-#line 220 "../SqlLexer.lpp"
-return TOKEN_GROUP;
+#line 221 "../SqlLexer.lpp"
+return TOKEN_FULL;
 	YY_BREAK
 case 55:
 YY_RULE_SETUP
-#line 221 "../SqlLexer.lpp"
-return TOKEN_HASH;
+#line 222 "../SqlLexer.lpp"
+return TOKEN_GROUP;
 	YY_BREAK
 case 56:
 YY_RULE_SETUP
-#line 222 "../SqlLexer.lpp"
-return TOKEN_HAVING;
+#line 223 "../SqlLexer.lpp"
+return TOKEN_HASH;
 	YY_BREAK
 case 57:
 YY_RULE_SETUP
-#line 223 "../SqlLexer.lpp"
-return TOKEN_HOUR;
+#line 224 "../SqlLexer.lpp"
+return TOKEN_HAVING;
 	YY_BREAK
 case 58:
 YY_RULE_SETUP
-#line 224 "../SqlLexer.lpp"
-return TOKEN_IN;
+#line 225 "../SqlLexer.lpp"
+return TOKEN_HOUR;
 	YY_BREAK
 case 59:
 YY_RULE_SETUP
-#line 225 "../SqlLexer.lpp"
-return TOKEN_INDEX;
+#line 226 "../SqlLexer.lpp"
+return TOKEN_IN;
 	YY_BREAK
 case 60:
 YY_RULE_SETUP
-#line 226 "../SqlLexer.lpp"
-return TOKEN_INNER;
+#line 227 "../SqlLexer.lpp"
+return TOKEN_INDEX;
 	YY_BREAK
 case 61:
 YY_RULE_SETUP
-#line 227 "../SqlLexer.lpp"
-return TOKEN_INSERT;
+#line 228 "../SqlLexer.lpp"
+return TOKEN_INNER;
 	YY_BREAK
 case 62:
 YY_RULE_SETUP
-#line 228 "../SqlLexer.lpp"
-return TOKEN_INTEGER;
+#line 229 "../SqlLexer.lpp"
+return TOKEN_INSERT;
 	YY_BREAK
 case 63:
 YY_RULE_SETUP
-#line 229 "../SqlLexer.lpp"
+#line 230 "../SqlLexer.lpp"
 return TOKEN_INTEGER;
 	YY_BREAK
 case 64:
 YY_RULE_SETUP
-#line 230 "../SqlLexer.lpp"
-return TOKEN_INTERSECT;
+#line 231 "../SqlLexer.lpp"
+return TOKEN_INTEGER;
 	YY_BREAK
 case 65:
 YY_RULE_SETUP
-#line 231 "../SqlLexer.lpp"
-return TOKEN_INTERVAL;
+#line 232 "../SqlLexer.lpp"
+return TOKEN_INTERSECT;
 	YY_BREAK
 case 66:
 YY_RULE_SETUP
-#line 232 "../SqlLexer.lpp"
-return TOKEN_INTO;
+#line 233 "../SqlLexer.lpp"
+return TOKEN_INTERVAL;
 	YY_BREAK
 case 67:
 YY_RULE_SETUP
-#line 233 "../SqlLexer.lpp"
-return TOKEN_IS;
+#line 234 "../SqlLexer.lpp"
+return TOKEN_INTO;
 	YY_BREAK
 case 68:
 YY_RULE_SETUP
-#line 234 "../SqlLexer.lpp"
-return TOKEN_JOIN;
+#line 235 "../SqlLexer.lpp"
+return TOKEN_IS;
 	YY_BREAK
 case 69:
 YY_RULE_SETUP
-#line 235 "../SqlLexer.lpp"
-return TOKEN_KEY;
+#line 236 "../SqlLexer.lpp"
+return TOKEN_JOIN;
 	YY_BREAK
 case 70:
 YY_RULE_SETUP
-#line 236 "../SqlLexer.lpp"
-return TOKEN_LAST;
+#line 237 "../SqlLexer.lpp"
+return TOKEN_KEY;
 	YY_BREAK
 case 71:
 YY_RULE_SETUP
-#line 237 "../SqlLexer.lpp"
-return TOKEN_LEFT;
+#line 238 "../SqlLexer.lpp"
+return TOKEN_LAST;
 	YY_BREAK
 case 72:
 YY_RULE_SETUP
-#line 238 "../SqlLexer.lpp"
-return TOKEN_LIKE;
+#line 239 "../SqlLexer.lpp"
+return TOKEN_LEFT;
 	YY_BREAK
 case 73:
 YY_RULE_SETUP
-#line 239 "../SqlLexer.lpp"
-return TOKEN_LIMIT;
+#line 240 "../SqlLexer.lpp"
+return TOKEN_LIKE;
 	YY_BREAK
 case 74:
 YY_RULE_SETUP
-#line 240 "../SqlLexer.lpp"
-return TOKEN_LONG;
+#line 241 "../SqlLexer.lpp"
+return TOKEN_LIMIT;
 	YY_BREAK
 case 75:
 YY_RULE_SETUP
-#line 241 "../SqlLexer.lpp"
-return TOKEN_MINUTE;
+#line 242 "../SqlLexer.lpp"
+return TOKEN_LONG;
 	YY_BREAK
 case 76:
 YY_RULE_SETUP
-#line 242 "../SqlLexer.lpp"
-return TOKEN_MONTH;
+#line 243 "../SqlLexer.lpp"
+return TOKEN_MINUTE;
 	YY_BREAK
 case 77:
 YY_RULE_SETUP
-#line 243 "../SqlLexer.lpp"
-return TOKEN_NOT;
+#line 244 "../SqlLexer.lpp"
+return TOKEN_MONTH;
 	YY_BREAK
 case 78:
 YY_RULE_SETUP
-#line 244 "../SqlLexer.lpp"
-return TOKEN_NULL;
+#line 245 "../SqlLexer.lpp"
+return TOKEN_NOT;
 	YY_BREAK
 case 79:
 YY_RULE_SETUP
-#line 245 "../SqlLexer.lpp"
-return TOKEN_NULLS;
+#line 246 "../SqlLexer.lpp"
+return TOKEN_NULL;
 	YY_BREAK
 case 80:
 YY_RULE_SETUP
-#line 246 "../SqlLexer.lpp"
-return TOKEN_OFF;
+#line 247 "../SqlLexer.lpp"
+return TOKEN_NULLS;
 	YY_BREAK
 case 81:
 YY_RULE_SETUP
-#line 247 "../SqlLexer.lpp"
-return TOKEN_ON;
+#line 248 "../SqlLexer.lpp"
+return TOKEN_OFF;
 	YY_BREAK
 case 82:
 YY_RULE_SETUP
-#line 248 "../SqlLexer.lpp"
-return TOKEN_OR;
+#line 249 "../SqlLexer.lpp"
+return TOKEN_ON;
 	YY_BREAK
 case 83:
 YY_RULE_SETUP
-#line 249 "../SqlLexer.lpp"
-return TOKEN_ORDER;
+#line 250 "../SqlLexer.lpp"
+return TOKEN_OR;
 	YY_BREAK
 case 84:
 YY_RULE_SETUP
-#line 250 "../SqlLexer.lpp"
-return TOKEN_OUTER;
+#line 251 "../SqlLexer.lpp"
+return TOKEN_ORDER;
 	YY_BREAK
 case 85:
 YY_RULE_SETUP
-#line 251 "../SqlLexer.lpp"
-return TOKEN_OVER;
+#line 252 "../SqlLexer.lpp"
+return TOKEN_OUTER;
 	YY_BREAK
 case 86:
 YY_RULE_SETUP
-#line 252 "../SqlLexer.lpp"
-return TOKEN_PARTITION;
+#line 253 "../SqlLexer.lpp"
+return TOKEN_OVER;
 	YY_BREAK
 case 87:
 YY_RULE_SETUP
-#line 253 "../SqlLexer.lpp"
-return TOKEN_PARTITIONS;
+#line 254 "../SqlLexer.lpp"
+return TOKEN_PARTITION;
 	YY_BREAK
 case 88:
 YY_RULE_SETUP
-#line 254 "../SqlLexer.lpp"
-return TOKEN_PERCENT;
+#line 255 "../SqlLexer.lpp"
+return TOKEN_PARTITIONS;
 	YY_BREAK
 case 89:
 YY_RULE_SETUP
-#line 255 "../SqlLexer.lpp"
-return TOKEN_PRECEDING;
+#line 256 "../SqlLexer.lpp"
+return TOKEN_PERCENT;
 	YY_BREAK
 case 90:
 YY_RULE_SETUP
-#line 256 "../SqlLexer.lpp"
-return TOKEN_PRIMARY;
+#line 257 "../SqlLexer.lpp"
+return TOKEN_PRECEDING;
 	YY_BREAK
 case 91:
 YY_RULE_SETUP
-#line 257 "../SqlLexer.lpp"
-return TOKEN_PRIORITY;
+#line 258 "../SqlLexer.lpp"
+return TOKEN_PRIMARY;
 	YY_BREAK
 case 92:
 YY_RULE_SETUP
-#line 258 "../SqlLexer.lpp"
-return TOKEN_QUIT;
+#line 259 "../SqlLexer.lpp"
+return TOKEN_PRIORITY;
 	YY_BREAK
 case 93:
 YY_RULE_SETUP
-#line 259 "../SqlLexer.lpp"
-return TOKEN_RANGE;
+#line 260 "../SqlLexer.lpp"
+return TOKEN_QUIT;
 	YY_BREAK
 case 94:
 YY_RULE_SETUP
-#line 260 "../SqlLexer.lpp"
-return TOKEN_REAL;
+#line 261 "../SqlLexer.lpp"
+return TOKEN_RANGE;
 	YY_BREAK
 case 95:
 YY_RULE_SETUP
-#line 261 "../SqlLexer.lpp"
-return TOKEN_REFERENCES;
+#line 262 "../SqlLexer.lpp"
+return TOKEN_REAL;
 	YY_BREAK
 case 96:
 YY_RULE_SETUP
-#line 262 "../SqlLexer.lpp"
-return TOKEN_REGEXP;
+#line 263 "../SqlLexer.lpp"
+return TOKEN_REFERENCES;
 	YY_BREAK
 case 97:
 YY_RULE_SETUP
-#line 263 "../SqlLexer.lpp"
-return TOKEN_RIGHT;
+#line 264 "../SqlLexer.lpp"
+return TOKEN_REGEXP;
 	YY_BREAK
 case 98:
 YY_RULE_SETUP
-#line 264 "../SqlLexer.lpp"
-return TOKEN_ROW;
+#line 265 "../SqlLexer.lpp"
+return TOKEN_RIGHT;
 	YY_BREAK
 case 99:
 YY_RULE_SETUP
-#line 265 "../SqlLexer.lpp"
-return TOKEN_ROW_DELIMITER;
+#line 266 "../SqlLexer.lpp"
+return TOKEN_ROW;
 	YY_BREAK
 case 100:
 YY_RULE_SETUP
-#line 266 "../SqlLexer.lpp"
-return TOKEN_ROWS;
+#line 267 "../SqlLexer.lpp"
+return TOKEN_ROW_DELIMITER;
 	YY_BREAK
 case 101:
 YY_RULE_SETUP
-#line 267 "../SqlLexer.lpp"
-return TOKEN_SECOND;
+#line 268 "../SqlLexer.lpp"
+return TOKEN_ROWS;
 	YY_BREAK
 case 102:
 YY_RULE_SETUP
-#line 268 "../SqlLexer.lpp"
-return TOKEN_SELECT;
+#line 269 "../SqlLexer.lpp"
+return TOKEN_SECOND;
 	YY_BREAK
 case 103:
 YY_RULE_SETUP
-#line 269 "../SqlLexer.lpp"
-return TOKEN_SET;
+#line 270 "../SqlLexer.lpp"
+return TOKEN_SELECT;
 	YY_BREAK
 case 104:
 YY_RULE_SETUP
-#line 270 "../SqlLexer.lpp"
-return TOKEN_SMA;
+#line 271 "../SqlLexer.lpp"
+return TOKEN_SET;
 	YY_BREAK
 case 105:
 YY_RULE_SETUP
-#line 271 "../SqlLexer.lpp"
-return TOKEN_SMALLINT;
+#line 272 "../SqlLexer.lpp"
+return TOKEN_SMA;
 	YY_BREAK
 case 106:
 YY_RULE_SETUP
-#line 272 "../SqlLexer.lpp"
-return TOKEN_STDERR;
+#line 273 "../SqlLexer.lpp"
+return TOKEN_SMALLINT;
 	YY_BREAK
 case 107:
 YY_RULE_SETUP
-#line 273 "../SqlLexer.lpp"
-return TOKEN_STDOUT;
+#line 274 "../SqlLexer.lpp"
+return TOKEN_STDERR;
 	YY_BREAK
 case 108:
 YY_RULE_SETUP
-#line 274 "../SqlLexer.lpp"
-return TOKEN_SUBSTRING;
+#line 275 "../SqlLexer.lpp"
+return TOKEN_STDOUT;
 	YY_BREAK
 case 109:
 YY_RULE_SETUP
-#line 275 "../SqlLexer.lpp"
-return TOKEN_TABLE;
+#line 276 "../SqlLexer.lpp"
+return TOKEN_SUBSTRING;
 	YY_BREAK
 case 110:
 YY_RULE_SETUP
-#line 276 "../SqlLexer.lpp"
-return TOKEN_THEN;
+#line 277 "../SqlLexer.lpp"
+return TOKEN_TABLE;
 	YY_BREAK
 case 111:
 YY_RULE_SETUP
-#line 277 "../SqlLexer.lpp"
-return TOKEN_TIME;
+#line 278 "../SqlLexer.lpp"
+return TOKEN_THEN;
 	YY_BREAK
 case 112:
 YY_RULE_SETUP
-#line 278 "../SqlLexer.lpp"
-return TOKEN_TIMESTAMP;
+#line 279 "../SqlLexer.lpp"
+return TOKEN_TIME;
 	YY_BREAK
 case 113:
 YY_RULE_SETUP
-#line 279 "../SqlLexer.lpp"
-return TOKEN_TO;
+#line 280 "../SqlLexer.lpp"
+return TOKEN_TIMESTAMP;
 	YY_BREAK
 case 114:
 YY_RULE_SETUP
-#line 280 "../SqlLexer.lpp"
-return TOKEN_TRUE;
+#line 281 "../SqlLexer.lpp"
+return TOKEN_TO;
 	YY_BREAK
 case 115:
 YY_RULE_SETUP
-#line 281 "../SqlLexer.lpp"
-return TOKEN_TUPLESAMPLE;
+#line 282 "../SqlLexer.lpp"
+return TOKEN_TRUE;
 	YY_BREAK
 case 116:
 YY_RULE_SETUP
-#line 282 "../SqlLexer.lpp"
-return TOKEN_UNBOUNDED;
+#line 283 "../SqlLexer.lpp"
+return TOKEN_TUPLESAMPLE;
 	YY_BREAK
 case 117:
 YY_RULE_SETUP
-#line 283 "../SqlLexer.lpp"
-return TOKEN_UNION;
+#line 284 "../SqlLexer.lpp"
+return TOKEN_UNBOUNDED;
 	YY_BREAK
 case 118:
 YY_RULE_SETUP
-#line 284 "../SqlLexer.lpp"
-return TOKEN_UNIQUE;
+#line 285 "../SqlLexer.lpp"
+return TOKEN_UNION;
 	YY_BREAK
 case 119:
 YY_RULE_SETUP
-#line 285 "../SqlLexer.lpp"
-return TOKEN_UPDATE;
+#line 286 "../SqlLexer.lpp"
+return TOKEN_UNIQUE;
 	YY_BREAK
 case 120:
 YY_RULE_SETUP
-#line 286 "../SqlLexer.lpp"
-return TOKEN_USING;
+#line 287 "../SqlLexer.lpp"
+return TOKEN_UPDATE;
 	YY_BREAK
 case 121:
 YY_RULE_SETUP
-#line 287 "../SqlLexer.lpp"
-return TOKEN_VALUES;
+#line 288 "../SqlLexer.lpp"
+return TOKEN_USING;
 	YY_BREAK
 case 122:
 YY_RULE_SETUP
-#line 288 "../SqlLexer.lpp"
-return TOKEN_VARCHAR;
+#line 289 "../SqlLexer.lpp"
+return TOKEN_VALUES;
 	YY_BREAK
 case 123:
 YY_RULE_SETUP
-#line 289 "../SqlLexer.lpp"
-return TOKEN_WHEN;
+#line 290 "../SqlLexer.lpp"
+return TOKEN_VARCHAR;
 	YY_BREAK
 case 124:
 YY_RULE_SETUP
-#line 290 "../SqlLexer.lpp"
-return TOKEN_WHERE;
+#line 291 "../SqlLexer.lpp"
+return TOKEN_WHEN;
 	YY_BREAK
 case 125:
 YY_RULE_SETUP
-#line 291 "../SqlLexer.lpp"
-return TOKEN_WINDOW;
+#line 292 "../SqlLexer.lpp"
+return TOKEN_WHERE;
 	YY_BREAK
 case 126:
 YY_RULE_SETUP
-#line 292 "../SqlLexer.lpp"
-return TOKEN_WITH;
+#line 293 "../SqlLexer.lpp"
+return TOKEN_WINDOW;
 	YY_BREAK
 case 127:
 YY_RULE_SETUP
-#line 293 "../SqlLexer.lpp"
-return TOKEN_YEAR;
+#line 294 "../SqlLexer.lpp"
+return TOKEN_WITH;
 	YY_BREAK
 case 128:
 YY_RULE_SETUP
-#line 294 "../SqlLexer.lpp"
-return TOKEN_YEARMONTH;
+#line 295 "../SqlLexer.lpp"
+return TOKEN_YEAR;
 	YY_BREAK
 case 129:
 YY_RULE_SETUP
 #line 296 "../SqlLexer.lpp"
-return TOKEN_EQ;
+return TOKEN_YEARMONTH;
 	YY_BREAK
 case 130:
 YY_RULE_SETUP
-#line 297 "../SqlLexer.lpp"
-return TOKEN_NEQ;
+#line 298 "../SqlLexer.lpp"
+return TOKEN_EQ;
 	YY_BREAK
 case 131:
 YY_RULE_SETUP
-#line 298 "../SqlLexer.lpp"
+#line 299 "../SqlLexer.lpp"
 return TOKEN_NEQ;
 	YY_BREAK
 case 132:
 YY_RULE_SETUP
-#line 299 "../SqlLexer.lpp"
-return TOKEN_LT;
+#line 300 "../SqlLexer.lpp"
+return TOKEN_NEQ;
 	YY_BREAK
 case 133:
 YY_RULE_SETUP
-#line 300 "../SqlLexer.lpp"
-return TOKEN_GT;
+#line 301 "../SqlLexer.lpp"
+return TOKEN_LT;
 	YY_BREAK
 case 134:
 YY_RULE_SETUP
-#line 301 "../SqlLexer.lpp"
-return TOKEN_LEQ;
+#line 302 "../SqlLexer.lpp"
+return TOKEN_GT;
 	YY_BREAK
 case 135:
 YY_RULE_SETUP
-#line 302 "../SqlLexer.lpp"
-return TOKEN_GEQ;
+#line 303 "../SqlLexer.lpp"
+return TOKEN_LEQ;
 	YY_BREAK
 case 136:
 YY_RULE_SETUP
 #line 304 "../SqlLexer.lpp"
-return yytext[0];
+return TOKEN_GEQ;
 	YY_BREAK
 case 137:
 YY_RULE_SETUP
-#line 305 "../SqlLexer.lpp"
+#line 306 "../SqlLexer.lpp"
+return yytext[0];
+	YY_BREAK
+case 138:
+YY_RULE_SETUP
+#line 307 "../SqlLexer.lpp"
 return yytext[0];
 	YY_BREAK
 /**
     * Quoted strings. Prefacing a string with an 'e' or 'E' causes escape
     * sequences to be processed (as in PostgreSQL).
     **/
-case 138:
+case 139:
 YY_RULE_SETUP
-#line 311 "../SqlLexer.lpp"
+#line 313 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED_ESCAPED);
   }
 	YY_BREAK
-case 139:
+case 140:
 YY_RULE_SETUP
-#line 316 "../SqlLexer.lpp"
+#line 318 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED);
   }
 	YY_BREAK
-case 140:
+case 141:
 YY_RULE_SETUP
-#line 321 "../SqlLexer.lpp"
+#line 323 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_DOUBLE_QUOTED);
@@ -2399,7 +2411,7 @@ YY_RULE_SETUP
 case YY_STATE_EOF(CONDITION_STRING_SINGLE_QUOTED):
 case YY_STATE_EOF(CONDITION_STRING_SINGLE_QUOTED_ESCAPED):
 case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
-#line 330 "../SqlLexer.lpp"
+#line 332 "../SqlLexer.lpp"
 {
     delete yylval->string_value_;
     BEGIN(INITIAL);
@@ -2410,9 +2422,9 @@ case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
 
 /* Process escape sequences. */
 
-case 141:
+case 142:
 YY_RULE_SETUP
-#line 340 "../SqlLexer.lpp"
+#line 342 "../SqlLexer.lpp"
 {
     /* Octal code */
     unsigned int code;
@@ -2426,9 +2438,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 142:
+case 143:
 YY_RULE_SETUP
-#line 352 "../SqlLexer.lpp"
+#line 354 "../SqlLexer.lpp"
 {
     /* Hexadecimal code */
     unsigned int code;
@@ -2436,9 +2448,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 143:
+case 144:
 YY_RULE_SETUP
-#line 358 "../SqlLexer.lpp"
+#line 360 "../SqlLexer.lpp"
 {
     /* A numeric escape sequence that isn't correctly specified. */
     delete yylval->string_value_;
@@ -2447,58 +2459,58 @@ YY_RULE_SETUP
     return TOKEN_LEX_ERROR;
   }
 	YY_BREAK
-case 144:
+case 145:
 YY_RULE_SETUP
-#line 365 "../SqlLexer.lpp"
+#line 367 "../SqlLexer.lpp"
 {
     /* Backspace */
     yylval->string_value_->push_back('\b');
   }
 	YY_BREAK
-case 145:
+case 146:
 YY_RULE_SETUP
-#line 369 "../SqlLexer.lpp"
+#line 371 "../SqlLexer.lpp"
 {
     /* Form-feed */
     yylval->string_value_->push_back('\f');
   }
 	YY_BREAK
-case 146:
+case 147:
 YY_RULE_SETUP
-#line 373 "../SqlLexer.lpp"
+#line 375 "../SqlLexer.lpp"
 {
     /* Newline */
     yylval->string_value_->push_back('\n');
   }
 	YY_BREAK
-case 147:
+case 148:
 YY_RULE_SETUP
-#line 377 "../SqlLexer.lpp"
+#line 379 "../SqlLexer.lpp"
 {
     /* Carriage-return */
     yylval->string_value_->push_back('\r');
   }
 	YY_BREAK
-case 148:
+case 149:
 YY_RULE_SETUP
-#line 381 "../SqlLexer.lpp"
+#line 383 "../SqlLexer.lpp"
 {
     /* Horizontal Tab */
     yylval->string_value_->push_back('\t');
   }
 	YY_BREAK
-case 149:
-/* rule 149 can match eol */
+case 150:
+/* rule 150 can match eol */
 YY_RULE_SETUP
-#line 385 "../SqlLexer.lpp"
+#line 387 "../SqlLexer.lpp"
 {
     /* Any other character (including actual newline or carriage return) */
     yylval->string_value_->push_back(yytext[1]);
   }
 	YY_BREAK
-case 150:
+case 151:
 YY_RULE_SETUP
-#line 389 "../SqlLexer.lpp"
+#line 391 "../SqlLexer.lpp"
 {
     /* This should only be encountered right before an EOF. */
     delete yylval->string_value_;
@@ -2509,17 +2521,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 151:
+case 152:
 YY_RULE_SETUP
-#line 399 "../SqlLexer.lpp"
+#line 401 "../SqlLexer.lpp"
 {
     /* Two quotes in a row become a single quote (this is specified by the SQL standard). */
     yylval->string_value_->push_back('\'');
   }
 	YY_BREAK
-case 152:
+case 153:
 YY_RULE_SETUP
-#line 403 "../SqlLexer.lpp"
+#line 405 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2528,17 +2540,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 153:
+case 154:
 YY_RULE_SETUP
-#line 411 "../SqlLexer.lpp"
+#line 413 "../SqlLexer.lpp"
 {
     /* Two quotes in a row become a single quote (this is specified by the SQL standard). */
     yylval->string_value_->push_back('"');
   }
 	YY_BREAK
-case 154:
+case 155:
 YY_RULE_SETUP
-#line 415 "../SqlLexer.lpp"
+#line 417 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2546,94 +2558,94 @@ YY_RULE_SETUP
   }
 	YY_BREAK
 
-case 155:
-/* rule 155 can match eol */
+case 156:
+/* rule 156 can match eol */
 YY_RULE_SETUP
-#line 422 "../SqlLexer.lpp"
+#line 424 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 156:
-/* rule 156 can match eol */
+case 157:
+/* rule 157 can match eol */
 YY_RULE_SETUP
-#line 427 "../SqlLexer.lpp"
+#line 429 "../SqlLexer.lpp"
 {
   /* Scan up to a quote or escape sequence. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 157:
-/* rule 157 can match eol */
+case 158:
+/* rule 158 can match eol */
 YY_RULE_SETUP
-#line 432 "../SqlLexer.lpp"
+#line 434 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
 
-case 158:
+case 159:
 YY_RULE_SETUP
-#line 438 "../SqlLexer.lpp"
+#line 440 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(
         yylloc->first_line, yylloc->first_column, std::string(yytext, yyleng));
     return TOKEN_NAME;
   }
 	YY_BREAK
-case 159:
+case 160:
 YY_RULE_SETUP
-#line 444 "../SqlLexer.lpp"
+#line 446 "../SqlLexer.lpp"
 {
     yylval->numeric_literal_value_ = new quickstep::NumericParseLiteralValue(
         yylloc->first_line, yylloc->first_column, yytext);
     return TOKEN_UNSIGNED_NUMVAL;
   }
 	YY_BREAK
-case 160:
-YY_RULE_SETUP
-#line 450 "../SqlLexer.lpp"
-/* comment */
-	YY_BREAK
 case 161:
-/* rule 161 can match eol */
 YY_RULE_SETUP
 #line 452 "../SqlLexer.lpp"
-{ yycolumn = 0; }
+/* comment */
 	YY_BREAK
 case 162:
+/* rule 162 can match eol */
 YY_RULE_SETUP
 #line 454 "../SqlLexer.lpp"
+{ yycolumn = 0; }
+	YY_BREAK
+case 163:
+YY_RULE_SETUP
+#line 456 "../SqlLexer.lpp"
 ; /* ignore white space */
 	YY_BREAK
 /* CONDITION_SQL */
 case YY_STATE_EOF(INITIAL):
 case YY_STATE_EOF(CONDITION_COMMAND):
 case YY_STATE_EOF(CONDITION_SQL):
-#line 458 "../SqlLexer.lpp"
+#line 460 "../SqlLexer.lpp"
 {
   /* All conditions except for mutli-state string extracting conditions. */
   BEGIN(INITIAL);
   return TOKEN_EOF;
 }
 	YY_BREAK
-case 163:
+case 164:
 YY_RULE_SETUP
-#line 464 "../SqlLexer.lpp"
+#line 466 "../SqlLexer.lpp"
 {
   BEGIN(INITIAL);
   quickstep_yyerror(NULL, yyscanner, NULL, "illegal character");
   return TOKEN_LEX_ERROR;
 }
 	YY_BREAK
-case 164:
+case 165:
 YY_RULE_SETUP
-#line 470 "../SqlLexer.lpp"
+#line 472 "../SqlLexer.lpp"
 YY_FATAL_ERROR( "flex scanner jammed" );
 	YY_BREAK
-#line 2636 "SqlLexer_gen.cpp"
+#line 2648 "SqlLexer_gen.cpp"
 
 	case YY_END_OF_BUFFER:
 		{
@@ -2931,7 +2943,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 		while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state )
 			{
 			yy_current_state = (int) yy_def[yy_current_state];
-			if ( yy_current_state >= 589 )
+			if ( yy_current_state >= 593 )
 				yy_c = yy_meta[yy_c];
 			}
 		yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
@@ -2960,11 +2972,11 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 	while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state )
 		{
 		yy_current_state = (int) yy_def[yy_current_state];
-		if ( yy_current_state >= 589 )
+		if ( yy_current_state >= 593 )
 			yy_c = yy_meta[yy_c];
 		}
 	yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
-	yy_is_jam = (yy_current_state == 588);
+	yy_is_jam = (yy_current_state == 592);
 
 	(void)yyg;
 	return yy_is_jam ? 0 : yy_current_state;
@@ -3794,6 +3806,6 @@ void yyfree (void * ptr , yyscan_t yyscanner)
 
 #define YYTABLES_NAME "yytables"
 
-#line 470 "../SqlLexer.lpp"
+#line 472 "../SqlLexer.lpp"
 
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/preprocessed/SqlLexer_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.hpp b/parser/preprocessed/SqlLexer_gen.hpp
index 5fafae5..9990de6 100644
--- a/parser/preprocessed/SqlLexer_gen.hpp
+++ b/parser/preprocessed/SqlLexer_gen.hpp
@@ -733,7 +733,7 @@ extern int yylex \
 #undef yyTABLES_NAME
 #endif
 
-#line 470 "../SqlLexer.lpp"
+#line 472 "../SqlLexer.lpp"
 
 
 #line 739 "SqlLexer_gen.hpp"


[03/11] incubator-quickstep git commit: Fix dag visualizer

Posted by ji...@apache.org.
Fix dag visualizer


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

Branch: refs/heads/transitive-closure
Commit: a57638e400d6174be9f644b89f8b9353582c5889
Parents: b45cdbc
Author: jianqiao <ji...@node-2.jianqiao.quickstep-pg0.wisc.cloudlab.us>
Authored: Mon Nov 27 18:27:27 2017 -0600
Committer: jianqiao <ji...@node-2.jianqiao.quickstep-pg0.wisc.cloudlab.us>
Committed: Mon Nov 27 18:27:27 2017 -0600

----------------------------------------------------------------------
 utility/ExecutionDAGVisualizer.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a57638e4/utility/ExecutionDAGVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/ExecutionDAGVisualizer.cpp b/utility/ExecutionDAGVisualizer.cpp
index 8059ef3..bb8fe8f 100644
--- a/utility/ExecutionDAGVisualizer.cpp
+++ b/utility/ExecutionDAGVisualizer.cpp
@@ -293,7 +293,7 @@ void ExecutionDAGVisualizer::bindProfilingStats(
           "effective concurrency: " + FormatDigits(concurrency, 2));
 
       DCHECK(workorders_count.find(node_index) != workorders_count.end());
-      const std::size_t workorders_count_for_node = workorders_count.at(node_index);
+      const std::size_t workorders_count_for_node = workorders_count[node_index];
       if (workorders_count_for_node > 0) {
         mean_time_per_workorder[node_index] =
             mean_time_per_workorder[node_index] /


[07/11] incubator-quickstep git commit: Initialize updates for transitive closure

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index 142059d..130f353 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -78,107 +78,108 @@ extern int quickstep_yydebug;
     TOKEN_BLOCKPROPERTIES = 288,
     TOKEN_BLOCKSAMPLE = 289,
     TOKEN_BLOOM_FILTER = 290,
-    TOKEN_CSB_TREE = 291,
-    TOKEN_BY = 292,
-    TOKEN_CASE = 293,
-    TOKEN_CHARACTER = 294,
-    TOKEN_CHECK = 295,
-    TOKEN_COLUMN = 296,
+    TOKEN_BY = 291,
+    TOKEN_CASE = 292,
+    TOKEN_CHARACTER = 293,
+    TOKEN_CHECK = 294,
+    TOKEN_COLUMN = 295,
+    TOKEN_CONNECT = 296,
     TOKEN_CONSTRAINT = 297,
     TOKEN_COPY = 298,
     TOKEN_CREATE = 299,
-    TOKEN_CURRENT = 300,
-    TOKEN_DATE = 301,
-    TOKEN_DATETIME = 302,
-    TOKEN_DAY = 303,
-    TOKEN_DECIMAL = 304,
-    TOKEN_DEFAULT = 305,
-    TOKEN_DELETE = 306,
-    TOKEN_DESC = 307,
-    TOKEN_DISTINCT = 308,
-    TOKEN_DOUBLE = 309,
-    TOKEN_DROP = 310,
-    TOKEN_ELSE = 311,
-    TOKEN_END = 312,
-    TOKEN_EXISTS = 313,
-    TOKEN_EXTRACT = 314,
-    TOKEN_FALSE = 315,
-    TOKEN_FIRST = 316,
-    TOKEN_FLOAT = 317,
-    TOKEN_FOLLOWING = 318,
-    TOKEN_FOR = 319,
-    TOKEN_FOREIGN = 320,
-    TOKEN_FROM = 321,
-    TOKEN_FULL = 322,
-    TOKEN_GROUP = 323,
-    TOKEN_HASH = 324,
-    TOKEN_HAVING = 325,
-    TOKEN_HOUR = 326,
-    TOKEN_IN = 327,
-    TOKEN_INDEX = 328,
-    TOKEN_INNER = 329,
-    TOKEN_INSERT = 330,
-    TOKEN_INTEGER = 331,
-    TOKEN_INTERVAL = 332,
-    TOKEN_INTO = 333,
-    TOKEN_JOIN = 334,
-    TOKEN_KEY = 335,
-    TOKEN_LAST = 336,
-    TOKEN_LEFT = 337,
-    TOKEN_LIMIT = 338,
-    TOKEN_LONG = 339,
-    TOKEN_MINUTE = 340,
-    TOKEN_MONTH = 341,
-    TOKEN_NULL = 342,
-    TOKEN_NULLS = 343,
-    TOKEN_OFF = 344,
-    TOKEN_ON = 345,
-    TOKEN_ORDER = 346,
-    TOKEN_OUTER = 347,
-    TOKEN_OVER = 348,
-    TOKEN_PARTITION = 349,
-    TOKEN_PARTITIONS = 350,
-    TOKEN_PERCENT = 351,
-    TOKEN_PRECEDING = 352,
-    TOKEN_PRIMARY = 353,
-    TOKEN_PRIORITY = 354,
-    TOKEN_QUIT = 355,
-    TOKEN_RANGE = 356,
-    TOKEN_REAL = 357,
-    TOKEN_REFERENCES = 358,
-    TOKEN_RIGHT = 359,
-    TOKEN_ROW = 360,
-    TOKEN_ROW_DELIMITER = 361,
-    TOKEN_ROWS = 362,
-    TOKEN_SECOND = 363,
-    TOKEN_SELECT = 364,
-    TOKEN_SET = 365,
-    TOKEN_SMA = 366,
-    TOKEN_SMALLINT = 367,
-    TOKEN_STDERR = 368,
-    TOKEN_STDOUT = 369,
-    TOKEN_SUBSTRING = 370,
-    TOKEN_TABLE = 371,
-    TOKEN_THEN = 372,
-    TOKEN_TIME = 373,
-    TOKEN_TIMESTAMP = 374,
-    TOKEN_TO = 375,
-    TOKEN_TRUE = 376,
-    TOKEN_TUPLESAMPLE = 377,
-    TOKEN_UNBOUNDED = 378,
-    TOKEN_UNIQUE = 379,
-    TOKEN_UPDATE = 380,
-    TOKEN_USING = 381,
-    TOKEN_VALUES = 382,
-    TOKEN_VARCHAR = 383,
-    TOKEN_WHEN = 384,
-    TOKEN_WHERE = 385,
-    TOKEN_WINDOW = 386,
-    TOKEN_WITH = 387,
-    TOKEN_YEAR = 388,
-    TOKEN_YEARMONTH = 389,
-    TOKEN_EOF = 390,
-    TOKEN_LEX_ERROR = 391
+    TOKEN_CSB_TREE = 300,
+    TOKEN_CURRENT = 301,
+    TOKEN_DATE = 302,
+    TOKEN_DATETIME = 303,
+    TOKEN_DAY = 304,
+    TOKEN_DECIMAL = 305,
+    TOKEN_DEFAULT = 306,
+    TOKEN_DELETE = 307,
+    TOKEN_DESC = 308,
+    TOKEN_DISTINCT = 309,
+    TOKEN_DOUBLE = 310,
+    TOKEN_DROP = 311,
+    TOKEN_ELSE = 312,
+    TOKEN_END = 313,
+    TOKEN_EXISTS = 314,
+    TOKEN_EXTRACT = 315,
+    TOKEN_FALSE = 316,
+    TOKEN_FIRST = 317,
+    TOKEN_FLOAT = 318,
+    TOKEN_FOLLOWING = 319,
+    TOKEN_FOR = 320,
+    TOKEN_FOREIGN = 321,
+    TOKEN_FROM = 322,
+    TOKEN_FULL = 323,
+    TOKEN_GROUP = 324,
+    TOKEN_HASH = 325,
+    TOKEN_HAVING = 326,
+    TOKEN_HOUR = 327,
+    TOKEN_IN = 328,
+    TOKEN_INDEX = 329,
+    TOKEN_INNER = 330,
+    TOKEN_INSERT = 331,
+    TOKEN_INTEGER = 332,
+    TOKEN_INTERVAL = 333,
+    TOKEN_INTO = 334,
+    TOKEN_JOIN = 335,
+    TOKEN_KEY = 336,
+    TOKEN_LAST = 337,
+    TOKEN_LEFT = 338,
+    TOKEN_LIMIT = 339,
+    TOKEN_LONG = 340,
+    TOKEN_MINUTE = 341,
+    TOKEN_MONTH = 342,
+    TOKEN_NULL = 343,
+    TOKEN_NULLS = 344,
+    TOKEN_OFF = 345,
+    TOKEN_ON = 346,
+    TOKEN_ORDER = 347,
+    TOKEN_OUTER = 348,
+    TOKEN_OVER = 349,
+    TOKEN_PARTITION = 350,
+    TOKEN_PARTITIONS = 351,
+    TOKEN_PERCENT = 352,
+    TOKEN_PRECEDING = 353,
+    TOKEN_PRIMARY = 354,
+    TOKEN_PRIORITY = 355,
+    TOKEN_QUIT = 356,
+    TOKEN_RANGE = 357,
+    TOKEN_REAL = 358,
+    TOKEN_REFERENCES = 359,
+    TOKEN_RIGHT = 360,
+    TOKEN_ROW = 361,
+    TOKEN_ROW_DELIMITER = 362,
+    TOKEN_ROWS = 363,
+    TOKEN_SECOND = 364,
+    TOKEN_SELECT = 365,
+    TOKEN_SET = 366,
+    TOKEN_SMA = 367,
+    TOKEN_SMALLINT = 368,
+    TOKEN_STDERR = 369,
+    TOKEN_STDOUT = 370,
+    TOKEN_SUBSTRING = 371,
+    TOKEN_TABLE = 372,
+    TOKEN_THEN = 373,
+    TOKEN_TIME = 374,
+    TOKEN_TIMESTAMP = 375,
+    TOKEN_TO = 376,
+    TOKEN_TRUE = 377,
+    TOKEN_TUPLESAMPLE = 378,
+    TOKEN_UNBOUNDED = 379,
+    TOKEN_UNIQUE = 380,
+    TOKEN_UPDATE = 381,
+    TOKEN_USING = 382,
+    TOKEN_VALUES = 383,
+    TOKEN_VARCHAR = 384,
+    TOKEN_WHEN = 385,
+    TOKEN_WHERE = 386,
+    TOKEN_WINDOW = 387,
+    TOKEN_WITH = 388,
+    TOKEN_YEAR = 389,
+    TOKEN_YEARMONTH = 390,
+    TOKEN_EOF = 391,
+    TOKEN_LEX_ERROR = 392
   };
 #endif
 
@@ -187,7 +188,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 121 "../SqlParser.ypp" /* yacc.c:1915  */
+#line 122 "../SqlParser.ypp" /* yacc.c:1915  */
 
   quickstep::ParseString *string_value_;
 
@@ -289,7 +290,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 293 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 294 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 791434a..305890f 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -222,6 +222,7 @@ target_link_libraries(quickstep_queryexecution_QueryContext
                       quickstep_storage_HashTableFactory
                       quickstep_storage_InsertDestination
                       quickstep_storage_InsertDestination_proto
+                      quickstep_storage_TransitiveClosureState
                       quickstep_storage_WindowAggregationOperationState
                       quickstep_threading_SpinSharedMutex
                       quickstep_types_TypedValue

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_execution/QueryContext.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 52aa4dc..becddb6 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -37,6 +37,7 @@
 #include "storage/HashTableFactory.hpp"
 #include "storage/InsertDestination.hpp"
 #include "storage/InsertDestination.pb.h"
+#include "storage/TransitiveClosureState.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/Tuple.hpp"
 #include "utility/SortConfiguration.hpp"
@@ -168,6 +169,11 @@ QueryContext::QueryContext(const serialization::QueryContext &proto,
                                                               database,
                                                               storage_manager));
   }
+
+  for (int i = 0; i < proto.transitive_closure_states_size(); ++i) {
+    transitive_closure_states_.emplace_back(
+        std::make_unique<TransitiveClosureState>(proto.transitive_closure_states(i).range()));
+  }
 }
 
 bool QueryContext::ProtoIsValid(const serialization::QueryContext &proto,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_execution/QueryContext.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index e65f096..7cee2b9 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -33,6 +33,7 @@
 #include "storage/AggregationOperationState.hpp"
 #include "storage/HashTable.hpp"
 #include "storage/InsertDestination.hpp"
+#include "storage/TransitiveClosureState.hpp"
 #include "storage/WindowAggregationOperationState.hpp"
 #include "threading/SpinSharedMutex.hpp"
 #include "types/containers/Tuple.hpp"
@@ -586,6 +587,16 @@ class QueryContext {
     return window_aggregation_states_[id].release();
   }
 
+  TransitiveClosureState* getTransitiveClosureState(const std::size_t id) const {
+    DCHECK_LT(id, transitive_closure_states_.size());
+    return transitive_closure_states_[id].get();
+  }
+
+  void destroyTransitiveClosureState(const std::size_t id) {
+    DCHECK_LT(id, transitive_closure_states_.size());
+    transitive_closure_states_[id].reset();
+  }
+
   /**
    * @brief Get the total memory footprint of the temporary data structures
    *        used for query execution (e.g. join hash tables, aggregation hash
@@ -650,6 +661,7 @@ class QueryContext {
   std::vector<std::unique_ptr<Tuple>> tuples_;
   std::vector<std::unordered_map<attribute_id, std::unique_ptr<const Scalar>>> update_groups_;
   std::vector<std::unique_ptr<WindowAggregationOperationState>> window_aggregation_states_;
+  std::vector<std::unique_ptr<TransitiveClosureState>> transitive_closure_states_;
 
   mutable SpinSharedMutex<false> hash_tables_mutex_;
   mutable SpinSharedMutex<false> aggregation_states_mutex_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_execution/QueryContext.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.proto b/query_execution/QueryContext.proto
index 6ad51b5..630fb34 100644
--- a/query_execution/QueryContext.proto
+++ b/query_execution/QueryContext.proto
@@ -54,6 +54,10 @@ message QueryContext {
     required int32 relation_id = 1;
     repeated UpdateAssignment update_assignments = 2;
   }
+  
+  message TransitiveClosureState {
+    required int32 range = 1;
+  }
 
   repeated AggregationOperationStateContext aggregation_states = 1;
   repeated GeneratorFunctionHandle generator_functions = 2;
@@ -70,10 +74,9 @@ message QueryContext {
   repeated UpdateGroup update_groups = 11;
 
   repeated WindowAggregationOperationState window_aggregation_states = 12;
+  repeated TransitiveClosureState transitive_closure_states = 13;
 
-  required uint64 query_id = 13;
-
-  repeated uint64 num_partitions_for_nested_loops_joins = 14;
-
-  optional uint64 num_partitions_for_lips = 15 [default = 1];
+  required uint64 query_id = 14;
+  repeated uint64 num_partitions_for_nested_loops_joins = 15;
+  optional uint64 num_partitions_for_lips = 16 [default = 1];
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 011cecb..b4bd108 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -117,6 +117,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_physical_TableGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_physical_TransitiveClosure
                       quickstep_queryoptimizer_physical_UnionAll
                       quickstep_queryoptimizer_physical_UpdateTable
                       quickstep_queryoptimizer_physical_WindowAggregate
@@ -124,6 +125,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_relationaloperators_BuildAggregationExistenceMapOperator
                       quickstep_relationaloperators_BuildHashOperator
                       quickstep_relationaloperators_BuildLIPFilterOperator
+                      quickstep_relationaloperators_BuildTransitiveClosureOperator
                       quickstep_relationaloperators_CreateIndexOperator
                       quickstep_relationaloperators_CreateTableOperator
                       quickstep_relationaloperators_DeleteOperator
@@ -133,6 +135,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_relationaloperators_HashJoinOperator
                       quickstep_relationaloperators_InitializeAggregationOperator
+                      quickstep_relationaloperators_InitializeTransitiveClosureOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_relationaloperators_RelationalOperator
@@ -144,6 +147,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_relationaloperators_TableExportOperator
                       quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_relationaloperators_TextScanOperator
+                      quickstep_relationaloperators_TransitiveClosureOperator
                       quickstep_relationaloperators_UnionAllOperator
                       quickstep_relationaloperators_UpdateOperator
                       quickstep_relationaloperators_WindowAggregationOperator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 3ef74ee..8f29271 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -101,6 +101,7 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/physical/TransitiveClosure.hpp"
 #include "query_optimizer/physical/UnionAll.hpp"
 #include "query_optimizer/physical/UpdateTable.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
@@ -108,6 +109,7 @@
 #include "relational_operators/BuildAggregationExistenceMapOperator.hpp"
 #include "relational_operators/BuildHashOperator.hpp"
 #include "relational_operators/BuildLIPFilterOperator.hpp"
+#include "relational_operators/BuildTransitiveClosureOperator.hpp"
 #include "relational_operators/CreateIndexOperator.hpp"
 #include "relational_operators/CreateTableOperator.hpp"
 #include "relational_operators/DeleteOperator.hpp"
@@ -117,6 +119,7 @@
 #include "relational_operators/FinalizeAggregationOperator.hpp"
 #include "relational_operators/HashJoinOperator.hpp"
 #include "relational_operators/InitializeAggregationOperator.hpp"
+#include "relational_operators/InitializeTransitiveClosureOperator.hpp"
 #include "relational_operators/InsertOperator.hpp"
 #include "relational_operators/NestedLoopsJoinOperator.hpp"
 #include "relational_operators/RelationalOperator.hpp"
@@ -128,6 +131,7 @@
 #include "relational_operators/TableExportOperator.hpp"
 #include "relational_operators/TableGeneratorOperator.hpp"
 #include "relational_operators/TextScanOperator.hpp"
+#include "relational_operators/TransitiveClosureOperator.hpp"
 #include "relational_operators/UnionAllOperator.hpp"
 #include "relational_operators/UpdateOperator.hpp"
 #include "relational_operators/WindowAggregationOperator.hpp"
@@ -245,6 +249,60 @@ bool CheckAggregatePartitioned(const std::size_t num_aggregate_functions,
   return estimated_num_groups >= FLAGS_partition_aggregation_num_groups_threshold;
 }
 
+void CheckExactPositiveRangeForAttribute(const P::PhysicalPtr &physical,
+                                         const E::AttributeReferencePtr &attribute,
+                                         cost::StarSchemaSimpleCostModel *cost_model,
+                                         int *range) {
+  if (attribute->getValueType().getTypeID() != kInt) {
+    THROW_SQL_ERROR() << "Can not apply fast transitive closure with non-integer types";
+  }
+
+  bool min_is_exact;
+  bool max_is_exact;
+  const TypedValue min_value =
+      cost_model->findMinValueStat(physical, attribute, &min_is_exact);
+  const TypedValue max_value =
+      cost_model->findMaxValueStat(physical, attribute, &max_is_exact);
+
+  if (min_value.isNull() || max_value.isNull() || (!min_is_exact) || (!max_is_exact)) {
+    THROW_SQL_ERROR() << "Can not apply fast transitive closure without exact statistics";
+  }
+
+  const int min_cpp_value = min_value.getLiteral<int>();
+  const int max_cpp_value = max_value.getLiteral<int>();
+  if (min_cpp_value < 0) {
+    THROW_SQL_ERROR() << "Can not apply fast transitive closure with negative values";
+  }
+  if (max_cpp_value > 1000000) {
+    THROW_SQL_ERROR() << "Can not apply fast transitive closure with values larger than 100000";
+  }
+  *range = max_cpp_value;
+}
+
+int GetTransitiveClosureValueRange(const P::TransitiveClosurePtr &transitive_closure,
+                                   cost::StarSchemaSimpleCostModel *cost_model) {
+  const P::PhysicalPtr start = transitive_closure->start();
+  const P::PhysicalPtr edge = transitive_closure->edge();
+
+  const auto start_attrs = start->getOutputAttributes();
+  const auto edge_attrs = edge->getOutputAttributes();
+  DCHECK_EQ(1u, start_attrs.size());
+  DCHECK_EQ(2u, edge_attrs.size());
+
+  for (const auto &attr : {start_attrs[0], edge_attrs[0], edge_attrs[1]}) {
+    if (attr->getValueType().getTypeID() != kInt) {
+      THROW_SQL_ERROR() << "Can not apply fast transitive closure with non-integer types";
+    }
+  }
+
+  int source_max_value;
+  int destination_max_value;
+  CheckExactPositiveRangeForAttribute(edge, edge_attrs[0], cost_model, &source_max_value);
+  CheckExactPositiveRangeForAttribute(edge, edge_attrs[1], cost_model, &destination_max_value);
+
+  return std::max(source_max_value, destination_max_value) + 1;
+}
+
 }  // namespace
 
 constexpr QueryPlan::DAGNodeIndex ExecutionGenerator::CatalogRelationInfo::kInvalidOperatorIndex;
@@ -396,6 +454,9 @@ void ExecutionGenerator::generatePlanInternal(
     case P::PhysicalType::kTableReference:
       return convertTableReference(
           std::static_pointer_cast<const P::TableReference>(physical_plan));
+    case P::PhysicalType::kTransitiveClosure:
+      return convertTransitiveClosure(
+          std::static_pointer_cast<const P::TransitiveClosure>(physical_plan));
     case P::PhysicalType::kUnionAll:
       return convertUnionAll(
           std::static_pointer_cast<const P::UnionAll>(physical_plan));
@@ -438,8 +499,7 @@ void ExecutionGenerator::createTemporaryCatalogRelation(
                              project_expression->getValueType(),
                              aid,
                              project_expression->attribute_alias()));
-    attribute_substitution_map_[project_expression->id()] =
-        catalog_attribute.get();
+    attribute_substitution_map_[project_expression->id()] = catalog_attribute.get();
     catalog_relation->addAttribute(catalog_attribute.release());
     ++aid;
   }
@@ -2362,5 +2422,85 @@ void ExecutionGenerator::convertWindowAggregate(
                                             output_relation);
 }
 
+void ExecutionGenerator::convertTransitiveClosure(
+    const physical::TransitiveClosurePtr &physical_plan) {
+  const int range = GetTransitiveClosureValueRange(
+      physical_plan, cost_model_for_aggregation_.get());
+
+  const std::size_t transitive_closure_state_index =
+      query_context_proto_->transitive_closure_states_size();
+  S::QueryContext::TransitiveClosureState *transitive_closure_state_proto =
+      query_context_proto_->add_transitive_closure_states();
+
+  transitive_closure_state_proto->set_range(range);
+
+  const CatalogRelationInfo *start_relation_info =
+      findRelationInfoOutputByPhysical(physical_plan->start());
+  const CatalogRelationInfo *edge_relation_info =
+      findRelationInfoOutputByPhysical(physical_plan->edge());
+
+  const attribute_id start_attr_id =
+      attribute_substitution_map_[physical_plan->start_attr()->id()]->getID();
+  const attribute_id source_attr_id =
+      attribute_substitution_map_[physical_plan->source_attr()->id()]->getID();
+  const attribute_id destination_attr_id =
+      attribute_substitution_map_[physical_plan->destination_attr()->id()]->getID();
+
+  const QueryPlan::DAGNodeIndex init_tc_operator_index =
+      execution_plan_->addRelationalOperator(
+          new InitializeTransitiveClosureOperator(query_handle_->query_id(),
+                                                  transitive_closure_state_index));
+
+  const QueryPlan::DAGNodeIndex build_tc_operator_index =
+      execution_plan_->addRelationalOperator(
+          new BuildTransitiveClosureOperator(query_handle_->query_id(),
+                                             transitive_closure_state_index,
+                                             *start_relation_info->relation,
+                                             *edge_relation_info->relation,
+                                             start_relation_info->isStoredRelation(),
+                                             edge_relation_info->isStoredRelation(),
+                                             start_attr_id,
+                                             source_attr_id,
+                                             destination_attr_id));
+
+  execution_plan_->addDirectDependency(build_tc_operator_index,
+                                       init_tc_operator_index,
+                                       true /* is_pipeline_breaker */);
+
+  if (!start_relation_info->isStoredRelation()) {
+    execution_plan_->addDirectDependency(build_tc_operator_index,
+                                         start_relation_info->producer_operator_index,
+                                         false /* is_pipeline_breaker */);
+  }
+  if (!edge_relation_info->isStoredRelation()) {
+    execution_plan_->addDirectDependency(build_tc_operator_index,
+                                         edge_relation_info->producer_operator_index,
+                                         false /* is_pipeline_breaker */);
+  }
+
+  // Create InsertDestination proto.
+  const CatalogRelation *output_relation = nullptr;
+  const QueryContext::insert_destination_id insert_destination_index =
+      query_context_proto_->insert_destinations_size();
+  S::InsertDestination *insert_destination_proto =
+      query_context_proto_->add_insert_destinations();
+  createTemporaryCatalogRelation(physical_plan,
+                                 &output_relation,
+                                 insert_destination_proto);
+
+
+  (void)insert_destination_index;
+
+  // TODO: fix
+  insert_destination_proto->set_relational_op_index(build_tc_operator_index /* FIX */);
+  physical_to_output_relation_map_.emplace(
+      std::piecewise_construct,
+      std::forward_as_tuple(physical_plan),
+      std::forward_as_tuple(build_tc_operator_index /* FIX */, output_relation));
+
+  temporary_relation_info_vec_.emplace_back(build_tc_operator_index /* FIX */,
+                                            output_relation);
+}
+
 }  // namespace optimizer
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index bc9f88b..8fe853f 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -63,6 +63,7 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/physical/TransitiveClosure.hpp"
 #include "query_optimizer/physical/UnionAll.hpp"
 #include "query_optimizer/physical/UpdateTable.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
@@ -374,6 +375,8 @@ class ExecutionGenerator {
    */
   void convertTableGenerator(const physical::TableGeneratorPtr &physical_plan);
 
+  void convertTransitiveClosure(const physical::TransitiveClosurePtr &physical_plan);
+
   /**
    * @brief Converts a physical WindowAggregate to a WindowAggregation operator.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 865cd11..b7b0db0 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -194,7 +194,7 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
              << physical_plan_->toString();
   }
 
-  DVLOG(4) << "Optimized physical plan:\n" << physical_plan_->toString();
+  std::cerr << "Optimized physical plan:\n" << physical_plan_->toString();
 
   if (FLAGS_visualize_plan) {
     quickstep::PlanVisualizer plan_visualizer;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index 3d4ee93..de8c32e 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -45,6 +45,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_SimpleCostModel
                       quickstep_queryoptimizer_physical_TableGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_physical_TransitiveClosure
                       quickstep_queryoptimizer_physical_UnionAll
                       quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_utility_Macros)
@@ -80,6 +81,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       quickstep_queryoptimizer_physical_TableGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_physical_TransitiveClosure
                       quickstep_queryoptimizer_physical_UnionAll
                       quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_types_NullType

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index cc46c00..ac48908 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -38,6 +38,7 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/physical/TransitiveClosure.hpp"
 #include "query_optimizer/physical/UnionAll.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
 
@@ -88,6 +89,9 @@ std::size_t SimpleCostModel::estimateCardinality(
     case P::PhysicalType::kSort:
       return estimateCardinalityForSort(
           std::static_pointer_cast<const P::Sort>(physical_plan));
+    case P::PhysicalType::kTransitiveClosure:
+      return estimateCardinalityForTransitiveClosure(
+          std::static_pointer_cast<const P::TransitiveClosure>(physical_plan));
     case P::PhysicalType::kWindowAggregate:
       return estimateCardinalityForWindowAggregate(
           std::static_pointer_cast<const P::WindowAggregate>(physical_plan));
@@ -176,6 +180,12 @@ std::size_t SimpleCostModel::estimateCardinalityForUnionAll(
   return cardinality;
 }
 
+std::size_t SimpleCostModel::estimateCardinalityForTransitiveClosure(
+    const physical::TransitiveClosurePtr &physical_plan) {
+  return std::max(estimateCardinality(physical_plan->start()),
+                  estimateCardinality(physical_plan->edge()));
+}
+
 }  // namespace cost
 }  // namespace optimizer
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/cost_model/SimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.hpp b/query_optimizer/cost_model/SimpleCostModel.hpp
index 653e115..7c04326 100644
--- a/query_optimizer/cost_model/SimpleCostModel.hpp
+++ b/query_optimizer/cost_model/SimpleCostModel.hpp
@@ -35,6 +35,7 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/physical/TransitiveClosure.hpp"
 #include "query_optimizer/physical/UnionAll.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
 #include "utility/Macros.hpp"
@@ -115,6 +116,9 @@ class SimpleCostModel : public CostModel {
   std::size_t estimateCardinalityForUnionAll(
       const physical::UnionAllPtr &physical_plan);
 
+  std::size_t estimateCardinalityForTransitiveClosure(
+      const physical::TransitiveClosurePtr &physical_plan);
+
   const std::vector<physical::PhysicalPtr> &shared_subplans_;
 
   DISALLOW_COPY_AND_ASSIGN(SimpleCostModel);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 729a563..875c672 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -55,6 +55,7 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/physical/TransitiveClosure.hpp"
 #include "query_optimizer/physical/UnionAll.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
@@ -116,6 +117,9 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinality(
     case P::PhysicalType::kSort:
       return estimateCardinalityForSort(
           std::static_pointer_cast<const P::Sort>(physical_plan));
+    case P::PhysicalType::kTransitiveClosure:
+      return estimateCardinalityForTransitiveClosure(
+          std::static_pointer_cast<const P::TransitiveClosure>(physical_plan));
     case P::PhysicalType::kWindowAggregate:
       return estimateCardinalityForWindowAggregate(
           std::static_pointer_cast<const P::WindowAggregate>(physical_plan));
@@ -216,6 +220,13 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForUnionAll(
   return cardinality;
 }
 
+std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTransitiveClosure(
+    const P::TransitiveClosurePtr &physical_plan) {
+  std::size_t left_cardinality = estimateCardinality(physical_plan->start());
+  std::size_t right_cardinality = estimateCardinality(physical_plan->edge());
+  return std::max(left_cardinality, right_cardinality);
+}
+
 std::size_t StarSchemaSimpleCostModel::estimateNumGroupsForAggregate(
     const physical::AggregatePtr &aggregate) {
   if (aggregate->grouping_expressions().empty()) {
@@ -286,6 +297,7 @@ std::size_t StarSchemaSimpleCostModel::estimateNumDistinctValues(
         return static_cast<std::size_t>(
             left_child_num_distinct_values * right_child_selectivity + 0.5);
       }
+      break;
     }
     case P::PhysicalType::kHashJoin: {
       const P::HashJoinPtr &hash_join =
@@ -306,6 +318,20 @@ std::size_t StarSchemaSimpleCostModel::estimateNumDistinctValues(
         return static_cast<std::size_t>(
             right_child_num_distinct_values * left_child_selectivity * filter_selectivity + 0.5);
       }
+      break;
+    }
+    case P::PhysicalType::kTransitiveClosure: {
+      const P::TransitiveClosurePtr &transitive_closure =
+          std::static_pointer_cast<const P::TransitiveClosure>(physical_plan);
+      const P::PhysicalPtr start = transitive_closure->start();
+      if (E::ContainsExprId(start->getOutputAttributes(), attribute_id)) {
+        return estimateNumDistinctValues(attribute_id, start);
+      }
+      const P::PhysicalPtr edge = transitive_closure->edge();
+      if (E::ContainsExprId(edge->getOutputAttributes(), attribute_id)) {
+        return estimateNumDistinctValues(attribute_id, edge);
+      }
+      break;
     }
     default:
       break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index a056866..2af0137 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -39,6 +39,7 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/physical/TransitiveClosure.hpp"
 #include "query_optimizer/physical/UnionAll.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
 #include "types/TypedValue.hpp"
@@ -227,6 +228,9 @@ class StarSchemaSimpleCostModel : public CostModel {
   std::size_t estimateCardinalityForTopLevelPlan(
       const physical::TopLevelPlanPtr &physical_plan);
 
+  std::size_t estimateCardinalityForTransitiveClosure(
+      const physical::TransitiveClosurePtr &physical_plan);
+
   std::size_t estimateCardinalityForWindowAggregate(
       const physical::WindowAggregatePtr &physical_plan);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/logical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CMakeLists.txt b/query_optimizer/logical/CMakeLists.txt
index 37e9735..ad723bc 100644
--- a/query_optimizer/logical/CMakeLists.txt
+++ b/query_optimizer/logical/CMakeLists.txt
@@ -44,6 +44,7 @@ add_library(quickstep_queryoptimizer_logical_Sort Sort.cpp Sort.hpp)
 add_library(quickstep_queryoptimizer_logical_TableReference TableReference.cpp TableReference.hpp)
 add_library(quickstep_queryoptimizer_logical_TableGenerator ../../empty_src.cpp TableGenerator.hpp)
 add_library(quickstep_queryoptimizer_logical_TopLevelPlan TopLevelPlan.cpp TopLevelPlan.hpp)
+add_library(quickstep_queryoptimizer_logical_TransitiveClosure TransitiveClosure.cpp TransitiveClosure.hpp)
 add_library(quickstep_queryoptimizer_logical_UpdateTable UpdateTable.cpp UpdateTable.hpp)
 add_library(quickstep_queryoptimizer_logical_WindowAggregate WindowAggregate.cpp WindowAggregate.hpp)
 
@@ -270,6 +271,13 @@ target_link_libraries(quickstep_queryoptimizer_logical_TopLevelPlan
                       quickstep_queryoptimizer_logical_LogicalType
                       quickstep_utility_Cast
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_logical_TransitiveClosure
+                      glog
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_logical_Logical
+                      quickstep_queryoptimizer_logical_LogicalType
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_logical_UpdateTable
                       glog
                       quickstep_queryoptimizer_OptimizerTree
@@ -324,5 +332,6 @@ target_link_libraries(quickstep_queryoptimizer_logical
                       quickstep_queryoptimizer_logical_TableGenerator
                       quickstep_queryoptimizer_logical_TableReference
                       quickstep_queryoptimizer_logical_TopLevelPlan
+                      quickstep_queryoptimizer_logical_TransitiveClosure
                       quickstep_queryoptimizer_logical_UpdateTable
                       quickstep_queryoptimizer_logical_WindowAggregate)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/logical/LogicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/LogicalType.hpp b/query_optimizer/logical/LogicalType.hpp
index d8b85dd..6f4ff24 100644
--- a/query_optimizer/logical/LogicalType.hpp
+++ b/query_optimizer/logical/LogicalType.hpp
@@ -53,6 +53,7 @@ enum class LogicalType {
   kTableGenerator,
   kTableReference,
   kTopLevelPlan,
+  kTransitiveClosure,
   kUpdateTable,
   kWindowAggregate
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/logical/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/PatternMatcher.hpp b/query_optimizer/logical/PatternMatcher.hpp
index 23c2872..dc49c3b 100644
--- a/query_optimizer/logical/PatternMatcher.hpp
+++ b/query_optimizer/logical/PatternMatcher.hpp
@@ -47,6 +47,7 @@ class SharedSubplanReference;
 class Sort;
 class TableReference;
 class TopLevelPlan;
+class TransitiveClosure;
 class UpdateTable;
 class WindowAggregate;
 
@@ -134,6 +135,7 @@ using SomeSharedSubplanReference = SomeLogicalNode<SharedSubplanReference, Logic
 using SomeSort = SomeLogicalNode<Sort, LogicalType::kSort>;
 using SomeTableReference = SomeLogicalNode<TableReference, LogicalType::kTableReference>;
 using SomeTopLevelPlan = SomeLogicalNode<TopLevelPlan, LogicalType::kTopLevelPlan>;
+using SomeTransitiveClosure = SomeLogicalNode<TransitiveClosure, LogicalType::kTransitiveClosure>;
 using SomeUpdateTable = SomeLogicalNode<UpdateTable, LogicalType::kUpdateTable>;
 using SomeWindowAggregate = SomeLogicalNode<WindowAggregate, LogicalType::kWindowAggregate>;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/logical/TransitiveClosure.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/TransitiveClosure.cpp b/query_optimizer/logical/TransitiveClosure.cpp
new file mode 100644
index 0000000..99b693a
--- /dev/null
+++ b/query_optimizer/logical/TransitiveClosure.cpp
@@ -0,0 +1,80 @@
+/**
+ * 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/logical/TransitiveClosure.hpp"
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/logical/Logical.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace logical {
+
+namespace E = ::quickstep::optimizer::expressions;
+
+LogicalPtr TransitiveClosure::copyWithNewChildren(
+    const std::vector<LogicalPtr> &new_children) const {
+  DCHECK_EQ(2u, new_children.size());
+  return Create(new_children[0],
+                new_children[1],
+                start_attr_,
+                source_attr_,
+                destination_attr_);
+}
+
+std::vector<E::AttributeReferencePtr> TransitiveClosure::getOutputAttributes() const {
+  return { start_attr_, destination_attr_ };
+}
+
+std::vector<E::AttributeReferencePtr> TransitiveClosure::getReferencedAttributes() const {
+  return { start_attr_, source_attr_, destination_attr_ };
+}
+
+void TransitiveClosure::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
+  non_container_child_field_names->push_back("start");
+  non_container_child_fields->push_back(start_);
+
+  non_container_child_field_names->push_back("edge");
+  non_container_child_fields->push_back(edge_);
+
+  non_container_child_field_names->push_back("start_attr");
+  non_container_child_fields->push_back(start_attr_);
+
+  non_container_child_field_names->push_back("source_attr");
+  non_container_child_fields->push_back(source_attr_);
+
+  non_container_child_field_names->push_back("destination_attr");
+  non_container_child_fields->push_back(destination_attr_);
+}
+
+}  // namespace logical
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/logical/TransitiveClosure.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/TransitiveClosure.hpp b/query_optimizer/logical/TransitiveClosure.hpp
new file mode 100644
index 0000000..a2ebe02
--- /dev/null
+++ b/query_optimizer/logical/TransitiveClosure.hpp
@@ -0,0 +1,134 @@
+/**
+ * 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_LOGICAL_TRANSITIVE_CLOSURE_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_TRANSITIVE_CLOSURE_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/logical/Logical.hpp"
+#include "query_optimizer/logical/LogicalType.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+namespace logical {
+
+/** \addtogroup OptimizerLogical
+ *  @{
+ */
+
+class TransitiveClosure;
+typedef std::shared_ptr<const TransitiveClosure> TransitiveClosurePtr;
+
+class TransitiveClosure : public Logical {
+ public:
+  ~TransitiveClosure() override {}
+
+  LogicalType getLogicalType() const override {
+    return LogicalType::kTransitiveClosure;
+  }
+
+  std::string getName() const override {
+    return "TransitiveClosure";
+  }
+
+  const LogicalPtr& start() const {
+    return start_;
+  }
+
+  const LogicalPtr& edge() const {
+    return edge_;
+  }
+
+  const expressions::AttributeReferencePtr& start_attr() const {
+    return start_attr_;
+  }
+
+  const expressions::AttributeReferencePtr& source_attr() const {
+    return source_attr_;
+  }
+
+  const expressions::AttributeReferencePtr& destination_attr() const {
+    return destination_attr_;
+  }
+
+  LogicalPtr copyWithNewChildren(const std::vector<LogicalPtr> &new_children) const override;
+
+  std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override;
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
+
+  static TransitiveClosurePtr Create(
+      const LogicalPtr &start,
+      const LogicalPtr &edge,
+      const expressions::AttributeReferencePtr &start_attr,
+      const expressions::AttributeReferencePtr &source_attr,
+      const expressions::AttributeReferencePtr &destination_attr) {
+    return TransitiveClosurePtr(
+        new TransitiveClosure(start,
+                              edge,
+                              start_attr,
+                              source_attr,
+                              destination_attr));
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
+
+ private:
+  TransitiveClosure(const LogicalPtr &start,
+                    const LogicalPtr &edge,
+                    const expressions::AttributeReferencePtr &start_attr,
+                    const expressions::AttributeReferencePtr &source_attr,
+                    const expressions::AttributeReferencePtr &destination_attr)
+      : start_(start),
+        edge_(edge),
+        start_attr_(start_attr),
+        source_attr_(source_attr),
+        destination_attr_(destination_attr) {
+    addChild(start_);
+    addChild(edge_);
+  }
+
+  const LogicalPtr start_;
+  const LogicalPtr edge_;
+  const expressions::AttributeReferencePtr start_attr_;
+  const expressions::AttributeReferencePtr source_attr_;
+  const expressions::AttributeReferencePtr destination_attr_;
+
+  DISALLOW_COPY_AND_ASSIGN(TransitiveClosure);
+};
+
+/** @} */
+
+}  // namespace logical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_TRANSITIVE_CLOSURE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/physical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CMakeLists.txt b/query_optimizer/physical/CMakeLists.txt
index a1a72f7..bfcbd34 100644
--- a/query_optimizer/physical/CMakeLists.txt
+++ b/query_optimizer/physical/CMakeLists.txt
@@ -47,6 +47,7 @@ add_library(quickstep_queryoptimizer_physical_Sort Sort.cpp Sort.hpp)
 add_library(quickstep_queryoptimizer_physical_TableGenerator ../../empty_src.cpp TableGenerator.hpp)
 add_library(quickstep_queryoptimizer_physical_TableReference TableReference.cpp TableReference.hpp)
 add_library(quickstep_queryoptimizer_physical_TopLevelPlan TopLevelPlan.cpp TopLevelPlan.hpp)
+add_library(quickstep_queryoptimizer_physical_TransitiveClosure TransitiveClosure.cpp TransitiveClosure.hpp)
 add_library(quickstep_queryoptimizer_physical_UnionAll ../../empty_src.cpp UnionAll.hpp)
 add_library(quickstep_queryoptimizer_physical_UpdateTable UpdateTable.cpp UpdateTable.hpp)
 add_library(quickstep_queryoptimizer_physical_WindowAggregate WindowAggregate.cpp WindowAggregate.hpp)
@@ -301,6 +302,13 @@ target_link_libraries(quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_utility_Cast
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_physical_TransitiveClosure
+                      glog
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_physical_UnionAll
                       quickstep_queryoptimizer_OptimizerTree
                       quickstep_queryoptimizer_expressions_AttributeReference
@@ -363,6 +371,7 @@ target_link_libraries(quickstep_queryoptimizer_physical
                       quickstep_queryoptimizer_physical_TableGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_physical_TransitiveClosure
                       quickstep_queryoptimizer_physical_UnionAll
                       quickstep_queryoptimizer_physical_UpdateTable
                       quickstep_queryoptimizer_physical_WindowAggregate)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/physical/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PatternMatcher.hpp b/query_optimizer/physical/PatternMatcher.hpp
index 0204504..eb70995 100644
--- a/query_optimizer/physical/PatternMatcher.hpp
+++ b/query_optimizer/physical/PatternMatcher.hpp
@@ -46,6 +46,7 @@ class SharedSubplanReference;
 class Sort;
 class TableReference;
 class TopLevelPlan;
+class TransitiveClosure;
 class UpdateTable;
 
 /** \addtogroup OptimizerPhysical
@@ -127,6 +128,7 @@ using SomeSharedSubplanReference = SomePhysicalNode<SharedSubplanReference, Phys
 using SomeSort = SomePhysicalNode<Sort, PhysicalType::kSort>;
 using SomeTableReference = SomePhysicalNode<TableReference, PhysicalType::kTableReference>;
 using SomeTopLevelPlan = SomePhysicalNode<TopLevelPlan, PhysicalType::kTopLevelPlan>;
+using SomeTransitiveClosure = SomePhysicalNode<TransitiveClosure, PhysicalType::kTransitiveClosure>;
 using SomeUpdateTable = SomePhysicalNode<UpdateTable, PhysicalType::kUpdateTable>;
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/physical/PhysicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PhysicalType.hpp b/query_optimizer/physical/PhysicalType.hpp
index 0a965af..5207436 100644
--- a/query_optimizer/physical/PhysicalType.hpp
+++ b/query_optimizer/physical/PhysicalType.hpp
@@ -52,6 +52,7 @@ enum class PhysicalType {
   kTableGenerator,
   kTableReference,
   kTopLevelPlan,
+  kTransitiveClosure,
   kUnionAll,
   kUpdateTable,
   kWindowAggregate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/physical/TransitiveClosure.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TransitiveClosure.cpp b/query_optimizer/physical/TransitiveClosure.cpp
new file mode 100644
index 0000000..26ed59a
--- /dev/null
+++ b/query_optimizer/physical/TransitiveClosure.cpp
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "query_optimizer/physical/TransitiveClosure.hpp"
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+namespace E = ::quickstep::optimizer::expressions;
+
+PhysicalPtr TransitiveClosure::copyWithNewChildren(
+    const std::vector<PhysicalPtr> &new_children) const {
+  DCHECK_EQ(2u, new_children.size());
+  return Create(new_children[0],
+                new_children[1],
+                start_attr_,
+                source_attr_,
+                destination_attr_);
+}
+
+std::vector<E::AttributeReferencePtr> TransitiveClosure::getOutputAttributes() const {
+  return { start_attr_, destination_attr_ };
+}
+
+std::vector<E::AttributeReferencePtr> TransitiveClosure::getReferencedAttributes() const {
+  return { start_attr_, source_attr_, destination_attr_ };
+}
+
+void TransitiveClosure::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
+  non_container_child_field_names->push_back("start");
+  non_container_child_fields->push_back(start_);
+
+  non_container_child_field_names->push_back("edge");
+  non_container_child_fields->push_back(edge_);
+
+  non_container_child_field_names->push_back("start_attr");
+  non_container_child_fields->push_back(start_attr_);
+
+  non_container_child_field_names->push_back("source_attr");
+  non_container_child_fields->push_back(source_attr_);
+
+  non_container_child_field_names->push_back("destination_attr");
+  non_container_child_fields->push_back(destination_attr_);
+}
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep
+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/physical/TransitiveClosure.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TransitiveClosure.hpp b/query_optimizer/physical/TransitiveClosure.hpp
new file mode 100644
index 0000000..512bae8
--- /dev/null
+++ b/query_optimizer/physical/TransitiveClosure.hpp
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_TRANSITIVE_CLOSURE_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_TRANSITIVE_CLOSURE_HPP_
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+/** \addtogroup OptimizerPhysical
+ *  @{
+ */
+
+class TransitiveClosure;
+typedef std::shared_ptr<const TransitiveClosure> TransitiveClosurePtr;
+
+class TransitiveClosure : public Physical {
+ public:
+  PhysicalType getPhysicalType() const override {
+    return PhysicalType::kTransitiveClosure;
+  }
+
+  std::string getName() const override {
+    return "TransitiveClosure";
+  }
+
+  const PhysicalPtr& start() const {
+    return start_;
+  }
+
+  const PhysicalPtr& edge() const {
+    return edge_;
+  }
+
+  const expressions::AttributeReferencePtr& start_attr() const {
+    return start_attr_;
+  }
+
+  const expressions::AttributeReferencePtr& source_attr() const {
+    return source_attr_;
+  }
+
+  const expressions::AttributeReferencePtr& destination_attr() const {
+    return destination_attr_;
+  }
+
+  PhysicalPtr copyWithNewChildren(
+      const std::vector<PhysicalPtr> &new_children) const override;
+
+  std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override;
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
+
+  bool maybeCopyWithPrunedExpressions(
+      const expressions::UnorderedNamedExpressionSet &referenced_expressions,
+      PhysicalPtr *output) const override {
+    return false;
+  }
+
+  static TransitiveClosurePtr Create(
+      const PhysicalPtr &start,
+      const PhysicalPtr &edge,
+      const expressions::AttributeReferencePtr &start_attr,
+      const expressions::AttributeReferencePtr &source_attr,
+      const expressions::AttributeReferencePtr &destination_attr) {
+    return TransitiveClosurePtr(
+        new TransitiveClosure(start,
+                              edge,
+                              start_attr,
+                              source_attr,
+                              destination_attr));
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
+
+ private:
+  TransitiveClosure(const PhysicalPtr &start,
+                    const PhysicalPtr &edge,
+                    const expressions::AttributeReferencePtr &start_attr,
+                    const expressions::AttributeReferencePtr &source_attr,
+                    const expressions::AttributeReferencePtr &destination_attr)
+      : start_(start),
+        edge_(edge),
+        start_attr_(start_attr),
+        source_attr_(source_attr),
+        destination_attr_(destination_attr) {
+    addChild(start_);
+    addChild(edge_);
+  }
+
+  const PhysicalPtr start_;
+  const PhysicalPtr edge_;
+  const expressions::AttributeReferencePtr start_attr_;
+  const expressions::AttributeReferencePtr source_attr_;
+  const expressions::AttributeReferencePtr destination_attr_;
+
+  DISALLOW_COPY_AND_ASSIGN(TransitiveClosure);
+};
+
+/** @} */
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_TRANSITIVE_CLOSURE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index 6feb1e8..6b005c8 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -67,6 +67,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_parser_ParseSubqueryExpression
                       quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseTableReference
+                      quickstep_parser_ParseTransitiveClosureTableReference
                       quickstep_parser_ParseWindow
                       quickstep_queryoptimizer_OptimizerContext
                       quickstep_queryoptimizer_Validator
@@ -116,6 +117,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_queryoptimizer_logical_TableGenerator
                       quickstep_queryoptimizer_logical_TableReference
                       quickstep_queryoptimizer_logical_TopLevelPlan
+                      quickstep_queryoptimizer_logical_TransitiveClosure
                       quickstep_queryoptimizer_logical_UpdateTable
                       quickstep_queryoptimizer_logical_WindowAggregate
                       quickstep_queryoptimizer_resolver_NameResolver

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index dc4bc2e..6593e1c 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -63,6 +63,7 @@
 #include "parser/ParseSubqueryExpression.hpp"
 #include "parser/ParseSubqueryTableReference.hpp"
 #include "parser/ParseTableReference.hpp"
+#include "parser/ParseTransitiveClosureTableReference.hpp"
 #include "parser/ParseWindow.hpp"
 #include "query_optimizer/OptimizerContext.hpp"
 #include "query_optimizer/Validator.hpp"
@@ -111,6 +112,7 @@
 #include "query_optimizer/logical/TableGenerator.hpp"
 #include "query_optimizer/logical/TableReference.hpp"
 #include "query_optimizer/logical/TopLevelPlan.hpp"
+#include "query_optimizer/logical/TransitiveClosure.hpp"
 #include "query_optimizer/logical/UpdateTable.hpp"
 #include "query_optimizer/logical/WindowAggregate.hpp"
 #include "query_optimizer/resolver/NameResolver.hpp"
@@ -1966,6 +1968,21 @@ L::LogicalPtr Resolver::resolveTableReference(const ParseTableReference &table_r
       name_resolver->merge(&joined_table_name_resolver);
       break;
     }
+    case ParseTableReference::kTransitiveClosureTableReference: {
+      DCHECK(reference_signature != nullptr)
+          << "Transitive closure subquery must be explicitly named";
+      DCHECK(reference_signature->table_alias() != nullptr);
+
+      logical_plan = resolveTransitiveClosureTableReference(
+          static_cast<const ParseTransitiveClosureTableReference&>(table_reference));
+
+      if (reference_signature->column_aliases() != nullptr) {
+        logical_plan = RenameOutputColumns(logical_plan, *reference_signature);
+      }
+
+      name_resolver->addRelation(reference_signature->table_alias(), logical_plan);
+      break;
+    }
     default:
       LOG(FATAL) << "Unhandeled table reference " << table_reference.toString();
   }
@@ -2257,6 +2274,34 @@ L::LogicalPtr Resolver::resolveJoinedTableReference(
   THROW_SQL_ERROR_AT(&joined_table_reference) << "Full outer join is not supported yet";
 }
 
+L::LogicalPtr Resolver::resolveTransitiveClosureTableReference(
+    const ParseTransitiveClosureTableReference &transitive_closure_table_reference) {
+  std::unique_ptr<NameResolver> local_name_resolver;
+
+  local_name_resolver = std::make_unique<NameResolver>();
+  L::LogicalPtr start_table =
+      resolveTableReference(*transitive_closure_table_reference.start_table(),
+                            local_name_resolver.get());
+  const std::vector<E::AttributeReferencePtr> start_attrs = start_table->getOutputAttributes();
+  if (start_attrs.size() != 1) {
+    THROW_SQL_ERROR_AT(transitive_closure_table_reference.start_table())
+        << "The table must contain exactly one column";
+  }
+
+  local_name_resolver = std::make_unique<NameResolver>();
+  L::LogicalPtr edge_table =
+      resolveTableReference(*transitive_closure_table_reference.edge_table(),
+                            local_name_resolver.get());
+  const std::vector<E::AttributeReferencePtr> edge_attrs = edge_table->getOutputAttributes();
+  if (edge_attrs.size() != 2) {
+    THROW_SQL_ERROR_AT(transitive_closure_table_reference.edge_table())
+        << "The table must contain exactly two columns";
+  }
+
+  return L::TransitiveClosure::Create(
+      start_table, edge_table, start_attrs[0], edge_attrs[0], edge_attrs[1]);
+}
+
 L::LogicalPtr Resolver::resolveSortInWindow(
     const L::LogicalPtr &logical_plan,
     const E::WindowInfo &window_info) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/resolver/Resolver.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.hpp b/query_optimizer/resolver/Resolver.hpp
index 1784782..b08012e 100644
--- a/query_optimizer/resolver/Resolver.hpp
+++ b/query_optimizer/resolver/Resolver.hpp
@@ -68,6 +68,7 @@ class ParseString;
 class ParseSubqueryExpression;
 class ParseTableReference;
 class ParseTableReferenceSignature;
+class ParseTransitiveClosureTableReference;
 class ParseTreeNode;
 class ParseWindow;
 template <class T>
@@ -366,7 +367,7 @@ class Resolver {
    * @return The logical plan to derive the table.
    */
   logical::LogicalPtr resolveTableReference(const ParseTableReference &table_reference,
-                                            NameResolver *name_resolver);
+                                            NameResolver *name_resolver = nullptr);
 
   /**
    * @brief Resolves a table reference by name, and converts it to a logical
@@ -403,6 +404,9 @@ class Resolver {
       const ParseJoinedTableReference &joined_table_reference,
       NameResolver *name_resolver);
 
+  logical::LogicalPtr resolveTransitiveClosureTableReference(
+      const ParseTransitiveClosureTableReference &transitive_closure_table_reference);
+
   /**
    * @brief Renames the output columns from \p logical_plan based on the table signature
    *        by wrapping it with a Project.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/strategy/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/CMakeLists.txt b/query_optimizer/strategy/CMakeLists.txt
index 20a4eb4..a8cf1be 100644
--- a/query_optimizer/strategy/CMakeLists.txt
+++ b/query_optimizer/strategy/CMakeLists.txt
@@ -75,6 +75,7 @@ target_link_libraries(quickstep_queryoptimizer_strategy_OneToOne
                       quickstep_queryoptimizer_OptimizerContext
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_ExpressionType
                       quickstep_queryoptimizer_logical_CopyFrom
                       quickstep_queryoptimizer_logical_CopyTo
                       quickstep_queryoptimizer_logical_CreateIndex
@@ -92,6 +93,7 @@ target_link_libraries(quickstep_queryoptimizer_strategy_OneToOne
                       quickstep_queryoptimizer_logical_TableGenerator
                       quickstep_queryoptimizer_logical_TableReference
                       quickstep_queryoptimizer_logical_TopLevelPlan
+                      quickstep_queryoptimizer_logical_TransitiveClosure
                       quickstep_queryoptimizer_logical_UpdateTable
                       quickstep_queryoptimizer_logical_WindowAggregate
                       quickstep_queryoptimizer_physical_Aggregate
@@ -103,13 +105,16 @@ target_link_libraries(quickstep_queryoptimizer_strategy_OneToOne
                       quickstep_queryoptimizer_physical_DropTable
                       quickstep_queryoptimizer_physical_InsertSelection
                       quickstep_queryoptimizer_physical_InsertTuple
+                      quickstep_queryoptimizer_physical_PatternMatcher
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_physical_Sample
+                      quickstep_queryoptimizer_physical_Selection
                       quickstep_queryoptimizer_physical_SharedSubplanReference
                       quickstep_queryoptimizer_physical_Sort
                       quickstep_queryoptimizer_physical_TableGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_physical_TransitiveClosure
                       quickstep_queryoptimizer_physical_UnionAll
                       quickstep_queryoptimizer_physical_UpdateTable
                       quickstep_queryoptimizer_physical_WindowAggregate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/query_optimizer/strategy/OneToOne.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/OneToOne.cpp b/query_optimizer/strategy/OneToOne.cpp
index 3cfe013..738724f 100644
--- a/query_optimizer/strategy/OneToOne.cpp
+++ b/query_optimizer/strategy/OneToOne.cpp
@@ -25,6 +25,7 @@
 #include "query_optimizer/LogicalToPhysicalMapper.hpp"
 #include "query_optimizer/OptimizerContext.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionType.hpp"
 #include "query_optimizer/expressions/ExpressionUtil.hpp"
 #include "query_optimizer/logical/CopyFrom.hpp"
 #include "query_optimizer/logical/CopyTo.hpp"
@@ -42,6 +43,7 @@
 #include "query_optimizer/logical/TableGenerator.hpp"
 #include "query_optimizer/logical/TableReference.hpp"
 #include "query_optimizer/logical/TopLevelPlan.hpp"
+#include "query_optimizer/logical/TransitiveClosure.hpp"
 #include "query_optimizer/logical/UpdateTable.hpp"
 #include "query_optimizer/logical/WindowAggregate.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
@@ -53,12 +55,15 @@
 #include "query_optimizer/physical/DropTable.hpp"
 #include "query_optimizer/physical/InsertSelection.hpp"
 #include "query_optimizer/physical/InsertTuple.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
 #include "query_optimizer/physical/Sample.hpp"
+#include "query_optimizer/physical/Selection.hpp"
 #include "query_optimizer/physical/SharedSubplanReference.hpp"
 #include "query_optimizer/physical/Sort.hpp"
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/physical/TransitiveClosure.hpp"
 #include "query_optimizer/physical/UnionAll.hpp"
 #include "query_optimizer/physical/UpdateTable.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
@@ -242,6 +247,42 @@ bool OneToOne::generatePlan(const L::LogicalPtr &logical_input,
           table_generator->attribute_list());
       return true;
     }
+    case L::LogicalType::kTransitiveClosure: {
+      const L::TransitiveClosurePtr transitive_closure =
+          std::static_pointer_cast<const L::TransitiveClosure>(logical_input);
+      P::PhysicalPtr start =
+          physical_mapper_->createOrGetPhysicalFromLogical(transitive_closure->start());
+      P::PhysicalPtr edge =
+          physical_mapper_->createOrGetPhysicalFromLogical(transitive_closure->edge());
+
+      const auto start_attrs = start->getOutputAttributes();
+      DCHECK_EQ(1u, start_attrs.size());
+      const auto edge_attrs = edge->getOutputAttributes();
+      DCHECK_EQ(2u, edge_attrs.size());
+
+      P::SelectionPtr selection;
+      if (P::SomeSelection::MatchesWithConditionalCast(start, &selection) &&
+          selection->filter_predicate() == nullptr) {
+        const auto &exprs = selection->project_expressions();
+        DCHECK_EQ(1u, exprs.size());
+        if (exprs.front()->id() == start_attrs.front()->id()) {
+          start = selection->input();
+        }
+      }
+      if (P::SomeSelection::MatchesWithConditionalCast(edge, &selection) &&
+          selection->filter_predicate() == nullptr) {
+        const auto &exprs = selection->project_expressions();
+        DCHECK_EQ(2u, exprs.size());
+        if (exprs[0]->id() == edge_attrs[0]->id() &&
+            exprs[1]->id() == edge_attrs[1]->id()) {
+          edge = selection->input();
+        }
+      }
+
+      *physical_output = P::TransitiveClosure::Create(
+          start, edge, start_attrs[0], edge_attrs[0], edge_attrs[1]);
+      return true;
+    }
     case L::LogicalType::kUpdateTable: {
       const L::UpdateTablePtr update_table =
           std::static_pointer_cast<const L::UpdateTable>(logical_input);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/relational_operators/BuildTransitiveClosureOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildTransitiveClosureOperator.cpp b/relational_operators/BuildTransitiveClosureOperator.cpp
new file mode 100644
index 0000000..e151756
--- /dev/null
+++ b/relational_operators/BuildTransitiveClosureOperator.cpp
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "relational_operators/BuildTransitiveClosureOperator.hpp"
+
+#include <memory>
+#include <cstddef>
+
+#include "cli/Flags.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
+#include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
+#include "storage/StorageBlock.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/TransitiveClosureState.hpp"
+#include "storage/TupleStorageSubBlock.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "utility/Range.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+bool BuildTransitiveClosureOperator::getAllWorkOrders(
+    WorkOrdersContainer *container,
+    QueryContext *query_context,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
+  DCHECK(query_context != nullptr);
+
+  if (doneWorkOrderGeneration()) {
+    return true;
+  }
+
+  TransitiveClosureState *state =
+      query_context->getTransitiveClosureState(transitive_closure_context_index_);
+
+  while (start_relation_block_stream_.hasNext()) {
+    container->addNormalWorkOrder(
+        new BuildTransitiveClosureWorkOrder(query_id_,
+                                            start_relation_,
+                                            true /* is_start_relation */,
+                                            start_relation_block_stream_.getNext(),
+                                            start_attr_id_,
+                                            source_attr_id_,
+                                            destination_attr_id_,
+                                            state,
+                                            storage_manager),
+        op_index_);
+  }
+
+  while (edge_relation_block_stream_.hasNext()) {
+    container->addNormalWorkOrder(
+        new BuildTransitiveClosureWorkOrder(query_id_,
+                                            edge_relation_,
+                                            false /* is_start_relation */,
+                                            edge_relation_block_stream_.getNext(),
+                                            start_attr_id_,
+                                            source_attr_id_,
+                                            destination_attr_id_,
+                                            state,
+                                            storage_manager),
+        op_index_);
+  }
+
+  return doneWorkOrderGeneration();
+}
+
+bool BuildTransitiveClosureOperator::getAllWorkOrderProtos(
+    WorkOrderProtosContainer *container)  {
+  LOG(FATAL) << "Not supported";
+}
+
+void BuildTransitiveClosureWorkOrder::execute() {
+  BlockReference block(storage_manager_->getBlock(block_, input_relation_));
+
+  std::unique_ptr<ValueAccessor> accessor(
+      block->getTupleStorageSubBlock().createValueAccessor());
+
+  if (is_start_relation_) {
+    buildStartRelation(accessor.get());
+  } else {
+    buildEdgeRelation(accessor.get());
+  }
+}
+
+void BuildTransitiveClosureWorkOrder::buildStartRelation(ValueAccessor *accessor) {
+  std::cout << "BuildStartRelation: " << block_ << "\n";
+  InvokeOnAnyValueAccessor(
+      accessor,
+      [&](auto *accessor) -> void {
+    while (accessor->next()) {
+      const void *value = accessor->getUntypedValue(this->start_attr_id_);
+      this->state_->addStart(*static_cast<const int*>(value));
+    }
+  });
+}
+
+void BuildTransitiveClosureWorkOrder::buildEdgeRelation(ValueAccessor *accessor) {
+  std::cout << "BuildEdgeRelation: " << block_ << "\n";
+  InvokeOnAnyValueAccessor(
+      accessor,
+      [&](auto *accessor) -> void {
+    while (accessor->next()) {
+      const void *source = accessor->getUntypedValue(this->source_attr_id_);
+      const void *destination = accessor->getUntypedValue(this->destination_attr_id_);
+      this->state_->addEdge(*static_cast<const int*>(source),
+                            *static_cast<const int*>(destination));
+    }
+  });
+}
+
+}  // namespace quickstep
+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/734ddc1e/relational_operators/BuildTransitiveClosureOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildTransitiveClosureOperator.hpp b/relational_operators/BuildTransitiveClosureOperator.hpp
new file mode 100644
index 0000000..299466b
--- /dev/null
+++ b/relational_operators/BuildTransitiveClosureOperator.hpp
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_RELATIONAL_OPERATORS_BUILD_TRANSITIVE_CLOSURE_OPERATOR_HPP_
+#define QUICKSTEP_RELATIONAL_OPERATORS_BUILD_TRANSITIVE_CLOSURE_OPERATOR_HPP_
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+#include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/TransitiveClosureState.hpp"
+#include "utility/BlockIDStream.hpp"
+#include "utility/Macros.hpp"
+#include "utility/Range.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb { class MessageBus; }
+
+namespace quickstep {
+
+class CatalogRelationSchema;
+class StorageManager;
+class WorkOrderProtosContainer;
+class WorkOrdersContainer;
+class ValueAccessor;
+
+/** \addtogroup RelationalOperators
+ *  @{
+ */
+
+class BuildTransitiveClosureOperator : public RelationalOperator {
+ public:
+  BuildTransitiveClosureOperator(const std::size_t query_id,
+                                 const std::size_t transitive_closure_context_index,
+                                 const CatalogRelation &start_relation,
+                                 const CatalogRelation &edge_relation,
+                                 const bool start_relation_is_stored,
+                                 const bool edge_relation_is_stored,
+                                 const attribute_id start_attr_id,
+                                 const attribute_id source_attr_id,
+                                 const attribute_id destination_attr_id)
+      : RelationalOperator(query_id, 1u),
+        transitive_closure_context_index_(transitive_closure_context_index),
+        start_relation_(start_relation),
+        edge_relation_(edge_relation),
+        start_attr_id_(start_attr_id),
+        source_attr_id_(source_attr_id),
+        destination_attr_id_(destination_attr_id),
+        start_relation_block_stream_(start_relation,
+                                     start_relation_is_stored,
+                                     done_feeding_input_relation_),
+        edge_relation_block_stream_(edge_relation,
+                                    edge_relation_is_stored,
+                                    done_feeding_input_relation_) {
+  }
+
+  ~BuildTransitiveClosureOperator() override {}
+
+  OperatorType getOperatorType() const override {
+    return kBuildTransitiveClosure;
+  }
+
+  std::string getName() const override {
+    return "BuildTransitiveClosureOperator";
+  }
+
+  bool getAllWorkOrders(WorkOrdersContainer *container,
+                        QueryContext *query_context,
+                        StorageManager *storage_manager,
+                        const tmb::client_id scheduler_client_id,
+                        tmb::MessageBus *bus) override;
+
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
+  void feedInputBlock(const block_id input_block_id,
+                      const relation_id input_relation_id,
+                      const partition_id part_id) override {
+    if (input_relation_id == edge_relation_.getID()) {
+      edge_relation_block_stream_.append(input_block_id);
+    } else {
+      DCHECK_EQ(start_relation_.getID(), input_relation_id);
+      start_relation_block_stream_.append(input_block_id);
+    }
+  }
+
+ private:
+  bool doneWorkOrderGeneration() const {
+    return start_relation_block_stream_.isEndOfStream()
+        && edge_relation_block_stream_.isEndOfStream();
+  }
+
+  const std::size_t transitive_closure_context_index_;
+
+  const CatalogRelation &start_relation_;
+  const CatalogRelation &edge_relation_;
+
+  const attribute_id start_attr_id_;
+  const attribute_id source_attr_id_;
+  const attribute_id destination_attr_id_;
+
+  BlockIDStream start_relation_block_stream_;
+  BlockIDStream edge_relation_block_stream_;
+
+  DISALLOW_COPY_AND_ASSIGN(BuildTransitiveClosureOperator);
+};
+
+class BuildTransitiveClosureWorkOrder : public WorkOrder {
+ public:
+  BuildTransitiveClosureWorkOrder(const std::size_t query_id,
+                                  const CatalogRelationSchema &input_relation,
+                                  const bool is_start_relation,
+                                  const block_id block,
+                                  const attribute_id start_attr_id,
+                                  const attribute_id source_attr_id,
+                                  const attribute_id destination_attr_id,
+                                  TransitiveClosureState *state,
+                                  StorageManager *storage_manager)
+      : WorkOrder(query_id, 1u),
+        input_relation_(input_relation),
+        is_start_relation_(is_start_relation),
+        block_(block),
+        start_attr_id_(start_attr_id),
+        source_attr_id_(source_attr_id),
+        destination_attr_id_(destination_attr_id),
+        state_(state),
+        storage_manager_(storage_manager) {
+  }
+
+  ~BuildTransitiveClosureWorkOrder() override {}
+
+  void execute() override;
+
+ private:
+  void buildStartRelation(ValueAccessor *accessor);
+  void buildEdgeRelation(ValueAccessor *accessor);
+
+  const CatalogRelationSchema &input_relation_;
+  const bool is_start_relation_;
+  const block_id block_;
+
+  const attribute_id start_attr_id_;
+  const attribute_id source_attr_id_;
+  const attribute_id destination_attr_id_;
+
+  TransitiveClosureState *state_;
+  StorageManager *storage_manager_;
+
+  DISALLOW_COPY_AND_ASSIGN(BuildTransitiveClosureWorkOrder);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_BUILD_TRANSITIVE_CLOSURE_OPERATOR_HPP_