You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by zu...@apache.org on 2016/07/08 01:25:43 UTC

[01/50] [abbrv] incubator-quickstep git commit: Bug fixed in isFull() function. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/tmp-relation-col-store e62dba059 -> 82e6f0900 (forced update)


Bug fixed in isFull() function.

- Earlier check for isFull() was that whether there is at least one
empty bucket available for insertion.
- Now isFull() can check if there is enough space adding specified
number of buckets.
- To reproduce the bug behind this fix: Run TPC-H Q2 on SF100 dataset after
  running the \analyze command. The optimizer allocates smaller than
  necessary space for one of the hash joins. The resize() is triggered
  but never gets executed because of the restrictive isFull() condition.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 83935e7d8c9ff95071906e4a6078d1ed096c7bae
Parents: 5ae5052
Author: Harshad Deshmukh <ha...@cs.wisc.edu>
Authored: Fri Jun 3 15:28:52 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed Jun 8 11:57:47 2016 -0700

----------------------------------------------------------------------
 storage/SimpleScalarSeparateChainingHashTable.hpp | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/83935e7d/storage/SimpleScalarSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/SimpleScalarSeparateChainingHashTable.hpp b/storage/SimpleScalarSeparateChainingHashTable.hpp
index eda6c86..962a66c 100644
--- a/storage/SimpleScalarSeparateChainingHashTable.hpp
+++ b/storage/SimpleScalarSeparateChainingHashTable.hpp
@@ -230,10 +230,10 @@ class SimpleScalarSeparateChainingHashTable : public HashTable<ValueT,
                                        HashTablePreallocationState *prealloc_state);
 
   // Determine whether it is actually necessary to resize this hash table.
-  // Checks that there is at least one unallocated bucket.
-  inline bool isFull() const {
-    return header_->buckets_allocated.load(std::memory_order_relaxed)
-           >= header_->num_buckets;
+  // Checks that there are at least ``extra_buckets`` unallocated buckets.
+  inline bool isFull(const std::size_t extra_buckets) const {
+    return (header_->buckets_allocated.load(std::memory_order_relaxed) +
+            extra_buckets) >= header_->num_buckets;
   }
 
   // Cache the TypeID of the key.
@@ -831,7 +831,7 @@ void SimpleScalarSeparateChainingHashTable<ValueT,
   // Recheck whether the hash table is still full. Note that multiple threads
   // might wait to rebuild this hash table simultaneously. Only the first one
   // should do the rebuild.
-  if (!isFull()) {
+  if (!isFull(extra_buckets)) {
     return;
   }
 


[34/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-18: Allow BasicColumnStoreTupleStorageSubBlock to be unsorted

Posted by zu...@apache.org.
QUICKSTEP-18: Allow BasicColumnStoreTupleStorageSubBlock to be unsorted

Review comments applied

Revert build fix for Lexer (separate PR has been opened)


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

Branch: refs/heads/tmp-relation-col-store
Commit: c1476d1e7637a943b00e104cdfd9498e3c7cfe03
Parents: 00ca1e4
Author: Craig Chasseur <sp...@gmail.com>
Authored: Sun Jun 12 04:01:17 2016 -0700
Committer: Craig Chasseur <sp...@gmail.com>
Committed: Sun Jun 19 21:58:27 2016 -0700

----------------------------------------------------------------------
 parser/CMakeLists.txt                           |   7 +-
 .../BasicColumnStoreTupleStorageSubBlock.cpp    |  68 +++++----
 .../BasicColumnStoreTupleStorageSubBlock.hpp    |  15 +-
 storage/StorageBlockLayout.proto                |   6 +-
 ...ColumnStoreTupleStorageSubBlock_unittest.cpp | 137 ++++++++++++++-----
 5 files changed, 158 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c1476d1e/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index 55c4a67..114ad14 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -384,15 +384,12 @@ set_property(SOURCE ${CMAKE_CURRENT_BINARY_DIR}/SqlParser_gen.cpp APPEND PROPERT
 include(CheckCXXCompilerFlag)
 CHECK_CXX_COMPILER_FLAG("-Wno-deprecated-register" COMPILER_HAS_WNO_DEPRECATED_REGISTER)
 if (COMPILER_HAS_WNO_DEPRECATED_REGISTER)
-  set_property(TARGET quickstep_parser_SqlLexer APPEND_STRING PROPERTY COMPILE_FLAGS " -Wno-deprecated-register")
+  set_target_properties(quickstep_parser_SqlLexer PROPERTIES COMPILE_FLAGS "-Wno-deprecated-register")
 endif()
 
 # GCC will make a warning for unsigned-signed comparisons which are inherent
 # in the lexer. For this, we turn off the sign compare.
-CHECK_CXX_COMPILER_FLAG("-Wno-sign-compare" COMPILER_HAS_WNO_SIGN_COMPARE)
-if (COMPILER_HAS_WNO_SIGN_COMPARE)
-  set_property(TARGET quickstep_parser_SqlLexer APPEND_STRING PROPERTY COMPILE_FLAGS " -Wno-sign-compare")
-endif()
+set_target_properties(quickstep_parser_SqlLexer PROPERTIES COMPILE_FLAGS "-Wno-sign-compare")
 
 add_subdirectory(tests)
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c1476d1e/storage/BasicColumnStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/BasicColumnStoreTupleStorageSubBlock.cpp b/storage/BasicColumnStoreTupleStorageSubBlock.cpp
index 0c913ff..a474437 100644
--- a/storage/BasicColumnStoreTupleStorageSubBlock.cpp
+++ b/storage/BasicColumnStoreTupleStorageSubBlock.cpp
@@ -52,6 +52,8 @@
 #include "utility/PtrVector.hpp"
 #include "utility/ScopedBuffer.hpp"
 
+#include "glog/logging.h"
+
 using std::memcpy;
 using std::memmove;
 using std::size_t;
@@ -104,14 +106,19 @@ BasicColumnStoreTupleStorageSubBlock::BasicColumnStoreTupleStorageSubBlock(
                            new_block,
                            sub_block_memory,
                            sub_block_memory_size),
+      sort_specified_(description_.HasExtension(
+          BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id)),
       sorted_(true),
       header_(static_cast<BasicColumnStoreHeader*>(sub_block_memory)) {
   if (!DescriptionIsValid(relation_, description_)) {
     FATAL_ERROR("Attempted to construct a BasicColumnStoreTupleStorageSubBlock from an invalid description.");
   }
 
-  sort_column_id_ = description_.GetExtension(BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id);
-  sort_column_type_ = &(relation_.getAttributeById(sort_column_id_)->getType());
+  if (sort_specified_) {
+    sort_column_id_ = description_.GetExtension(
+        BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id);
+    sort_column_type_ = &(relation_.getAttributeById(sort_column_id_)->getType());
+  }
 
   if (sub_block_memory_size < sizeof(BasicColumnStoreHeader)) {
     throw BlockMemoryTooSmall("BasicColumnStoreTupleStorageSubBlock", sub_block_memory_size_);
@@ -183,26 +190,27 @@ bool BasicColumnStoreTupleStorageSubBlock::DescriptionIsValid(
   if (description.sub_block_type() != TupleStorageSubBlockDescription::BASIC_COLUMN_STORE) {
     return false;
   }
-  // Make sure a sort_attribute_id is specified.
-  if (!description.HasExtension(BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id)) {
-    return false;
-  }
 
   // Make sure relation is not variable-length.
   if (relation.isVariableLength()) {
     return false;
   }
 
-  // Check that the specified sort attribute exists and can be ordered by LessComparison.
-  attribute_id sort_attribute_id = description.GetExtension(
-      BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id);
-  if (!relation.hasAttributeWithId(sort_attribute_id)) {
-    return false;
-  }
-  const Type &sort_attribute_type = relation.getAttributeById(sort_attribute_id)->getType();
-  if (!ComparisonFactory::GetComparison(ComparisonID::kLess).canCompareTypes(sort_attribute_type,
-                                                                             sort_attribute_type)) {
-    return false;
+  // If a sort attribute is specified, check that it exists and can be ordered
+  // by LessComparison.
+  if (description.HasExtension(
+          BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id)) {
+    const attribute_id sort_attribute_id = description.GetExtension(
+        BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id);
+    if (!relation.hasAttributeWithId(sort_attribute_id)) {
+      return false;
+    }
+    const Type &sort_attribute_type =
+        relation.getAttributeById(sort_attribute_id)->getType();
+    if (!ComparisonFactory::GetComparison(ComparisonID::kLess).canCompareTypes(
+            sort_attribute_type, sort_attribute_type)) {
+      return false;
+    }
   }
 
   return true;
@@ -230,9 +238,9 @@ TupleStorageSubBlock::InsertResult BasicColumnStoreTupleStorageSubBlock::insertT
   }
 
   tuple_id insert_position = header_->num_tuples;
-  // If sort column is NULL, skip the search and put the new tuple at the end
-  // of everything else.
-  if (!tuple.getAttributeValue(sort_column_id_).isNull()) {
+  // If this column store is unsorted, or the value of the sort column is NULL,
+  // skip the search and put the new tuple at the end of everything else.
+  if (sort_specified_ && !tuple.getAttributeValue(sort_column_id_).isNull()) {
     // Binary search for the appropriate insert location.
     ColumnStripeIterator begin_it(column_stripes_[sort_column_id_],
                                   relation_.getAttributeById(sort_column_id_)->getType().maximumByteLength(),
@@ -435,6 +443,9 @@ bool BasicColumnStoreTupleStorageSubBlock::canSetAttributeValuesInPlaceTyped(
     const tuple_id tuple,
     const std::unordered_map<attribute_id, TypedValue> &new_values) const {
   DEBUG_ASSERT(hasTupleWithID(tuple));
+  if (!sort_specified_) {
+    return true;
+  }
   for (std::unordered_map<attribute_id, TypedValue>::const_iterator it
            = new_values.begin();
        it != new_values.end();
@@ -453,7 +464,7 @@ void BasicColumnStoreTupleStorageSubBlock::setAttributeValueInPlaceTyped(
     const tuple_id tuple,
     const attribute_id attr,
     const TypedValue &value) {
-  DEBUG_ASSERT(attr != sort_column_id_);
+  DCHECK(!sort_specified_ || (attr != sort_column_id_));
 
   const Type &attr_type = relation_.getAttributeById(attr)->getType();
   void *value_position = static_cast<char*>(column_stripes_[attr])
@@ -473,10 +484,10 @@ void BasicColumnStoreTupleStorageSubBlock::setAttributeValueInPlaceTyped(
 bool BasicColumnStoreTupleStorageSubBlock::deleteTuple(const tuple_id tuple) {
   DEBUG_ASSERT(hasTupleWithID(tuple));
 
-  if (!column_null_bitmaps_.elementIsNull(sort_column_id_)) {
-    if (column_null_bitmaps_[sort_column_id_].getBit(tuple)) {
-      --(header_->nulls_in_sort_column);
-    }
+  if (sort_specified_
+      && !column_null_bitmaps_.elementIsNull(sort_column_id_)
+      && column_null_bitmaps_[sort_column_id_].getBit(tuple)) {
+    --(header_->nulls_in_sort_column);
   }
 
   if (tuple == header_->num_tuples - 1) {
@@ -563,7 +574,7 @@ bool BasicColumnStoreTupleStorageSubBlock::bulkDeleteTuples(TupleIdSequence *tup
 
 predicate_cost_t BasicColumnStoreTupleStorageSubBlock::estimatePredicateEvaluationCost(
     const ComparisonPredicate &predicate) const {
-  if (predicate.isAttributeLiteralComparisonPredicate()) {
+  if (sort_specified_ && predicate.isAttributeLiteralComparisonPredicate()) {
     std::pair<bool, attribute_id> comparison_attr = predicate.getAttributeFromAttributeLiteralComparison();
     if (comparison_attr.second == sort_column_id_) {
       return predicate_cost::kBinarySearch;
@@ -575,6 +586,11 @@ predicate_cost_t BasicColumnStoreTupleStorageSubBlock::estimatePredicateEvaluati
 TupleIdSequence* BasicColumnStoreTupleStorageSubBlock::getMatchesForPredicate(
     const ComparisonPredicate &predicate,
     const TupleIdSequence *filter) const {
+  DCHECK(sort_specified_) <<
+      "Called BasicColumnStoreTupleStorageSubBlock::getMatchesForPredicate() "
+      "for an unsorted column store (predicate should have been evaluated "
+      "with a scan instead).";
+
   TupleIdSequence *matches = SortColumnPredicateEvaluator::EvaluatePredicateForUncompressedSortColumn(
       predicate,
       relation_,
@@ -671,6 +687,8 @@ void BasicColumnStoreTupleStorageSubBlock::shiftNullBitmaps(
 // total size of tuples contained in this sub-block. It could be done with
 // less memory, although the implementation would be more complex.
 bool BasicColumnStoreTupleStorageSubBlock::rebuildInternal() {
+  DCHECK(sort_specified_);
+
   const tuple_id num_tuples = header_->num_tuples;
   // Immediately return if 1 or 0 tuples.
   if (num_tuples <= 1) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c1476d1e/storage/BasicColumnStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/BasicColumnStoreTupleStorageSubBlock.hpp b/storage/BasicColumnStoreTupleStorageSubBlock.hpp
index 6c72c78..b36eb20 100644
--- a/storage/BasicColumnStoreTupleStorageSubBlock.hpp
+++ b/storage/BasicColumnStoreTupleStorageSubBlock.hpp
@@ -108,7 +108,9 @@ class BasicColumnStoreTupleStorageSubBlock : public TupleStorageSubBlock {
   }
 
   bool adHocInsertIsEfficient() const override {
-    return false;
+    // Ad-hoc insert is only efficient if it doesn't require resorting the
+    // whole block.
+    return !sort_specified_;
   }
 
   TupleStorageSubBlockType getTupleStorageSubBlockType() const override {
@@ -170,14 +172,14 @@ class BasicColumnStoreTupleStorageSubBlock : public TupleStorageSubBlock {
                                           const TupleIdSequence *filter) const override;
 
   void rebuild() override {
-    if (!sorted_) {
+    if (sort_specified_ && !sorted_) {
       rebuildInternal();
     }
   }
 
   bool isInsertOrderPreserving() const override {
-    // Rebuild reorders inserts based on sort column.
-    return false;
+    // Rebuild reorders inserts based on sort column if any is specified.
+    return !sort_specified_;
   }
 
  private:
@@ -211,11 +213,12 @@ class BasicColumnStoreTupleStorageSubBlock : public TupleStorageSubBlock {
                         const tuple_id distance);
 
   // Sort all columns according to ascending order of values in the sort
-  // column. Returns true if any reordering occured.
+  // column. Returns true if any reordering occured. This should only be called
+  // when 'sort_specified_' is true, otherwise the block is always "built".
   bool rebuildInternal();
 
   tuple_id max_tuples_;
-  bool sorted_;
+  bool sort_specified_, sorted_;
 
   attribute_id sort_column_id_;
   const Type *sort_column_type_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c1476d1e/storage/StorageBlockLayout.proto
----------------------------------------------------------------------
diff --git a/storage/StorageBlockLayout.proto b/storage/StorageBlockLayout.proto
index 9e13fbe..cafc0e9 100644
--- a/storage/StorageBlockLayout.proto
+++ b/storage/StorageBlockLayout.proto
@@ -1,7 +1,7 @@
 //   Copyright 2011-2015 Quickstep Technologies LLC.
 //   Copyright 2015-2016 Pivotal Software, Inc.
 //   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-//    University of Wisconsin\u2014Madison.
+//    University of Wisconsin-Madison.
 //
 //   Licensed under the Apache License, Version 2.0 (the "License");
 //   you may not use this file except in compliance with the License.
@@ -38,8 +38,8 @@ message TupleStorageSubBlockDescription {
 
 message BasicColumnStoreTupleStorageSubBlockDescription {
   extend TupleStorageSubBlockDescription {
-    // Extentions may not be marked required in protobuf 2.6 or higher. This
-    // field is considered required when sub_block_type == BASIC_COLUMN_STORE.
+    // Indicates the attribute to sort the column store by. If unset, the
+    // column store will be unsorted.
     optional int32 sort_attribute_id = 64;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c1476d1e/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
index e5476f7..df8b479 100644
--- a/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
@@ -109,8 +109,11 @@ class BasicColumnStoreTupleStorageSubBlockTest : public ::testing::TestWithParam
 
     tuple_store_description_.reset(new TupleStorageSubBlockDescription());
     tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::BASIC_COLUMN_STORE);
-    tuple_store_description_->SetExtension(BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id,
-                                           sort_column);
+    if (sort_column != kInvalidCatalogId) {
+      tuple_store_description_->SetExtension(
+          BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id,
+          sort_column);
+    }
 
     tuple_store_.reset(new BasicColumnStoreTupleStorageSubBlock(*relation_,
                                                                 *tuple_store_description_,
@@ -197,7 +200,7 @@ class BasicColumnStoreTupleStorageSubBlockTest : public ::testing::TestWithParam
   }
 
   int computeNullsInSortColumn(const attribute_id sort_column_id) {
-    if (!GetParam()) {
+    if (sort_column_id == kInvalidCatalogId || !GetParam()) {
       return 0;
     }
 
@@ -341,11 +344,15 @@ class BasicColumnStoreTupleStorageSubBlockTest : public ::testing::TestWithParam
     EXPECT_EQ(computeRowCapacity() - regular_tuples_deleted - last_tuples_deleted,
               static_cast<std::size_t>(tuple_store_->getMaxTupleID() + 1));
 
-    const Type &sort_attribute_type = relation_->getAttributeById(sort_attribute_id)->getType();
-    std::unique_ptr<UncheckedComparator> sort_attribute_comparator(
-        ComparisonFactory::GetComparison(ComparisonID::kLessOrEqual).makeUncheckedComparatorForTypes(
-            sort_attribute_type,
-            sort_attribute_type));
+    std::unique_ptr<UncheckedComparator> sort_attribute_comparator;
+    if (sort_attribute_id != kInvalidCatalogId) {
+      const Type &sort_attribute_type =
+          relation_->getAttributeById(sort_attribute_id)->getType();
+      sort_attribute_comparator.reset(
+          ComparisonFactory::GetComparison(ComparisonID::kLessOrEqual)
+              .makeUncheckedComparatorForTypes(sort_attribute_type,
+                                               sort_attribute_type));
+    }
 
     std::vector<bool> existence_check_vector;
     existence_check_vector.resize(computeRowCapacity() - 2, false);
@@ -390,17 +397,21 @@ class BasicColumnStoreTupleStorageSubBlockTest : public ::testing::TestWithParam
         }
       }
 
-      // Check ordering.
-      TypedValue sort_attribute_value = tuple_store_->getAttributeValueTyped(tid, sort_attribute_id);
-      if (previous_sort_attribute_value.get() != nullptr) {
-        if (previous_sort_attribute_value->isNull()) {
-          EXPECT_TRUE(sort_attribute_value.isNull());
-        } else if (!sort_attribute_value.isNull()) {
-          EXPECT_TRUE(sort_attribute_comparator->compareTypedValues(*previous_sort_attribute_value,
-                                                                    sort_attribute_value));
+      // Check ordering if block is sorted.
+      if (sort_attribute_id != kInvalidCatalogId) {
+        TypedValue sort_attribute_value =
+            tuple_store_->getAttributeValueTyped(tid, sort_attribute_id);
+        if (previous_sort_attribute_value) {
+          if (previous_sort_attribute_value->isNull()) {
+            EXPECT_TRUE(sort_attribute_value.isNull());
+          } else if (!sort_attribute_value.isNull()) {
+            EXPECT_TRUE(sort_attribute_comparator->compareTypedValues(
+                *previous_sort_attribute_value,
+                sort_attribute_value));
+          }
         }
+        previous_sort_attribute_value.reset(new TypedValue(sort_attribute_value));
       }
-      previous_sort_attribute_value.reset(new TypedValue(sort_attribute_value));
     }
 
     EXPECT_EQ(2 - last_tuples_deleted, num_last_tuples);
@@ -435,13 +446,18 @@ class BasicColumnStoreTupleStorageSubBlockTest : public ::testing::TestWithParam
       delete_sequence.set(tid, true);
     }
     // One of the special "last" values.
-    if (GetParam()) {
+    if (sort_attribute_id != kInvalidCatalogId && GetParam()) {
       delete_sequence.set(row_capacity - computeNullsInSortColumn(sort_attribute_id) - 2, true);
     }
 
     EXPECT_TRUE(tuple_store_->bulkDeleteTuples(&delete_sequence));
 
-    checkBlockValues(sort_attribute_id, delete_sequence.numTuples() + 2 - 1, 1);
+    if (sort_attribute_id == kInvalidCatalogId) {
+      // There is no special treatment of "last" values without sorting.
+      checkBlockValues(sort_attribute_id, delete_sequence.numTuples(), 2);
+    } else {
+      checkBlockValues(sort_attribute_id, delete_sequence.numTuples() + 2 - 1, 1);
+    }
   }
 
   // Create a ComparisonPredicate of the form "attribute comp literal".
@@ -702,23 +718,26 @@ class BasicColumnStoreTupleStorageSubBlockTest : public ::testing::TestWithParam
     TypedValue new_wide_char_value(kChar, new_wide_char_lit, 32);
 
     std::unordered_map<attribute_id, TypedValue> proposed_values;
-    switch (sort_attribute_id) {
-      case 0:
-        proposed_values.emplace(0, new_int_value);
-        break;
-      case 1:
-        proposed_values.emplace(1, new_double_value);
-        break;
-      case 2:
-        proposed_values.emplace(2, new_narrow_char_value);
-        break;
-      case 3:
-        proposed_values.emplace(3, new_wide_char_value);
-        break;
-    }
+    if (sort_attribute_id != kInvalidCatalogId) {
+      switch (sort_attribute_id) {
+        case 0:
+          proposed_values.emplace(0, new_int_value);
+          break;
+        case 1:
+          proposed_values.emplace(1, new_double_value);
+          break;
+        case 2:
+          proposed_values.emplace(2, new_narrow_char_value);
+          break;
+        case 3:
+          proposed_values.emplace(3, new_wide_char_value);
+          break;
+      }
 
-    // Can't overwrite a sort column value in-place.
-    EXPECT_FALSE(tuple_store_->canSetAttributeValuesInPlaceTyped(target_tid, proposed_values));
+      // Can't overwrite a sort column value in-place.
+      EXPECT_FALSE(tuple_store_->canSetAttributeValuesInPlaceTyped(
+          target_tid, proposed_values));
+    }
 
     // Other column values should be OK.
     proposed_values.clear();
@@ -726,7 +745,9 @@ class BasicColumnStoreTupleStorageSubBlockTest : public ::testing::TestWithParam
     proposed_values.emplace(1, new_double_value);
     proposed_values.emplace(2, new_narrow_char_value);
     proposed_values.emplace(3, new_wide_char_value);
-    proposed_values.erase(sort_attribute_id);
+    if (sort_attribute_id != kInvalidCatalogId) {
+      proposed_values.erase(sort_attribute_id);
+    }
     EXPECT_TRUE(tuple_store_->canSetAttributeValuesInPlaceTyped(target_tid, proposed_values));
 
     // Actually set values.
@@ -820,6 +841,14 @@ TEST_P(BasicColumnStoreTupleStorageSubBlockTest, DescriptionIsValidTest) {
         *tuple_store_description_));
   }
 
+  // Also check a description that doesn't specify a sort column.
+  tuple_store_description_.reset(new TupleStorageSubBlockDescription());
+  tuple_store_description_->set_sub_block_type(
+      TupleStorageSubBlockDescription::BASIC_COLUMN_STORE);
+  EXPECT_TRUE(BasicColumnStoreTupleStorageSubBlock::DescriptionIsValid(
+      *relation_,
+      *tuple_store_description_));
+
   // An uninitialized description is not valid.
   tuple_store_description_.reset(new TupleStorageSubBlockDescription());
   EXPECT_FALSE(BasicColumnStoreTupleStorageSubBlock::DescriptionIsValid(
@@ -859,6 +888,24 @@ TEST_P(BasicColumnStoreTupleStorageSubBlockTest, BlockTooSmallTest) {
   EXPECT_THROW(createBlock(0, 32), BlockMemoryTooSmall);
 }
 
+TEST_P(BasicColumnStoreTupleStorageSubBlockTest, InsertWithNoSortColumnTest) {
+  // Non-random, batch insert.
+  createBlock(kInvalidCatalogId, kSubBlockSize);
+  fillBlockWithSampleData(false, false);
+
+  // Non-random, ad-hoc insert.
+  createBlock(kInvalidCatalogId, kSubBlockSize);
+  fillBlockWithSampleData(false, true);
+
+  // Random order, batch insert.
+  createBlock(kInvalidCatalogId, kSubBlockSize);
+  fillBlockWithSampleData(true, false);
+
+  // Random order, ad-hoc insert.
+  createBlock(kInvalidCatalogId, kSubBlockSize);
+  fillBlockWithSampleData(true, true);
+}
+
 TEST_P(BasicColumnStoreTupleStorageSubBlockTest, InsertWithIntSortColumnTest) {
   // Non-random, batch insert.
   createBlock(0, kSubBlockSize);
@@ -931,6 +978,12 @@ TEST_P(BasicColumnStoreTupleStorageSubBlockTest, InsertWithWideCharSortColumnTes
   fillBlockWithSampleData(true, true);
 }
 
+TEST_P(BasicColumnStoreTupleStorageSubBlockTest, GetValuesWithNoSortColumnTest) {
+  createBlock(kInvalidCatalogId, kSubBlockSize);
+  fillBlockWithSampleData(true, false);
+  checkBlockValues(kInvalidCatalogId, 0, 0);
+}
+
 TEST_P(BasicColumnStoreTupleStorageSubBlockTest, GetValuesWithIntSortColumnTest) {
   createBlock(0, kSubBlockSize);
   fillBlockWithSampleData(true, false);
@@ -955,6 +1008,10 @@ TEST_P(BasicColumnStoreTupleStorageSubBlockTest, GetValuesWithWideCharSortColumn
   checkBlockValues(3, 0, 0);
 }
 
+TEST_P(BasicColumnStoreTupleStorageSubBlockTest, DeleteWithNoSortColumnTest) {
+  runDeleteTest(kInvalidCatalogId);
+}
+
 TEST_P(BasicColumnStoreTupleStorageSubBlockTest, DeleteWithIntSortColumnTest) {
   runDeleteTest(0);
 }
@@ -971,6 +1028,10 @@ TEST_P(BasicColumnStoreTupleStorageSubBlockTest, DeleteWithWideCharSortColumnTes
   runDeleteTest(3);
 }
 
+TEST_P(BasicColumnStoreTupleStorageSubBlockTest, GetMatchesForPredicateWithNoSortColumnTest) {
+  runCheckPredicateTest(kInvalidCatalogId);
+}
+
 TEST_P(BasicColumnStoreTupleStorageSubBlockTest, GetMatchesForPredicateWithIntSortColumnTest) {
   runCheckPredicateTest(0);
 }
@@ -987,6 +1048,10 @@ TEST_P(BasicColumnStoreTupleStorageSubBlockTest, GetMatchesForPredicateWithWideC
   runCheckPredicateTest(3);
 }
 
+TEST_P(BasicColumnStoreTupleStorageSubBlockTest, SetAttributeValueInPlaceTypedWithNoSortColumnTest) {
+  runSetAttributeValueInPlaceTypedTest(kInvalidCatalogId);
+}
+
 TEST_P(BasicColumnStoreTupleStorageSubBlockTest, SetAttributeValueInPlaceTypedWithIntSortColumnTest) {
   runSetAttributeValueInPlaceTypedTest(0);
 }


[46/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-33: Fixed the bug in NumericCast.

Posted by zu...@apache.org.
QUICKSTEP-33: Fixed the bug in NumericCast.


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

Branch: refs/heads/tmp-relation-col-store
Commit: b258821ef6c00df199e52249eb1099a6d885bbb1
Parents: 3347003
Author: Hakan Memisoglu <ha...@gmail.com>
Authored: Wed Jun 29 14:01:13 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Jun 30 13:26:50 2016 -0700

----------------------------------------------------------------------
 types/operations/unary_operations/NumericCastOperation.hpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b258821e/types/operations/unary_operations/NumericCastOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/NumericCastOperation.hpp b/types/operations/unary_operations/NumericCastOperation.hpp
index 250df6d..6662796 100644
--- a/types/operations/unary_operations/NumericCastOperation.hpp
+++ b/types/operations/unary_operations/NumericCastOperation.hpp
@@ -126,7 +126,7 @@ class UncheckedNumericCastOperator : public UncheckedUnaryOperator {
           result->appendNullValue();
         } else {
           *static_cast<typename TargetType::cpptype*>(result->getPtrForDirectWrite())
-              = static_cast<typename SourceType::cpptype>(*scalar_arg);
+              = static_cast<typename TargetType::cpptype>(*scalar_arg);
         }
       }
       return result;


[03/50] [abbrv] incubator-quickstep git commit: Added Query ID to Relational operators and WorkOrders.

Posted by zu...@apache.org.
Added Query ID to Relational operators and WorkOrders.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 405426823d0e12c75cef447c130fb2c92cacb90f
Parents: 83935e7
Author: Harshad Deshmukh <ha...@cs.wisc.edu>
Authored: Sun May 1 23:01:39 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed Jun 8 11:57:49 2016 -0700

----------------------------------------------------------------------
 query_execution/Foreman.cpp                     |  3 +
 query_execution/QueryManager.cpp                |  1 +
 query_execution/tests/Foreman_unittest.cpp      |  2 +-
 query_execution/tests/QueryManager_unittest.cpp |  2 +-
 .../tests/WorkOrdersContainer_unittest.cpp      |  2 +-
 query_optimizer/ExecutionGenerator.cpp          | 68 +++++++++++++-------
 .../tests/ExecutionHeuristics_unittest.cpp      |  6 +-
 relational_operators/AggregationOperator.cpp    | 12 ++--
 relational_operators/AggregationOperator.hpp    | 12 +++-
 relational_operators/BuildHashOperator.cpp      |  2 +
 relational_operators/BuildHashOperator.hpp      | 17 +++--
 relational_operators/CreateIndexOperator.hpp    |  8 ++-
 relational_operators/CreateTableOperator.hpp    | 10 ++-
 relational_operators/DeleteOperator.cpp         |  2 +
 relational_operators/DeleteOperator.hpp         | 12 +++-
 relational_operators/DestroyHashOperator.cpp    |  5 +-
 relational_operators/DestroyHashOperator.hpp    | 12 +++-
 relational_operators/DropTableOperator.cpp      |  3 +-
 relational_operators/DropTableOperator.hpp      | 13 +++-
 .../FinalizeAggregationOperator.cpp             |  6 +-
 .../FinalizeAggregationOperator.hpp             | 15 +++--
 relational_operators/HashJoinOperator.cpp       |  4 ++
 relational_operators/HashJoinOperator.hpp       | 49 ++++++++++----
 relational_operators/InsertOperator.cpp         |  6 +-
 relational_operators/InsertOperator.hpp         | 15 +++--
 .../NestedLoopsJoinOperator.cpp                 |  4 ++
 .../NestedLoopsJoinOperator.hpp                 | 12 +++-
 relational_operators/RebuildWorkOrder.hpp       |  5 +-
 relational_operators/RelationalOperator.hpp     |  8 ++-
 relational_operators/SampleOperator.cpp         |  7 +-
 relational_operators/SampleOperator.hpp         | 13 ++--
 relational_operators/SaveBlocksOperator.cpp     |  1 +
 relational_operators/SaveBlocksOperator.hpp     | 12 +++-
 relational_operators/SelectOperator.cpp         |  4 ++
 relational_operators/SelectOperator.hpp         | 24 +++++--
 relational_operators/SortMergeRunOperator.cpp   |  1 +
 relational_operators/SortMergeRunOperator.hpp   | 12 +++-
 .../SortRunGenerationOperator.cpp               |  2 +
 .../SortRunGenerationOperator.hpp               | 12 +++-
 relational_operators/TableGeneratorOperator.cpp |  7 +-
 relational_operators/TableGeneratorOperator.hpp | 13 ++--
 relational_operators/TextScanOperator.cpp       | 11 +++-
 relational_operators/TextScanOperator.hpp       | 16 ++++-
 relational_operators/UpdateOperator.cpp         |  1 +
 relational_operators/UpdateOperator.hpp         | 12 +++-
 relational_operators/WorkOrder.hpp              |  9 ++-
 relational_operators/WorkOrder.proto            |  1 +
 relational_operators/WorkOrderFactory.cpp       | 32 +++++++--
 .../tests/AggregationOperator_unittest.cpp      | 16 +++--
 .../tests/HashJoinOperator_unittest.cpp         | 48 ++++++++------
 .../tests/SortMergeRunOperator_unittest.cpp     |  6 +-
 .../SortRunGenerationOperator_unittest.cpp      |  3 +-
 .../tests/TextScanOperator_unittest.cpp         |  3 +-
 53 files changed, 430 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index 304c429..b358f70 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -518,11 +518,14 @@ void Foreman::getRebuildWorkOrders(const dag_node_index index, WorkOrdersContain
   for (vector<MutableBlockReference>::size_type i = 0;
        i < partially_filled_block_refs.size();
        ++i) {
+    // Note: The query ID used below is dummy for now, it will be replaced with
+    // the true query ID when QueryManager gets used in Foreman.
     container->addRebuildWorkOrder(
         new RebuildWorkOrder(move(partially_filled_block_refs[i]),
                             index,
                             op.getOutputRelationID(),
                             foreman_client_id_,
+                            0,
                             bus_),
         index);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/query_execution/QueryManager.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManager.cpp b/query_execution/QueryManager.cpp
index 02c5d4c..21f5820 100644
--- a/query_execution/QueryManager.cpp
+++ b/query_execution/QueryManager.cpp
@@ -461,6 +461,7 @@ void QueryManager::getRebuildWorkOrders(const dag_node_index index,
                             index,
                             op.getOutputRelationID(),
                             foreman_client_id_,
+                            query_id_,
                             bus_),
         index);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/query_execution/tests/Foreman_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/Foreman_unittest.cpp b/query_execution/tests/Foreman_unittest.cpp
index 47cc641..d2f43a4 100644
--- a/query_execution/tests/Foreman_unittest.cpp
+++ b/query_execution/tests/Foreman_unittest.cpp
@@ -61,7 +61,7 @@ namespace quickstep {
 class MockWorkOrder : public WorkOrder {
  public:
   explicit MockWorkOrder(const int op_index)
-      : op_index_(op_index) {}
+      : WorkOrder(0), op_index_(op_index) {}
 
   void execute() override {
     VLOG(3) << "WorkOrder[" << op_index_ << "] executing.";

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 1b9be48..80876f2 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -62,7 +62,7 @@ namespace quickstep {
 class MockWorkOrder : public WorkOrder {
  public:
   explicit MockWorkOrder(const int op_index)
-      : op_index_(op_index) {}
+      : WorkOrder(0), op_index_(op_index) {}
 
   void execute() override {
     VLOG(3) << "WorkOrder[" << op_index_ << "] executing.";

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/query_execution/tests/WorkOrdersContainer_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/WorkOrdersContainer_unittest.cpp b/query_execution/tests/WorkOrdersContainer_unittest.cpp
index d7db9a6..cf133c4 100644
--- a/query_execution/tests/WorkOrdersContainer_unittest.cpp
+++ b/query_execution/tests/WorkOrdersContainer_unittest.cpp
@@ -30,7 +30,7 @@ namespace quickstep {
 class MockNUMAWorkOrder : public WorkOrder {
  public:
   MockNUMAWorkOrder(const int id, const std::vector<int> &numa_nodes)
-      : id_(id) {
+      : WorkOrder(0), id_(id) {
     for (int numa_node : numa_nodes) {
       preferred_numa_nodes_.push_back(numa_node);
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index c590b6e..30dfa8e 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -194,6 +194,7 @@ void ExecutionGenerator::generatePlan(const P::PhysicalPtr &physical_plan) {
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
             new DropTableOperator(*temporary_relation,
+                                  query_handle_->query_id(),
                                   optimizer_context_->catalog_database(),
                                   false /* only_drop_blocks */));
     DCHECK(!temporary_relation_info.isStoredRelation());
@@ -415,7 +416,8 @@ void ExecutionGenerator::convertSample(const P::SamplePtr &physical_sample) {
                                                  insert_destination_index,
                                                  input_relation_info->isStoredRelation(),
                                                  physical_sample->is_block_sample(),
-                                                 physical_sample->percentage());
+                                                 physical_sample->percentage(),
+                                                 query_handle_->query_id());
   const QueryPlan::DAGNodeIndex sample_index =
       execution_plan_->addRelationalOperator(sample_op);
   insert_destination_proto->set_relational_op_index(sample_index);
@@ -531,13 +533,15 @@ void ExecutionGenerator::convertSelection(
                              insert_destination_index,
                              execution_predicate_index,
                              move(attributes),
-                             input_relation_info->isStoredRelation())
+                             input_relation_info->isStoredRelation(),
+                             query_handle_->query_id())
         : new SelectOperator(*input_relation_info->relation,
                              *output_relation,
                              insert_destination_index,
                              execution_predicate_index,
                              project_expressions_group_index,
-                             input_relation_info->isStoredRelation());
+                             input_relation_info->isStoredRelation(),
+                             query_handle_->query_id());
 
   const QueryPlan::DAGNodeIndex select_index =
       execution_plan_->addRelationalOperator(op);
@@ -741,7 +745,8 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
               build_relation_info->isStoredRelation(),
               build_attribute_ids,
               any_build_attributes_nullable,
-              join_hash_table_index));
+              join_hash_table_index,
+              query_handle_->query_id()));
 
   // Create InsertDestination proto.
   const CatalogRelation *output_relation = nullptr;
@@ -787,13 +792,14 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
               join_hash_table_index,
               residual_predicate_index,
               project_expressions_group_index,
+              query_handle_->query_id(),
               is_selection_on_build.get(),
               join_type));
   insert_destination_proto->set_relational_op_index(join_operator_index);
 
   const QueryPlan::DAGNodeIndex destroy_operator_index =
-      execution_plan_->addRelationalOperator(
-          new DestroyHashOperator(join_hash_table_index));
+      execution_plan_->addRelationalOperator(new DestroyHashOperator(
+          join_hash_table_index, query_handle_->query_id()));
 
   if (!build_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(build_operator_index,
@@ -887,7 +893,8 @@ void ExecutionGenerator::convertNestedLoopsJoin(
           execution_join_predicate_index,
           project_expressions_group_index,
           left_relation_info->isStoredRelation(),
-          right_relation_info->isStoredRelation()));
+          right_relation_info->isStoredRelation(),
+          query_handle_->query_id()));
   insert_destination_proto->set_relational_op_index(join_operator_index);
 
   if (!left_relation_info->isStoredRelation()) {
@@ -938,12 +945,13 @@ void ExecutionGenerator::convertCopyFrom(
               physical_plan->escape_strings(),
               FLAGS_parallelize_load,
               *output_relation,
-              insert_destination_index));
+              insert_destination_index,
+              query_handle_->query_id()));
   insert_destination_proto->set_relational_op_index(scan_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_operator_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator());
+          new SaveBlocksOperator(query_handle_->query_id()));
   execution_plan_->addDirectDependency(save_blocks_operator_index,
                                        scan_operator_index,
                                        false /* is_pipeline_breaker */);
@@ -991,6 +999,7 @@ void ExecutionGenerator::convertCreateIndex(
   }
   execution_plan_->addRelationalOperator(new CreateIndexOperator(input_relation,
                                                                  physical_plan->index_name(),
+                                                                 query_handle_->query_id(),
                                                                  std::move(index_description)));
 }
 
@@ -1031,7 +1040,8 @@ void ExecutionGenerator::convertCreateTable(
   }
 
   execution_plan_->addRelationalOperator(
-      new CreateTableOperator(catalog_relation.release(),
+      new CreateTableOperator(query_handle_->query_id(),
+                              catalog_relation.release(),
                               optimizer_context_->catalog_database()));
 }
 
@@ -1058,6 +1068,7 @@ void ExecutionGenerator::convertDeleteTuples(
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
             new DropTableOperator(*input_relation_info->relation,
+                                  query_handle_->query_id(),
                                   optimizer_context_->catalog_database(),
                                   true /* only_drop_blocks */));
     if (!input_relation_info->isStoredRelation()) {
@@ -1073,7 +1084,8 @@ void ExecutionGenerator::convertDeleteTuples(
         execution_plan_->addRelationalOperator(new DeleteOperator(
             *input_relation_info->relation,
             execution_predicate_index,
-            input_relation_info->isStoredRelation()));
+            input_relation_info->isStoredRelation(),
+            query_handle_->query_id()));
     if (!input_relation_info->isStoredRelation()) {
       execution_plan_->addDirectDependency(delete_tuples_index,
                                            input_relation_info->producer_operator_index,
@@ -1082,7 +1094,7 @@ void ExecutionGenerator::convertDeleteTuples(
 
     const QueryPlan::DAGNodeIndex save_blocks_index =
         execution_plan_->addRelationalOperator(
-            new SaveBlocksOperator());
+            new SaveBlocksOperator(query_handle_->query_id()));
     execution_plan_->addDirectDependency(save_blocks_index,
                                          delete_tuples_index,
                                          false /* is_pipeline_breaker */);
@@ -1100,6 +1112,7 @@ void ExecutionGenerator::convertDropTable(
 
   execution_plan_->addRelationalOperator(
       new DropTableOperator(catalog_relation,
+                            query_handle_->query_id(),
                             optimizer_context_->catalog_database()));
 }
 
@@ -1153,12 +1166,13 @@ void ExecutionGenerator::convertInsertTuple(
       execution_plan_->addRelationalOperator(
           new InsertOperator(input_relation,
                              insert_destination_index,
-                             tuple_index));
+                             tuple_index,
+                             query_handle_->query_id()));
   insert_destination_proto->set_relational_op_index(insert_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator());
+          new SaveBlocksOperator(query_handle_->query_id()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(insert_operator_index,
                                          input_relation_info->producer_operator_index,
@@ -1231,14 +1245,15 @@ void ExecutionGenerator::convertInsertSelection(
                          insert_destination_index,
                          QueryContext::kInvalidPredicateId,
                          move(attributes),
-                         selection_relation_info->isStoredRelation());
+                         selection_relation_info->isStoredRelation(),
+                         query_handle_->query_id());
 
   const QueryPlan::DAGNodeIndex insert_selection_index =
       execution_plan_->addRelationalOperator(insert_selection_op);
   insert_destination_proto->set_relational_op_index(insert_selection_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
-      execution_plan_->addRelationalOperator(new SaveBlocksOperator());
+      execution_plan_->addRelationalOperator(new SaveBlocksOperator(query_handle_->query_id()));
 
   if (!selection_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(insert_selection_index,
@@ -1311,12 +1326,13 @@ void ExecutionGenerator::convertUpdateTable(
               *optimizer_context_->catalog_database()->getRelationById(input_rel_id),
               relocation_destination_index,
               execution_predicate_index,
-              update_group_index));
+              update_group_index,
+              query_handle_->query_id()));
   relocation_destination_proto->set_relational_op_index(update_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator());
+          new SaveBlocksOperator(query_handle_->query_id()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(update_operator_index,
                                          input_relation_info->producer_operator_index,
@@ -1415,7 +1431,8 @@ void ExecutionGenerator::convertAggregate(
           new AggregationOperator(
               *input_relation_info->relation,
               input_relation_info->isStoredRelation(),
-              aggr_state_index));
+              aggr_state_index,
+              query_handle_->query_id()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(aggregation_operator_index,
                                          input_relation_info->producer_operator_index,
@@ -1435,7 +1452,8 @@ void ExecutionGenerator::convertAggregate(
       execution_plan_->addRelationalOperator(
           new FinalizeAggregationOperator(aggr_state_index,
                                           *output_relation,
-                                          insert_destination_index));
+                                          insert_destination_index,
+                                          query_handle_->query_id()));
   insert_destination_proto->set_relational_op_index(finalize_aggregation_operator_index);
 
   execution_plan_->addDirectDependency(finalize_aggregation_operator_index,
@@ -1486,7 +1504,8 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
                                         *initial_runs_relation,
                                         initial_runs_destination_id,
                                         sort_run_gen_config_id,
-                                        input_relation_info->isStoredRelation()));
+                                        input_relation_info->isStoredRelation(),
+                                        query_handle_->query_id()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(run_generator_index,
                                          input_relation_info->producer_operator_index,
@@ -1543,7 +1562,8 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
                                    sort_merge_run_config_id,
                                    64 /* merge_factor */,
                                    physical_sort->limit(),
-                                   false /* input_relation_is_stored */));
+                                   false /* input_relation_is_stored */,
+                                   query_handle_->query_id()));
   execution_plan_->addDirectDependency(merge_run_operator_index,
                                        run_generator_index,
                                        false /* is_pipeline_breaker */);
@@ -1557,6 +1577,7 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
       execution_plan_->addRelationalOperator(
           new DropTableOperator(
               *merged_runs_relation,
+              query_handle_->query_id(),
               optimizer_context_->catalog_database(),
               false /* only_drop_blocks */));
   execution_plan_->addDirectDependency(
@@ -1594,7 +1615,8 @@ void ExecutionGenerator::convertTableGenerator(
   TableGeneratorOperator *op =
       new TableGeneratorOperator(*output_relation,
                                  insert_destination_index,
-                                 generator_function_index);
+                                 generator_function_index,
+                                 query_handle_->query_id());
 
   const QueryPlan::DAGNodeIndex tablegen_index =
       execution_plan_->addRelationalOperator(op);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
index 12acaff..a08a476 100644
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
@@ -79,7 +79,8 @@ class ExecutionHeuristicsTest : public ::testing::Test {
                                                                 true,
                                                                 build_attribute_ids,
                                                                 false,
-                                                                join_hash_table_index));
+                                                                join_hash_table_index,
+                                                                0  /* dummy query ID */));
     return build_operator_index;
   }
 
@@ -100,7 +101,8 @@ class ExecutionHeuristicsTest : public ::testing::Test {
                                                                0,
                                                                join_hash_table_index,
                                                                0,
-                                                               0));
+                                                               0,
+                                                               0  /* dummy query ID */));
     return join_operator_index;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/AggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.cpp b/relational_operators/AggregationOperator.cpp
index 1b935ee..94ba901 100644
--- a/relational_operators/AggregationOperator.cpp
+++ b/relational_operators/AggregationOperator.cpp
@@ -38,8 +38,10 @@ bool AggregationOperator::getAllWorkOrders(
     if (!started_) {
       for (const block_id input_block_id : input_relation_block_ids_) {
         container->addNormalWorkOrder(
-            new AggregationWorkOrder(input_block_id,
-                                     query_context->getAggregationState(aggr_state_index_)),
+            new AggregationWorkOrder(
+                input_block_id,
+                query_id_,
+                query_context->getAggregationState(aggr_state_index_)),
             op_index_);
       }
       started_ = true;
@@ -48,8 +50,10 @@ bool AggregationOperator::getAllWorkOrders(
   } else {
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
-          new AggregationWorkOrder(input_relation_block_ids_[num_workorders_generated_],
-                                   query_context->getAggregationState(aggr_state_index_)),
+          new AggregationWorkOrder(
+              input_relation_block_ids_[num_workorders_generated_],
+              query_id_,
+              query_context->getAggregationState(aggr_state_index_)),
           op_index_);
       ++num_workorders_generated_;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 0e74dfc..0fbc381 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -58,11 +58,14 @@ class AggregationOperator : public RelationalOperator {
    *        is fully available to the operator before it can start generating
    *        workorders.
    * @param aggr_state_index The index of the AggregationState in QueryContext.
+   * @param query_id The ID of this query.
    **/
   AggregationOperator(const CatalogRelation &input_relation,
                       bool input_relation_is_stored,
-                      const QueryContext::aggregation_state_id aggr_state_index)
-      : input_relation_is_stored_(input_relation_is_stored),
+                      const QueryContext::aggregation_state_id aggr_state_index,
+                      const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_is_stored_(input_relation_is_stored),
         input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
                                                            : std::vector<block_id>()),
         aggr_state_index_(aggr_state_index),
@@ -107,11 +110,14 @@ class AggregationWorkOrder : public WorkOrder {
    * @brief Constructor
    *
    * @param input_block_id The block id.
+   * @param query_id The ID of this query.
    * @param state The AggregationState to use.
    **/
   AggregationWorkOrder(const block_id input_block_id,
+                       const std::size_t query_id,
                        AggregationOperationState *state)
-      : input_block_id_(input_block_id),
+      : WorkOrder(query_id),
+        input_block_id_(input_block_id),
         state_(DCHECK_NOTNULL(state)) {}
 
   ~AggregationWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index df92159..c6f6f96 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -73,6 +73,7 @@ bool BuildHashOperator::getAllWorkOrders(
                                    join_key_attributes_,
                                    any_join_key_attributes_nullable_,
                                    input_block_id,
+                                   query_id_,
                                    hash_table,
                                    storage_manager),
             op_index_);
@@ -88,6 +89,7 @@ bool BuildHashOperator::getAllWorkOrders(
               join_key_attributes_,
               any_join_key_attributes_nullable_,
               input_relation_block_ids_[num_workorders_generated_],
+              query_id_,
               hash_table,
               storage_manager),
           op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index f9d830f..5a46d8b 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -69,13 +69,16 @@ class BuildHashOperator : public RelationalOperator {
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
    *        The HashTable's key Type(s) should be the Type(s) of the
    *        join_key_attributes in input_relation.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   BuildHashOperator(const CatalogRelation &input_relation,
                     const bool input_relation_is_stored,
                     const std::vector<attribute_id> &join_key_attributes,
                     const bool any_join_key_attributes_nullable,
-                    const QueryContext::join_hash_table_id hash_table_index)
-    : input_relation_(input_relation),
+                    const QueryContext::join_hash_table_id hash_table_index,
+                    const std::size_t query_id)
+    : RelationalOperator(query_id),
+      input_relation_(input_relation),
       input_relation_is_stored_(input_relation_is_stored),
       join_key_attributes_(join_key_attributes),
       any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -133,6 +136,7 @@ class BuildHashWorkOrder : public WorkOrder {
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
    * @param build_block_id The block id.
+   * @param query_id The ID of the query.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
    **/
@@ -140,9 +144,11 @@ class BuildHashWorkOrder : public WorkOrder {
                      const std::vector<attribute_id> &join_key_attributes,
                      const bool any_join_key_attributes_nullable,
                      const block_id build_block_id,
+                     const std::size_t query_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
         build_block_id_(build_block_id),
@@ -156,6 +162,7 @@ class BuildHashWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param query_id The ID of the query.
    * @param build_block_id The block id.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
@@ -164,9 +171,11 @@ class BuildHashWorkOrder : public WorkOrder {
                      std::vector<attribute_id> &&join_key_attributes,
                      const bool any_join_key_attributes_nullable,
                      const block_id build_block_id,
+                     const std::size_t query_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
         build_block_id_(build_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index 2bfacc4..ede3f02 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_CREATE_INDEX_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_CREATE_INDEX_OPERATOR_HPP_
 
+#include <cstddef>
 #include <string>
 
 #include "catalog/CatalogRelation.hpp"
@@ -52,19 +53,22 @@ class CreateIndexOperator : public RelationalOperator {
    *
    * @param relation The relation to create index upon.
    * @param index_name The index to create.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param index_description The index_description associated with this index.
    **/
   CreateIndexOperator(CatalogRelation *relation,
                       const std::string &index_name,
+                      const std::size_t query_id,
                       IndexSubBlockDescription &&index_description)  // NOLINT(whitespace/operators)
-      : relation_(DCHECK_NOTNULL(relation)),
+      : RelationalOperator(query_id),
+        relation_(DCHECK_NOTNULL(relation)),
         index_name_(index_name),
         index_description_(index_description) {}
 
   ~CreateIndexOperator() override {}
 
   /**
-   * @note no WorkOrder generated for this operator.
+   * @note No WorkOrder generated for this operator.
    **/
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 98f3253..60bcef4 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_CREATE_TABLE_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_CREATE_TABLE_OPERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 
 #include "catalog/CatalogRelation.hpp"
@@ -49,20 +50,23 @@ class CreateTableOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param relation The relation to add. This CreateTableOperator owns
    *        relation until the WorkOrder it produces is successfully executed,
    *        at which point it is owned by database.
    * @param database The database to add a relation to.
    **/
-  CreateTableOperator(CatalogRelation *relation,
+  CreateTableOperator(const std::size_t query_id,
+                      CatalogRelation *relation,
                       CatalogDatabase *database)
-      : relation_(DCHECK_NOTNULL(relation)),
+      : RelationalOperator(query_id),
+        relation_(DCHECK_NOTNULL(relation)),
         database_(DCHECK_NOTNULL(database)) {}
 
   ~CreateTableOperator() override {}
 
   /**
-   * @note no WorkOrder generated for this operator.
+   * @note No WorkOrder generated for this operator.
    **/
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 2c2c6de..94169ed 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -60,6 +60,7 @@ bool DeleteOperator::getAllWorkOrders(
                                 storage_manager,
                                 op_index_,
                                 scheduler_client_id,
+                                query_id_,
                                 bus),
             op_index_);
       }
@@ -75,6 +76,7 @@ bool DeleteOperator::getAllWorkOrders(
                               storage_manager,
                               op_index_,
                               scheduler_client_id,
+                              query_id_,
                               bus),
           op_index_);
       ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 1d44552..ba1f825 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -61,11 +61,14 @@ class DeleteOperator : public RelationalOperator {
    *        tuples will be deleted).
    * @param relation_is_stored If relation is a stored relation and is fully
    *        available to the operator before it can start generating workorders.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   DeleteOperator(const CatalogRelation &relation,
                  const QueryContext::predicate_id predicate_index,
-                 const bool relation_is_stored)
-     :  relation_(relation),
+                 const bool relation_is_stored,
+                 const std::size_t query_id)
+     :  RelationalOperator(query_id),
+        relation_(relation),
         predicate_index_(predicate_index),
         relation_is_stored_(relation_is_stored),
         started_(false),
@@ -127,6 +130,7 @@ class DeleteWorkOrder : public WorkOrder {
    * @param delete_operator_index The index of the Delete Operator in the query
    *        plan DAG.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
+   * @param query_id The ID of the query to which this workorder belongs.
    * @param bus A pointer to the TMB.
    **/
   DeleteWorkOrder(const CatalogRelationSchema &input_relation,
@@ -135,8 +139,10 @@ class DeleteWorkOrder : public WorkOrder {
                   StorageManager *storage_manager,
                   const std::size_t delete_operator_index,
                   const tmb::client_id scheduler_client_id,
+                  const std::size_t query_id,
                   MessageBus *bus)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
         storage_manager_(DCHECK_NOTNULL(storage_manager)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/DestroyHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.cpp b/relational_operators/DestroyHashOperator.cpp
index c2220d5..c92117a 100644
--- a/relational_operators/DestroyHashOperator.cpp
+++ b/relational_operators/DestroyHashOperator.cpp
@@ -32,8 +32,9 @@ bool DestroyHashOperator::getAllWorkOrders(
     tmb::MessageBus *bus) {
   if (blocking_dependencies_met_ && !work_generated_) {
     work_generated_ = true;
-    container->addNormalWorkOrder(new DestroyHashWorkOrder(hash_table_index_, query_context),
-                                  op_index_);
+    container->addNormalWorkOrder(
+        new DestroyHashWorkOrder(hash_table_index_, query_id_, query_context),
+        op_index_);
   }
   return work_generated_;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 46331ba..086c279 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -47,9 +47,12 @@ class DestroyHashOperator : public RelationalOperator {
    * @brief Constructor.
    *
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
-  explicit DestroyHashOperator(const QueryContext::join_hash_table_id hash_table_index)
-      : hash_table_index_(hash_table_index),
+  DestroyHashOperator(const QueryContext::join_hash_table_id hash_table_index,
+                      const std::size_t query_id)
+      : RelationalOperator(query_id),
+        hash_table_index_(hash_table_index),
         work_generated_(false) {}
 
   ~DestroyHashOperator() override {}
@@ -76,11 +79,14 @@ class DestroyHashWorkOrder : public WorkOrder {
    * @brief Constructor.
    *
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param query_context The QueryContext to use.
    **/
   DestroyHashWorkOrder(const QueryContext::join_hash_table_id hash_table_index,
+                       const std::size_t query_id,
                        QueryContext *query_context)
-      : hash_table_index_(hash_table_index),
+      : WorkOrder(query_id),
+        hash_table_index_(hash_table_index),
         query_context_(DCHECK_NOTNULL(query_context)) {}
 
   ~DestroyHashWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/DropTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.cpp b/relational_operators/DropTableOperator.cpp
index f3a3a2c..256f6a1 100644
--- a/relational_operators/DropTableOperator.cpp
+++ b/relational_operators/DropTableOperator.cpp
@@ -45,7 +45,8 @@ bool DropTableOperator::getAllWorkOrders(
 
     // DropTableWorkOrder only drops blocks, if any.
     container->addNormalWorkOrder(
-        new DropTableWorkOrder(std::move(relation_blocks), storage_manager),
+        new DropTableWorkOrder(
+            query_id_, std::move(relation_blocks), storage_manager),
         op_index_);
 
     database_->setStatus(CatalogDatabase::Status::kPendingBlockDeletions);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index bf9b1b1..0bbb718 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_DROP_TABLE_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_DROP_TABLE_OPERATOR_HPP_
 
+#include <cstddef>
 #include <utility>
 #include <vector>
 
@@ -55,14 +56,17 @@ class DropTableOperator : public RelationalOperator {
    * @brief Constructor.
    *
    * @param relation The relation to drop.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param database The databse where to drop \c relation.
    * @param only_drop_blocks If true, only drop the blocks belonging to \c
    *        relation, but leave \c relation in \c database.
    **/
   DropTableOperator(const CatalogRelation &relation,
+                    const std::size_t query_id,
                     CatalogDatabase *database,
                     const bool only_drop_blocks = false)
-      : relation_(relation),
+      : RelationalOperator(query_id),
+        relation_(relation),
         database_(database),
         only_drop_blocks_(only_drop_blocks),
         work_generated_(false) {}
@@ -95,17 +99,20 @@ class DropTableWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param blocks The blocks to drop.
    * @param storage_manager The StorageManager to use.
    * @param rel_id The relation id to drop.
    * @param catalog_database_cache The CatalogDatabaseCache in the distributed
    *        version.
    **/
-  DropTableWorkOrder(std::vector<block_id> &&blocks,
+  DropTableWorkOrder(const std::size_t query_id,
+                     std::vector<block_id> &&blocks,
                      StorageManager *storage_manager,
                      const relation_id rel_id = kInvalidCatalogId,
                      CatalogDatabaseLite *catalog_database_cache = nullptr)
-      : blocks_(std::move(blocks)),
+      : WorkOrder(query_id),
+        blocks_(std::move(blocks)),
         storage_manager_(DCHECK_NOTNULL(storage_manager)),
         rel_id_(rel_id),
         catalog_database_cache_(catalog_database_cache) {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 410ec69..1dc4188 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -38,8 +38,10 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
   if (blocking_dependencies_met_ && !started_) {
     started_ = true;
     container->addNormalWorkOrder(
-        new FinalizeAggregationWorkOrder(query_context->releaseAggregationState(aggr_state_index_),
-                                         query_context->getInsertDestination(output_destination_index_)),
+        new FinalizeAggregationWorkOrder(
+            query_id_,
+            query_context->releaseAggregationState(aggr_state_index_),
+            query_context->getInsertDestination(output_destination_index_)),
         op_index_);
   }
   return started_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index fb9608a..51e55dc 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_FINALIZE_AGGREGATION_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_FINALIZE_AGGREGATION_OPERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 
 #include "catalog/CatalogRelation.hpp"
@@ -57,11 +58,14 @@ class FinalizeAggregationOperator : public RelationalOperator {
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert aggregation results.
+   * @param query_id The ID of the query to which this operator belongs.
    */
   FinalizeAggregationOperator(const QueryContext::aggregation_state_id aggr_state_index,
                               const CatalogRelation &output_relation,
-                              const QueryContext::insert_destination_id output_destination_index)
-      : aggr_state_index_(aggr_state_index),
+                              const QueryContext::insert_destination_id output_destination_index,
+                              const std::size_t query_id)
+      : RelationalOperator(query_id),
+        aggr_state_index_(aggr_state_index),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         started_(false) {}
@@ -101,13 +105,16 @@ class FinalizeAggregationWorkOrder : public WorkOrder {
    *
    * @note InsertWorkOrder takes ownership of \c state.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param state The AggregationState to use.
    * @param output_destination The InsertDestination to insert aggregation
    *        results.
    */
-  FinalizeAggregationWorkOrder(AggregationOperationState *state,
+  FinalizeAggregationWorkOrder(const std::size_t query_id,
+                               AggregationOperationState *state,
                                InsertDestination *output_destination)
-      : state_(DCHECK_NOTNULL(state)),
+      : WorkOrder(query_id),
+        state_(DCHECK_NOTNULL(state)),
         output_destination_(DCHECK_NOTNULL(output_destination)) {}
 
   ~FinalizeAggregationWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index aa03794..d8c7304 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -298,6 +298,7 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
                                      residual_predicate,
                                      selection,
                                      hash_table,
+                                     query_id_,
                                      output_destination,
                                      storage_manager),
               op_index_);
@@ -316,6 +317,7 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
                                    residual_predicate,
                                    selection,
                                    hash_table,
+                                   query_id_,
                                    output_destination,
                                    storage_manager),
             op_index_);
@@ -356,6 +358,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
                   selection,
                   is_selection_on_build_,
                   hash_table,
+                  query_id_,
                   output_destination,
                   storage_manager),
               op_index_);
@@ -375,6 +378,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
                 selection,
                 is_selection_on_build_,
                 hash_table,
+                query_id_,
                 output_destination,
                 storage_manager),
             op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index fcc087a..825f360 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -108,6 +108,7 @@ class HashJoinOperator : public RelationalOperator {
    *        corresponding to the attributes of the relation referred by
    *        output_relation_id. Each Scalar is evaluated for the joined tuples,
    *        and the resulting value is inserted into the join result.
+   * @param query_id The ID of the query.
    * @param is_selection_on_build Whether each selection Scalar is using attributes
    *        from the build relation as input. Can be NULL for inner/semi/anti
    *        joins since this information is not utilized by these joins.
@@ -123,9 +124,11 @@ class HashJoinOperator : public RelationalOperator {
                    const QueryContext::join_hash_table_id hash_table_index,
                    const QueryContext::predicate_id residual_predicate_index,
                    const QueryContext::scalar_group_id selection_index,
+                   const std::size_t query_id,
                    const std::vector<bool> *is_selection_on_build = nullptr,
                    const JoinType join_type = JoinType::kInnerJoin)
-      : build_relation_(build_relation),
+      : RelationalOperator(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         probe_relation_is_stored_(probe_relation_is_stored),
         join_key_attributes_(join_key_attributes),
@@ -243,6 +246,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -254,9 +258,11 @@ class HashInnerJoinWorkOrder : public WorkOrder {
                          const Predicate *residual_predicate,
                          const std::vector<std::unique_ptr<const Scalar>> &selection,
                          const JoinHashTable &hash_table,
+                         const std::size_t query_id,
                          InsertDestination *output_destination,
                          StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -286,6 +292,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -297,9 +304,11 @@ class HashInnerJoinWorkOrder : public WorkOrder {
                          const Predicate *residual_predicate,
                          const std::vector<std::unique_ptr<const Scalar>> &selection,
                          const JoinHashTable &hash_table,
+                         const std::size_t query_id,
                          InsertDestination *output_destination,
                          StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -366,6 +375,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -377,9 +387,11 @@ class HashSemiJoinWorkOrder : public WorkOrder {
                         const Predicate *residual_predicate,
                         const std::vector<std::unique_ptr<const Scalar>> &selection,
                         const JoinHashTable &hash_table,
+                        const std::size_t query_id,
                         InsertDestination *output_destination,
                         StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -409,6 +421,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -420,9 +433,11 @@ class HashSemiJoinWorkOrder : public WorkOrder {
                         const Predicate *residual_predicate,
                         const std::vector<std::unique_ptr<const Scalar>> &selection,
                         const JoinHashTable &hash_table,
+                        const std::size_t query_id,
                         InsertDestination *output_destination,
                         StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -482,6 +497,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -493,9 +509,11 @@ class HashAntiJoinWorkOrder : public WorkOrder {
                         const Predicate *residual_predicate,
                         const std::vector<std::unique_ptr<const Scalar>> &selection,
                         const JoinHashTable &hash_table,
+                        const std::size_t query_id,
                         InsertDestination *output_destination,
                         StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -525,6 +543,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -536,9 +555,11 @@ class HashAntiJoinWorkOrder : public WorkOrder {
                         const Predicate *residual_predicate,
                         const std::vector<std::unique_ptr<const Scalar>> &selection,
                         const JoinHashTable &hash_table,
+                        const std::size_t query_id,
                         InsertDestination *output_destination,
                         StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -602,6 +623,7 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
    * @param lookup_block_id The block id of the probe_relation.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -613,9 +635,11 @@ class HashOuterJoinWorkOrder : public WorkOrder {
                          const std::vector<std::unique_ptr<const Scalar>> &selection,
                          const std::vector<bool> &is_selection_on_build,
                          const JoinHashTable &hash_table,
+                         const std::size_t query_id,
                          InsertDestination *output_destination,
                          StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -636,14 +660,15 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param hash_table The JoinHashTable to use.
+   * @param lookup_block_id The block id of the probe_relation.
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param is_selection_on_build Whether each Scalar in the \p selection vector
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
-   * @param lookup_block_id The block id of the probe_relation.
+   * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -655,9 +680,11 @@ class HashOuterJoinWorkOrder : public WorkOrder {
                          const std::vector<std::unique_ptr<const Scalar>> &selection,
                          std::vector<bool> &&is_selection_on_build,
                          const JoinHashTable &hash_table,
+                         const std::size_t query_id,
                          InsertDestination *output_destination,
                          StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/InsertOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.cpp b/relational_operators/InsertOperator.cpp
index 8d083e5..3ec9933 100644
--- a/relational_operators/InsertOperator.cpp
+++ b/relational_operators/InsertOperator.cpp
@@ -40,8 +40,10 @@ bool InsertOperator::getAllWorkOrders(
 
     work_generated_ = true;
     container->addNormalWorkOrder(
-        new InsertWorkOrder(query_context->getInsertDestination(output_destination_index_),
-                            query_context->releaseTuple(tuple_index_)),
+        new InsertWorkOrder(
+            query_id_,
+            query_context->getInsertDestination(output_destination_index_),
+            query_context->releaseTuple(tuple_index_)),
         op_index_);
   }
   return work_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index 8a06c94..ccef444 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_INSERT_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_INSERT_OPERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 
 #include "catalog/CatalogRelation.hpp"
@@ -56,11 +57,14 @@ class InsertOperator : public RelationalOperator {
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert the tuple.
    * @param tuple_index The index of the tuple to insert in the QueryContext.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   InsertOperator(const CatalogRelation &output_relation,
                  const QueryContext::insert_destination_id output_destination_index,
-                 const QueryContext::tuple_id tuple_index)
-      : output_relation_(output_relation),
+                 const QueryContext::tuple_id tuple_index,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         tuple_index_(tuple_index),
         work_generated_(false) {}
@@ -100,12 +104,15 @@ class InsertWorkOrder : public WorkOrder {
    *
    * @note InsertWorkOrder takes ownership of \c tuple.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the tuple.
    * @param tuple The tuple to insert.
    **/
-  InsertWorkOrder(InsertDestination *output_destination,
+  InsertWorkOrder(const std::size_t query_id,
+                  InsertDestination *output_destination,
                   Tuple *tuple)
-      : output_destination_(DCHECK_NOTNULL(output_destination)),
+      : WorkOrder(query_id),
+        output_destination_(DCHECK_NOTNULL(output_destination)),
         tuple_(DCHECK_NOTNULL(tuple)) {}
 
   ~InsertWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/NestedLoopsJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.cpp b/relational_operators/NestedLoopsJoinOperator.cpp
index 5cc498b..317cc5d 100644
--- a/relational_operators/NestedLoopsJoinOperator.cpp
+++ b/relational_operators/NestedLoopsJoinOperator.cpp
@@ -82,6 +82,7 @@ bool NestedLoopsJoinOperator::getAllWorkOrders(
                                            right_block_id,
                                            query_context->getPredicate(join_predicate_index_),
                                            query_context->getScalarGroup(selection_index_),
+                                           query_id_,
                                            query_context->getInsertDestination(output_destination_index_),
                                            storage_manager),
               op_index_);
@@ -171,6 +172,7 @@ std::size_t NestedLoopsJoinOperator::getAllWorkOrdersHelperBothNotStored(WorkOrd
                                        right_relation_block_ids_[right_index],
                                        query_context->getPredicate(join_predicate_index_),
                                        query_context->getScalarGroup(selection_index_),
+                                       query_id_,
                                        query_context->getInsertDestination(output_destination_index_),
                                        storage_manager),
           op_index_);
@@ -205,6 +207,7 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
                 right_relation_block_ids_[right_index],
                 join_predicate,
                 selection,
+                query_id_,
                 output_destination,
                 storage_manager),
             op_index_);
@@ -224,6 +227,7 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
                                          right_block_id,
                                          join_predicate,
                                          selection,
+                                         query_id_,
                                          output_destination,
                                          storage_manager),
             op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index a52ca25..f165442 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -76,6 +76,7 @@ class NestedLoopsJoinOperator : public RelationalOperator {
    * @param left_relation_is_stored If left_input_relation is a stored relation.
    * @param right_relation_is_stored If right_input_relation is a stored
    *                                 relation.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   NestedLoopsJoinOperator(const CatalogRelation &left_input_relation,
                           const CatalogRelation &right_input_relation,
@@ -84,8 +85,10 @@ class NestedLoopsJoinOperator : public RelationalOperator {
                           const QueryContext::predicate_id join_predicate_index,
                           const QueryContext::scalar_group_id selection_index,
                           bool left_relation_is_stored,
-                          bool right_relation_is_stored)
-      : left_input_relation_(left_input_relation),
+                          bool right_relation_is_stored,
+                          const std::size_t query_id)
+      : RelationalOperator(query_id),
+        left_input_relation_(left_input_relation),
         right_input_relation_(right_input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
@@ -230,6 +233,7 @@ class NestedLoopsJoinWorkOrder : public WorkOrder {
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -239,9 +243,11 @@ class NestedLoopsJoinWorkOrder : public WorkOrder {
                            const block_id right_block_id,
                            const Predicate *join_predicate,
                            const std::vector<std::unique_ptr<const Scalar>> &selection,
+                           const std::size_t query_id,
                            InsertDestination *output_destination,
                            StorageManager *storage_manager)
-      : left_input_relation_(left_input_relation),
+      : WorkOrder(query_id),
+        left_input_relation_(left_input_relation),
         right_input_relation_(right_input_relation),
         left_block_id_(left_block_id),
         right_block_id_(right_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index 5443d48..ae876ba 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -55,14 +55,17 @@ class RebuildWorkOrder : public WorkOrder {
    * @param input_relation_id The ID of the CatalogRelation to which the given
    *        storage block belongs to.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
+   * @param query_id The ID of the query to which this RebuildWorkOrder belongs.
    * @param bus A pointer to the TMB.
    **/
   RebuildWorkOrder(MutableBlockReference &&block_ref,
                    const std::size_t input_operator_index,
                    const relation_id input_relation_id,
                    const client_id scheduler_client_id,
+                   const std::size_t query_id,
                    MessageBus *bus)
-      : block_ref_(std::move(block_ref)),
+      : WorkOrder(query_id),
+        block_ref_(std::move(block_ref)),
         input_operator_index_(input_operator_index),
         input_relation_id_(input_relation_id),
         scheduler_client_id_(scheduler_client_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index 75fde17..aa93018 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -208,16 +208,20 @@ class RelationalOperator {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param blocking_dependencies_met If those dependencies which break the
    *        pipeline have been met.
    **/
-  explicit RelationalOperator(bool blocking_dependencies_met = false)
+  explicit RelationalOperator(const std::size_t query_id = 0,
+                              const bool blocking_dependencies_met = false)
       : blocking_dependencies_met_(blocking_dependencies_met),
-        done_feeding_input_relation_(false) {}
+        done_feeding_input_relation_(false),
+        query_id_(query_id) {}
 
   bool blocking_dependencies_met_;
   bool done_feeding_input_relation_;
   std::size_t op_index_;
+  const std::size_t query_id_;
 
  private:
   DISALLOW_COPY_AND_ASSIGN(RelationalOperator);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SampleOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.cpp b/relational_operators/SampleOperator.cpp
index 6842b28..b318ce4 100644
--- a/relational_operators/SampleOperator.cpp
+++ b/relational_operators/SampleOperator.cpp
@@ -58,6 +58,7 @@ bool SampleOperator::getAllWorkOrders(
                                     input_block_id,
                                     is_block_sample_,
                                     percentage_,
+                                    query_id_,
                                     output_destination,
                                     storage_manager),
                 op_index_);
@@ -72,6 +73,7 @@ bool SampleOperator::getAllWorkOrders(
                                   input_block_id,
                                   is_block_sample_,
                                   percentage_,
+                                  query_id_,
                                   output_destination,
                                   storage_manager),
               op_index_);
@@ -89,6 +91,7 @@ bool SampleOperator::getAllWorkOrders(
                                       input_relation_block_ids_[num_workorders_generated_],
                                       is_block_sample_,
                                       percentage_,
+                                      query_id_,
                                       output_destination,
                                       storage_manager),
                   op_index_);
@@ -101,7 +104,9 @@ bool SampleOperator::getAllWorkOrders(
               new SampleWorkOrder(input_relation_,
                                   input_relation_block_ids_[num_workorders_generated_],
                                   is_block_sample_,
-                                  percentage_, output_destination,
+                                  percentage_,
+                                  query_id_,
+                                  output_destination,
                                   storage_manager),
               op_index_);
           ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index 305de34..f65f28a 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_SAMPLE_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_SAMPLE_OPERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <vector>
 
@@ -64,15 +65,17 @@ class SampleOperator : public RelationalOperator {
    *        workorders.
    * @param is_block_sample Flag indicating whether the sample type is block or tuple.
    * @param percentage The percentage of data to be sampled.
-   *
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SampleOperator(const CatalogRelation &input_relation,
                  const CatalogRelationSchema &output_relation,
                  const QueryContext::insert_destination_id output_destination_index,
                  const bool input_relation_is_stored,
                  const bool is_block_sample,
-                 const int percentage)
-      : input_relation_(input_relation),
+                 const int percentage,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         input_relation_is_stored_(input_relation_is_stored),
@@ -134,9 +137,11 @@ class SampleWorkOrder : public WorkOrder {
                   const block_id input_block_id,
                   const bool is_block_sample,
                   const int percentage,
+                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         is_block_sample_(is_block_sample),
         percentage_(percentage),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index ac61407..3581090 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -38,6 +38,7 @@ bool SaveBlocksOperator::getAllWorkOrders(
         new SaveBlocksWorkOrder(
             destination_block_ids_[num_workorders_generated_],
             force_,
+            query_id_,
             storage_manager),
         op_index_);
     ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 49195ea..f4650bb 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_SAVE_BLOCKS_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_SAVE_BLOCKS_OPERATOR_HPP_
 
+#include <cstddef>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
@@ -50,11 +51,13 @@ class SaveBlocksOperator : public RelationalOperator {
   /**
    * @brief Constructor for saving only modified blocks in a relation.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
    **/
-  explicit SaveBlocksOperator(bool force = false)
-      : force_(force),
+  explicit SaveBlocksOperator(const std::size_t query_id, bool force = false)
+      : RelationalOperator(query_id),
+        force_(force),
         num_workorders_generated_(0) {}
 
   ~SaveBlocksOperator() override {}
@@ -96,12 +99,15 @@ class SaveBlocksWorkOrder : public WorkOrder {
    * @param save_block_id The id of the block to save.
    * @param force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param storage_manager The StorageManager to use.
    **/
   SaveBlocksWorkOrder(const block_id save_block_id,
                       const bool force,
+                      const std::size_t query_id,
                       StorageManager *storage_manager)
-      : save_block_id_(save_block_id),
+      : WorkOrder(query_id),
+        save_block_id_(save_block_id),
         force_(force),
         storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index 69bb434..350890d 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -49,6 +49,7 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
                               simple_projection_,
                               simple_selection_,
                               selection,
+                              query_id_,
                               output_destination,
                               storage_manager),
           op_index_);
@@ -63,6 +64,7 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
               simple_projection_,
               simple_selection_,
               selection,
+              query_id_,
               output_destination,
               storage_manager),
           op_index_);
@@ -91,6 +93,7 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
                 simple_projection_,
                 simple_selection_,
                 selection,
+                query_id_,
                 output_destination,
                 storage_manager,
                 placement_scheme_->getNUMANodeForBlock(input_block_id)),
@@ -111,6 +114,7 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
                 simple_projection_,
                 simple_selection_,
                 selection,
+                query_id_,
                 output_destination,
                 storage_manager,
                 placement_scheme_->getNUMANodeForBlock(block_in_partition)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 76f4cb6..4f5b8ca 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -76,14 +76,17 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SelectOperator(const CatalogRelation &input_relation,
                  const CatalogRelation &output_relation,
                  const QueryContext::insert_destination_id output_destination_index,
                  const QueryContext::predicate_id predicate_index,
                  const QueryContext::scalar_group_id selection_index,
-                 const bool input_relation_is_stored)
-      : input_relation_(input_relation),
+                 const bool input_relation_is_stored,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         predicate_index_(predicate_index),
@@ -133,14 +136,17 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SelectOperator(const CatalogRelation &input_relation,
                  const CatalogRelation &output_relation,
                  const QueryContext::insert_destination_id output_destination_index,
                  const QueryContext::predicate_id predicate_index,
                  std::vector<attribute_id> &&selection,
-                 const bool input_relation_is_stored)
-      : input_relation_(input_relation),
+                 const bool input_relation_is_stored,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         predicate_index_(predicate_index),
@@ -281,6 +287,7 @@ class SelectWorkOrder : public WorkOrder {
    *        simple_projection is true.
    * @param selection A list of Scalars which will be evaluated to project
    *        input tuples, used if \c simple_projection is false.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the selection
    *        results.
    * @param storage_manager The StorageManager to use.
@@ -291,10 +298,12 @@ class SelectWorkOrder : public WorkOrder {
                   const bool simple_projection,
                   const std::vector<attribute_id> &simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
+                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
                   const numa_node_id numa_node = 0)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
         simple_projection_(simple_projection),
@@ -320,6 +329,7 @@ class SelectWorkOrder : public WorkOrder {
    *        simple_projection is true.
    * @param selection A list of Scalars which will be evaluated to project
    *        input tuples, used if \c simple_projection is false.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the selection
    *        results.
    * @param storage_manager The StorageManager to use.
@@ -330,10 +340,12 @@ class SelectWorkOrder : public WorkOrder {
                   const bool simple_projection,
                   std::vector<attribute_id> &&simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
+                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
                   const numa_node_id numa_node = 0)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
         simple_projection_(simple_projection),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.cpp b/relational_operators/SortMergeRunOperator.cpp
index 7427d44..9db8de1 100644
--- a/relational_operators/SortMergeRunOperator.cpp
+++ b/relational_operators/SortMergeRunOperator.cpp
@@ -90,6 +90,7 @@ WorkOrder *SortMergeRunOperator::createWorkOrder(
       std::move(job->runs),
       top_k_,
       job->level,
+      query_id_,
       output_destination,
       storage_manager,
       op_index_,



[41/50] [abbrv] incubator-quickstep git commit: Go back to using LLVM Apt repositories

Posted by zu...@apache.org.
Go back to using LLVM Apt repositories


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

Branch: refs/heads/tmp-relation-col-store
Commit: c0c13f67e330ccc4b233d9fd9033453211d9c584
Parents: 714874c
Author: Navneet Potti <na...@gmail.com>
Authored: Mon Jun 27 11:06:58 2016 -0500
Committer: Navneet Potti <na...@gmail.com>
Committed: Mon Jun 27 11:06:58 2016 -0500

----------------------------------------------------------------------
 .travis.yml | 15 ++-------------
 1 file changed, 2 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0c13f67/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index fce3ba0..54a0c8a 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -20,19 +20,6 @@ env:
   - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=selection
   - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=selection
 
-before_install:
-  - LLVM_VERSION=3.7.1
-  - LLVM_ARCHIVE_PATH=$HOME/clang+llvm.tar.xz
-  - if [[ $CC = "clang" ]]; then
-      wget http://llvm.org/releases/$LLVM_VERSION/clang+llvm-$LLVM_VERSION-x86_64-linux-gnu-ubuntu-14.04.tar.xz -O $LLVM_ARCHIVE_PATH;
-      mkdir -p $HOME/clang-$LLVM_VERSION;
-      tar xf $LLVM_ARCHIVE_PATH -C $HOME/clang-$LLVM_VERSION --strip-components 1;
-      ln -sf $HOME/clang-$LLVM_VERSION/bin/clang++ $HOME/clang-$LLVM_VERSION/bin/clang++-3.7;
-      export PATH=$HOME/clang-$LLVM_VERSION/bin:$PATH;
-      export CPPFLAGS="-I $HOME/clang-$LLVM_VERSION/include/c++/v1";
-      echo "Using clang at " `which $CC-3.7` " and $CXX at " `which $CXX-3.7`;
-    fi
-
 install:
   - if [ "$CC" = "gcc" ]; then
       export MAKE_JOBS=1;
@@ -91,9 +78,11 @@ addons:
   apt:
     sources:
       - ubuntu-toolchain-r-test
+      - llvm-toolchain-precise-3.7
     packages:
       - gcc-5
       - g++-5
+      - clang-3.7
       - binutils-gold
       - libprotobuf-dev
       - protobuf-compiler


[50/50] [abbrv] incubator-quickstep git commit: Used the basic column store for tmp relations w/o variable length attributes.

Posted by zu...@apache.org.
Used the basic column store for tmp relations w/o variable length attributes.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 82e6f0900de4b8d3d19e8e083b73c3ad0ab6abad
Parents: 04c8224
Author: Zuyu Zhang <zz...@pivotal.io>
Authored: Thu May 19 12:48:48 2016 -0700
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Jul 7 20:25:24 2016 -0500

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt         |  1 +
 query_optimizer/ExecutionGenerator.cpp | 23 +++++++++++++++++++++++
 storage/StorageBlockLayout.proto       |  2 +-
 3 files changed, 25 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/82e6f090/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 8912414..a91ab3f 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -131,6 +131,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_relationaloperators_TextScanOperator
                       quickstep_relationaloperators_UpdateOperator
                       quickstep_storage_AggregationOperationState_proto
+                      quickstep_storage_BasicColumnStoreTupleStorageSubBlock
                       quickstep_storage_HashTableFactory
                       quickstep_storage_HashTable_proto
                       quickstep_storage_InsertDestination_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/82e6f090/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 45f5f78..d6db2a4 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -105,6 +105,7 @@
 #include "relational_operators/TextScanOperator.hpp"
 #include "relational_operators/UpdateOperator.hpp"
 #include "storage/AggregationOperationState.pb.h"
+#include "storage/BasicColumnStoreTupleStorageSubBlock.hpp"
 #include "storage/HashTable.pb.h"
 #include "storage/HashTableFactory.hpp"
 #include "storage/InsertDestination.pb.h"
@@ -149,6 +150,8 @@ DEFINE_bool(parallelize_load, true, "Parallelize loading data files.");
 DEFINE_bool(optimize_joins, false,
             "Enable post execution plan generation optimizations for joins.");
 
+DEFINE_bool(use_column_store, true, "Use the column store for blocks in temporary relations.");
+
 namespace E = ::quickstep::optimizer::expressions;
 namespace P = ::quickstep::optimizer::physical;
 namespace S = ::quickstep::serialization;
@@ -325,6 +328,23 @@ void ExecutionGenerator::createTemporaryCatalogRelation(
     ++aid;
   }
 
+  if (FLAGS_use_column_store && !catalog_relation->isVariableLength()) {
+    StorageBlockLayoutDescription layout_description;
+    layout_description.set_num_slots(1);
+    layout_description.mutable_tuple_store_description()
+        ->set_sub_block_type(TupleStorageSubBlockDescription::BASIC_COLUMN_STORE);
+
+    DCHECK(BasicColumnStoreTupleStorageSubBlock::DescriptionIsValid(*catalog_relation,
+                                                                    layout_description.tuple_store_description()));
+
+    unique_ptr<StorageBlockLayout> layout(
+        new StorageBlockLayout(*catalog_relation, layout_description));
+    layout->finalize();
+    catalog_relation->setDefaultStorageBlockLayout(layout.release());
+  } else {
+    // Use the default row store for variable length attributes.
+  }
+
   *catalog_relation_output = catalog_relation.get();
   const relation_id output_rel_id = optimizer_context_->catalog_database()->addRelation(
       catalog_relation.release());
@@ -335,6 +355,9 @@ void ExecutionGenerator::createTemporaryCatalogRelation(
 
   insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
   insert_destination_proto->set_relation_id(output_rel_id);
+
+  insert_destination_proto->mutable_layout()->MergeFrom(
+      (*catalog_relation_output)->getDefaultStorageBlockLayout().getDescription());
 }
 
 void ExecutionGenerator::dropAllTemporaryRelations() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/82e6f090/storage/StorageBlockLayout.proto
----------------------------------------------------------------------
diff --git a/storage/StorageBlockLayout.proto b/storage/StorageBlockLayout.proto
index cafc0e9..4db6f06 100644
--- a/storage/StorageBlockLayout.proto
+++ b/storage/StorageBlockLayout.proto
@@ -66,7 +66,7 @@ message IndexSubBlockDescription {
     SMA = 1;
     BLOOM_FILTER = 2;
     BITWEAVING_H = 3;
-    BITWEAVING_V = 4;    
+    BITWEAVING_V = 4;
   }
 
   required IndexSubBlockType sub_block_type = 1;


[07/50] [abbrv] incubator-quickstep git commit: Fix typos in CMakeLists.txt.

Posted by zu...@apache.org.
Fix typos in CMakeLists.txt.


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

Branch: refs/heads/tmp-relation-col-store
Commit: fd75e17576118ef539d696e3663691ce12518a3b
Parents: 1be47dc
Author: Navneet Potti <na...@gmail.com>
Authored: Wed Jun 8 16:57:18 2016 -0500
Committer: Navneet Potti <na...@gmail.com>
Committed: Wed Jun 8 17:21:33 2016 -0500

----------------------------------------------------------------------
 expressions/CMakeLists.txt             | 2 +-
 expressions/aggregation/CMakeLists.txt | 1 -
 2 files changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/fd75e175/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/CMakeLists.txt b/expressions/CMakeLists.txt
index d8452b2..53ad5d4 100644
--- a/expressions/CMakeLists.txt
+++ b/expressions/CMakeLists.txt
@@ -63,6 +63,6 @@ add_library(quickstep_expressions ../empty_src.cpp ExpressionsModule.hpp)
 target_link_libraries(quickstep_expressions
                       quickstep_expressions_ExpressionFactories
                       quickstep_expressions_Expressions_proto
-                      quickstep_expressions_aggregate
+                      quickstep_expressions_aggregation
                       quickstep_expressions_predicate
                       quickstep_expressions_scalar)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/fd75e175/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index 416c4c6..5744c52 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -266,7 +266,6 @@ target_link_libraries(quickstep_expressions_aggregation
                       quickstep_expressions_aggregation_AggregationHandleMax
                       quickstep_expressions_aggregation_AggregationHandleMin
                       quickstep_expressions_aggregation_AggregationHandleSum
-                      quickstep_expressions_aggregation_AggregationHandleUtil
                       quickstep_expressions_aggregation_AggregationID)
 
 # Tests:


[02/50] [abbrv] incubator-quickstep git commit: Added Query ID to Relational operators and WorkOrders.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index f92affe..f54e925 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -88,6 +88,7 @@ class SortMergeRunOperator : public RelationalOperator {
    *              \c top_k is 0.
    * @param input_relation_is_stored Boolean to indicate is input relation is
    *                                 stored or streamed.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SortMergeRunOperator(const CatalogRelation &input_relation,
                        const CatalogRelation &output_relation,
@@ -97,8 +98,10 @@ class SortMergeRunOperator : public RelationalOperator {
                        const QueryContext::sort_config_id sort_config_index,
                        const std::size_t merge_factor,
                        const std::size_t top_k,
-                       const bool input_relation_is_stored)
-      : input_relation_(input_relation),
+                       const bool input_relation_is_stored,
+                       const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         sort_config_index_(sort_config_index),
@@ -216,6 +219,7 @@ class SortMergeRunWorkOrder : public WorkOrder {
    * @param input_runs Input runs to merge.
    * @param top_k If non-zero will merge only \c top_k tuples.
    * @param merge_level Merge level in the merge tree.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to create new blocks.
    * @param storage_manager The StorageManager to use.
    * @param operator_index Merge-run operator index to send feedback messages
@@ -229,12 +233,14 @@ class SortMergeRunWorkOrder : public WorkOrder {
       std::vector<merge_run_operator::Run> &&input_runs,
       const std::size_t top_k,
       const std::size_t merge_level,
+      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
       const std::size_t operator_index,
       const tmb::client_id scheduler_client_id,
       MessageBus *bus)
-      : sort_config_(sort_config),
+      : WorkOrder(query_id),
+        sort_config_(sort_config),
         run_relation_(run_relation),
         input_runs_(std::move(input_runs)),
         top_k_(top_k),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SortRunGenerationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.cpp b/relational_operators/SortRunGenerationOperator.cpp
index 9bb3f51..e352f9e 100644
--- a/relational_operators/SortRunGenerationOperator.cpp
+++ b/relational_operators/SortRunGenerationOperator.cpp
@@ -54,6 +54,7 @@ bool SortRunGenerationOperator::getAllWorkOrders(
             new SortRunGenerationWorkOrder(input_relation_,
                                            input_block_id,
                                            sort_config,
+                                           query_id_,
                                            output_destination,
                                            storage_manager),
             op_index_);
@@ -69,6 +70,7 @@ bool SortRunGenerationOperator::getAllWorkOrders(
               input_relation_,
               input_relation_block_ids_[num_workorders_generated_],
               sort_config,
+              query_id_,
               output_destination,
               storage_manager),
           op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index 04290a9..3da9813 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -83,13 +83,16 @@ class SortRunGenerationOperator : public RelationalOperator {
    * @param input_relation_is_stored Does the input relation contain the blocks
    *                                 to sort. If \c false, the blocks are
    *                                 streamed.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SortRunGenerationOperator(const CatalogRelation &input_relation,
                             const CatalogRelation &output_relation,
                             const QueryContext::insert_destination_id output_destination_index,
                             const QueryContext::sort_config_id sort_config_index,
-                            bool input_relation_is_stored)
-      : input_relation_(input_relation),
+                            bool input_relation_is_stored,
+                            const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         sort_config_index_(sort_config_index),
@@ -152,6 +155,7 @@ class SortRunGenerationWorkOrder : public WorkOrder {
    * @param input_block_id The block id.
    * @param sort_config The Sort configuration specifying ORDER BY, ordering,
    *        and null ordering.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to store the sorted blocks
    *        of runs.
    * @param storage_manager The StorageManager to use.
@@ -159,9 +163,11 @@ class SortRunGenerationWorkOrder : public WorkOrder {
   SortRunGenerationWorkOrder(const CatalogRelationSchema &input_relation,
                              const block_id input_block_id,
                              const SortConfiguration &sort_config,
+                             const std::size_t query_id,
                              InsertDestination *output_destination,
                              StorageManager *storage_manager)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         sort_config_(sort_config),
         output_destination_(DCHECK_NOTNULL(output_destination)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/TableGeneratorOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.cpp b/relational_operators/TableGeneratorOperator.cpp
index 886d05f..fb1f743 100644
--- a/relational_operators/TableGeneratorOperator.cpp
+++ b/relational_operators/TableGeneratorOperator.cpp
@@ -42,8 +42,11 @@ bool TableGeneratorOperator::getAllWorkOrders(
     // Currently the generator function is not abstracted to be parallelizable,
     // so just produce one work order.
     container->addNormalWorkOrder(
-        new TableGeneratorWorkOrder(query_context->getGeneratorFunctionHandle(generator_function_index_),
-                                    query_context->getInsertDestination(output_destination_index_)),
+        new TableGeneratorWorkOrder(
+            query_context->getGeneratorFunctionHandle(
+                generator_function_index_),
+            query_id_,
+            query_context->getInsertDestination(output_destination_index_)),
         op_index_);
     started_ = true;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index a26b227..bfc70c5 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -60,12 +60,14 @@ class TableGeneratorOperator : public RelationalOperator {
    *        QueryContext to insert the generated output.
    * @param generator_function_index The index of the GeneratorFunctionHandle in
    *        the QueryContext.
-   *
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   TableGeneratorOperator(const CatalogRelation &output_relation,
                          const QueryContext::insert_destination_id output_destination_index,
-                         const QueryContext::generator_function_id generator_function_index)
-      : output_relation_(output_relation),
+                         const QueryContext::generator_function_id generator_function_index,
+                         const std::size_t query_id)
+      : RelationalOperator(query_id),
+        output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         generator_function_index_(generator_function_index),
         started_(false) {
@@ -112,12 +114,15 @@ class TableGeneratorWorkOrder : public WorkOrder {
    * @brief Constructor.
    *
    * @param generator_function The GeneratorFunctionHandle to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the generated
    *        output.
    **/
   TableGeneratorWorkOrder(const GeneratorFunctionHandle &function_handle,
+                          const std::size_t query_id,
                           InsertDestination *output_destination)
-      : function_handle_(function_handle),
+      : WorkOrder(query_id),
+        function_handle_(function_handle),
         output_destination_(DCHECK_NOTNULL(output_destination)) {}
 
   ~TableGeneratorWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index 5ede6f7..8db5ef1 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -167,6 +167,7 @@ bool TextScanOperator::getAllWorkOrders(
           container->addNormalWorkOrder(
               new TextSplitWorkOrder(file,
                                      process_escape_sequences_,
+                                     query_id_,
                                      storage_manager,
                                      op_index_,
                                      scheduler_client_id,
@@ -185,6 +186,7 @@ bool TextScanOperator::getAllWorkOrders(
                                     blob_work.size,
                                     field_terminator_,
                                     process_escape_sequences_,
+                                    query_id_,
                                     output_destination,
                                     storage_manager),
               op_index_);
@@ -204,6 +206,7 @@ bool TextScanOperator::getAllWorkOrders(
             new TextScanWorkOrder(file,
                                   field_terminator_,
                                   process_escape_sequences_,
+                                  query_id_,
                                   output_destination,
                                   storage_manager),
             op_index_);
@@ -235,9 +238,11 @@ void TextScanOperator::receiveFeedbackMessage(const WorkOrder::FeedbackMessage &
 TextScanWorkOrder::TextScanWorkOrder(const std::string &filename,
                                      const char field_terminator,
                                      const bool process_escape_sequences,
+                                     const std::size_t query_id,
                                      InsertDestination *output_destination,
                                      StorageManager *storage_manager)
-    : is_file_(true),
+    : WorkOrder(query_id),
+      is_file_(true),
       filename_(filename),
       field_terminator_(field_terminator),
       text_blob_(0),
@@ -253,9 +258,11 @@ TextScanWorkOrder::TextScanWorkOrder(const block_id text_blob,
                                      const std::size_t text_size,
                                      const char field_terminator,
                                      const bool process_escape_sequences,
+                                     const std::size_t query_id,
                                      InsertDestination *output_destination,
                                      StorageManager *storage_manager)
-    : is_file_(false),
+    : WorkOrder(query_id),
+      is_file_(false),
       field_terminator_(field_terminator),
       text_blob_(text_blob),
       text_size_(text_size),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index a2d4ced..1d0c04f 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -134,14 +134,17 @@ class TextScanOperator : public RelationalOperator {
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert tuples.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   TextScanOperator(const std::string &file_pattern,
                    const char field_terminator,
                    const bool process_escape_sequences,
                    const bool parallelize_load,
                    const CatalogRelation &output_relation,
-                   const QueryContext::insert_destination_id output_destination_index)
-      : file_pattern_(file_pattern),
+                   const QueryContext::insert_destination_id output_destination_index,
+                   const std::size_t query_id)
+      : RelationalOperator(query_id),
+        file_pattern_(file_pattern),
         field_terminator_(field_terminator),
         process_escape_sequences_(process_escape_sequences),
         parallelize_load_(parallelize_load),
@@ -202,6 +205,7 @@ class TextScanWorkOrder : public WorkOrder {
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert tuples.
    * @param storage_manager The StorageManager to use.
    **/
@@ -209,6 +213,7 @@ class TextScanWorkOrder : public WorkOrder {
       const std::string &filename,
       const char field_terminator,
       const bool process_escape_sequences,
+      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager);
 
@@ -221,6 +226,7 @@ class TextScanWorkOrder : public WorkOrder {
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to write the read tuples.
    * @param storage_manager The StorageManager to use.
    */
@@ -229,6 +235,7 @@ class TextScanWorkOrder : public WorkOrder {
       const std::size_t text_size,
       const char field_terminator,
       const bool process_escape_sequences,
+      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager);
 
@@ -318,6 +325,7 @@ class TextSplitWorkOrder : public WorkOrder {
    * @param filename File to split into row-aligned blobs.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param storage_manager The StorageManager to use.
    * @param operator_index Operator index of the current operator. This is used
    *                       to send new-work available message to Foreman.
@@ -326,11 +334,13 @@ class TextSplitWorkOrder : public WorkOrder {
    */
   TextSplitWorkOrder(const std::string &filename,
                      const bool process_escape_sequences,
+                     const std::size_t query_id,
                      StorageManager *storage_manager,
                      const std::size_t operator_index,
                      const tmb::client_id scheduler_client_id,
                      MessageBus *bus)
-      : filename_(filename),
+      : WorkOrder(query_id),
+        filename_(filename),
         process_escape_sequences_(process_escape_sequences),
         storage_manager_(DCHECK_NOTNULL(storage_manager)),
         operator_index_(operator_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index 7585db1..b331a9c 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -57,6 +57,7 @@ bool UpdateOperator::getAllWorkOrders(
                               input_block_id,
                               query_context->getPredicate(predicate_index_),
                               query_context->getUpdateGroup(update_group_index_),
+                              query_id_,
                               query_context->getInsertDestination(relocation_destination_index_),
                               storage_manager,
                               op_index_,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index 78f8fe0..ba2d6cf 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -72,6 +72,7 @@ class UpdateOperator : public RelationalOperator {
    * @param update_group_index The index of a update group (the map of
    *        attribute_ids to Scalars) which should be evaluated to get the new
    *        value for the corresponding attribute.
+   * @param query_id The ID of the query to which this operator belongs.
    *
    * @warning The constructed InsertDestination should belong to relation, but
    *          must NOT contain any pre-existing blocks.
@@ -79,8 +80,10 @@ class UpdateOperator : public RelationalOperator {
   UpdateOperator(const CatalogRelation &relation,
                  const QueryContext::insert_destination_id relocation_destination_index,
                  const QueryContext::predicate_id predicate_index,
-                 const QueryContext::update_group_id update_group_index)
-      : relation_(relation),
+                 const QueryContext::update_group_id update_group_index,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        relation_(relation),
         relocation_destination_index_(relocation_destination_index),
         predicate_index_(predicate_index),
         update_group_index_(update_group_index),
@@ -130,6 +133,7 @@ class UpdateWorkOrder : public WorkOrder {
    * @param assignments The assignments (the map of attribute_ids to Scalars)
    *        which should be evaluated to get the new value for the corresponding
    *        attribute.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_block_id The block id.
    * @param relocation_destination The InsertDestination to relocate tuples
    *        which can not be updated in-place.
@@ -143,12 +147,14 @@ class UpdateWorkOrder : public WorkOrder {
                   const block_id input_block_id,
                   const Predicate *predicate,
                   const std::unordered_map<attribute_id, std::unique_ptr<const Scalar>> &assignments,
+                  const std::size_t query_id,
                   InsertDestination *relocation_destination,
                   StorageManager *storage_manager,
                   const std::size_t update_operator_index,
                   const tmb::client_id scheduler_client_id,
                   MessageBus *bus)
-      : relation_(relation),
+      : WorkOrder(query_id),
+        relation_(relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
         assignments_(assignments),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index 42cec2a..059865d 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -286,8 +286,15 @@ class WorkOrder {
   }
 
  protected:
-  WorkOrder() {}
+  /**
+   * @brief Constructor.
+   *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
+   **/
+  explicit WorkOrder(const std::size_t query_id)
+      : query_id_(query_id) {}
 
+  const std::size_t query_id_;
   // A vector of preferred NUMA node IDs where this workorder should be executed.
   // These node IDs typically indicate the NUMA node IDs of the input(s) of the
   // workorder. Derived classes should ensure that there are no duplicate entries

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 5d0619a..fd731f7 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -45,6 +45,7 @@ enum WorkOrderType {
 
 message WorkOrder {
   required WorkOrderType work_order_type = 1;
+  required uint64 query_id = 2;
 
   // The convention for extension numbering is that extensions for a particular
   // WorkOrderID should begin from (operator_type + 1) * 16.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 4157d0f..fdd694f 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -76,6 +76,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       LOG(INFO) << "Creating AggregationWorkOrder";
       return new AggregationWorkOrder(
           proto.GetExtension(serialization::AggregationWorkOrder::block_id),
+          proto.query_id(),
           query_context->getAggregationState(
               proto.GetExtension(serialization::AggregationWorkOrder::aggr_state_index)));
     }
@@ -93,6 +94,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           move(join_key_attributes),
           proto.GetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable),
           proto.GetExtension(serialization::BuildHashWorkOrder::block_id),
+          proto.query_id(),
           query_context->getJoinHashTable(
               proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index)),
           storage_manager);
@@ -108,12 +110,15 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           storage_manager,
           proto.GetExtension(serialization::DeleteWorkOrder::operator_index),
           shiftboss_client_id,
+          proto.query_id(),
           bus);
     }
     case serialization::DESTROY_HASH: {
       LOG(INFO) << "Creating DestroyHashWorkOrder";
       return new DestroyHashWorkOrder(
-          proto.GetExtension(serialization::DestroyHashWorkOrder::join_hash_table_index),
+          proto.GetExtension(
+              serialization::DestroyHashWorkOrder::join_hash_table_index),
+          proto.query_id(),
           query_context);
     }
     case serialization::DROP_TABLE: {
@@ -125,6 +130,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       }
 
       return new DropTableWorkOrder(
+          proto.query_id(),
           move(blocks),
           storage_manager,
           proto.HasExtension(serialization::DropTableWorkOrder::relation_id)
@@ -135,10 +141,12 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::FINALIZE_AGGREGATION: {
       LOG(INFO) << "Creating FinalizeAggregationWorkOrder";
       return new FinalizeAggregationWorkOrder(
-          query_context->releaseAggregationState(
-              proto.GetExtension(serialization::FinalizeAggregationWorkOrder::aggr_state_index)),
+          proto.query_id(),
+          query_context->releaseAggregationState(proto.GetExtension(
+              serialization::FinalizeAggregationWorkOrder::aggr_state_index)),
           query_context->getInsertDestination(
-              proto.GetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index)));
+              proto.GetExtension(serialization::FinalizeAggregationWorkOrder::
+                                     insert_destination_index)));
     }
     case serialization::HASH_JOIN: {
       const auto hash_join_work_order_type =
@@ -193,6 +201,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
+              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -207,6 +216,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
+              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -229,6 +239,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               selection,
               move(is_selection_on_build),
               hash_table,
+              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -243,6 +254,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
+              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -253,6 +265,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::INSERT: {
       LOG(INFO) << "Creating InsertWorkOrder";
       return new InsertWorkOrder(
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::InsertWorkOrder::insert_destination_index)),
           query_context->releaseTuple(
@@ -271,6 +284,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::join_predicate_index)),
           query_context->getScalarGroup(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::selection_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::insert_destination_index)),
           storage_manager);
@@ -283,6 +297,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           proto.GetExtension(serialization::SampleWorkOrder::block_id),
           proto.GetExtension(serialization::SampleWorkOrder::is_block_sample),
           proto.GetExtension(serialization::SampleWorkOrder::percentage),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SampleWorkOrder::insert_destination_index)),
           storage_manager);
@@ -292,6 +307,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       return new SaveBlocksWorkOrder(
           proto.GetExtension(serialization::SaveBlocksWorkOrder::block_id),
           proto.GetExtension(serialization::SaveBlocksWorkOrder::force),
+          proto.query_id(),
           storage_manager);
     }
     case serialization::SELECT: {
@@ -315,6 +331,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           simple_projection ? nullptr
                             : &query_context->getScalarGroup(
                                   proto.GetExtension(serialization::SelectWorkOrder::selection_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SelectWorkOrder::insert_destination_index)),
           storage_manager);
@@ -340,6 +357,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           move(runs),
           proto.GetExtension(serialization::SortMergeRunWorkOrder::top_k),
           proto.GetExtension(serialization::SortMergeRunWorkOrder::merge_level),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SortMergeRunWorkOrder::insert_destination_index)),
           storage_manager,
@@ -355,6 +373,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           proto.GetExtension(serialization::SortRunGenerationWorkOrder::block_id),
           query_context->getSortConfig(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::sort_config_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::insert_destination_index)),
           storage_manager);
@@ -364,6 +383,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       return new TableGeneratorWorkOrder(
           query_context->getGeneratorFunctionHandle(
               proto.GetExtension(serialization::TableGeneratorWorkOrder::generator_function_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::TableGeneratorWorkOrder::insert_destination_index)));
     }
@@ -374,6 +394,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
             proto.GetExtension(serialization::TextScanWorkOrder::filename),
             proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
             proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
+            proto.query_id(),
             query_context->getInsertDestination(
                 proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
             storage_manager);
@@ -386,6 +407,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
             text_blob_proto.size(),
             proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
             proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
+            proto.query_id(),
             query_context->getInsertDestination(
                 proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
             storage_manager);
@@ -396,6 +418,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       return new TextSplitWorkOrder(
           proto.GetExtension(serialization::TextSplitWorkOrder::filename),
           proto.GetExtension(serialization::TextSplitWorkOrder::process_escape_sequences),
+          proto.query_id(),
           storage_manager,
           proto.GetExtension(serialization::TextSplitWorkOrder::operator_index),
           shiftboss_client_id,
@@ -411,6 +434,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::UpdateWorkOrder::predicate_index)),
           query_context->getUpdateGroup(
               proto.GetExtension(serialization::UpdateWorkOrder::update_group_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::UpdateWorkOrder::insert_destination_index)),
           storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index f2207c2..ace7951 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -270,7 +270,7 @@ class AggregationOperatorTest : public ::testing::Test {
     aggr_state_proto->set_estimated_num_entries(estimated_entries);
 
     // Create Operators.
-    op_.reset(new AggregationOperator(*table_, true, aggr_state_index));
+    op_.reset(new AggregationOperator(*table_, true, aggr_state_index, 0));
 
     // Setup the InsertDestination proto in the query context proto.
     const QueryContext::insert_destination_id insert_destination_index =
@@ -281,8 +281,10 @@ class AggregationOperatorTest : public ::testing::Test {
     insert_destination_proto->set_relation_id(result_table_->getID());
     insert_destination_proto->set_relational_op_index(kOpIndex);
 
-    finalize_op_.reset(
-        new FinalizeAggregationOperator(aggr_state_index, *result_table_, insert_destination_index));
+    finalize_op_.reset(new FinalizeAggregationOperator(aggr_state_index,
+                                                       *result_table_,
+                                                       insert_destination_index,
+                                                       0 /* dummy query ID */));
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,
@@ -352,7 +354,7 @@ class AggregationOperatorTest : public ::testing::Test {
         serialization::HashTableImplType::LINEAR_OPEN_ADDRESSING);
 
     // Create Operators.
-    op_.reset(new AggregationOperator(*table_, true, aggr_state_index));
+    op_.reset(new AggregationOperator(*table_, true, aggr_state_index, 0));
 
     // Setup the InsertDestination proto in the query context proto.
     const QueryContext::insert_destination_id insert_destination_index =
@@ -363,8 +365,10 @@ class AggregationOperatorTest : public ::testing::Test {
     insert_destination_proto->set_relation_id(result_table_->getID());
     insert_destination_proto->set_relational_op_index(kOpIndex);
 
-    finalize_op_.reset(
-        new FinalizeAggregationOperator(aggr_state_index, *result_table_, insert_destination_index));
+    finalize_op_.reset(new FinalizeAggregationOperator(aggr_state_index,
+                                                       *result_table_,
+                                                       insert_destination_index,
+                                                       0 /* dummy query ID */));
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 333c3f0..4ef5a5c 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -336,7 +336,8 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_long.getID()),
                             dim_col_long.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID
 
   // Create the prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -368,7 +369,8 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -421,7 +423,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -477,7 +479,8 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_int.getID()),
                             dim_col_int.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -514,7 +517,8 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -588,7 +592,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -636,7 +640,8 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_char.getID()),
                             dim_col_char.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID.
 
   // Create prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -668,7 +673,8 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -721,7 +727,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -770,7 +776,8 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_varchar.getID()),
                             dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID.
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -807,7 +814,8 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -885,7 +893,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   }
 
   // Create the cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -939,7 +947,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID.
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -980,7 +989,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -1058,7 +1068,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -1113,7 +1123,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID.
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -1164,7 +1175,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
                            output_destination_index,
                            join_hash_table_index,
                            residual_pred_index,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -1242,7 +1254,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index 50c508d..244091f 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -1573,7 +1573,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             true));
+                                             true,
+                                             0  /* dummy query ID */));
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.
@@ -1616,7 +1617,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             false));
+                                             false,
+                                             0  /* dummy query ID */));
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index 7491778..6f24b92 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -358,7 +358,8 @@ class SortRunGenerationOperatorTest : public ::testing::Test {
                                       *result_table_,
                                       insert_destination_index,
                                       sort_config_index,
-                                      true /* is_stored */));
+                                      true /* is_stored */,
+                                      0  /* dummy query ID */));
     run_gen->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index 1dfad7b..7626686 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -193,7 +193,8 @@ TEST_F(TextScanOperatorTest, ScanTest) {
                            true,
                            false,
                            *relation_,
-                           output_destination_index));
+                           output_destination_index,
+                           0  /* dummy query ID */));
 
   // Setup query_context_.
   query_context_.reset(new QueryContext(query_context_proto,


[28/50] [abbrv] incubator-quickstep git commit: Measure execution time of WorkOrders.

Posted by zu...@apache.org.
Measure execution time of WorkOrders.

- Measure order execution time for both normal and rebuild work
  orders.
- Split the WorkOrderCompletion proto message in two: Normal and Rebuild
  work orders, with a possibility that in the future the two messages
  will have different fields.
- Add execution time field in both WorkOrderComplete proto messages.
- Include the recorded time in both WorkOrderComplete proto messages
  that are sent back to Foreman.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 8e825f1525d5080e6c4bd02ee2ce0e14d81ab6c5
Parents: a701362
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Tue Jun 14 14:02:45 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Jun 15 15:59:35 2016 -0500

----------------------------------------------------------------------
 query_execution/PolicyEnforcer.cpp              | 23 ++++-
 query_execution/QueryExecutionMessages.proto    | 29 +++++--
 query_execution/QueryManager.cpp                |  4 +-
 query_execution/Worker.cpp                      | 89 ++++++++++++--------
 query_execution/Worker.hpp                      | 38 +++++++--
 query_execution/tests/QueryManager_unittest.cpp |  9 +-
 6 files changed, 132 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8e825f15/query_execution/PolicyEnforcer.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcer.cpp b/query_execution/PolicyEnforcer.cpp
index 4501026..9f0502d 100644
--- a/query_execution/PolicyEnforcer.cpp
+++ b/query_execution/PolicyEnforcer.cpp
@@ -67,13 +67,26 @@ void PolicyEnforcer::processMessage(const TaggedMessage &tagged_message) {
   // TaggedMessage only once.
   std::size_t query_id;
   switch (tagged_message.message_type()) {
-    case kWorkOrderCompleteMessage:  // Fall through.
+    case kWorkOrderCompleteMessage: {
+      serialization::NormalWorkOrderCompletionMessage proto;
+      // Note: This proto message contains the time it took to execute the
+      // WorkOrder. It can be accessed in this scope.
+      CHECK(proto.ParseFromArray(tagged_message.message(),
+                                 tagged_message.message_bytes()));
+      query_id = proto.query_id();
+      worker_directory_->decrementNumQueuedWorkOrders(
+          proto.worker_thread_index());
+      break;
+    }
     case kRebuildWorkOrderCompleteMessage: {
-      serialization::WorkOrderCompletionMessage proto;
+      serialization::RebuildWorkOrderCompletionMessage proto;
+      // Note: This proto message contains the time it took to execute the
+      // rebuild WorkOrder. It can be accessed in this scope.
       CHECK(proto.ParseFromArray(tagged_message.message(),
                                  tagged_message.message_bytes()));
       query_id = proto.query_id();
-      worker_directory_->decrementNumQueuedWorkOrders(proto.worker_thread_index());
+      worker_directory_->decrementNumQueuedWorkOrders(
+          proto.worker_thread_index());
       break;
     }
     case kCatalogRelationNewBlockMessage: {
@@ -98,7 +111,9 @@ void PolicyEnforcer::processMessage(const TaggedMessage &tagged_message) {
       break;
     }
     case kWorkOrderFeedbackMessage: {
-      WorkOrder::FeedbackMessage msg(const_cast<void *>(tagged_message.message()), tagged_message.message_bytes());
+      WorkOrder::FeedbackMessage msg(
+          const_cast<void *>(tagged_message.message()),
+          tagged_message.message_bytes());
       query_id = msg.header().query_id;
       break;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8e825f15/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 9d9a9e5..65a8946 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -20,14 +20,33 @@ package quickstep.serialization;
 message EmptyMessage {
 }
 
-// Used for both Normal WorkOrders and RebuildWorkOrders.
-// NOTE(zuyu): we might need to seperate the completion messages to contain
-// run-time information for Foreman to make better decisions on scheduling
-// WorkOrders.
-message WorkOrderCompletionMessage {
+// Note: There are different types of completion messages for normal work orders
+// rebuild work orders. This can be potentially helpful when we want to collect
+// different statistics for executing different types of work orders. 
+// e.g. In select normal work order completion message, we could be interested 
+// in the selectivity of the block whose work order got execute. In rebuild work 
+// order completion message, we may be interested in adding the compression 
+// ratio or dictionary size of the rebuilt block.
+
+// TODO(harshad) : If there are different fields in the two message types below, 
+// create a base message class called WorkOrderCompletionMessage and make the
+// two classes below extend the base class. All the common fields in both the
+// classes can be moved to the base class. 
+
+// A message sent upon completion of a normal (not rebuild) WorkOrder execution.
+message NormalWorkOrderCompletionMessage {
+  required uint64 operator_index = 1;
+  required uint64 worker_thread_index = 2;
+  required uint64 query_id = 3;
+  optional uint64 execution_time_in_microseconds = 4;
+}
+
+// A message sent upon completion of a rebuild WorkOrder execution.
+message RebuildWorkOrderCompletionMessage {
   required uint64 operator_index = 1;
   required uint64 worker_thread_index = 2;
   required uint64 query_id = 3;
+  optional uint64 execution_time_in_microseconds = 4;
 }
 
 message CatalogRelationNewBlockMessage {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8e825f15/query_execution/QueryManager.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManager.cpp b/query_execution/QueryManager.cpp
index e4e4c9d..d20b592 100644
--- a/query_execution/QueryManager.cpp
+++ b/query_execution/QueryManager.cpp
@@ -161,7 +161,7 @@ QueryManager::QueryStatusCode QueryManager::processMessage(
   dag_node_index op_index;
   switch (tagged_message.message_type()) {
     case kWorkOrderCompleteMessage: {
-      serialization::WorkOrderCompletionMessage proto;
+      serialization::NormalWorkOrderCompletionMessage proto;
       CHECK(proto.ParseFromArray(tagged_message.message(),
                                  tagged_message.message_bytes()));
 
@@ -170,7 +170,7 @@ QueryManager::QueryStatusCode QueryManager::processMessage(
       break;
     }
     case kRebuildWorkOrderCompleteMessage: {
-      serialization::WorkOrderCompletionMessage proto;
+      serialization::RebuildWorkOrderCompletionMessage proto;
       CHECK(proto.ParseFromArray(tagged_message.message(),
                                  tagged_message.message_bytes()));
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8e825f15/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index ef596e1..ae889c7 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -16,7 +16,9 @@
 
 #include "query_execution/Worker.hpp"
 
+#include <chrono>
 #include <cstddef>
+#include <cstdint>
 #include <cstdlib>
 #include <utility>
 
@@ -36,6 +38,7 @@
 #include "tmb/tagged_message.h"
 
 using std::size_t;
+using std::uint64_t;
 
 using tmb::TaggedMessage;
 
@@ -50,22 +53,27 @@ void Worker::run() {
   for (;;) {
     // Receive() is a blocking call, causing this thread to sleep until next
     // message is received.
-    const AnnotatedMessage annotated_msg = bus_->Receive(worker_client_id_, 0, true);
+    const AnnotatedMessage annotated_msg =
+        bus_->Receive(worker_client_id_, 0, true);
     const TaggedMessage &tagged_message = annotated_msg.tagged_message;
     switch (tagged_message.message_type()) {
-      case kWorkOrderMessage:  // Fall through.
+      case kWorkOrderMessage: {
+        serialization::NormalWorkOrderCompletionMessage proto;
+        executeWorkOrderHelper<serialization::NormalWorkOrderCompletionMessage>(
+            tagged_message, &proto);
+        sendWorkOrderCompleteMessage<
+            serialization::NormalWorkOrderCompletionMessage>(
+            annotated_msg.sender, proto, kWorkOrderCompleteMessage);
+        break;
+      }
       case kRebuildWorkOrderMessage: {
-        WorkerMessage message(*static_cast<const WorkerMessage*>(tagged_message.message()));
-        DCHECK(message.getWorkOrder() != nullptr);
-        message.getWorkOrder()->execute();
-        const std::size_t query_id_for_workorder =
-            message.getWorkOrder()->getQueryID();
-        delete message.getWorkOrder();
-
-        sendWorkOrderCompleteMessage(
-            annotated_msg.sender, message.getRelationalOpIndex(),
-            query_id_for_workorder,
-            tagged_message.message_type() == kRebuildWorkOrderMessage);
+        serialization::RebuildWorkOrderCompletionMessage proto;
+        executeWorkOrderHelper<
+            serialization::RebuildWorkOrderCompletionMessage>(tagged_message,
+                                                              &proto);
+        sendWorkOrderCompleteMessage<
+            serialization::RebuildWorkOrderCompletionMessage>(
+            annotated_msg.sender, proto, kRebuildWorkOrderCompleteMessage);
         break;
       }
       case kPoisonMessage: {
@@ -77,34 +85,49 @@ void Worker::run() {
   }
 }
 
+template <typename CompletionMessageProtoT>
 void Worker::sendWorkOrderCompleteMessage(const tmb::client_id receiver,
-                                          const size_t op_index,
-                                          const size_t query_id,
-                                          const bool is_rebuild_work_order) {
-  serialization::WorkOrderCompletionMessage proto;
-  proto.set_operator_index(op_index);
-  proto.set_worker_thread_index(worker_thread_index_);
-  proto.set_query_id(query_id);
-
+                                          const CompletionMessageProtoT &proto,
+                                          const message_type_id message_type) {
   // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
   const size_t proto_length = proto.ByteSize();
-  char *proto_bytes = static_cast<char*>(std::malloc(proto_length));
+  char *proto_bytes = static_cast<char *>(std::malloc(proto_length));
   CHECK(proto.SerializeToArray(proto_bytes, proto_length));
 
-  TaggedMessage message(static_cast<const void*>(proto_bytes),
-                        proto_length,
-                        is_rebuild_work_order ? kRebuildWorkOrderCompleteMessage
-                                              : kWorkOrderCompleteMessage);
+  TaggedMessage tagged_message(
+      static_cast<const void *>(proto_bytes), proto_length, message_type);
   std::free(proto_bytes);
 
   const tmb::MessageBus::SendStatus send_status =
-      QueryExecutionUtil::SendTMBMessage(bus_,
-                                         worker_client_id_,
-                                         receiver,
-                                         std::move(message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could not "
-      "be sent from worker with TMB client ID " << worker_client_id_ << " to "
-      "Foreman with TMB client ID " << receiver;
+      QueryExecutionUtil::SendTMBMessage(
+          bus_, worker_client_id_, receiver, std::move(tagged_message));
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+      << "Message could not be sent from worker with TMB client ID "
+      << worker_client_id_ << " to Foreman with TMB client ID " << receiver;
+}
+
+template <typename CompletionMessageProtoT>
+void Worker::executeWorkOrderHelper(const TaggedMessage &tagged_message,
+                                    CompletionMessageProtoT *proto) {
+  std::chrono::time_point<std::chrono::steady_clock> start, end;
+  WorkerMessage worker_message(
+      *static_cast<const WorkerMessage *>(tagged_message.message()));
+  DCHECK(worker_message.getWorkOrder() != nullptr);
+  const size_t query_id_for_workorder = worker_message.getWorkOrder()->getQueryID();
+
+  // Start measuring the execution time.
+  start = std::chrono::steady_clock::now();
+  worker_message.getWorkOrder()->execute();
+  end = std::chrono::steady_clock::now();
+  delete worker_message.getWorkOrder();
+  const uint64_t execution_time_microseconds =
+      std::chrono::duration_cast<std::chrono::milliseconds>(end - start)
+          .count();
+  // Construct the proto message.
+  proto->set_operator_index(worker_message.getRelationalOpIndex());
+  proto->set_query_id(query_id_for_workorder);
+  proto->set_worker_thread_index(worker_thread_index_);
+  proto->set_execution_time_in_microseconds(execution_time_microseconds);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8e825f15/query_execution/Worker.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.hpp b/query_execution/Worker.hpp
index c0bafdc..71d3ec9 100644
--- a/query_execution/Worker.hpp
+++ b/query_execution/Worker.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_QUERY_EXECUTION_WORKER_HPP_
 
 #include <cstddef>
+#include <cstdint>
 
 #include "query_execution/QueryExecutionTypedefs.hpp"
 #include "threading/Thread.hpp"
@@ -27,6 +28,8 @@
 #include "tmb/id_typedefs.h"
 #include "tmb/message_bus.h"
 
+namespace tmb { class TaggedMessge; }
+
 namespace quickstep {
 
 /** \addtogroup QueryExecution
@@ -93,18 +96,35 @@ class Worker : public Thread {
 
  private:
   /**
-   * @brief Send the response WorkOrder completion message.
+   * @brief A helper method to execute the WorkOrder and construct a
+   *        completion message.
+   *
+   * @note CompletionMessageProtoT is the type of the completion message.
+   * @note Right now a single helper method works for all message types.
+   *       If different message types need to collect different statistics for
+   *       the WorkOrder execution, we need to create different helper methods,
+   *       one for each message type.
+   *
+   * @param tagged_message The TaggedMessage which consists of the WorkOrder.
+   * @param proto The proto message to be sent.
+   **/
+  template <typename CompletionMessageProtoT>
+  void executeWorkOrderHelper(const TaggedMessage &tagged_message,
+                              CompletionMessageProtoT *proto);
+
+  /**
+   * @brief A helper method to send the WorkOrder completion message.
+   *
+   * @note CompletionMessageProtoT is the type of the completion message.
    *
-   * @param receiver The id of the TMB client which should receive the response.
-   * @param op_index The index of the operator to which the WorkOrder belongs.
-   * @param query_id The ID of the query which the WorkOrder belongs to.
-   * @param is_rebuild_work_order True if it is a RebuildWorkOrder. Otherwise
-   *        false.
+   * @param receiver The TMB client ID of the receiver.
+   * @param proto The proto message to be sent.
+   * @param message_type The ID of the type of the message being sent.
    **/
+  template <typename CompletionMessageProtoT>
   void sendWorkOrderCompleteMessage(const tmb::client_id receiver,
-                                    const std::size_t op_index,
-                                    const std::size_t query_id,
-                                    const bool is_rebuild_work_order);
+                                    const CompletionMessageProtoT &proto,
+                                    const message_type_id message_type);
 
   const std::size_t worker_thread_index_;
   MessageBus *bus_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8e825f15/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 4f98748..37e2cdd 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -275,7 +275,7 @@ class QueryManagerTest : public ::testing::Test {
   inline bool placeWorkOrderCompleteMessage(const QueryPlan::DAGNodeIndex index) {
     VLOG(3) << "Place WorkOrderComplete message for Op[" << index << "]";
     TaggedMessage tagged_message;
-    serialization::WorkOrderCompletionMessage proto;
+    serialization::NormalWorkOrderCompletionMessage proto;
     proto.set_operator_index(index);
     proto.set_worker_thread_index(1);  // dummy worker ID.
     proto.set_query_id(0);  // dummy query ID.
@@ -296,8 +296,7 @@ class QueryManagerTest : public ::testing::Test {
 
   inline bool placeRebuildWorkOrderCompleteMessage(const QueryPlan::DAGNodeIndex index) {
     VLOG(3) << "Place RebuildWorkOrderComplete message for Op[" << index << "]";
-    // foreman_->processRebuildWorkOrderCompleteMessage(index, 0 /* worker id */);
-    serialization::WorkOrderCompletionMessage proto;
+    serialization::RebuildWorkOrderCompletionMessage proto;
     proto.set_operator_index(index);
     proto.set_worker_thread_index(1);  // dummy worker thread ID.
     proto.set_query_id(0);  // dummy query ID.
@@ -346,7 +345,6 @@ class QueryManagerTest : public ::testing::Test {
   unique_ptr<QueryHandle> query_handle_;
   unique_ptr<QueryManager> query_manager_;
 
-  // unique_ptr<Foreman> foreman_;
   MessageBusImpl bus_;
 
   client_id worker_client_id_;
@@ -357,7 +355,6 @@ class QueryManagerTest : public ::testing::Test {
 TEST_F(QueryManagerTest, SingleNodeDAGNoWorkOrdersTest) {
   // This test creates a DAG of a single node. No workorders are generated.
   query_plan_->addRelationalOperator(new MockOperator(false, false));
-  // foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
 
   const MockOperator &op = static_cast<const MockOperator &>(
       query_plan_->getQueryPlanDAG().getNodePayload(0));
@@ -377,7 +374,6 @@ TEST_F(QueryManagerTest, SingleNodeDAGStaticWorkOrdersTest) {
   // This test creates a DAG of a single node. Static workorders are generated.
   const QueryPlan::DAGNodeIndex id =
       query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
-  // foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
 
   const MockOperator &op = static_cast<const MockOperator &>(
       query_plan_->getQueryPlanDAG().getNodePayload(id));
@@ -429,7 +425,6 @@ TEST_F(QueryManagerTest, SingleNodeDAGDynamicWorkOrdersTest) {
   // scaffolding of mocking. If we use gMock, we can do much better.
   const QueryPlan::DAGNodeIndex id =
       query_plan_->addRelationalOperator(new MockOperator(true, false, 4, 3));
-  // foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
 
   const MockOperator &op = static_cast<const MockOperator &>(
       query_plan_->getQueryPlanDAG().getNodePayload(id));


[49/50] [abbrv] incubator-quickstep git commit: Minor changes in profiling work order output.

Posted by zu...@apache.org.
Minor changes in profiling work order output.

- Now prints query ID along with each work order entry.
- Removed spaces between two columns.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 04c8224b1584b982412c7023d041c1060d5c0342
Parents: 31f1bbb
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Wed Jul 6 11:38:49 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Jul 6 22:28:28 2016 -0500

----------------------------------------------------------------------
 query_execution/Foreman.cpp | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/04c8224b/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index f9f2e7a..98146e2 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -238,16 +238,17 @@ void Foreman::printWorkOrderProfilingResults(const std::size_t query_id,
   const std::vector<
       std::tuple<std::size_t, std::size_t, std::size_t>>
       &recorded_times = policy_enforcer_->getProfilingResults(query_id);
-  fputs("Worker ID, NUMA Socket, Operator ID, Time (microseconds)\n", out);
+  fputs("Query ID,Worker ID,NUMA Socket,Operator ID,Time (microseconds)\n", out);
   for (auto workorder_entry : recorded_times) {
     // Note: Index of the "worker thread index" in the tuple is 0.
     const std::size_t worker_id = std::get<0>(workorder_entry);
     fprintf(out,
-            "%lu, %d, %lu, %lu\n",
+            "%lu,%lu,%d,%lu,%lu\n",
+            query_id,
             worker_id,
             worker_directory_->getNUMANode(worker_id),
-            std::get<1>(workorder_entry),
-            std::get<2>(workorder_entry));
+            std::get<1>(workorder_entry),  // Operator ID.
+            std::get<2>(workorder_entry));  // Time.
   }
 }
 


[40/50] [abbrv] incubator-quickstep git commit: Added Window Aggregation Function in optimizer.

Posted by zu...@apache.org.
Added Window Aggregation Function in optimizer.

  - The resolver could understand optional window clause w/ aggregation functions.
  - Only one window aggregation function is allowed.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 714874ce54e12972285a43f92784ef6954a8b6fd
Parents: d642891
Author: shixuan <sh...@wisc.edu>
Authored: Tue Jun 21 20:08:52 2016 +0000
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Fri Jun 24 18:59:47 2016 -0700

----------------------------------------------------------------------
 query_optimizer/expressions/CMakeLists.txt      |  17 +-
 query_optimizer/expressions/ExpressionType.hpp  |   3 +-
 query_optimizer/expressions/PatternMatcher.hpp  |   3 +
 .../expressions/WindowAggregateFunction.cpp     | 193 ++++++++++++
 .../expressions/WindowAggregateFunction.hpp     | 246 +++++++++++++++
 query_optimizer/logical/CMakeLists.txt          |  18 +-
 query_optimizer/logical/LogicalType.hpp         |   3 +-
 query_optimizer/logical/PatternMatcher.hpp      |   2 +
 query_optimizer/logical/WindowAggregate.cpp     |  85 +++++
 query_optimizer/logical/WindowAggregate.hpp     | 123 ++++++++
 query_optimizer/resolver/CMakeLists.txt         |   2 +
 query_optimizer/resolver/Resolver.cpp           | 314 ++++++++++++++++++-
 query_optimizer/resolver/Resolver.hpp           |  66 +++-
 query_optimizer/strategy/CMakeLists.txt         |   3 +-
 query_optimizer/strategy/OneToOne.cpp           |   5 +
 .../tests/logical_generator/Select.test         | 162 ++++++++++
 query_optimizer/tests/resolver/Select.test      | 162 ++++++++++
 17 files changed, 1387 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CMakeLists.txt b/query_optimizer/expressions/CMakeLists.txt
index 6c40741..08d7df5 100644
--- a/query_optimizer/expressions/CMakeLists.txt
+++ b/query_optimizer/expressions/CMakeLists.txt
@@ -43,6 +43,7 @@ add_library(quickstep_queryoptimizer_expressions_SearchedCase SearchedCase.cpp S
 add_library(quickstep_queryoptimizer_expressions_SimpleCase SimpleCase.cpp SimpleCase.hpp)
 add_library(quickstep_queryoptimizer_expressions_SubqueryExpression SubqueryExpression.cpp SubqueryExpression.hpp)
 add_library(quickstep_queryoptimizer_expressions_UnaryExpression UnaryExpression.cpp UnaryExpression.hpp)
+add_library(quickstep_queryoptimizer_expressions_WindowAggregateFunction WindowAggregateFunction.cpp WindowAggregateFunction.hpp)
 
 # Link dependencies:
 target_link_libraries(quickstep_queryoptimizer_expressions_AggregateFunction
@@ -301,6 +302,19 @@ target_link_libraries(quickstep_queryoptimizer_expressions_UnaryExpression
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_expressions_WindowAggregateFunction
+                      glog
+                      quickstep_expressions_aggregation_AggregateFunction
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_Expression
+                      quickstep_queryoptimizer_expressions_ExpressionType
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Scalar
+                      quickstep_types_Type
+                      quickstep_utility_Cast
+                      quickstep_utility_Macros)
+
 
 # Module all-in-one library:
 add_library(quickstep_queryoptimizer_expressions ../../empty_src.cpp OptimizerExpressionsModule.hpp)
@@ -330,4 +344,5 @@ target_link_libraries(quickstep_queryoptimizer_expressions
                       quickstep_queryoptimizer_expressions_SearchedCase
                       quickstep_queryoptimizer_expressions_SimpleCase
                       quickstep_queryoptimizer_expressions_SubqueryExpression
-                      quickstep_queryoptimizer_expressions_UnaryExpression)
+                      quickstep_queryoptimizer_expressions_UnaryExpression
+                      quickstep_queryoptimizer_expressions_WindowAggregateFunction)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/expressions/ExpressionType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionType.hpp b/query_optimizer/expressions/ExpressionType.hpp
index 23770e0..77e0874 100644
--- a/query_optimizer/expressions/ExpressionType.hpp
+++ b/query_optimizer/expressions/ExpressionType.hpp
@@ -49,7 +49,8 @@ enum class ExpressionType {
   kSearchedCase,
   kSimpleCase,
   kSubqueryExpression,
-  kUnaryExpression
+  kUnaryExpression,
+  kWindowAggregateFunction
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/expressions/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/PatternMatcher.hpp b/query_optimizer/expressions/PatternMatcher.hpp
index 87bc52a..2cc91d6 100644
--- a/query_optimizer/expressions/PatternMatcher.hpp
+++ b/query_optimizer/expressions/PatternMatcher.hpp
@@ -52,6 +52,7 @@ class Scalar;
 class ScalarLiteral;
 class Sum;
 class UnaryExpression;
+class WindowAggregateFunction;
 
 /** \addtogroup OptimizerExpressions
  *  @{
@@ -155,6 +156,8 @@ using SomeScalar = SomeExpressionNode<Scalar,
                                       ExpressionType::kUnaryExpression>;
 using SomeScalarLiteral = SomeExpressionNode<ScalarLiteral, ExpressionType::kScalarLiteral>;
 using SomeUnaryExpression = SomeExpressionNode<UnaryExpression, ExpressionType::kUnaryExpression>;
+using SomeWindowAggregateFunction = SomeExpressionNode<WindowAggregateFunction,
+                                                       ExpressionType::kWindowAggregateFunction>;
 
 using SomeAggregateFunction = SomeExpressionNode<AggregateFunction,
                                                  ExpressionType::kAggregateFunction>;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/expressions/WindowAggregateFunction.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/WindowAggregateFunction.cpp b/query_optimizer/expressions/WindowAggregateFunction.cpp
new file mode 100644
index 0000000..7b1f304
--- /dev/null
+++ b/query_optimizer/expressions/WindowAggregateFunction.cpp
@@ -0,0 +1,193 @@
+/**
+ *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "query_optimizer/expressions/WindowAggregateFunction.hpp"
+
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "expressions/aggregation/AggregateFunction.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/Expression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
+#include "types/Type.hpp"
+#include "utility/Cast.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace expressions {
+
+bool WindowAggregateFunction::isNullable() const {
+  std::vector<const Type*> argument_types;
+  for (const ScalarPtr &argument : arguments_) {
+    argument_types.emplace_back(&argument->getValueType());
+  }
+
+  const Type *return_type = window_aggregate_.resultTypeForArgumentTypes(argument_types);
+  DCHECK(return_type != nullptr);
+  return return_type->isNullable();
+}
+
+const Type& WindowAggregateFunction::getValueType() const {
+  std::vector<const Type*> argument_types;
+  for (const ScalarPtr &argument : arguments_) {
+    argument_types.emplace_back(&argument->getValueType());
+  }
+
+  const Type *return_type = window_aggregate_.resultTypeForArgumentTypes(argument_types);
+  DCHECK(return_type != nullptr);
+  return *return_type;
+}
+
+WindowAggregateFunctionPtr WindowAggregateFunction::Create(
+    const ::quickstep::AggregateFunction &window_aggregate,
+    const std::vector<ScalarPtr> &arguments,
+    const WindowInfo &window_info,
+    const std::string &window_name,
+    const bool is_distinct) {
+#ifdef QUICKSTEP_DEBUG
+  std::vector<const Type*> argument_types;
+  for (const ScalarPtr &argument : arguments) {
+    argument_types.emplace_back(&argument->getValueType());
+  }
+  DCHECK(window_aggregate.canApplyToTypes(argument_types));
+#endif  // QUICKSTEP_DEBUG
+
+  return WindowAggregateFunctionPtr(
+      new WindowAggregateFunction(window_aggregate, arguments, window_info, window_name, is_distinct));
+}
+
+ExpressionPtr WindowAggregateFunction::copyWithNewChildren(
+    const std::vector<ExpressionPtr> &new_children) const {
+  std::vector<ScalarPtr> new_arguments;
+  for (const ExpressionPtr &expression_ptr : new_children) {
+    ScalarPtr expr_as_scalar;
+    CHECK(SomeScalar::MatchesWithConditionalCast(expression_ptr, &expr_as_scalar))
+        << expression_ptr->toString();
+    new_arguments.emplace_back(std::move(expr_as_scalar));
+  }
+
+  return WindowAggregateFunctionPtr(
+      new WindowAggregateFunction(window_aggregate_,
+                                  new_arguments,
+                                  window_info_,
+                                  window_name_,
+                                  is_distinct_));
+}
+
+std::vector<AttributeReferencePtr> WindowAggregateFunction::getReferencedAttributes() const {
+  std::vector<AttributeReferencePtr> referenced_attributes;
+  for (const ScalarPtr &argument : arguments_) {
+    const std::vector<AttributeReferencePtr> referenced_attributes_in_argument =
+        argument->getReferencedAttributes();
+    referenced_attributes.insert(referenced_attributes.end(),
+                                 referenced_attributes_in_argument.begin(),
+                                 referenced_attributes_in_argument.end());
+  }
+
+  referenced_attributes.insert(referenced_attributes.end(),
+                               window_info_.partition_by_attributes.begin(),
+                               window_info_.partition_by_attributes.end());
+
+  referenced_attributes.insert(referenced_attributes.end(),
+                               window_info_.order_by_attributes.begin(),
+                               window_info_.order_by_attributes.end());
+
+  return referenced_attributes;
+}
+
+void WindowAggregateFunction::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
+  inline_field_names->push_back("function");
+  inline_field_values->push_back(window_aggregate_.getName());
+
+  container_child_field_names->push_back("arguments");
+  container_child_fields->emplace_back(CastSharedPtrVector<OptimizerTreeBase>(arguments_));
+
+  inline_field_names->push_back("window_name");
+  inline_field_values->push_back(window_name_);
+
+  container_child_field_names->push_back("partition_by");
+  container_child_fields->emplace_back(
+      CastSharedPtrVector<OptimizerTreeBase>(window_info_.partition_by_attributes));
+
+  container_child_field_names->push_back("order_by");
+  container_child_fields->emplace_back(
+      CastSharedPtrVector<OptimizerTreeBase>(window_info_.order_by_attributes));
+
+  inline_field_names->push_back("is_ascending");
+  std::string ascending_list("[");
+  for (const bool is_ascending : window_info_.order_by_directions) {
+    if (is_ascending) {
+      ascending_list.append("true,");
+    } else {
+      ascending_list.append("false,");
+    }
+  }
+  if (!window_info_.order_by_directions.empty()) {
+    ascending_list.pop_back();
+  }
+  ascending_list.append("]");
+  inline_field_values->push_back(ascending_list);
+
+  inline_field_names->push_back("nulls_first");
+  std::string nulls_first_flags("[");
+  for (const bool nulls_first_flag : window_info_.nulls_first) {
+    if (nulls_first_flag) {
+      nulls_first_flags.append("true,");
+    } else {
+      nulls_first_flags.append("false,");
+    }
+  }
+  if (!window_info_.nulls_first.empty()) {
+    nulls_first_flags.pop_back();
+  }
+  nulls_first_flags.append("]");
+  inline_field_values->push_back(nulls_first_flags);
+
+  if (window_info_.frame_info != nullptr) {
+    const WindowFrameInfo *frame_info = window_info_.frame_info;
+
+    inline_field_names->push_back("frame_mode");
+    inline_field_values->push_back(frame_info->is_row ? "row" : "range");
+
+    inline_field_names->push_back("num_preceding");
+    inline_field_values->push_back(std::to_string(frame_info->num_preceding));
+
+    inline_field_names->push_back("num_following");
+    inline_field_values->push_back(std::to_string(frame_info->num_following));
+  }
+
+  if (is_distinct_) {
+    inline_field_names->push_back("is_distinct");
+    inline_field_values->push_back("true");
+  }
+}
+
+}  // namespace expressions
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/expressions/WindowAggregateFunction.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/WindowAggregateFunction.hpp b/query_optimizer/expressions/WindowAggregateFunction.hpp
new file mode 100644
index 0000000..0bee28f
--- /dev/null
+++ b/query_optimizer/expressions/WindowAggregateFunction.hpp
@@ -0,0 +1,246 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_WINDOW_AGGREGATE_FUNCTION_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_WINDOW_AGGREGATE_FUNCTION_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/Expression.hpp"
+#include "query_optimizer/expressions/ExpressionType.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class AggregateFunction;
+class Type;
+
+namespace optimizer {
+namespace expressions {
+
+/** \addtogroup OptimizerExpressions
+ *  @{
+ */
+
+struct WindowFrameInfo {
+  /**
+   * @brief Cosntructor.
+   *
+   * @param is_row_in True if this window frame is defined by ROWS, false if
+   *                  defined by RANGE.
+   * @param num_preceding_in The number of preceding tuples the window frame
+   *                         will cover, -1 means UNBOUNDED.
+   * @param num_following_in The number of following tuples the window frame
+   *                         will cover, -1 means UNBOUNDED.
+   **/
+  WindowFrameInfo(const bool is_row_in,
+                  const int num_preceding_in,
+                  const int num_following_in)
+      : is_row(is_row_in),
+        num_preceding(num_preceding_in),
+        num_following(num_following_in) {}
+
+  const bool is_row;
+  const int num_preceding;
+  const int num_following;
+};
+
+struct WindowInfo {
+  /**
+   * @brief Constructor.
+   *
+   * @param partition_by_attributes_in The partition keys for the window.
+   * @param order_by_attributes_in The order keys for the window.
+   * @param order_by_directions_in The order direction for order key.
+   * @param nulls_first_in The nulls' position for order key.
+   * @param frame_info_in The window frame information for the window. Null
+   *        means there is no explicit window frame definition.
+   **/
+  WindowInfo(const std::vector<AttributeReferencePtr> &partition_by_attributes_in,
+             const std::vector<AttributeReferencePtr> &order_by_attributes_in,
+             const std::vector<bool> &order_by_directions_in,
+             const std::vector<bool> &nulls_first_in,
+             const WindowFrameInfo *frame_info_in)
+      : partition_by_attributes(partition_by_attributes_in),
+        order_by_attributes(order_by_attributes_in),
+        order_by_directions(order_by_directions_in),
+        nulls_first(nulls_first_in),
+        frame_info(frame_info_in) {}
+
+  const std::vector<AttributeReferencePtr> partition_by_attributes;
+  const std::vector<AttributeReferencePtr> order_by_attributes;
+  const std::vector<bool> order_by_directions;
+  const std::vector<bool> nulls_first;
+  const WindowFrameInfo *frame_info;
+};
+
+class WindowAggregateFunction;
+typedef std::shared_ptr<const WindowAggregateFunction> WindowAggregateFunctionPtr;
+
+/**
+ * @brief Represents a window aggregate function and its arguments in the
+ *        optimizer. This class wraps some of the functionality from
+ *        quickstep::AggregateFunction and represents a particular instance
+ *        of an aggregate during query optimization.
+ **/
+class WindowAggregateFunction : public Expression {
+ public:
+  /**
+   * @brief Destructor.
+   */
+  ~WindowAggregateFunction() override {}
+
+  ExpressionType getExpressionType() const override {
+    return ExpressionType::kWindowAggregateFunction;
+  }
+
+  std::string getName() const override {
+    return "WindowAggregateFunction";
+  }
+
+  const Type& getValueType() const override;
+
+  bool isConstant() const override {
+    // Window aggregate function is never considered as a constant expression.
+    return false;
+  }
+
+  ExpressionPtr copyWithNewChildren(
+      const std::vector<ExpressionPtr> &new_children) const override;
+
+  std::vector<AttributeReferencePtr> getReferencedAttributes() const override;
+
+  /**
+   * @return Whether the type of the return value is nullable.
+   **/
+  bool isNullable() const;
+
+  /**
+   * @return The WindowAggregateFunction singleton (from the expression system)
+   *         for this node.
+   **/
+  inline const ::quickstep::AggregateFunction& window_aggregate() const {
+    return window_aggregate_;
+  }
+
+  /**
+   * @return The list of scalar arguments to this aggregate.
+   **/
+  inline const std::vector<ScalarPtr>& arguments() const {
+    return arguments_;
+  }
+
+  /**
+   * @return The window info of this window aggregate function.
+   **/
+  inline const WindowInfo window_info() const {
+    return window_info_;
+  }
+
+  /**
+   * @return The name of the window.
+   **/
+  inline const std::string window_name() const {
+    return window_name_;
+  }
+
+  /**
+   * @return Whether this is a DISTINCT aggregation.
+   **/
+  inline bool is_distinct() const {
+    return is_distinct_;
+  }
+
+  /**
+   * @brief Create a new WindowAggregateFunction by directly defined window.
+   *
+   * @warning It is an error to call this with arguments that the given
+   *          aggregate can not apply to.
+   *
+   * @param aggregate The underlying WindowAggregateFunction from the expression
+   *        system.
+   * @param arguments A list of arguments to the window aggregate function.
+   * @param window_info The window info of the window aggregate function.
+   * @param is_distinct Whether this is a DISTINCT aggregation.
+   * @return A new AggregateFunctionPtr.
+   **/
+  static WindowAggregateFunctionPtr Create(const ::quickstep::AggregateFunction &window_aggregate,
+                                           const std::vector<ScalarPtr> &arguments,
+                                           const WindowInfo &window_info,
+                                           const std::string &window_name,
+                                           const bool is_distinct);
+
+ 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:
+  /**
+   * @brief Constructor.
+   *
+   * @param window_aggregate The actual AggregateFunction to use.
+   * @param arguments A list of arguments to the window aggregate function.
+   * @param window_info The window info of the window aggregate function.
+   * @param is_distinct Indicates whether this is a DISTINCT aggregation.
+   */
+  WindowAggregateFunction(const ::quickstep::AggregateFunction &window_aggregate,
+                          const std::vector<ScalarPtr> &arguments,
+                          const WindowInfo &window_info,
+                          const std::string &window_name,
+                          const bool is_distinct)
+      : window_aggregate_(window_aggregate),
+        arguments_(arguments),
+        window_info_(window_info),
+        window_name_(window_name),
+        is_distinct_(is_distinct) {
+    for (const ScalarPtr &child : arguments_) {
+      addChild(child);
+    }
+  }
+
+  // TODO(Shixuan): Currently this class uses AggregationFunction as
+  // window_aggregate_. If it really needs to be seperated from the
+  // AggregationFunction, a new class for WindowAggregationFunction should be
+  // created as quickstep::WindowAggregateFunction.
+  const ::quickstep::AggregateFunction &window_aggregate_;
+  std::vector<ScalarPtr> arguments_;
+  const WindowInfo window_info_;
+  const std::string window_name_;
+  const bool is_distinct_;
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregateFunction);
+};
+
+/** @} */
+
+}  // namespace expressions
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_WINDOW_AGGREGATE_FUNCTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/logical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CMakeLists.txt b/query_optimizer/logical/CMakeLists.txt
index 61c6234..b787c60 100644
--- a/query_optimizer/logical/CMakeLists.txt
+++ b/query_optimizer/logical/CMakeLists.txt
@@ -43,6 +43,7 @@ add_library(quickstep_queryoptimizer_logical_TableReference TableReference.cpp T
 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_UpdateTable UpdateTable.cpp UpdateTable.hpp)
+add_library(quickstep_queryoptimizer_logical_WindowAggregate WindowAggregate.cpp WindowAggregate.hpp)
 
 # Link dependencies:
 target_link_libraries(quickstep_queryoptimizer_logical_Aggregate
@@ -259,6 +260,20 @@ target_link_libraries(quickstep_queryoptimizer_logical_UpdateTable
                       quickstep_queryoptimizer_logical_LogicalType
                       quickstep_utility_Cast
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_logical_WindowAggregate
+                      glog
+                      quickstep_queryoptimizer_expressions_Alias
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_Expression
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_logical_Logical
+                      quickstep_queryoptimizer_logical_LogicalType
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_utility_Cast
+                      quickstep_utility_Macros)
+
 
 # Module all-in-one library:
 add_library(quickstep_queryoptimizer_logical ../../empty_src.cpp OptimizerLogicalModule.hpp)
@@ -287,4 +302,5 @@ target_link_libraries(quickstep_queryoptimizer_logical
                       quickstep_queryoptimizer_logical_TableGenerator
                       quickstep_queryoptimizer_logical_TableReference
                       quickstep_queryoptimizer_logical_TopLevelPlan
-                      quickstep_queryoptimizer_logical_UpdateTable)
+                      quickstep_queryoptimizer_logical_UpdateTable
+                      quickstep_queryoptimizer_logical_WindowAggregate)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/logical/LogicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/LogicalType.hpp b/query_optimizer/logical/LogicalType.hpp
index 1b9366e..c82fb47 100644
--- a/query_optimizer/logical/LogicalType.hpp
+++ b/query_optimizer/logical/LogicalType.hpp
@@ -49,7 +49,8 @@ enum class LogicalType {
   kTableGenerator,
   kTableReference,
   kTopLevelPlan,
-  kUpdateTable
+  kUpdateTable,
+  kWindowAggregate
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/logical/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/PatternMatcher.hpp b/query_optimizer/logical/PatternMatcher.hpp
index ff8f3d0..de8609e 100644
--- a/query_optimizer/logical/PatternMatcher.hpp
+++ b/query_optimizer/logical/PatternMatcher.hpp
@@ -45,6 +45,7 @@ class Sort;
 class TableReference;
 class TopLevelPlan;
 class UpdateTable;
+class WindowAggregate;
 
 /** \addtogroup OptimizerLogical
  *  @{
@@ -130,6 +131,7 @@ using SomeSort = SomeLogicalNode<Sort, LogicalType::kSort>;
 using SomeTableReference = SomeLogicalNode<TableReference, LogicalType::kTableReference>;
 using SomeTopLevelPlan = SomeLogicalNode<TopLevelPlan, LogicalType::kTopLevelPlan>;
 using SomeUpdateTable = SomeLogicalNode<UpdateTable, LogicalType::kUpdateTable>;
+using SomeWindowAggregate = SomeLogicalNode<WindowAggregate, LogicalType::kWindowAggregate>;
 
 /** @} */
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/logical/WindowAggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/WindowAggregate.cpp b/query_optimizer/logical/WindowAggregate.cpp
new file mode 100644
index 0000000..0d747b6
--- /dev/null
+++ b/query_optimizer/logical/WindowAggregate.cpp
@@ -0,0 +1,85 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "query_optimizer/logical/WindowAggregate.hpp"
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/Alias.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "utility/Cast.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace logical {
+
+namespace E = ::quickstep::optimizer::expressions;
+
+LogicalPtr WindowAggregate::copyWithNewChildren(
+    const std::vector<LogicalPtr> &new_children) const {
+  DCHECK_EQ(getNumChildren(), new_children.size());
+  return Create(new_children[0], window_aggregate_expression_);
+}
+
+std::vector<E::AttributeReferencePtr> WindowAggregate::getOutputAttributes() const {
+  std::vector<E::AttributeReferencePtr> output_attributes(input_->getOutputAttributes());
+  output_attributes.push_back(E::ToRef(window_aggregate_expression_));
+  return output_attributes;
+}
+
+std::vector<E::AttributeReferencePtr> WindowAggregate::getReferencedAttributes() const {
+  return window_aggregate_expression_->getReferencedAttributes();
+}
+
+LogicalPtr WindowAggregate::copyWithNewInputExpressions(
+    const std::vector<E::ExpressionPtr> &input_expressions) const {
+  // Only one expression needed
+  DCHECK_EQ(1u, input_expressions.size());
+
+  E::AliasPtr window_aggregate_expression;
+  E::SomeAlias::MatchesWithConditionalCast(input_expressions[0],
+                                           &window_aggregate_expression);
+
+  return Create(input_, window_aggregate_expression);
+}
+
+void WindowAggregate::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("input");
+  non_container_child_fields->push_back(input_);
+
+  non_container_child_field_names->push_back("window_aggregate_expression");
+  non_container_child_fields->push_back(window_aggregate_expression_);
+}
+
+}  // namespace logical
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/logical/WindowAggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/WindowAggregate.hpp b/query_optimizer/logical/WindowAggregate.hpp
new file mode 100644
index 0000000..dcd9a7d
--- /dev/null
+++ b/query_optimizer/logical/WindowAggregate.hpp
@@ -0,0 +1,123 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_WINDOW_AGGREGATE_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_WINDOW_AGGREGATE_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/Alias.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/Expression.hpp"
+#include "query_optimizer/expressions/NamedExpression.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 WindowAggregate;
+typedef std::shared_ptr<const WindowAggregate> WindowAggregatePtr;
+
+/**
+ * @brief Window aggregate operator that computes window aggregate expressions.
+ */
+class WindowAggregate : public Logical {
+ public:
+  LogicalType getLogicalType() const override {
+    return LogicalType::kWindowAggregate;
+  }
+
+  std::string getName() const override { return "WindowAggregate"; }
+
+  /**
+   * @return The input logical node.
+   */
+  const LogicalPtr& input() const { return input_; }
+
+  /**
+   * @return PARTITION BY expressions.
+   */
+  const expressions::AliasPtr window_aggregate_expression() const {
+    return window_aggregate_expression_;
+  }
+
+  LogicalPtr copyWithNewChildren(
+      const std::vector<LogicalPtr> &new_children) const override;
+
+  LogicalPtr copyWithNewInputExpressions(
+      const std::vector<expressions::ExpressionPtr> &input_expressions) const override;
+
+  std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override;
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
+
+  /**
+   * @brief Creates an Aggregate logical node.
+   *
+   * @param input The input node.
+   * @param window_aggregate_expression The window aggregate expression.
+   * @return An immutable WindowAggregate node.
+   */
+  static WindowAggregatePtr Create(
+      const LogicalPtr &input,
+      const expressions::AliasPtr &window_aggregate_expression) {
+    return WindowAggregatePtr(new WindowAggregate(input, window_aggregate_expression));
+  }
+
+ 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:
+  WindowAggregate(const LogicalPtr &input,
+                  const expressions::AliasPtr &window_aggregate_expression)
+      : input_(input),
+        window_aggregate_expression_(window_aggregate_expression) {
+    addChild(input_);
+    addInputExpression(window_aggregate_expression_);
+  }
+
+  const LogicalPtr input_;
+  const expressions::AliasPtr window_aggregate_expression_;
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregate);
+};
+
+/** @} */
+
+}  // namespace logical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_WINDOW_AGGREGATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index dc7eac0..fb75767 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -89,6 +89,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_queryoptimizer_expressions_SimpleCase
                       quickstep_queryoptimizer_expressions_SubqueryExpression
                       quickstep_queryoptimizer_expressions_UnaryExpression
+                      quickstep_queryoptimizer_expressions_WindowAggregateFunction
                       quickstep_queryoptimizer_logical_Aggregate
                       quickstep_queryoptimizer_logical_CopyFrom
                       quickstep_queryoptimizer_logical_CreateIndex
@@ -109,6 +110,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_queryoptimizer_logical_TableReference
                       quickstep_queryoptimizer_logical_TopLevelPlan
                       quickstep_queryoptimizer_logical_UpdateTable
+                      quickstep_queryoptimizer_logical_WindowAggregate
                       quickstep_queryoptimizer_resolver_NameResolver
                       quickstep_storage_StorageBlockLayout_proto
                       quickstep_storage_StorageConstants

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index ffc173a..f880ce7 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -85,6 +85,7 @@
 #include "query_optimizer/expressions/SimpleCase.hpp"
 #include "query_optimizer/expressions/SubqueryExpression.hpp"
 #include "query_optimizer/expressions/UnaryExpression.hpp"
+#include "query_optimizer/expressions/WindowAggregateFunction.hpp"
 #include "query_optimizer/logical/Aggregate.hpp"
 #include "query_optimizer/logical/CopyFrom.hpp"
 #include "query_optimizer/logical/CreateIndex.hpp"
@@ -104,6 +105,7 @@
 #include "query_optimizer/logical/TableReference.hpp"
 #include "query_optimizer/logical/TopLevelPlan.hpp"
 #include "query_optimizer/logical/UpdateTable.hpp"
+#include "query_optimizer/logical/WindowAggregate.hpp"
 #include "query_optimizer/resolver/NameResolver.hpp"
 #include "storage/StorageBlockLayout.pb.h"
 #include "storage/StorageConstants.hpp"
@@ -164,9 +166,11 @@ struct Resolver::ExpressionResolutionInfo {
    */
   ExpressionResolutionInfo(const NameResolver &name_resolver_in,
                            QueryAggregationInfo *query_aggregation_info_in,
+                           WindowAggregationInfo *window_aggregation_info_in,
                            SelectListInfo *select_list_info_in)
       : name_resolver(name_resolver_in),
         query_aggregation_info(query_aggregation_info_in),
+        window_aggregation_info(window_aggregation_info_in),
         select_list_info(select_list_info_in) {}
 
   /**
@@ -180,6 +184,7 @@ struct Resolver::ExpressionResolutionInfo {
       : name_resolver(parent.name_resolver),
         not_allow_aggregate_here(parent.not_allow_aggregate_here),
         query_aggregation_info(parent.query_aggregation_info),
+        window_aggregation_info(parent.window_aggregation_info),
         select_list_info(parent.select_list_info) {}
 
   /**
@@ -187,16 +192,29 @@ struct Resolver::ExpressionResolutionInfo {
    */
   bool hasAggregate() const { return parse_aggregate_expression != nullptr; }
 
+  /**
+   * @return True if the expression contains a window aggregate function.
+   **/
+  bool hasWindowAggregate() const {
+    return parse_window_aggregate_expression != nullptr;
+  }
+
   const NameResolver &name_resolver;
   // Empty if aggregations are allowed.
   const std::string not_allow_aggregate_here;
 
   // Can be NULL if aggregations are not allowed.
   QueryAggregationInfo *query_aggregation_info = nullptr;
+
+  // Alias expressions that wraps window aggregate functions.
+  WindowAggregationInfo *window_aggregation_info = nullptr;
+
   // Can be NULL if alias references to SELECT-list expressions are not allowed.
   SelectListInfo *select_list_info = nullptr;
   // The first aggregate in the expression.
   const ParseTreeNode *parse_aggregate_expression = nullptr;
+  // The first window aggregate in the expression.
+  const ParseTreeNode *parse_window_aggregate_expression = nullptr;
 };
 
 struct Resolver::QueryAggregationInfo {
@@ -209,6 +227,26 @@ struct Resolver::QueryAggregationInfo {
   std::vector<E::AliasPtr> aggregate_expressions;
 };
 
+struct Resolver::WindowPlan {
+  WindowPlan(const E::WindowInfo &window_info_in,
+             const L::LogicalPtr &logical_plan_in)
+      : window_info(window_info_in),
+        logical_plan(logical_plan_in) {}
+
+  const E::WindowInfo window_info;
+  const L::LogicalPtr logical_plan;
+};
+
+struct Resolver::WindowAggregationInfo {
+  explicit WindowAggregationInfo(const std::unordered_map<std::string, WindowPlan> &window_map_in)
+      : window_map(window_map_in) {}
+
+  // Whether the current query block has a GROUP BY.
+  const std::unordered_map<std::string, WindowPlan> window_map;
+  // Alias expressions that wraps window aggregate functions.
+  std::vector<E::AliasPtr> window_aggregate_expressions;
+};
+
 struct Resolver::SelectListInfo {
  public:
   /**
@@ -973,8 +1011,36 @@ L::LogicalPtr Resolver::resolveSelect(
         logical_plan, resolvePredicate(parse_predicate, &expr_resolution_info));
   }
 
+  // Resolve WINDOW clause.
+  std::unordered_map<std::string, WindowPlan> sorted_window_map;
+  if (select_query.window_list() != nullptr) {
+    if (select_query.window_list()->size() > 1) {
+      THROW_SQL_ERROR_AT(&(*select_query.window_list()->begin()))
+          << "Currently we don't support multiple window aggregation functions";
+    }
+
+    // Sort the table according to the window.
+    for (const ParseWindow &window : *select_query.window_list()) {
+      // Check for duplicate definition.
+      // Currently this is useless since we only support one window.
+      if (sorted_window_map.find(window.name()->value()) != sorted_window_map.end()) {
+        THROW_SQL_ERROR_AT(window.name())
+            << "Duplicate definition of window " << window.name()->value();
+      }
+
+      E::WindowInfo resolved_window = resolveWindow(window, *name_resolver);
+      L::LogicalPtr sorted_logical_plan = resolveSortInWindow(logical_plan,
+                                                              resolved_window);
+
+      WindowPlan window_plan(resolved_window, sorted_logical_plan);
+
+      sorted_window_map.emplace(window.name()->value(), window_plan);
+    }
+  }
+
   QueryAggregationInfo query_aggregation_info(
       (select_query.group_by() != nullptr));
+  WindowAggregationInfo window_aggregation_info(sorted_window_map);
 
   // Resolve SELECT-list clause.
   std::vector<E::NamedExpressionPtr> select_list_expressions;
@@ -984,6 +1050,7 @@ L::LogicalPtr Resolver::resolveSelect(
                       type_hints,
                       *name_resolver,
                       &query_aggregation_info,
+                      &window_aggregation_info,
                       &select_list_expressions,
                       &has_aggregate_per_expression);
   DCHECK_EQ(has_aggregate_per_expression.size(),
@@ -992,6 +1059,29 @@ L::LogicalPtr Resolver::resolveSelect(
   SelectListInfo select_list_info(select_list_expressions,
                                   has_aggregate_per_expression);
 
+  // Create window aggregate node if needed
+  for (const E::AliasPtr &alias : window_aggregation_info.window_aggregate_expressions) {
+    E::WindowAggregateFunctionPtr window_aggregate_function;
+    if (!E::SomeWindowAggregateFunction::MatchesWithConditionalCast(alias->expression(),
+                                                                    &window_aggregate_function)) {
+      THROW_SQL_ERROR()
+          << "Unexpected expression in window aggregation function";
+    }
+    L::LogicalPtr sorted_logical_plan;
+
+    // Get the sorted logical plan
+    const std::string window_name = window_aggregate_function->window_name();
+    if (!window_name.empty()) {
+      sorted_logical_plan = sorted_window_map.at(window_name).logical_plan;
+    } else {
+      sorted_logical_plan = resolveSortInWindow(logical_plan,
+                                                window_aggregate_function->window_info());
+    }
+
+    logical_plan = L::WindowAggregate::Create(sorted_logical_plan,
+                                              alias);
+  }
+
   // Resolve GROUP BY.
   std::vector<E::NamedExpressionPtr> group_by_expressions;
   if (select_query.group_by() != nullptr) {
@@ -1039,7 +1129,7 @@ L::LogicalPtr Resolver::resolveSelect(
   E::PredicatePtr having_predicate;
   if (select_query.having() != nullptr) {
     ExpressionResolutionInfo expr_resolution_info(
-        *name_resolver, &query_aggregation_info, &select_list_info);
+        *name_resolver, &query_aggregation_info, &window_aggregation_info, &select_list_info);
     having_predicate = resolvePredicate(
         *select_query.having()->having_predicate(), &expr_resolution_info);
   }
@@ -1053,7 +1143,7 @@ L::LogicalPtr Resolver::resolveSelect(
     for (const ParseOrderByItem &order_by_item :
          *select_query.order_by()->order_by_items()) {
       ExpressionResolutionInfo expr_resolution_info(
-          *name_resolver, &query_aggregation_info, &select_list_info);
+          *name_resolver, &query_aggregation_info, &window_aggregation_info, &select_list_info);
       E::ScalarPtr order_by_scalar = resolveExpression(
           *order_by_item.ordering_expression(),
           nullptr,  // No Type hint.
@@ -1528,6 +1618,89 @@ L::LogicalPtr Resolver::RenameOutputColumns(
   return L::Project::Create(logical_plan, project_expressions);
 }
 
+E::WindowInfo Resolver::resolveWindow(const ParseWindow &parse_window,
+                                      const NameResolver &name_resolver) {
+  std::vector<E::AttributeReferencePtr> partition_by_attributes;
+  std::vector<E::AttributeReferencePtr> order_by_attributes;
+  std::vector<bool> order_by_directions;
+  std::vector<bool> nulls_first;
+  E::WindowFrameInfo *frame_info = nullptr;
+
+  // Resolve PARTITION BY
+  if (parse_window.partition_by_expressions() != nullptr) {
+    for (const ParseExpression &unresolved_partition_by_expression :
+         *parse_window.partition_by_expressions()) {
+      ExpressionResolutionInfo expr_resolution_info(
+          name_resolver,
+          "PARTITION BY clause" /* clause_name */,
+          nullptr /* select_list_info */);
+      E::ScalarPtr partition_by_scalar = resolveExpression(
+          unresolved_partition_by_expression,
+          nullptr,  // No Type hint.
+          &expr_resolution_info);
+
+      if (partition_by_scalar->isConstant()) {
+        THROW_SQL_ERROR_AT(&unresolved_partition_by_expression)
+            << "Constant expression not allowed in PARTITION BY";
+      }
+
+      E::AttributeReferencePtr partition_by_attribute;
+      if (!E::SomeAttributeReference::MatchesWithConditionalCast(partition_by_scalar,
+                                                                 &partition_by_attribute)) {
+        THROW_SQL_ERROR_AT(&unresolved_partition_by_expression)
+            << "Only attribute name allowed in PARTITION BY in window definition";
+      }
+
+      partition_by_attributes.push_back(partition_by_attribute);
+    }
+  }
+
+  // Resolve ORDER BY
+  if (parse_window.order_by_expressions() != nullptr) {
+    for (const ParseOrderByItem &order_by_item :
+         *parse_window.order_by_expressions()) {
+      ExpressionResolutionInfo expr_resolution_info(
+          name_resolver,
+          "ORDER BY clause" /* clause name */,
+          nullptr /* select_list_info */);
+      E::ScalarPtr order_by_scalar = resolveExpression(
+          *order_by_item.ordering_expression(),
+          nullptr,  // No Type hint.
+          &expr_resolution_info);
+
+      if (order_by_scalar->isConstant()) {
+        THROW_SQL_ERROR_AT(&order_by_item)
+            << "Constant expression not allowed in ORDER BY";
+      }
+
+      E::AttributeReferencePtr order_by_attribute;
+      if (!E::SomeAttributeReference::MatchesWithConditionalCast(order_by_scalar,
+                                                                 &order_by_attribute)) {
+        THROW_SQL_ERROR_AT(&order_by_item)
+            << "Only attribute name allowed in ORDER BY in window definition";
+      }
+
+      order_by_attributes.push_back(order_by_attribute);
+      order_by_directions.push_back(order_by_item.is_ascending());
+      nulls_first.push_back(order_by_item.nulls_first());
+    }
+  }
+
+  // Resolve window frame
+  if (parse_window.frame_info() != nullptr) {
+    const quickstep::ParseFrameInfo *parse_frame_info = parse_window.frame_info();
+    frame_info = new E::WindowFrameInfo(parse_frame_info->is_row,
+                                        parse_frame_info->num_preceding,
+                                        parse_frame_info->num_following);
+  }
+
+  return E::WindowInfo(partition_by_attributes,
+                       order_by_attributes,
+                       order_by_directions,
+                       nulls_first,
+                       frame_info);
+}
+
 const CatalogRelation* Resolver::resolveRelationName(
     const ParseString *relation_name) {
   const CatalogRelation *relation =
@@ -1684,13 +1857,45 @@ L::LogicalPtr Resolver::resolveJoinedTableReference(
   THROW_SQL_ERROR_AT(&joined_table_reference) << "Full outer join is not supported yet";
 }
 
+L::LogicalPtr Resolver::resolveSortInWindow(
+    const L::LogicalPtr &logical_plan,
+    const E::WindowInfo &window_info) {
+  // Sort the table by (p_key, o_key)
+  std::vector<E::AttributeReferencePtr> sort_attributes(window_info.partition_by_attributes);
+  sort_attributes.insert(sort_attributes.end(),
+                         window_info.order_by_attributes.begin(),
+                         window_info.order_by_attributes.end());
+
+  std::vector<bool> sort_directions(
+      window_info.partition_by_attributes.size(), true);
+  sort_directions.insert(sort_directions.end(),
+                         window_info.order_by_directions.begin(),
+                         window_info.order_by_directions.end());
+
+  std::vector<bool> sort_nulls_first(
+      window_info.partition_by_attributes.size(), false);
+  sort_nulls_first.insert(sort_nulls_first.end(),
+                          window_info.nulls_first.begin(),
+                          window_info.nulls_first.end());
+
+  L::LogicalPtr sorted_logical_plan =
+      L::Sort::Create(logical_plan,
+                      sort_attributes,
+                      sort_directions,
+                      sort_nulls_first,
+                      -1 /* limit */);
+
+  return sorted_logical_plan;
+}
+
 void Resolver::resolveSelectClause(
     const ParseSelectionClause &parse_selection,
     const std::string &select_name,
     const std::vector<const Type*> *type_hints,
     const NameResolver &name_resolver,
     QueryAggregationInfo *query_aggregation_info,
-    std::vector<expressions::NamedExpressionPtr> *project_expressions,
+    WindowAggregationInfo *window_aggregation_info,
+    std::vector<E::NamedExpressionPtr> *project_expressions,
     std::vector<bool> *has_aggregate_per_expression) {
   project_expressions->clear();
   switch (parse_selection.getSelectionType()) {
@@ -1720,6 +1925,7 @@ void Resolver::resolveSelectClause(
         ExpressionResolutionInfo expr_resolution_info(
             name_resolver,
             query_aggregation_info,
+            window_aggregation_info,
             nullptr /* select_list_info */);
         const E::ScalarPtr project_scalar =
             resolveExpression(*parse_project_expression,
@@ -2362,16 +2568,12 @@ E::ScalarPtr Resolver::resolveSimpleCaseExpression(
 
 // TODO(chasseur): For now this only handles resolving aggregate functions. In
 // the future it should be extended to resolve scalar functions as well.
+// TODO(Shixuan): This will handle resolving window aggregation function as well,
+// which could be extended to general scalar functions.
 E::ScalarPtr Resolver::resolveFunctionCall(
     const ParseFunctionCall &parse_function_call,
     ExpressionResolutionInfo *expression_resolution_info) {
-  std::string function_name = ToLower(parse_function_call.name()->value());
-
-  // TODO(Shixuan): Add support for window aggregation function.
-  if (parse_function_call.isWindow()) {
-    THROW_SQL_ERROR_AT(&parse_function_call)
-        << "Window Aggregation Function is not supported currently";
-  }
+  const std::string function_name = ToLower(parse_function_call.name()->value());
 
   // First check for the special case COUNT(*).
   bool count_star = false;
@@ -2386,8 +2588,9 @@ E::ScalarPtr Resolver::resolveFunctionCall(
   std::vector<E::ScalarPtr> resolved_arguments;
   const PtrList<ParseExpression> *unresolved_arguments =
       parse_function_call.arguments();
-  // The first aggregate function in the arguments.
+  // The first aggregate function and window aggregate function in the arguments.
   const ParseTreeNode *first_aggregate_function = nullptr;
+  const ParseTreeNode *first_window_aggregate_function = nullptr;
   if (unresolved_arguments != nullptr) {
     for (const ParseExpression &unresolved_argument : *unresolved_arguments) {
       ExpressionResolutionInfo expr_resolution_info(
@@ -2401,6 +2604,13 @@ E::ScalarPtr Resolver::resolveFunctionCall(
         first_aggregate_function =
             expr_resolution_info.parse_aggregate_expression;
       }
+
+      if (expr_resolution_info.hasWindowAggregate() &&
+          first_window_aggregate_function == nullptr &&
+          parse_function_call.isWindow()) {
+          first_window_aggregate_function =
+              expr_resolution_info.parse_window_aggregate_expression;
+      }
     }
   }
 
@@ -2431,6 +2641,15 @@ E::ScalarPtr Resolver::resolveFunctionCall(
         << "Aggregation of Aggregates are not allowed";
   }
 
+  // TODO(Shixuan): We currently don't support nested window aggregation since
+  // TPC-DS doesn't have that. However, it is essentially a nested scalar
+  // function, which should be supported in the future version of Quickstep.
+  if (parse_function_call.isWindow() &&
+      first_window_aggregate_function != nullptr) {
+    THROW_SQL_ERROR_AT(first_window_aggregate_function)
+        << "Window aggregation of window aggregates is not allowed";
+  }
+
   // Make sure a naked COUNT() with no arguments wasn't specified.
   if ((aggregate->getAggregationID() == AggregationID::kCount)
       && (resolved_arguments.empty())
@@ -2452,6 +2671,13 @@ E::ScalarPtr Resolver::resolveFunctionCall(
         << " can not apply to the given argument(s).";
   }
 
+  if (parse_function_call.isWindow()) {
+    return resolveWindowAggregateFunction(parse_function_call,
+                                          expression_resolution_info,
+                                          aggregate,
+                                          resolved_arguments);
+  }
+
   // Create the optimizer representation of the resolved aggregate and an alias
   // for it to appear in the output relation.
   const E::AggregateFunctionPtr aggregate_function
@@ -2471,6 +2697,62 @@ E::ScalarPtr Resolver::resolveFunctionCall(
   return E::ToRef(aggregate_alias);
 }
 
+E::ScalarPtr Resolver::resolveWindowAggregateFunction(
+    const ParseFunctionCall &parse_function_call,
+    ExpressionResolutionInfo *expression_resolution_info,
+    const ::quickstep::AggregateFunction *window_aggregate,
+    const std::vector<E::ScalarPtr> &resolved_arguments) {
+  // A window aggregate function might be defined OVER a window name or a window.
+  E::WindowAggregateFunctionPtr window_aggregate_function;
+  if (parse_function_call.window_name() != nullptr) {
+    std::unordered_map<std::string, WindowPlan> window_map
+        = expression_resolution_info->window_aggregation_info->window_map;
+    std::string window_name = parse_function_call.window_name()->value();
+    std::unordered_map<std::string, WindowPlan>::const_iterator map_it
+        = window_map.find(window_name);
+
+    if (map_it == window_map.end()) {
+      THROW_SQL_ERROR_AT(parse_function_call.window_name())
+          << "Undefined window " << window_name;
+    }
+
+    window_aggregate_function =
+        E::WindowAggregateFunction::Create(*window_aggregate,
+                                           resolved_arguments,
+                                           map_it->second.window_info,
+                                           parse_function_call.window_name()->value(),
+                                           parse_function_call.is_distinct());
+  } else {
+    E::WindowInfo resolved_window = resolveWindow(*parse_function_call.window(),
+                                                  expression_resolution_info->name_resolver);
+
+    window_aggregate_function =
+        E::WindowAggregateFunction::Create(*window_aggregate,
+                                           resolved_arguments,
+                                           resolved_window,
+                                           "" /* window name */,
+                                           parse_function_call.is_distinct());
+  }
+
+  const std::string internal_alias = GenerateWindowAggregateAttributeAlias(
+      expression_resolution_info->query_aggregation_info->aggregate_expressions.size());
+  const E::AliasPtr aggregate_alias = E::Alias::Create(context_->nextExprId(),
+                                                       window_aggregate_function,
+                                                       "" /* attribute_name */,
+                                                       internal_alias,
+                                                       "$window_aggregate" /* relation_name */);
+
+  if (!expression_resolution_info->window_aggregation_info->window_aggregate_expressions.empty()) {
+    THROW_SQL_ERROR_AT(&parse_function_call)
+        << "Currently we only support single window aggregate in the query";
+  }
+
+  expression_resolution_info->window_aggregation_info
+      ->window_aggregate_expressions.emplace_back(aggregate_alias);
+  expression_resolution_info->parse_window_aggregate_expression = &parse_function_call;
+  return E::ToRef(aggregate_alias);
+}
+
 std::vector<E::PredicatePtr> Resolver::resolvePredicates(
     const PtrList<ParsePredicate> &parse_predicates,
     ExpressionResolutionInfo *expression_resolution_info) {
@@ -2794,16 +3076,20 @@ void Resolver::rewriteIfOrdinalReference(
   }
 }
 
+std::string Resolver::GenerateWindowAggregateAttributeAlias(int index) {
+  return "$window_aggregate" + std::to_string(index);
+}
+
 std::string Resolver::GenerateAggregateAttributeAlias(int index) {
-  return std::string("$aggregate").append(std::to_string(index));
+  return "$aggregate" + std::to_string(index);
 }
 
 std::string Resolver::GenerateGroupingAttributeAlias(int index) {
-  return std::string("$groupby").append(std::to_string(index));
+  return "$groupby" + std::to_string(index);
 }
 
 std::string Resolver::GenerateOrderingAttributeAlias(int index) {
-  return std::string("$orderby").append(std::to_string(index));
+  return "$orderby" + std::to_string(index);
 }
 
 }  // namespace resolver

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/resolver/Resolver.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.hpp b/query_optimizer/resolver/Resolver.hpp
index a84c61c..f4024e9 100644
--- a/query_optimizer/resolver/Resolver.hpp
+++ b/query_optimizer/resolver/Resolver.hpp
@@ -24,11 +24,13 @@
 #include <vector>
 
 #include "query_optimizer/expressions/AggregateFunction.hpp"
+#include "query_optimizer/expressions/Alias.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/NamedExpression.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
 #include "query_optimizer/expressions/SubqueryExpression.hpp"
 #include "query_optimizer/expressions/Scalar.hpp"
+#include "query_optimizer/expressions/WindowAggregateFunction.hpp"
 #include "query_optimizer/logical/Logical.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
@@ -65,6 +67,7 @@ class ParseSubqueryExpression;
 class ParseTableReference;
 class ParseTableReferenceSignature;
 class ParseTreeNode;
+class ParseWindow;
 template <class T>
 class PtrList;
 class StorageBlockLayoutDescription;
@@ -123,6 +126,17 @@ class Resolver {
   struct QueryAggregationInfo;
 
   /**
+   * @brief Query-scoped info that contains window aggregate expressions and a
+   *        window map.
+   **/
+  struct WindowAggregationInfo;
+
+  /**
+   * @brief A wrapper for resolved window and the corresponding sorted plan.
+   **/
+  struct WindowPlan;
+
+  /**
    * @brief Query-scoped info that contains select-list expressions
    *        and whether an expression is referenced by an
    *        ordinal or alias reference.
@@ -271,6 +285,8 @@ class Resolver {
    * @param name_resolver NameResolver to resolve the relation/attribute names.
    * @param query_aggregation_info Passed down to each expression to collects
    *                               aggregate expressions.
+   * @param window_aggregate_expressions Passed down to each expressions to
+   *                                     collects window aggregate expressions.
    * @param project_expressions Converted SELECT-list expressions.
    * @param has_aggregate_per_expression For each SELECT-list expression,
    *                                     indicates whether it contains
@@ -282,6 +298,7 @@ class Resolver {
       const std::vector<const Type*> *type_hints,
       const NameResolver &name_resolver,
       QueryAggregationInfo *query_aggregation_info,
+      WindowAggregationInfo *window_aggregation_info,
       std::vector<expressions::NamedExpressionPtr> *project_expressions,
       std::vector<bool> *has_aggregate_per_expression);
 
@@ -359,6 +376,17 @@ class Resolver {
       const ParseTableReferenceSignature &table_signature);
 
   /**
+   * @brief Sort the input table in (p_key, o_key) order specified by the window.
+   *
+   * @param logical_plan The input logical node.
+   * @param window_info The window that the input table has to be sorted accordingly.
+   * @return A logical plan that sorts the table according to window_info.
+   **/
+  logical::LogicalPtr resolveSortInWindow(
+      const logical::LogicalPtr &logical_plan,
+      const expressions::WindowInfo &window_info);
+
+  /**
    * @brief Resolves a parse expression and converts it to a scalar expression
    *        in the query optimizer. A non-scalar parse expression is resolved
    *        to an AttributeReference to another optimizer expression.
@@ -412,7 +440,8 @@ class Resolver {
    * @brief Resolves a function call. For a non-scalar function, the returned
    *        expression is an AttributeReference to the actual resolved expression.
    *
-   * @note This currently only handles resolving aggregate functions.
+   * @note This currently only handles resolving aggregate functions and window
+   *       aggregate functions.
    *
    * @param parse_function_call The function call to be resolved.
    * @param expression_resolution_info Resolution info that contains the name
@@ -425,6 +454,23 @@ class Resolver {
       ExpressionResolutionInfo *expression_resolution_info);
 
   /**
+   * @brief Resolves a window aggregate function.
+   *
+   * @param parse_function_call The function call to be resolved.
+   * @param expression_resolution_info Resolution info that contains the name
+   *                                   resolver and info to be updated after
+   *                                   resolution.
+   * @param aggregate The aggregate function.
+   * @param resolved_arguments The resolved arguments.
+   * @return An expression in the query optimizer.
+   */
+  expressions::ScalarPtr resolveWindowAggregateFunction(
+      const ParseFunctionCall &parse_function_call,
+      ExpressionResolutionInfo *expression_resolution_info,
+      const ::quickstep::AggregateFunction *aggregate,
+      const std::vector<expressions::ScalarPtr> &resolved_arguments);
+
+  /**
    * @brief Resolves a parse Predicate and converts it to a predicate in the
    *        query optimizer.
    *
@@ -469,6 +515,15 @@ class Resolver {
       const bool has_single_column);
 
   /**
+   * @brief Resolves a window definition.
+   *
+   * @param parse_window The parsed window definition.
+   * @param name_resolver The resolver to resolve names.
+   **/
+  expressions::WindowInfo resolveWindow(const ParseWindow &parse_window,
+                                        const NameResolver &name_resolver);
+
+  /**
    * @brief Resolves a relation name to a pointer to the corresponding
    *        CatalogRelation with the name.
    *
@@ -501,6 +556,15 @@ class Resolver {
   static std::string GenerateAggregateAttributeAlias(int index);
 
   /**
+   * @brief Generates an internal alias for a window aggregate attribute.
+   *
+   * @param index The index of the window aggregate attribute used for
+   *              generating the name.
+   * @return A string for the name.
+   */
+  static std::string GenerateWindowAggregateAttributeAlias(int index);
+
+  /**
    * @brief Generates an internal alias for a grouping attribute.
    *
    * @param index The index of the grouping attribute used for generating the

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/strategy/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/CMakeLists.txt b/query_optimizer/strategy/CMakeLists.txt
index 74f5a4b..84e151e 100644
--- a/query_optimizer/strategy/CMakeLists.txt
+++ b/query_optimizer/strategy/CMakeLists.txt
@@ -105,7 +105,8 @@ target_link_libraries(quickstep_queryoptimizer_strategy_OneToOne
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_UpdateTable
                       quickstep_queryoptimizer_strategy_Strategy
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      quickstep_utility_SqlError)
 target_link_libraries(quickstep_queryoptimizer_strategy_Selection
                       glog
                       quickstep_queryoptimizer_LogicalToPhysicalMapper

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/strategy/OneToOne.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/OneToOne.cpp b/query_optimizer/strategy/OneToOne.cpp
index 7f59151..f49a25c 100644
--- a/query_optimizer/strategy/OneToOne.cpp
+++ b/query_optimizer/strategy/OneToOne.cpp
@@ -55,6 +55,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 #include "query_optimizer/physical/UpdateTable.hpp"
+#include "utility/SqlError.hpp"
 
 namespace quickstep {
 namespace optimizer {
@@ -208,6 +209,10 @@ bool OneToOne::generatePlan(const L::LogicalPtr &logical_input,
           update_table->predicate());
       return true;
     }
+    case L::LogicalType::kWindowAggregate: {
+      THROW_SQL_ERROR()
+          << "Window aggregate function is not supported currently :(";
+    }
     default:
       return false;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/tests/logical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Select.test b/query_optimizer/tests/logical_generator/Select.test
index 3c152e8..e0003bf 100644
--- a/query_optimizer/tests/logical_generator/Select.test
+++ b/query_optimizer/tests/logical_generator/Select.test
@@ -1354,3 +1354,165 @@ TopLevelPlan
 +-output_attributes=
   +-AttributeReference[id=5,name=x,relation=,type=Int]
   +-AttributeReference[id=6,name=y,relation=,type=Int]
+==
+
+# Window Aggregate Function Test.
+SELECT avg(int_col) OVER w FROM test
+WINDOW w AS
+(PARTITION BY char_col
+ ORDER BY long_col DESC NULLS LAST
+ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW);
+--
+TopLevelPlan
++-plan=Project
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,false],nulls_first=[false,false]]
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_expressions=
+| | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=AVG,window_name=w,is_ascending=[false],
+| |     nulls_first=[false],frame_mode=row,num_preceding=-1,num_following=0]
+| |     +-arguments=
+| |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| |     +-order_by=
+| |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+==
+
+SELECT int_col, sum(float_col) OVER
+(PARTITION BY vchar_col, long_col
+ ORDER BY double_col DESC NULLS LAST, int_col ASC NULLS FIRST
+ RANGE BETWEEN 3 PRECEDING AND 3 FOLLOWING)
+FROM test;
+--
+TopLevelPlan
++-plan=Project
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,true,false,true],
+| | | nulls_first=[false,false,false,true]]
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_expressions=
+| | |   +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   | type=VarChar(20) NULL]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | |   +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=SUM,window_name=,
+| |     is_ascending=[false,true],nulls_first=[false,true],frame_mode=range,
+| |     num_preceding=3,num_following=3]
+| |     +-arguments=
+| |     | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| |     | | type=VarChar(20) NULL]
+| |     | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |     +-order_by=
+| |       +-AttributeReference[id=3,name=double_col,relation=test,
+| |       | type=Double NULL]
+| |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| +-project_list=
+|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   +-Alias[id=6,name=,alias=sum(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+  +-AttributeReference[id=6,name=,alias=sum(float_col),relation=,
+    type=Double NULL]
+==
+
+SELECT sum(avg(int_col) OVER w) FROM test
+WINDOW w AS
+(PARTITION BY char_col
+ ORDER BY long_col
+ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW);
+--
+TopLevelPlan
++-plan=Project
+| +-input=Aggregate
+| | +-input=WindowAggregate
+| | | +-input=Sort[is_ascending=[true,true],nulls_first=[false,false]]
+| | | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | | +-AttributeReference[id=3,name=double_col,relation=test,
+| | | | | | type=Double NULL]
+| | | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | | |   type=VarChar(20) NULL]
+| | | | +-sort_expressions=
+| | | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| | |   relation=$window_aggregate,type=Double NULL]
+| | |   +-WindowAggregateFunction[function=AVG,window_name=w,
+| | |     is_ascending=[true],nulls_first=[false],frame_mode=row,
+| | |     num_preceding=-1,num_following=0]
+| | |     +-arguments=
+| | |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |     +-partition_by=
+| | |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |     +-order_by=
+| | |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,type=Double NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+| |         relation=$window_aggregate,type=Double NULL]
+| +-project_list=
+|   +-Alias[id=7,name=,alias=sum(avg(int_col)),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=7,name=,alias=sum(avg(int_col)),relation=,
+    type=Double NULL]
+==
+
+SELECT int_col, sum(float_col) OVER w1 FROM test
+WINDOW w2 AS
+(PARTITION BY vchar_col, long_col
+ ORDER BY double_col DESC NULLS LAST, int_col ASC NULLS FIRST
+ RANGE BETWEEN 3 PRECEDING AND 3 FOLLOWING);
+--
+ERROR: Undefined window w1 (1 : 37)
+SELECT int_col, sum(float_col) OVER w1 FROM test
+                                    ^
+==
+
+SELECT sum(avg(int_col)) OVER w FROM test
+WINDOW w AS
+(PARTITION BY double_col
+ ORDER BY char_col)
+--
+ERROR: Aggregation of Aggregates are not allowed (1 : 12)
+SELECT sum(avg(int_col)) OVER w FROM test
+           ^

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/714874ce/query_optimizer/tests/resolver/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Select.test b/query_optimizer/tests/resolver/Select.test
index 141bfa0..89ab84d 100644
--- a/query_optimizer/tests/resolver/Select.test
+++ b/query_optimizer/tests/resolver/Select.test
@@ -3126,3 +3126,165 @@ FROM test;
 ERROR: The substring length must be greater than 0 (1 : 8)
 SELECT SUBSTRING(char_col FROM 1 FOR ...
        ^
+==
+
+# Window Aggregate Function Test.
+SELECT avg(int_col) OVER w FROM test
+WINDOW w AS
+(PARTITION BY char_col
+ ORDER BY long_col DESC NULLS LAST
+ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW);
+--
+TopLevelPlan
++-plan=Project
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,false],nulls_first=[false,false]]
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_expressions=
+| | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=AVG,window_name=w,is_ascending=[false],
+| |     nulls_first=[false],frame_mode=row,num_preceding=-1,num_following=0]
+| |     +-arguments=
+| |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| |     +-order_by=
+| |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+==
+
+SELECT int_col, sum(float_col) OVER
+(PARTITION BY vchar_col, long_col
+ ORDER BY double_col DESC NULLS LAST, int_col ASC NULLS FIRST
+ RANGE BETWEEN 3 PRECEDING AND 3 FOLLOWING)
+FROM test;
+--
+TopLevelPlan
++-plan=Project
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,true,false,true],
+| | | nulls_first=[false,false,false,true]]
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_expressions=
+| | |   +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   | type=VarChar(20) NULL]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | |   +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=SUM,window_name=,
+| |     is_ascending=[false,true],nulls_first=[false,true],frame_mode=range,
+| |     num_preceding=3,num_following=3]
+| |     +-arguments=
+| |     | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| |     | | type=VarChar(20) NULL]
+| |     | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |     +-order_by=
+| |       +-AttributeReference[id=3,name=double_col,relation=test,
+| |       | type=Double NULL]
+| |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| +-project_list=
+|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   +-Alias[id=6,name=,alias=sum(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+  +-AttributeReference[id=6,name=,alias=sum(float_col),relation=,
+    type=Double NULL]
+==
+
+SELECT sum(avg(int_col) OVER w) FROM test
+WINDOW w AS
+(PARTITION BY char_col
+ ORDER BY long_col
+ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW);
+--
+TopLevelPlan
++-plan=Project
+| +-input=Aggregate
+| | +-input=WindowAggregate
+| | | +-input=Sort[is_ascending=[true,true],nulls_first=[false,false]]
+| | | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | | +-AttributeReference[id=3,name=double_col,relation=test,
+| | | | | | type=Double NULL]
+| | | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | | |   type=VarChar(20) NULL]
+| | | | +-sort_expressions=
+| | | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| | |   relation=$window_aggregate,type=Double NULL]
+| | |   +-WindowAggregateFunction[function=AVG,window_name=w,
+| | |     is_ascending=[true],nulls_first=[false],frame_mode=row,
+| | |     num_preceding=-1,num_following=0]
+| | |     +-arguments=
+| | |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |     +-partition_by=
+| | |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |     +-order_by=
+| | |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,type=Double NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+| |         relation=$window_aggregate,type=Double NULL]
+| +-project_list=
+|   +-Alias[id=7,name=,alias=sum(avg(int_col)),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=7,name=,alias=sum(avg(int_col)),relation=,
+    type=Double NULL]
+==
+
+SELECT int_col, sum(float_col) OVER w1 FROM test
+WINDOW w2 AS
+(PARTITION BY vchar_col, long_col
+ ORDER BY double_col DESC NULLS LAST, int_col ASC NULLS FIRST
+ RANGE BETWEEN 3 PRECEDING AND 3 FOLLOWING);
+--
+ERROR: Undefined window w1 (1 : 37)
+SELECT int_col, sum(float_col) OVER w1 FROM test
+                                    ^
+==
+
+SELECT sum(avg(int_col)) OVER w FROM test
+WINDOW w AS
+(PARTITION BY double_col
+ ORDER BY char_col)
+--
+ERROR: Aggregation of Aggregates are not allowed (1 : 12)
+SELECT sum(avg(int_col)) OVER w FROM test
+           ^


[05/50] [abbrv] incubator-quickstep git commit: Reordered Query ID in operators and work orders.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index f165442..0b13842 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -59,6 +59,7 @@ class NestedLoopsJoinOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param left_input_relation The first relation in the join (order is not
    *        actually important).
    * @param right_input_relation The second relation in the join (order is not
@@ -76,17 +77,17 @@ class NestedLoopsJoinOperator : public RelationalOperator {
    * @param left_relation_is_stored If left_input_relation is a stored relation.
    * @param right_relation_is_stored If right_input_relation is a stored
    *                                 relation.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  NestedLoopsJoinOperator(const CatalogRelation &left_input_relation,
-                          const CatalogRelation &right_input_relation,
-                          const CatalogRelation &output_relation,
-                          const QueryContext::insert_destination_id output_destination_index,
-                          const QueryContext::predicate_id join_predicate_index,
-                          const QueryContext::scalar_group_id selection_index,
-                          bool left_relation_is_stored,
-                          bool right_relation_is_stored,
-                          const std::size_t query_id)
+  NestedLoopsJoinOperator(
+      const std::size_t query_id,
+      const CatalogRelation &left_input_relation,
+      const CatalogRelation &right_input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::predicate_id join_predicate_index,
+      const QueryContext::scalar_group_id selection_index,
+      bool left_relation_is_stored,
+      bool right_relation_is_stored)
       : RelationalOperator(query_id),
         left_input_relation_(left_input_relation),
         right_input_relation_(right_input_relation),
@@ -96,10 +97,12 @@ class NestedLoopsJoinOperator : public RelationalOperator {
         selection_index_(selection_index),
         left_relation_is_stored_(left_relation_is_stored),
         right_relation_is_stored_(right_relation_is_stored),
-        left_relation_block_ids_(left_relation_is_stored ? left_input_relation.getBlocksSnapshot()
-                                                         : std::vector<block_id>()),
-        right_relation_block_ids_(right_relation_is_stored ? right_input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        left_relation_block_ids_(left_relation_is_stored
+                                     ? left_input_relation.getBlocksSnapshot()
+                                     : std::vector<block_id>()),
+        right_relation_block_ids_(right_relation_is_stored
+                                      ? right_input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_left_workorders_generated_(0),
         num_right_workorders_generated_(0),
         done_feeding_left_relation_(false),
@@ -222,6 +225,7 @@ class NestedLoopsJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param left_input_relation The first relation in the join (order is not
    *        actually important).
    * @param right_input_relation The second relation in the join (order is not
@@ -233,19 +237,19 @@ class NestedLoopsJoinWorkOrder : public WorkOrder {
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
-   * @param query_id The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  NestedLoopsJoinWorkOrder(const CatalogRelationSchema &left_input_relation,
-                           const CatalogRelationSchema &right_input_relation,
-                           const block_id left_block_id,
-                           const block_id right_block_id,
-                           const Predicate *join_predicate,
-                           const std::vector<std::unique_ptr<const Scalar>> &selection,
-                           const std::size_t query_id,
-                           InsertDestination *output_destination,
-                           StorageManager *storage_manager)
+  NestedLoopsJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &left_input_relation,
+      const CatalogRelationSchema &right_input_relation,
+      const block_id left_block_id,
+      const block_id right_block_id,
+      const Predicate *join_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         left_input_relation_(left_input_relation),
         right_input_relation_(right_input_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index ae876ba..86f8eaf 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -48,6 +48,7 @@ class RebuildWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this RebuildWorkOrder belongs.
    * @param block_ref A MutableBlockReference to the storage block which needs
    *        to be rebuilt.
    * @param input_operator_index The index of the relational operator in the
@@ -55,15 +56,15 @@ class RebuildWorkOrder : public WorkOrder {
    * @param input_relation_id The ID of the CatalogRelation to which the given
    *        storage block belongs to.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
-   * @param query_id The ID of the query to which this RebuildWorkOrder belongs.
    * @param bus A pointer to the TMB.
    **/
-  RebuildWorkOrder(MutableBlockReference &&block_ref,
-                   const std::size_t input_operator_index,
-                   const relation_id input_relation_id,
-                   const client_id scheduler_client_id,
-                   const std::size_t query_id,
-                   MessageBus *bus)
+  RebuildWorkOrder(
+      const std::size_t query_id,
+      MutableBlockReference &&block_ref,  // NOLINT(whitespace/operators)
+      const std::size_t input_operator_index,
+      const relation_id input_relation_id,
+      const client_id scheduler_client_id,
+      MessageBus *bus)
       : WorkOrder(query_id),
         block_ref_(std::move(block_ref)),
         input_operator_index_(input_operator_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index aa93018..c173a0a 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -212,16 +212,17 @@ class RelationalOperator {
    * @param blocking_dependencies_met If those dependencies which break the
    *        pipeline have been met.
    **/
-  explicit RelationalOperator(const std::size_t query_id = 0,
+  explicit RelationalOperator(const std::size_t query_id,
                               const bool blocking_dependencies_met = false)
-      : blocking_dependencies_met_(blocking_dependencies_met),
-        done_feeding_input_relation_(false),
-        query_id_(query_id) {}
+      : query_id_(query_id),
+        blocking_dependencies_met_(blocking_dependencies_met),
+        done_feeding_input_relation_(false) {}
+
+  const std::size_t query_id_;
 
   bool blocking_dependencies_met_;
   bool done_feeding_input_relation_;
   std::size_t op_index_;
-  const std::size_t query_id_;
 
  private:
   DISALLOW_COPY_AND_ASSIGN(RelationalOperator);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SampleOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.cpp b/relational_operators/SampleOperator.cpp
index b318ce4..8d5fade 100644
--- a/relational_operators/SampleOperator.cpp
+++ b/relational_operators/SampleOperator.cpp
@@ -54,11 +54,11 @@ bool SampleOperator::getAllWorkOrders(
         for (const block_id input_block_id : input_relation_block_ids_) {
           if (distribution(generator) <= probability) {
             container->addNormalWorkOrder(
-                new SampleWorkOrder(input_relation_,
+                new SampleWorkOrder(query_id_,
+                                    input_relation_,
                                     input_block_id,
                                     is_block_sample_,
                                     percentage_,
-                                    query_id_,
                                     output_destination,
                                     storage_manager),
                 op_index_);
@@ -68,15 +68,14 @@ bool SampleOperator::getAllWorkOrders(
         // Add all the blocks for tuple sampling which would handle
         // the sampling from each block
         for (const block_id input_block_id : input_relation_block_ids_) {
-          container->addNormalWorkOrder(
-              new SampleWorkOrder(input_relation_,
-                                  input_block_id,
-                                  is_block_sample_,
-                                  percentage_,
-                                  query_id_,
-                                  output_destination,
-                                  storage_manager),
-              op_index_);
+          container->addNormalWorkOrder(new SampleWorkOrder(query_id_,
+                                                            input_relation_,
+                                                            input_block_id,
+                                                            is_block_sample_,
+                                                            percentage_,
+                                                            output_destination,
+                                                            storage_manager),
+                                        op_index_);
         }
       }
       started_ = true;
@@ -87,13 +86,14 @@ bool SampleOperator::getAllWorkOrders(
           while (num_workorders_generated_ < input_relation_block_ids_.size()) {
             if (distribution(generator) <= probability) {
               container->addNormalWorkOrder(
-                  new SampleWorkOrder(input_relation_,
-                                      input_relation_block_ids_[num_workorders_generated_],
-                                      is_block_sample_,
-                                      percentage_,
-                                      query_id_,
-                                      output_destination,
-                                      storage_manager),
+                  new SampleWorkOrder(
+                      query_id_,
+                      input_relation_,
+                      input_relation_block_ids_[num_workorders_generated_],
+                      is_block_sample_,
+                      percentage_,
+                      output_destination,
+                      storage_manager),
                   op_index_);
             ++num_workorders_generated_;
           }
@@ -101,13 +101,14 @@ bool SampleOperator::getAllWorkOrders(
       } else  {
         while (num_workorders_generated_ < input_relation_block_ids_.size()) {
           container->addNormalWorkOrder(
-              new SampleWorkOrder(input_relation_,
-                                  input_relation_block_ids_[num_workorders_generated_],
-                                  is_block_sample_,
-                                  percentage_,
-                                  query_id_,
-                                  output_destination,
-                                  storage_manager),
+              new SampleWorkOrder(
+                  query_id_,
+                  input_relation_,
+                  input_relation_block_ids_[num_workorders_generated_],
+                  is_block_sample_,
+                  percentage_,
+                  output_destination,
+                  storage_manager),
               op_index_);
           ++num_workorders_generated_;
         }
@@ -115,6 +116,7 @@ bool SampleOperator::getAllWorkOrders(
     return done_feeding_input_relation_;
   }
 }
+
 void SampleWorkOrder::execute() {
   BlockReference block(
       storage_manager_->getBlock(input_block_id_, input_relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index f65f28a..505daa2 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -54,8 +54,9 @@ class WorkOrdersContainer;
 class SampleOperator : public RelationalOperator {
  public:
   /**
-   * @brief Constructor for SampleOperator  with the sampling percentage and type of sampling.
+   * @brief Constructor for SampleOperator with the sampling percentage and type of sampling.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to perform sampling over.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -65,15 +66,15 @@ class SampleOperator : public RelationalOperator {
    *        workorders.
    * @param is_block_sample Flag indicating whether the sample type is block or tuple.
    * @param percentage The percentage of data to be sampled.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SampleOperator(const CatalogRelation &input_relation,
-                 const CatalogRelationSchema &output_relation,
-                 const QueryContext::insert_destination_id output_destination_index,
-                 const bool input_relation_is_stored,
-                 const bool is_block_sample,
-                 const int percentage,
-                 const std::size_t query_id)
+  SampleOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelationSchema &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const bool input_relation_is_stored,
+      const bool is_block_sample,
+      const int percentage)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
@@ -81,8 +82,9 @@ class SampleOperator : public RelationalOperator {
         input_relation_is_stored_(input_relation_is_stored),
         is_block_sample_(is_block_sample),
         percentage_(percentage),
-        input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        input_relation_block_ids_(input_relation_is_stored
+                                      ? input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         started_(false) {}
 
@@ -133,11 +135,22 @@ class SampleOperator : public RelationalOperator {
  **/
 class SampleWorkOrder : public WorkOrder {
  public:
-  SampleWorkOrder(const CatalogRelationSchema &input_relation,
+  /**
+   * @brief Constructor.
+   *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
+   * @param input_relation The relation to perform sampling over.
+   * @param input_block_id The block to sample.
+   * @param is_block_sample Flag indicating whether the sample type is block or tuple.
+   * @param percentage The percentage of data to be sampled.
+   * @param output_destination The InsertDestination to insert the sample results.
+   * @param storage_manager The StorageManager to use.
+   **/
+  SampleWorkOrder(const std::size_t query_id,
+                  const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const bool is_block_sample,
                   const int percentage,
-                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager)
       : WorkOrder(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index 3581090..8127d88 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -36,9 +36,9 @@ bool SaveBlocksOperator::getAllWorkOrders(
   while (num_workorders_generated_ < destination_block_ids_.size()) {
     container->addNormalWorkOrder(
         new SaveBlocksWorkOrder(
+            query_id_,
             destination_block_ids_[num_workorders_generated_],
             force_,
-            query_id_,
             storage_manager),
         op_index_);
     ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index f4650bb..6e2c72b 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -55,7 +55,8 @@ class SaveBlocksOperator : public RelationalOperator {
    * @param force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
    **/
-  explicit SaveBlocksOperator(const std::size_t query_id, bool force = false)
+  explicit SaveBlocksOperator(const std::size_t query_id,
+                              const bool force = false)
       : RelationalOperator(query_id),
         force_(force),
         num_workorders_generated_(0) {}
@@ -96,15 +97,15 @@ class SaveBlocksWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param save_block_id The id of the block to save.
    * @param force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
-   * @param query_id The ID of the query to which this operator belongs.
    * @param storage_manager The StorageManager to use.
    **/
-  SaveBlocksWorkOrder(const block_id save_block_id,
+  SaveBlocksWorkOrder(const std::size_t query_id,
+                      const block_id save_block_id,
                       const bool force,
-                      const std::size_t query_id,
                       StorageManager *storage_manager)
       : WorkOrder(query_id),
         save_block_id_(save_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index 350890d..e9a96f3 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -42,29 +42,28 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
                                    InsertDestination *output_destination) {
   if (input_relation_is_stored_) {
     for (const block_id input_block_id : input_relation_block_ids_) {
-      container->addNormalWorkOrder(
-          new SelectWorkOrder(input_relation_,
-                              input_block_id,
-                              predicate,
-                              simple_projection_,
-                              simple_selection_,
-                              selection,
-                              query_id_,
-                              output_destination,
-                              storage_manager),
-          op_index_);
+      container->addNormalWorkOrder(new SelectWorkOrder(query_id_,
+                                                        input_relation_,
+                                                        input_block_id,
+                                                        predicate,
+                                                        simple_projection_,
+                                                        simple_selection_,
+                                                        selection,
+                                                        output_destination,
+                                                        storage_manager),
+                                    op_index_);
     }
   } else {
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
           new SelectWorkOrder(
+              query_id_,
               input_relation_,
               input_relation_block_ids_[num_workorders_generated_],
               predicate,
               simple_projection_,
               simple_selection_,
               selection,
-              query_id_,
               output_destination,
               storage_manager),
           op_index_);
@@ -87,13 +86,13 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
            input_relation_block_ids_in_partition_[part_id]) {
         container->addNormalWorkOrder(
             new SelectWorkOrder(
+                query_id_,
                 input_relation_,
                 input_block_id,
                 predicate,
                 simple_projection_,
                 simple_selection_,
                 selection,
-                query_id_,
                 output_destination,
                 storage_manager,
                 placement_scheme_->getNUMANodeForBlock(input_block_id)),
@@ -108,13 +107,13 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
             = input_relation_block_ids_in_partition_[part_id][num_workorders_generated_in_partition_[part_id]];
         container->addNormalWorkOrder(
             new SelectWorkOrder(
+                query_id_,
                 input_relation_,
                 block_in_partition,
                 predicate,
                 simple_projection_,
                 simple_selection_,
                 selection,
-                query_id_,
                 output_destination,
                 storage_manager,
                 placement_scheme_->getNUMANodeForBlock(block_in_partition)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 4f5b8ca..ac7b038 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -64,6 +64,7 @@ class SelectOperator : public RelationalOperator {
    * @brief Constructor for selection with arbitrary expressions in projection
    *        list.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to perform selection over.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -76,23 +77,24 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SelectOperator(const CatalogRelation &input_relation,
-                 const CatalogRelation &output_relation,
-                 const QueryContext::insert_destination_id output_destination_index,
-                 const QueryContext::predicate_id predicate_index,
-                 const QueryContext::scalar_group_id selection_index,
-                 const bool input_relation_is_stored,
-                 const std::size_t query_id)
+  SelectOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::predicate_id predicate_index,
+      const QueryContext::scalar_group_id selection_index,
+      const bool input_relation_is_stored)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         predicate_index_(predicate_index),
         selection_index_(selection_index),
-        input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        input_relation_block_ids_(input_relation_is_stored
+                                      ? input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         simple_projection_(false),
         input_relation_is_stored_(input_relation_is_stored),
@@ -124,6 +126,7 @@ class SelectOperator : public RelationalOperator {
    *
    * @note selection_index_ is invalid, and will not be used for projection.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to perform selection over.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -136,15 +139,15 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SelectOperator(const CatalogRelation &input_relation,
-                 const CatalogRelation &output_relation,
-                 const QueryContext::insert_destination_id output_destination_index,
-                 const QueryContext::predicate_id predicate_index,
-                 std::vector<attribute_id> &&selection,
-                 const bool input_relation_is_stored,
-                 const std::size_t query_id)
+  SelectOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::predicate_id predicate_index,
+      std::vector<attribute_id> &&selection,
+      const bool input_relation_is_stored)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
@@ -152,8 +155,9 @@ class SelectOperator : public RelationalOperator {
         predicate_index_(predicate_index),
         selection_index_(QueryContext::kInvalidScalarGroupId),
         simple_selection_(std::move(selection)),
-        input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        input_relation_block_ids_(input_relation_is_stored
+                                      ? input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         simple_projection_(true),
         input_relation_is_stored_(input_relation_is_stored),
@@ -278,6 +282,7 @@ class SelectWorkOrder : public WorkOrder {
    * @note Reference parameter selection is NOT owned by this class and must
    *       remain valid until after execute() is called.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_relation The relation to perform selection over.
    * @param input_block_id The block id.
    * @param predicate All tuples matching \c predicate will be selected (or NULL
@@ -287,18 +292,17 @@ class SelectWorkOrder : public WorkOrder {
    *        simple_projection is true.
    * @param selection A list of Scalars which will be evaluated to project
    *        input tuples, used if \c simple_projection is false.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the selection
    *        results.
    * @param storage_manager The StorageManager to use.
    **/
-  SelectWorkOrder(const CatalogRelationSchema &input_relation,
+  SelectWorkOrder(const std::size_t query_id,
+                  const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const Predicate *predicate,
                   const bool simple_projection,
                   const std::vector<attribute_id> &simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
-                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
                   const numa_node_id numa_node = 0)
@@ -320,6 +324,7 @@ class SelectWorkOrder : public WorkOrder {
    * @note Reference parameter selection is NOT owned by this class and must
    *       remain valid until after execute() is called.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_relation The relation to perform selection over.
    * @param input_block_id The block id.
    * @param predicate All tuples matching \c predicate will be selected (or NULL
@@ -329,18 +334,17 @@ class SelectWorkOrder : public WorkOrder {
    *        simple_projection is true.
    * @param selection A list of Scalars which will be evaluated to project
    *        input tuples, used if \c simple_projection is false.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the selection
    *        results.
    * @param storage_manager The StorageManager to use.
    **/
-  SelectWorkOrder(const CatalogRelationSchema &input_relation,
+  SelectWorkOrder(const std::size_t query_id,
+                  const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const Predicate *predicate,
                   const bool simple_projection,
                   std::vector<attribute_id> &&simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
-                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
                   const numa_node_id numa_node = 0)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.cpp b/relational_operators/SortMergeRunOperator.cpp
index 9db8de1..6bf5719 100644
--- a/relational_operators/SortMergeRunOperator.cpp
+++ b/relational_operators/SortMergeRunOperator.cpp
@@ -85,12 +85,12 @@ WorkOrder *SortMergeRunOperator::createWorkOrder(
 
   // Create a work order from the merge job from merge tree.
   return new SortMergeRunWorkOrder(
+      query_id_,
       query_context->getSortConfig(sort_config_index_),
       job->level > 0 ? run_relation_ : input_relation_,
       std::move(job->runs),
       top_k_,
       job->level,
-      query_id_,
       output_destination,
       storage_manager,
       op_index_,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index f54e925..cfff8b9 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -72,6 +72,7 @@ class SortMergeRunOperator : public RelationalOperator {
   /**
    * @brief Constructor for merging sorted runs to generate a sorted relation.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to merge sorted blocks.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -88,18 +89,18 @@ class SortMergeRunOperator : public RelationalOperator {
    *              \c top_k is 0.
    * @param input_relation_is_stored Boolean to indicate is input relation is
    *                                 stored or streamed.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SortMergeRunOperator(const CatalogRelation &input_relation,
-                       const CatalogRelation &output_relation,
-                       const QueryContext::insert_destination_id output_destination_index,
-                       const CatalogRelation &run_relation,
-                       const QueryContext::insert_destination_id run_block_destination_index,
-                       const QueryContext::sort_config_id sort_config_index,
-                       const std::size_t merge_factor,
-                       const std::size_t top_k,
-                       const bool input_relation_is_stored,
-                       const std::size_t query_id)
+  SortMergeRunOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const CatalogRelation &run_relation,
+      const QueryContext::insert_destination_id run_block_destination_index,
+      const QueryContext::sort_config_id sort_config_index,
+      const std::size_t merge_factor,
+      const std::size_t top_k,
+      const bool input_relation_is_stored)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
@@ -214,12 +215,12 @@ class SortMergeRunWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param sort_config The Sort configuration.
    * @param run_relation The relation to which the run blocks belong to.
    * @param input_runs Input runs to merge.
    * @param top_k If non-zero will merge only \c top_k tuples.
    * @param merge_level Merge level in the merge tree.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to create new blocks.
    * @param storage_manager The StorageManager to use.
    * @param operator_index Merge-run operator index to send feedback messages
@@ -228,12 +229,12 @@ class SortMergeRunWorkOrder : public WorkOrder {
    * @param bus TMB to send the feedback message on.
    **/
   SortMergeRunWorkOrder(
+      const std::size_t query_id,
       const SortConfiguration &sort_config,
       const CatalogRelationSchema &run_relation,
       std::vector<merge_run_operator::Run> &&input_runs,
       const std::size_t top_k,
       const std::size_t merge_level,
-      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
       const std::size_t operator_index,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SortRunGenerationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.cpp b/relational_operators/SortRunGenerationOperator.cpp
index e352f9e..37b8fb8 100644
--- a/relational_operators/SortRunGenerationOperator.cpp
+++ b/relational_operators/SortRunGenerationOperator.cpp
@@ -51,10 +51,10 @@ bool SortRunGenerationOperator::getAllWorkOrders(
     if (!started_) {
       for (const block_id input_block_id : input_relation_block_ids_) {
         container->addNormalWorkOrder(
-            new SortRunGenerationWorkOrder(input_relation_,
+            new SortRunGenerationWorkOrder(query_id_,
+                                           input_relation_,
                                            input_block_id,
                                            sort_config,
-                                           query_id_,
                                            output_destination,
                                            storage_manager),
             op_index_);
@@ -67,10 +67,10 @@ bool SortRunGenerationOperator::getAllWorkOrders(
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
           new SortRunGenerationWorkOrder(
+              query_id_,
               input_relation_,
               input_relation_block_ids_[num_workorders_generated_],
               sort_config,
-              query_id_,
               output_destination,
               storage_manager),
           op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index 3da9813..f96e6a6 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -73,6 +73,7 @@ class SortRunGenerationOperator : public RelationalOperator {
    * @brief Constructor for sorting tuples in blocks based on the sort
    * configuration and writing to output destination.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to generate sorted runs of.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -83,21 +84,22 @@ class SortRunGenerationOperator : public RelationalOperator {
    * @param input_relation_is_stored Does the input relation contain the blocks
    *                                 to sort. If \c false, the blocks are
    *                                 streamed.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SortRunGenerationOperator(const CatalogRelation &input_relation,
-                            const CatalogRelation &output_relation,
-                            const QueryContext::insert_destination_id output_destination_index,
-                            const QueryContext::sort_config_id sort_config_index,
-                            bool input_relation_is_stored,
-                            const std::size_t query_id)
+  SortRunGenerationOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::sort_config_id sort_config_index,
+      bool input_relation_is_stored)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         sort_config_index_(sort_config_index),
-        input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        input_relation_block_ids_(input_relation_is_stored
+                                      ? input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         started_(false),
         input_relation_is_stored_(input_relation_is_stored) {}
@@ -151,19 +153,19 @@ class SortRunGenerationWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_relation The relation to generate sorted runs of.
    * @param input_block_id The block id.
    * @param sort_config The Sort configuration specifying ORDER BY, ordering,
    *        and null ordering.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to store the sorted blocks
    *        of runs.
    * @param storage_manager The StorageManager to use.
    **/
-  SortRunGenerationWorkOrder(const CatalogRelationSchema &input_relation,
+  SortRunGenerationWorkOrder(const std::size_t query_id,
+                             const CatalogRelationSchema &input_relation,
                              const block_id input_block_id,
                              const SortConfiguration &sort_config,
-                             const std::size_t query_id,
                              InsertDestination *output_destination,
                              StorageManager *storage_manager)
       : WorkOrder(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/TableGeneratorOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.cpp b/relational_operators/TableGeneratorOperator.cpp
index fb1f743..a3f9340 100644
--- a/relational_operators/TableGeneratorOperator.cpp
+++ b/relational_operators/TableGeneratorOperator.cpp
@@ -43,9 +43,9 @@ bool TableGeneratorOperator::getAllWorkOrders(
     // so just produce one work order.
     container->addNormalWorkOrder(
         new TableGeneratorWorkOrder(
+            query_id_,
             query_context->getGeneratorFunctionHandle(
                 generator_function_index_),
-            query_id_,
             query_context->getInsertDestination(output_destination_index_)),
         op_index_);
     started_ = true;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index bfc70c5..6a6af4b 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -55,23 +55,23 @@ class TableGeneratorOperator : public RelationalOperator {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert the generated output.
    * @param generator_function_index The index of the GeneratorFunctionHandle in
    *        the QueryContext.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  TableGeneratorOperator(const CatalogRelation &output_relation,
-                         const QueryContext::insert_destination_id output_destination_index,
-                         const QueryContext::generator_function_id generator_function_index,
-                         const std::size_t query_id)
+  TableGeneratorOperator(
+      const std::size_t query_id,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::generator_function_id generator_function_index)
       : RelationalOperator(query_id),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         generator_function_index_(generator_function_index),
-        started_(false) {
-  }
+        started_(false) {}
 
   ~TableGeneratorOperator() override {}
 
@@ -113,13 +113,13 @@ class TableGeneratorWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
-   * @param generator_function The GeneratorFunctionHandle to use.
    * @param query_id The ID of the query to which this WorkOrder belongs.
+   * @param generator_function The GeneratorFunctionHandle to use.
    * @param output_destination The InsertDestination to insert the generated
    *        output.
    **/
-  TableGeneratorWorkOrder(const GeneratorFunctionHandle &function_handle,
-                          const std::size_t query_id,
+  TableGeneratorWorkOrder(const std::size_t query_id,
+                          const GeneratorFunctionHandle &function_handle,
                           InsertDestination *output_destination)
       : WorkOrder(query_id),
         function_handle_(function_handle),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index 8db5ef1..5acecbf 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -165,9 +165,9 @@ bool TextScanOperator::getAllWorkOrders(
         // First, generate text-split work orders.
         for (const auto &file : files) {
           container->addNormalWorkOrder(
-              new TextSplitWorkOrder(file,
+              new TextSplitWorkOrder(query_id_,
+                                     file,
                                      process_escape_sequences_,
-                                     query_id_,
                                      storage_manager,
                                      op_index_,
                                      scheduler_client_id,
@@ -182,11 +182,11 @@ bool TextScanOperator::getAllWorkOrders(
         while (!text_blob_queue_.empty()) {
           const TextBlob blob_work = text_blob_queue_.popOne();
           container->addNormalWorkOrder(
-              new TextScanWorkOrder(blob_work.blob_id,
+              new TextScanWorkOrder(query_id_,
+                                    blob_work.blob_id,
                                     blob_work.size,
                                     field_terminator_,
                                     process_escape_sequences_,
-                                    query_id_,
                                     output_destination,
                                     storage_manager),
               op_index_);
@@ -203,10 +203,10 @@ bool TextScanOperator::getAllWorkOrders(
     if (blocking_dependencies_met_ && !work_generated_) {
       for (const auto &file : files) {
         container->addNormalWorkOrder(
-            new TextScanWorkOrder(file,
+            new TextScanWorkOrder(query_id_,
+                                  file,
                                   field_terminator_,
                                   process_escape_sequences_,
-                                  query_id_,
                                   output_destination,
                                   storage_manager),
             op_index_);
@@ -234,11 +234,10 @@ void TextScanOperator::receiveFeedbackMessage(const WorkOrder::FeedbackMessage &
   }
 }
 
-
-TextScanWorkOrder::TextScanWorkOrder(const std::string &filename,
+TextScanWorkOrder::TextScanWorkOrder(const std::size_t query_id,
+                                     const std::string &filename,
                                      const char field_terminator,
                                      const bool process_escape_sequences,
-                                     const std::size_t query_id,
                                      InsertDestination *output_destination,
                                      StorageManager *storage_manager)
     : WorkOrder(query_id),
@@ -254,11 +253,11 @@ TextScanWorkOrder::TextScanWorkOrder(const std::string &filename,
   DCHECK(storage_manager_ != nullptr);
 }
 
-TextScanWorkOrder::TextScanWorkOrder(const block_id text_blob,
+TextScanWorkOrder::TextScanWorkOrder(const std::size_t query_id,
+                                     const block_id text_blob,
                                      const std::size_t text_size,
                                      const char field_terminator,
                                      const bool process_escape_sequences,
-                                     const std::size_t query_id,
                                      InsertDestination *output_destination,
                                      StorageManager *storage_manager)
     : WorkOrder(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index 1d0c04f..3cda65b 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -121,6 +121,7 @@ class TextScanOperator : public RelationalOperator {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param file_pattern The glob-like file pattern of the sources to load. The
    *        pattern could include * (wildcard for multiple chars) and ?
    *        (wildcard for single char). It defaults to single file load, if a
@@ -134,15 +135,15 @@ class TextScanOperator : public RelationalOperator {
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert tuples.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  TextScanOperator(const std::string &file_pattern,
-                   const char field_terminator,
-                   const bool process_escape_sequences,
-                   const bool parallelize_load,
-                   const CatalogRelation &output_relation,
-                   const QueryContext::insert_destination_id output_destination_index,
-                   const std::size_t query_id)
+  TextScanOperator(
+      const std::size_t query_id,
+      const std::string &file_pattern,
+      const char field_terminator,
+      const bool process_escape_sequences,
+      const bool parallelize_load,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index)
       : RelationalOperator(query_id),
         file_pattern_(file_pattern),
         field_terminator_(field_terminator),
@@ -200,42 +201,42 @@ class TextScanWorkOrder : public WorkOrder {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param filename The name of the text file to bulk insert.
    * @param field_terminator The string which separates attribute values in
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert tuples.
    * @param storage_manager The StorageManager to use.
    **/
   TextScanWorkOrder(
+      const std::size_t query_id,
       const std::string &filename,
       const char field_terminator,
       const bool process_escape_sequences,
-      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager);
 
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param text_blob Blob ID containing the data to be scanned.
    * @param text_size Size of the data in the blob.
    * @param field_terminator The character which separates attribute values in
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to write the read tuples.
    * @param storage_manager The StorageManager to use.
    */
   TextScanWorkOrder(
+      const std::size_t query_id,
       const block_id text_blob,
       const std::size_t text_size,
       const char field_terminator,
       const bool process_escape_sequences,
-      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager);
 
@@ -322,19 +323,20 @@ class TextSplitWorkOrder : public WorkOrder {
  public:
   /**
    * @brief Constructor.
+   *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param filename File to split into row-aligned blobs.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param storage_manager The StorageManager to use.
    * @param operator_index Operator index of the current operator. This is used
    *                       to send new-work available message to Foreman.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
    * @param bus A pointer to the TMB.
    */
-  TextSplitWorkOrder(const std::string &filename,
+  TextSplitWorkOrder(const std::size_t query_id,
+                     const std::string &filename,
                      const bool process_escape_sequences,
-                     const std::size_t query_id,
                      StorageManager *storage_manager,
                      const std::size_t operator_index,
                      const tmb::client_id scheduler_client_id,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index b331a9c..2130563 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -53,16 +53,18 @@ bool UpdateOperator::getAllWorkOrders(
 
     for (const block_id input_block_id : input_blocks_) {
       container->addNormalWorkOrder(
-          new UpdateWorkOrder(relation_,
-                              input_block_id,
-                              query_context->getPredicate(predicate_index_),
-                              query_context->getUpdateGroup(update_group_index_),
-                              query_id_,
-                              query_context->getInsertDestination(relocation_destination_index_),
-                              storage_manager,
-                              op_index_,
-                              scheduler_client_id,
-                              bus),
+          new UpdateWorkOrder(
+              query_id_,
+              relation_,
+              input_block_id,
+              query_context->getPredicate(predicate_index_),
+              query_context->getUpdateGroup(update_group_index_),
+              query_context->getInsertDestination(
+                  relocation_destination_index_),
+              storage_manager,
+              op_index_,
+              scheduler_client_id,
+              bus),
           op_index_);
     }
     started_ = true;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index ba2d6cf..cebb9b5 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -62,6 +62,7 @@ class UpdateOperator : public RelationalOperator {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param relation The relation to perform the UPDATE over.
    * @param relocation_destination_index The index of the InsertDestination in
    *        the QueryContext to relocate tuples which can not be updated
@@ -72,16 +73,16 @@ class UpdateOperator : public RelationalOperator {
    * @param update_group_index The index of a update group (the map of
    *        attribute_ids to Scalars) which should be evaluated to get the new
    *        value for the corresponding attribute.
-   * @param query_id The ID of the query to which this operator belongs.
    *
    * @warning The constructed InsertDestination should belong to relation, but
    *          must NOT contain any pre-existing blocks.
    **/
-  UpdateOperator(const CatalogRelation &relation,
-                 const QueryContext::insert_destination_id relocation_destination_index,
-                 const QueryContext::predicate_id predicate_index,
-                 const QueryContext::update_group_id update_group_index,
-                 const std::size_t query_id)
+  UpdateOperator(
+      const std::size_t query_id,
+      const CatalogRelation &relation,
+      const QueryContext::insert_destination_id relocation_destination_index,
+      const QueryContext::predicate_id predicate_index,
+      const QueryContext::update_group_id update_group_index)
       : RelationalOperator(query_id),
         relation_(relation),
         relocation_destination_index_(relocation_destination_index),
@@ -127,13 +128,13 @@ class UpdateWorkOrder : public WorkOrder {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param relation The relation to perform the UPDATE over.
    * @param predicate All tuples matching \c predicate will be updated (or NULL
    *        to update all tuples).
    * @param assignments The assignments (the map of attribute_ids to Scalars)
    *        which should be evaluated to get the new value for the corresponding
    *        attribute.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_block_id The block id.
    * @param relocation_destination The InsertDestination to relocate tuples
    *        which can not be updated in-place.
@@ -143,16 +144,18 @@ class UpdateWorkOrder : public WorkOrder {
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
    * @param bus A pointer to the TMB.
    **/
-  UpdateWorkOrder(const CatalogRelationSchema &relation,
-                  const block_id input_block_id,
-                  const Predicate *predicate,
-                  const std::unordered_map<attribute_id, std::unique_ptr<const Scalar>> &assignments,
-                  const std::size_t query_id,
-                  InsertDestination *relocation_destination,
-                  StorageManager *storage_manager,
-                  const std::size_t update_operator_index,
-                  const tmb::client_id scheduler_client_id,
-                  MessageBus *bus)
+  UpdateWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &relation,
+      const block_id input_block_id,
+      const Predicate *predicate,
+      const std::unordered_map<attribute_id, std::unique_ptr<const Scalar>>
+          &assignments,
+      InsertDestination *relocation_destination,
+      StorageManager *storage_manager,
+      const std::size_t update_operator_index,
+      const tmb::client_id scheduler_client_id,
+      MessageBus *bus)
       : WorkOrder(query_id),
         relation_(relation),
         input_block_id_(input_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index fdd694f..489b666 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -75,8 +75,8 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::AGGREGATION: {
       LOG(INFO) << "Creating AggregationWorkOrder";
       return new AggregationWorkOrder(
-          proto.GetExtension(serialization::AggregationWorkOrder::block_id),
           proto.query_id(),
+          proto.GetExtension(serialization::AggregationWorkOrder::block_id),
           query_context->getAggregationState(
               proto.GetExtension(serialization::AggregationWorkOrder::aggr_state_index)));
     }
@@ -89,12 +89,12 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       }
 
       return new BuildHashWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::BuildHashWorkOrder::relation_id)),
           move(join_key_attributes),
           proto.GetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable),
           proto.GetExtension(serialization::BuildHashWorkOrder::block_id),
-          proto.query_id(),
           query_context->getJoinHashTable(
               proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index)),
           storage_manager);
@@ -102,6 +102,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::DELETE: {
       LOG(INFO) << "Creating DeleteWorkOrder";
       return new DeleteWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::DeleteWorkOrder::relation_id)),
           proto.GetExtension(serialization::DeleteWorkOrder::block_id),
@@ -110,15 +111,14 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           storage_manager,
           proto.GetExtension(serialization::DeleteWorkOrder::operator_index),
           shiftboss_client_id,
-          proto.query_id(),
           bus);
     }
     case serialization::DESTROY_HASH: {
       LOG(INFO) << "Creating DestroyHashWorkOrder";
       return new DestroyHashWorkOrder(
+          proto.query_id(),
           proto.GetExtension(
               serialization::DestroyHashWorkOrder::join_hash_table_index),
-          proto.query_id(),
           query_context);
     }
     case serialization::DROP_TABLE: {
@@ -193,6 +193,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
         case serialization::HashJoinWorkOrder::HASH_ANTI_JOIN: {
           LOG(INFO) << "Creating HashAntiJoinWorkOrder";
           return new HashAntiJoinWorkOrder(
+              proto.query_id(),
               build_relation,
               probe_relation,
               move(join_key_attributes),
@@ -201,13 +202,13 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
-              proto.query_id(),
               output_destination,
               storage_manager);
         }
         case serialization::HashJoinWorkOrder::HASH_INNER_JOIN: {
           LOG(INFO) << "Creating HashInnerJoinWorkOrder";
           return new HashInnerJoinWorkOrder(
+              proto.query_id(),
               build_relation,
               probe_relation,
               move(join_key_attributes),
@@ -216,7 +217,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
-              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -231,6 +231,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
 
           LOG(INFO) << "Creating HashOuterJoinWorkOrder";
           return new HashOuterJoinWorkOrder(
+              proto.query_id(),
               build_relation,
               probe_relation,
               move(join_key_attributes),
@@ -239,13 +240,13 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               selection,
               move(is_selection_on_build),
               hash_table,
-              proto.query_id(),
               output_destination,
               storage_manager);
         }
         case serialization::HashJoinWorkOrder::HASH_SEMI_JOIN: {
           LOG(INFO) << "Creating HashSemiJoinWorkOrder";
           return new HashSemiJoinWorkOrder(
+              proto.query_id(),
               build_relation,
               probe_relation,
               move(join_key_attributes),
@@ -254,7 +255,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
-              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -274,6 +274,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::NESTED_LOOP_JOIN: {
       LOG(INFO) << "Creating NestedLoopsJoinWorkOrder";
       return new NestedLoopsJoinWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::left_relation_id)),
           catalog_database->getRelationSchemaById(
@@ -284,7 +285,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::join_predicate_index)),
           query_context->getScalarGroup(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::selection_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::insert_destination_index)),
           storage_manager);
@@ -292,12 +292,12 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::SAMPLE: {
       LOG(INFO) << "Creating SampleWorkOrder";
       return new SampleWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::SampleWorkOrder::relation_id)),
           proto.GetExtension(serialization::SampleWorkOrder::block_id),
           proto.GetExtension(serialization::SampleWorkOrder::is_block_sample),
           proto.GetExtension(serialization::SampleWorkOrder::percentage),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SampleWorkOrder::insert_destination_index)),
           storage_manager);
@@ -305,9 +305,9 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::SAVE_BLOCKS: {
       LOG(INFO) << "Creating SaveBlocksWorkOrder";
       return new SaveBlocksWorkOrder(
+          proto.query_id(),
           proto.GetExtension(serialization::SaveBlocksWorkOrder::block_id),
           proto.GetExtension(serialization::SaveBlocksWorkOrder::force),
-          proto.query_id(),
           storage_manager);
     }
     case serialization::SELECT: {
@@ -321,6 +321,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       }
 
       return new SelectWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::SelectWorkOrder::relation_id)),
           proto.GetExtension(serialization::SelectWorkOrder::block_id),
@@ -331,7 +332,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           simple_projection ? nullptr
                             : &query_context->getScalarGroup(
                                   proto.GetExtension(serialization::SelectWorkOrder::selection_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SelectWorkOrder::insert_destination_index)),
           storage_manager);
@@ -350,6 +350,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       }
 
       return new SortMergeRunWorkOrder(
+          proto.query_id(),
           query_context->getSortConfig(
               proto.GetExtension(serialization::SortMergeRunWorkOrder::sort_config_index)),
           catalog_database->getRelationSchemaById(
@@ -357,7 +358,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           move(runs),
           proto.GetExtension(serialization::SortMergeRunWorkOrder::top_k),
           proto.GetExtension(serialization::SortMergeRunWorkOrder::merge_level),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SortMergeRunWorkOrder::insert_destination_index)),
           storage_manager,
@@ -368,12 +368,12 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::SORT_RUN_GENERATION: {
       LOG(INFO) << "Creating SortRunGenerationWorkOrder";
       return new SortRunGenerationWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::relation_id)),
           proto.GetExtension(serialization::SortRunGenerationWorkOrder::block_id),
           query_context->getSortConfig(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::sort_config_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::insert_destination_index)),
           storage_manager);
@@ -381,9 +381,9 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::TABLE_GENERATOR: {
       LOG(INFO) << "Creating SortRunGenerationWorkOrder";
       return new TableGeneratorWorkOrder(
+          proto.query_id(),
           query_context->getGeneratorFunctionHandle(
               proto.GetExtension(serialization::TableGeneratorWorkOrder::generator_function_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::TableGeneratorWorkOrder::insert_destination_index)));
     }
@@ -391,10 +391,10 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       LOG(INFO) << "Creating TextScanWorkOrder";
       if (proto.HasExtension(serialization::TextScanWorkOrder::filename)) {
         return new TextScanWorkOrder(
+            proto.query_id(),
             proto.GetExtension(serialization::TextScanWorkOrder::filename),
             proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
             proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
-            proto.query_id(),
             query_context->getInsertDestination(
                 proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
             storage_manager);
@@ -403,11 +403,11 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
             proto.GetExtension(serialization::TextScanWorkOrder::text_blob);
 
         return new TextScanWorkOrder(
+            proto.query_id(),
             text_blob_proto.blob_id(),
             text_blob_proto.size(),
             proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
             proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
-            proto.query_id(),
             query_context->getInsertDestination(
                 proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
             storage_manager);
@@ -416,9 +416,9 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::TEXT_SPLIT: {
       LOG(INFO) << "Creating TextSplitWorkOrder";
       return new TextSplitWorkOrder(
+          proto.query_id(),
           proto.GetExtension(serialization::TextSplitWorkOrder::filename),
           proto.GetExtension(serialization::TextSplitWorkOrder::process_escape_sequences),
-          proto.query_id(),
           storage_manager,
           proto.GetExtension(serialization::TextSplitWorkOrder::operator_index),
           shiftboss_client_id,
@@ -427,6 +427,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::UPDATE: {
       LOG(INFO) << "Creating UpdateWorkOrder";
       return new UpdateWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::UpdateWorkOrder::relation_id)),
           proto.GetExtension(serialization::UpdateWorkOrder::block_id),
@@ -434,7 +435,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::UpdateWorkOrder::predicate_index)),
           query_context->getUpdateGroup(
               proto.GetExtension(serialization::UpdateWorkOrder::update_group_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::UpdateWorkOrder::insert_destination_index)),
           storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index ace7951..fdcc54f 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -77,6 +77,7 @@ using std::unique_ptr;
 namespace quickstep {
 
 namespace {
+constexpr std::size_t kQueryId = 0;
 constexpr int kOpIndex = 0;
 }  // namespace
 
@@ -270,7 +271,7 @@ class AggregationOperatorTest : public ::testing::Test {
     aggr_state_proto->set_estimated_num_entries(estimated_entries);
 
     // Create Operators.
-    op_.reset(new AggregationOperator(*table_, true, aggr_state_index, 0));
+    op_.reset(new AggregationOperator(0, *table_, true, aggr_state_index));
 
     // Setup the InsertDestination proto in the query context proto.
     const QueryContext::insert_destination_id insert_destination_index =
@@ -281,10 +282,11 @@ class AggregationOperatorTest : public ::testing::Test {
     insert_destination_proto->set_relation_id(result_table_->getID());
     insert_destination_proto->set_relational_op_index(kOpIndex);
 
-    finalize_op_.reset(new FinalizeAggregationOperator(aggr_state_index,
-                                                       *result_table_,
-                                                       insert_destination_index,
-                                                       0 /* dummy query ID */));
+    finalize_op_.reset(
+        new FinalizeAggregationOperator(kQueryId,
+                                        aggr_state_index,
+                                        *result_table_,
+                                        insert_destination_index));
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,
@@ -354,7 +356,7 @@ class AggregationOperatorTest : public ::testing::Test {
         serialization::HashTableImplType::LINEAR_OPEN_ADDRESSING);
 
     // Create Operators.
-    op_.reset(new AggregationOperator(*table_, true, aggr_state_index, 0));
+    op_.reset(new AggregationOperator(0, *table_, true, aggr_state_index));
 
     // Setup the InsertDestination proto in the query context proto.
     const QueryContext::insert_destination_id insert_destination_index =
@@ -365,10 +367,11 @@ class AggregationOperatorTest : public ::testing::Test {
     insert_destination_proto->set_relation_id(result_table_->getID());
     insert_destination_proto->set_relational_op_index(kOpIndex);
 
-    finalize_op_.reset(new FinalizeAggregationOperator(aggr_state_index,
-                                                       *result_table_,
-                                                       insert_destination_index,
-                                                       0 /* dummy query ID */));
+    finalize_op_.reset(
+        new FinalizeAggregationOperator(kQueryId,
+                                        aggr_state_index,
+                                        *result_table_,
+                                        insert_destination_index));
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,


[37/50] [abbrv] incubator-quickstep git commit: Added PRIORITY clause in parser.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index c02baf4..8fdf490 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -1,19 +1,19 @@
-/* A Bison parser, made by GNU Bison 3.0.4.  */
+/* A Bison parser, made by GNU Bison 2.7.  */
 
 /* Bison implementation for Yacc-like parsers in C
-
-   Copyright (C) 1984, 1989-1990, 2000-2015 Free Software Foundation, Inc.
-
+   
+      Copyright (C) 1984, 1989-1990, 2000-2012 Free Software Foundation, Inc.
+   
    This program is free software: you can redistribute it and/or modify
    it under the terms of the GNU General Public License as published by
    the Free Software Foundation, either version 3 of the License, or
    (at your option) any later version.
-
+   
    This program is distributed in the hope that it will be useful,
    but WITHOUT ANY WARRANTY; without even the implied warranty of
    MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
    GNU General Public License for more details.
-
+   
    You should have received a copy of the GNU General Public License
    along with this program.  If not, see <http://www.gnu.org/licenses/>.  */
 
@@ -26,7 +26,7 @@
    special exception, which will cause the skeleton and the resulting
    Bison output files to be licensed under the GNU General Public
    License without this special exception.
-
+   
    This special exception was added by the Free Software Foundation in
    version 2.2 of Bison.  */
 
@@ -44,7 +44,7 @@
 #define YYBISON 1
 
 /* Bison version.  */
-#define YYBISON_VERSION "3.0.4"
+#define YYBISON_VERSION "2.7"
 
 /* Skeleton name.  */
 #define YYSKELETON_NAME "yacc.c"
@@ -63,12 +63,15 @@
 #define yyparse         quickstep_yyparse
 #define yylex           quickstep_yylex
 #define yyerror         quickstep_yyerror
+#define yylval          quickstep_yylval
+#define yychar          quickstep_yychar
 #define yydebug         quickstep_yydebug
 #define yynerrs         quickstep_yynerrs
-
+#define yylloc          quickstep_yylloc
 
 /* Copy the first part of user declarations.  */
-#line 35 "../SqlParser.ypp" /* yacc.c:339  */
+/* Line 371 of yacc.c  */
+#line 35 "../SqlParser.ypp"
 
 
 /* Override the default definition, as we only need <first_line> and <first_column>. */
@@ -96,7 +99,8 @@ typedef struct YYLTYPE {
     }                                                           \
   } while (0)
 
-#line 64 "../SqlParser.ypp" /* yacc.c:339  */
+/* Line 371 of yacc.c  */
+#line 64 "../SqlParser.ypp"
 
 #include <cstdlib>
 #include <string>
@@ -121,6 +125,7 @@ typedef struct YYLTYPE {
 #include "parser/ParsePredicate.hpp"
 #include "parser/ParsePredicateExists.hpp"
 #include "parser/ParsePredicateInTableQuery.hpp"
+#include "parser/ParsePriority.hpp"
 #include "parser/ParserUtil.hpp"
 #include "parser/ParseSample.hpp"
 #include "parser/ParseSelect.hpp"
@@ -151,13 +156,14 @@ typedef struct YYLTYPE {
 // Needed for Bison 2.6 and higher, which do not automatically provide this typedef.
 typedef void* yyscan_t;
 
-#line 155 "SqlParser_gen.cpp" /* yacc.c:339  */
+/* Line 371 of yacc.c  */
+#line 161 "SqlParser_gen.cpp"
 
-# ifndef YY_NULLPTR
+# ifndef YY_NULL
 #  if defined __cplusplus && 201103L <= __cplusplus
-#   define YY_NULLPTR nullptr
+#   define YY_NULL nullptr
 #  else
-#   define YY_NULLPTR 0
+#   define YY_NULL 0
 #  endif
 # endif
 
@@ -173,7 +179,7 @@ typedef void* yyscan_t;
    by #include "SqlParser_gen.hpp".  */
 #ifndef YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED
 # define YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED
-/* Debug traces.  */
+/* Enabling traces.  */
 #ifndef YYDEBUG
 # define YYDEBUG 0
 #endif
@@ -181,150 +187,152 @@ typedef void* yyscan_t;
 extern int quickstep_yydebug;
 #endif
 
-/* Token type.  */
+/* Tokens.  */
 #ifndef YYTOKENTYPE
 # define YYTOKENTYPE
-  enum yytokentype
-  {
-    TOKEN_COMMAND = 258,
-    TOKEN_NAME = 259,
-    TOKEN_STRING_SINGLE_QUOTED = 260,
-    TOKEN_STRING_DOUBLE_QUOTED = 261,
-    TOKEN_UNSIGNED_NUMVAL = 262,
-    TOKEN_OR = 263,
-    TOKEN_AND = 264,
-    TOKEN_NOT = 265,
-    TOKEN_EQ = 266,
-    TOKEN_LT = 267,
-    TOKEN_LEQ = 268,
-    TOKEN_GT = 269,
-    TOKEN_GEQ = 270,
-    TOKEN_NEQ = 271,
-    TOKEN_LIKE = 272,
-    TOKEN_REGEXP = 273,
-    TOKEN_BETWEEN = 274,
-    TOKEN_IS = 275,
-    UNARY_PLUS = 276,
-    UNARY_MINUS = 277,
-    TOKEN_ADD = 278,
-    TOKEN_ALL = 279,
-    TOKEN_ALTER = 280,
-    TOKEN_AS = 281,
-    TOKEN_ASC = 282,
-    TOKEN_BIGINT = 283,
-    TOKEN_BIT = 284,
-    TOKEN_BITWEAVING = 285,
-    TOKEN_BLOCKPROPERTIES = 286,
-    TOKEN_BLOCKSAMPLE = 287,
-    TOKEN_BLOOM_FILTER = 288,
-    TOKEN_CSB_TREE = 289,
-    TOKEN_BY = 290,
-    TOKEN_CASE = 291,
-    TOKEN_CHARACTER = 292,
-    TOKEN_CHECK = 293,
-    TOKEN_COLUMN = 294,
-    TOKEN_CONSTRAINT = 295,
-    TOKEN_COPY = 296,
-    TOKEN_CREATE = 297,
-    TOKEN_CURRENT = 298,
-    TOKEN_DATE = 299,
-    TOKEN_DATETIME = 300,
-    TOKEN_DAY = 301,
-    TOKEN_DECIMAL = 302,
-    TOKEN_DEFAULT = 303,
-    TOKEN_DELETE = 304,
-    TOKEN_DELIMITER = 305,
-    TOKEN_DESC = 306,
-    TOKEN_DISTINCT = 307,
-    TOKEN_DOUBLE = 308,
-    TOKEN_DROP = 309,
-    TOKEN_ELSE = 310,
-    TOKEN_END = 311,
-    TOKEN_ESCAPE_STRINGS = 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_QUIT = 354,
-    TOKEN_RANGE = 355,
-    TOKEN_REAL = 356,
-    TOKEN_REFERENCES = 357,
-    TOKEN_RIGHT = 358,
-    TOKEN_ROW = 359,
-    TOKEN_ROW_DELIMITER = 360,
-    TOKEN_ROWS = 361,
-    TOKEN_SECOND = 362,
-    TOKEN_SELECT = 363,
-    TOKEN_SET = 364,
-    TOKEN_SMA = 365,
-    TOKEN_SMALLINT = 366,
-    TOKEN_SUBSTRING = 367,
-    TOKEN_TABLE = 368,
-    TOKEN_THEN = 369,
-    TOKEN_TIME = 370,
-    TOKEN_TIMESTAMP = 371,
-    TOKEN_TRUE = 372,
-    TOKEN_TUPLESAMPLE = 373,
-    TOKEN_UNBOUNDED = 374,
-    TOKEN_UNIQUE = 375,
-    TOKEN_UPDATE = 376,
-    TOKEN_USING = 377,
-    TOKEN_VALUES = 378,
-    TOKEN_VARCHAR = 379,
-    TOKEN_WHEN = 380,
-    TOKEN_WHERE = 381,
-    TOKEN_WINDOW = 382,
-    TOKEN_WITH = 383,
-    TOKEN_YEAR = 384,
-    TOKEN_YEARMONTH = 385,
-    TOKEN_EOF = 386,
-    TOKEN_LEX_ERROR = 387
-  };
+   /* Put the tokens into the symbol table, so that GDB and other debuggers
+      know about them.  */
+   enum yytokentype {
+     TOKEN_COMMAND = 258,
+     TOKEN_NAME = 259,
+     TOKEN_STRING_SINGLE_QUOTED = 260,
+     TOKEN_STRING_DOUBLE_QUOTED = 261,
+     TOKEN_UNSIGNED_NUMVAL = 262,
+     TOKEN_OR = 263,
+     TOKEN_AND = 264,
+     TOKEN_NOT = 265,
+     TOKEN_EQ = 266,
+     TOKEN_NEQ = 267,
+     TOKEN_GEQ = 268,
+     TOKEN_GT = 269,
+     TOKEN_LEQ = 270,
+     TOKEN_LT = 271,
+     TOKEN_REGEXP = 272,
+     TOKEN_LIKE = 273,
+     TOKEN_BETWEEN = 274,
+     TOKEN_IS = 275,
+     UNARY_MINUS = 276,
+     UNARY_PLUS = 277,
+     TOKEN_ADD = 278,
+     TOKEN_ALL = 279,
+     TOKEN_ALTER = 280,
+     TOKEN_AS = 281,
+     TOKEN_ASC = 282,
+     TOKEN_BIGINT = 283,
+     TOKEN_BIT = 284,
+     TOKEN_BITWEAVING = 285,
+     TOKEN_BLOCKPROPERTIES = 286,
+     TOKEN_BLOCKSAMPLE = 287,
+     TOKEN_BLOOM_FILTER = 288,
+     TOKEN_CSB_TREE = 289,
+     TOKEN_BY = 290,
+     TOKEN_CASE = 291,
+     TOKEN_CHARACTER = 292,
+     TOKEN_CHECK = 293,
+     TOKEN_COLUMN = 294,
+     TOKEN_CONSTRAINT = 295,
+     TOKEN_COPY = 296,
+     TOKEN_CREATE = 297,
+     TOKEN_CURRENT = 298,
+     TOKEN_DATE = 299,
+     TOKEN_DATETIME = 300,
+     TOKEN_DAY = 301,
+     TOKEN_DECIMAL = 302,
+     TOKEN_DEFAULT = 303,
+     TOKEN_DELETE = 304,
+     TOKEN_DELIMITER = 305,
+     TOKEN_DESC = 306,
+     TOKEN_DISTINCT = 307,
+     TOKEN_DOUBLE = 308,
+     TOKEN_DROP = 309,
+     TOKEN_ELSE = 310,
+     TOKEN_END = 311,
+     TOKEN_ESCAPE_STRINGS = 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_SUBSTRING = 368,
+     TOKEN_TABLE = 369,
+     TOKEN_THEN = 370,
+     TOKEN_TIME = 371,
+     TOKEN_TIMESTAMP = 372,
+     TOKEN_TRUE = 373,
+     TOKEN_TUPLESAMPLE = 374,
+     TOKEN_UNBOUNDED = 375,
+     TOKEN_UNIQUE = 376,
+     TOKEN_UPDATE = 377,
+     TOKEN_USING = 378,
+     TOKEN_VALUES = 379,
+     TOKEN_VARCHAR = 380,
+     TOKEN_WHEN = 381,
+     TOKEN_WHERE = 382,
+     TOKEN_WINDOW = 383,
+     TOKEN_WITH = 384,
+     TOKEN_YEAR = 385,
+     TOKEN_YEARMONTH = 386,
+     TOKEN_EOF = 387,
+     TOKEN_LEX_ERROR = 388
+   };
 #endif
 
-/* Value type.  */
-#if ! defined YYSTYPE && ! defined YYSTYPE_IS_DECLARED
 
-union YYSTYPE
+#if ! defined YYSTYPE && ! defined YYSTYPE_IS_DECLARED
+typedef union YYSTYPE
 {
-#line 119 "../SqlParser.ypp" /* yacc.c:355  */
+/* Line 387 of yacc.c  */
+#line 120 "../SqlParser.ypp"
 
   quickstep::ParseString *string_value_;
 
@@ -422,42 +430,57 @@ union YYSTYPE
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
 
-#line 426 "SqlParser_gen.cpp" /* yacc.c:355  */
-};
+  quickstep::ParsePriority *opt_priority_clause_;
 
-typedef union YYSTYPE YYSTYPE;
+
+/* Line 387 of yacc.c  */
+#line 438 "SqlParser_gen.cpp"
+} YYSTYPE;
 # define YYSTYPE_IS_TRIVIAL 1
+# define yystype YYSTYPE /* obsolescent; will be withdrawn */
 # define YYSTYPE_IS_DECLARED 1
 #endif
 
-/* Location type.  */
 #if ! defined YYLTYPE && ! defined YYLTYPE_IS_DECLARED
-typedef struct YYLTYPE YYLTYPE;
-struct YYLTYPE
+typedef struct YYLTYPE
 {
   int first_line;
   int first_column;
   int last_line;
   int last_column;
-};
+} YYLTYPE;
+# define yyltype YYLTYPE /* obsolescent; will be withdrawn */
 # define YYLTYPE_IS_DECLARED 1
 # define YYLTYPE_IS_TRIVIAL 1
 #endif
 
 
-
+#ifdef YYPARSE_PARAM
+#if defined __STDC__ || defined __cplusplus
+int quickstep_yyparse (void *YYPARSE_PARAM);
+#else
+int quickstep_yyparse ();
+#endif
+#else /* ! YYPARSE_PARAM */
+#if defined __STDC__ || defined __cplusplus
 int quickstep_yyparse (yyscan_t yyscanner, quickstep::ParseStatement **parsedStatement);
+#else
+int quickstep_yyparse ();
+#endif
+#endif /* ! YYPARSE_PARAM */
 
 #endif /* !YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED  */
 
 /* Copy the second part of user declarations.  */
-#line 217 "../SqlParser.ypp" /* yacc.c:358  */
+/* Line 390 of yacc.c  */
+#line 220 "../SqlParser.ypp"
 
 /* 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 461 "SqlParser_gen.cpp" /* yacc.c:358  */
+/* Line 390 of yacc.c  */
+#line 484 "SqlParser_gen.cpp"
 
 #ifdef short
 # undef short
@@ -471,8 +494,11 @@ typedef unsigned char yytype_uint8;
 
 #ifdef YYTYPE_INT8
 typedef YYTYPE_INT8 yytype_int8;
-#else
+#elif (defined __STDC__ || defined __C99__FUNC__ \
+     || defined __cplusplus || defined _MSC_VER)
 typedef signed char yytype_int8;
+#else
+typedef short int yytype_int8;
 #endif
 
 #ifdef YYTYPE_UINT16
@@ -492,7 +518,8 @@ typedef short int yytype_int16;
 #  define YYSIZE_T __SIZE_TYPE__
 # elif defined size_t
 #  define YYSIZE_T size_t
-# elif ! defined YYSIZE_T
+# elif ! defined YYSIZE_T && (defined __STDC__ || defined __C99__FUNC__ \
+     || defined __cplusplus || defined _MSC_VER)
 #  include <stddef.h> /* INFRINGES ON USER NAME SPACE */
 #  define YYSIZE_T size_t
 # else
@@ -514,33 +541,6 @@ typedef short int yytype_int16;
 # endif
 #endif
 
-#ifndef YY_ATTRIBUTE
-# if (defined __GNUC__                                               \
-      && (2 < __GNUC__ || (__GNUC__ == 2 && 96 <= __GNUC_MINOR__)))  \
-     || defined __SUNPRO_C && 0x5110 <= __SUNPRO_C
-#  define YY_ATTRIBUTE(Spec) __attribute__(Spec)
-# else
-#  define YY_ATTRIBUTE(Spec) /* empty */
-# endif
-#endif
-
-#ifndef YY_ATTRIBUTE_PURE
-# define YY_ATTRIBUTE_PURE   YY_ATTRIBUTE ((__pure__))
-#endif
-
-#ifndef YY_ATTRIBUTE_UNUSED
-# define YY_ATTRIBUTE_UNUSED YY_ATTRIBUTE ((__unused__))
-#endif
-
-#if !defined _Noreturn \
-     && (!defined __STDC_VERSION__ || __STDC_VERSION__ < 201112)
-# if defined _MSC_VER && 1200 <= _MSC_VER
-#  define _Noreturn __declspec (noreturn)
-# else
-#  define _Noreturn YY_ATTRIBUTE ((__noreturn__))
-# endif
-#endif
-
 /* Suppress unused-variable warnings by "using" E.  */
 #if ! defined lint || defined __GNUC__
 # define YYUSE(E) ((void) (E))
@@ -548,26 +548,24 @@ typedef short int yytype_int16;
 # define YYUSE(E) /* empty */
 #endif
 
-#if defined __GNUC__ && 407 <= __GNUC__ * 100 + __GNUC_MINOR__
-/* Suppress an incorrect diagnostic about yylval being uninitialized.  */
-# define YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN \
-    _Pragma ("GCC diagnostic push") \
-    _Pragma ("GCC diagnostic ignored \"-Wuninitialized\"")\
-    _Pragma ("GCC diagnostic ignored \"-Wmaybe-uninitialized\"")
-# define YY_IGNORE_MAYBE_UNINITIALIZED_END \
-    _Pragma ("GCC diagnostic pop")
+/* Identity function, used to suppress warnings about constant conditions.  */
+#ifndef lint
+# define YYID(N) (N)
 #else
-# define YY_INITIAL_VALUE(Value) Value
-#endif
-#ifndef YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN
-# define YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN
-# define YY_IGNORE_MAYBE_UNINITIALIZED_END
+#if (defined __STDC__ || defined __C99__FUNC__ \
+     || defined __cplusplus || defined _MSC_VER)
+static int
+YYID (int yyi)
+#else
+static int
+YYID (yyi)
+    int yyi;
 #endif
-#ifndef YY_INITIAL_VALUE
-# define YY_INITIAL_VALUE(Value) /* Nothing. */
+{
+  return yyi;
+}
 #endif
 
-
 #if ! defined yyoverflow || YYERROR_VERBOSE
 
 /* The parser invokes alloca or malloc; define the necessary symbols.  */
@@ -585,7 +583,8 @@ typedef short int yytype_int16;
 #    define alloca _alloca
 #   else
 #    define YYSTACK_ALLOC alloca
-#    if ! defined _ALLOCA_H && ! defined EXIT_SUCCESS
+#    if ! defined _ALLOCA_H && ! defined EXIT_SUCCESS && (defined __STDC__ || defined __C99__FUNC__ \
+     || defined __cplusplus || defined _MSC_VER)
 #     include <stdlib.h> /* INFRINGES ON USER NAME SPACE */
       /* Use EXIT_SUCCESS as a witness for stdlib.h.  */
 #     ifndef EXIT_SUCCESS
@@ -597,8 +596,8 @@ typedef short int yytype_int16;
 # endif
 
 # ifdef YYSTACK_ALLOC
-   /* Pacify GCC's 'empty if-body' warning.  */
-#  define YYSTACK_FREE(Ptr) do { /* empty */; } while (0)
+   /* Pacify GCC's `empty if-body' warning.  */
+#  define YYSTACK_FREE(Ptr) do { /* empty */; } while (YYID (0))
 #  ifndef YYSTACK_ALLOC_MAXIMUM
     /* The OS might guarantee only one guard page at the bottom of the stack,
        and a page size can be as small as 4096 bytes.  So we cannot safely
@@ -614,7 +613,7 @@ typedef short int yytype_int16;
 #  endif
 #  if (defined __cplusplus && ! defined EXIT_SUCCESS \
        && ! ((defined YYMALLOC || defined malloc) \
-             && (defined YYFREE || defined free)))
+	     && (defined YYFREE || defined free)))
 #   include <stdlib.h> /* INFRINGES ON USER NAME SPACE */
 #   ifndef EXIT_SUCCESS
 #    define EXIT_SUCCESS 0
@@ -622,13 +621,15 @@ typedef short int yytype_int16;
 #  endif
 #  ifndef YYMALLOC
 #   define YYMALLOC malloc
-#   if ! defined malloc && ! defined EXIT_SUCCESS
+#   if ! defined malloc && ! defined EXIT_SUCCESS && (defined __STDC__ || defined __C99__FUNC__ \
+     || defined __cplusplus || defined _MSC_VER)
 void *malloc (YYSIZE_T); /* INFRINGES ON USER NAME SPACE */
 #   endif
 #  endif
 #  ifndef YYFREE
 #   define YYFREE free
-#   if ! defined free && ! defined EXIT_SUCCESS
+#   if ! defined free && ! defined EXIT_SUCCESS && (defined __STDC__ || defined __C99__FUNC__ \
+     || defined __cplusplus || defined _MSC_VER)
 void free (void *); /* INFRINGES ON USER NAME SPACE */
 #   endif
 #  endif
@@ -638,8 +639,8 @@ void free (void *); /* INFRINGES ON USER NAME SPACE */
 
 #if (! defined yyoverflow \
      && (! defined __cplusplus \
-         || (defined YYLTYPE_IS_TRIVIAL && YYLTYPE_IS_TRIVIAL \
-             && defined YYSTYPE_IS_TRIVIAL && YYSTYPE_IS_TRIVIAL)))
+	 || (defined YYLTYPE_IS_TRIVIAL && YYLTYPE_IS_TRIVIAL \
+	     && defined YYSTYPE_IS_TRIVIAL && YYSTYPE_IS_TRIVIAL)))
 
 /* A type that is properly aligned for any stack member.  */
 union yyalloc
@@ -665,16 +666,16 @@ union yyalloc
    elements in the stack, and YYPTR gives the new location of the
    stack.  Advance YYPTR to a properly aligned location for the next
    stack.  */
-# define YYSTACK_RELOCATE(Stack_alloc, Stack)                           \
-    do                                                                  \
-      {                                                                 \
-        YYSIZE_T yynewbytes;                                            \
-        YYCOPY (&yyptr->Stack_alloc, Stack, yysize);                    \
-        Stack = &yyptr->Stack_alloc;                                    \
-        yynewbytes = yystacksize * sizeof (*Stack) + YYSTACK_GAP_MAXIMUM; \
-        yyptr += yynewbytes / sizeof (*yyptr);                          \
-      }                                                                 \
-    while (0)
+# define YYSTACK_RELOCATE(Stack_alloc, Stack)				\
+    do									\
+      {									\
+	YYSIZE_T yynewbytes;						\
+	YYCOPY (&yyptr->Stack_alloc, Stack, yysize);			\
+	Stack = &yyptr->Stack_alloc;					\
+	yynewbytes = yystacksize * sizeof (*Stack) + YYSTACK_GAP_MAXIMUM; \
+	yyptr += yynewbytes / sizeof (*yyptr);				\
+      }									\
+    while (YYID (0))
 
 #endif
 
@@ -693,7 +694,7 @@ union yyalloc
           for (yyi = 0; yyi < (Count); yyi++)   \
             (Dst)[yyi] = (Src)[yyi];            \
         }                                       \
-      while (0)
+      while (YYID (0))
 #  endif
 # endif
 #endif /* !YYCOPY_NEEDED */
@@ -701,35 +702,33 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  47
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1347
+#define YYLAST   1346
 
 /* YYNTOKENS -- Number of terminals.  */
-#define YYNTOKENS  144
+#define YYNTOKENS  145
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  106
+#define YYNNTS  107
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  287
-/* YYNSTATES -- Number of states.  */
-#define YYNSTATES  531
+#define YYNRULES  289
+/* YYNRULES -- Number of states.  */
+#define YYNSTATES  536
 
-/* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
-   by yylex, with out-of-bounds checking.  */
+/* YYTRANSLATE(YYLEX) -- Bison symbol number corresponding to YYLEX.  */
 #define YYUNDEFTOK  2
-#define YYMAXUTOK   387
+#define YYMAXUTOK   388
 
-#define YYTRANSLATE(YYX)                                                \
+#define YYTRANSLATE(YYX)						\
   ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK)
 
-/* YYTRANSLATE[TOKEN-NUM] -- Symbol number corresponding to TOKEN-NUM
-   as returned by yylex, without out-of-bounds checking.  */
+/* YYTRANSLATE[YYLEX] -- Bison symbol number corresponding to YYLEX.  */
 static const yytype_uint8 yytranslate[] =
 {
        0,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-     139,     2,     2,     2,     2,     2,     2,     2,     2,     2,
+     140,     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,   143,     2,     2,
-     140,   141,    23,    21,   142,    22,    27,    24,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,     2,     2,   138,
+       2,     2,     2,     2,     2,     2,     2,   144,     2,     2,
+     141,   142,    23,    21,   143,    22,    27,    24,     2,     2,
+       2,     2,     2,     2,     2,     2,     2,     2,     2,   139,
        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,
@@ -762,42 +761,174 @@ static const yytype_uint8 yytranslate[] =
      100,   101,   102,   103,   104,   105,   106,   107,   108,   109,
      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
+     130,   131,   132,   133,   134,   135,   136,   137,   138
 };
 
 #if YYDEBUG
-  /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
+/* YYPRHS[YYN] -- Index of the first RHS symbol of rule number YYN in
+   YYRHS.  */
+static const yytype_uint16 yyprhs[] =
+{
+       0,     0,     3,     6,     9,    12,    15,    17,    19,    21,
+      23,    25,    27,    29,    31,    33,    35,    37,    39,    41,
+      48,    55,    62,    69,    79,    89,    93,    97,    99,   103,
+     105,   107,   109,   111,   113,   115,   117,   119,   121,   123,
+     125,   127,   129,   131,   134,   137,   142,   147,   149,   152,
+     154,   157,   160,   165,   171,   174,   176,   177,   179,   184,
+     190,   201,   206,   210,   212,   213,   215,   216,   220,   221,
+     227,   228,   237,   239,   241,   243,   247,   249,   251,   253,
+     256,   259,   264,   267,   269,   271,   273,   275,   276,   280,
+     291,   299,   304,   310,   316,   317,   322,   325,   328,   333,
+     338,   344,   349,   353,   355,   359,   362,   366,   367,   371,
+     374,   376,   380,   384,   395,   396,   398,   400,   402,   404,
+     406,   410,   414,   417,   419,   422,   426,   427,   431,   435,
+     436,   438,   440,   443,   445,   448,   450,   453,   460,   462,
+     465,   469,   472,   475,   477,   481,   483,   486,   488,   493,
+     495,   499,   500,   504,   505,   508,   509,   513,   514,   517,
+     518,   520,   522,   525,   532,   536,   537,   541,   542,   546,
+     547,   553,   555,   557,   560,   563,   566,   569,   572,   575,
+     577,   581,   585,   586,   588,   590,   591,   594,   597,   598,
+     600,   603,   607,   609,   613,   615,   618,   620,   626,   633,
+     638,   642,   646,   650,   653,   657,   663,   668,   675,   679,
+     681,   685,   687,   690,   692,   694,   696,   698,   702,   708,
+     710,   712,   714,   718,   720,   724,   729,   734,   740,   747,
+     754,   763,   769,   774,   776,   779,   784,   786,   789,   794,
+     795,   798,   800,   804,   806,   808,   811,   814,   816,   819,
+     823,   826,   828,   830,   832,   834,   836,   838,   840,   844,
+     846,   850,   852,   856,   858,   860,   862,   864,   866,   868,
+     870,   873,   875,   878,   880,   882,   884,   886,   888,   890,
+     892,   896,   898,   900,   902,   904,   906,   908,   911,   914
+};
+
+/* YYRHS -- A `-1'-separated list of the rules' RHS.  */
+static const yytype_int16 yyrhs[] =
+{
+     146,     0,    -1,   147,   139,    -1,   147,   137,    -1,   250,
+     140,    -1,   250,   137,    -1,     1,    -1,   137,    -1,   149,
+      -1,   174,    -1,   150,    -1,   151,    -1,   178,    -1,   152,
+      -1,   173,    -1,   148,    -1,   181,    -1,   177,    -1,   105,
+      -1,    30,   119,   248,    28,    44,   153,    -1,    30,   119,
+     248,    28,    45,   159,    -1,    30,   119,   248,    59,    44,
+     248,    -1,    30,   119,   248,    59,    45,   248,    -1,    47,
+     119,   248,   141,   154,   142,   161,   163,   164,    -1,    47,
+      78,   248,    95,   248,   162,   128,   171,   172,    -1,    59,
+     119,   248,    -1,   248,   155,   158,    -1,   153,    -1,   154,
+     143,   153,    -1,    34,    -1,    49,    -1,    50,    -1,   121,
+      -1,   122,    -1,    52,    -1,   107,    -1,    58,    -1,    67,
+      -1,   117,    -1,    81,    -1,    33,    -1,    89,    -1,    82,
+      -1,    50,    82,    -1,   136,    82,    -1,    42,   141,     7,
+     142,    -1,   130,   141,     7,   142,    -1,    92,    -1,    10,
+      92,    -1,   126,    -1,   103,    85,    -1,    53,   238,    -1,
+      43,   141,   220,   142,    -1,   108,   248,   141,   248,   142,
+      -1,   157,   156,    -1,   156,    -1,    -1,   157,    -1,   126,
+     141,   247,   142,    -1,   103,    85,   141,   247,   142,    -1,
+      70,    85,   141,   247,   142,   108,   248,   141,   247,   142,
+      -1,    43,   141,   220,   142,    -1,   160,   143,   159,    -1,
+     159,    -1,    -1,   160,    -1,    -1,   141,   242,   142,    -1,
+      -1,   134,    36,   141,   166,   142,    -1,    -1,    99,    40,
+     165,   141,   247,   142,   100,     7,    -1,    74,    -1,   106,
+      -1,   167,    -1,   166,   143,   167,    -1,   168,    -1,   169,
+      -1,   170,    -1,   248,   248,    -1,   248,    29,    -1,   248,
+     141,   247,   142,    -1,   248,     7,    -1,    35,    -1,    38,
+      -1,    39,    -1,   116,    -1,    -1,   141,   166,   142,    -1,
+      80,    83,   248,   141,   247,   142,   129,   141,   240,   142,
+      -1,    80,    83,   248,   129,   141,   240,   142,    -1,    80,
+      83,   248,   186,    -1,   183,    80,    83,   248,   186,    -1,
+      46,   248,    71,     5,   175,    -1,    -1,   134,   141,   176,
+     142,    -1,    55,     5,    -1,    62,   249,    -1,   176,   143,
+      55,     5,    -1,   176,   143,    62,   249,    -1,   127,   248,
+     115,   179,   218,    -1,    54,    71,   248,   218,    -1,   179,
+     143,   180,    -1,   180,    -1,   248,    11,   224,    -1,   186,
+     182,    -1,   183,   186,   182,    -1,    -1,   134,   104,     7,
+      -1,   134,   184,    -1,   185,    -1,   184,   143,   185,    -1,
+     198,    31,   192,    -1,   114,   187,   188,   191,   218,   200,
+     201,   202,   203,   204,    -1,    -1,    29,    -1,    57,    -1,
+      23,    -1,   189,    -1,   190,    -1,   189,   143,   190,    -1,
+     224,    31,   248,    -1,   224,   248,    -1,   224,    -1,    71,
+     199,    -1,   141,   186,   142,    -1,    -1,    37,     7,   101,
+      -1,   124,     7,   101,    -1,    -1,    79,    -1,    87,    -1,
+      87,    97,    -1,   109,    -1,   109,    97,    -1,    72,    -1,
+      72,    97,    -1,   195,   194,    84,   196,    95,   220,    -1,
+     196,    -1,   192,   197,    -1,   248,   193,   197,    -1,   248,
+     193,    -1,   228,   197,    -1,   228,    -1,   141,   195,   142,
+      -1,   198,    -1,    31,   198,    -1,   248,    -1,   248,   141,
+     247,   142,    -1,   195,    -1,   199,   143,   195,    -1,    -1,
+      73,    40,   237,    -1,    -1,    75,   220,    -1,    -1,    96,
+      40,   214,    -1,    -1,    88,     7,    -1,    -1,   205,    -1,
+     206,    -1,   205,   206,    -1,   133,   248,    31,   141,   207,
+     142,    -1,   208,   209,   210,    -1,    -1,    99,    40,   237,
+      -1,    -1,    96,    40,   214,    -1,    -1,   211,    19,   212,
+       9,   213,    -1,   112,    -1,   106,    -1,     7,   102,    -1,
+     125,   102,    -1,    48,   110,    -1,     7,    68,    -1,   125,
+      68,    -1,    48,   110,    -1,   215,    -1,   214,   143,   215,
+      -1,   224,   216,   217,    -1,    -1,    32,    -1,    56,    -1,
+      -1,    93,    66,    -1,    93,    86,    -1,    -1,   219,    -1,
+     132,   220,    -1,   220,     8,   221,    -1,   221,    -1,   221,
+       9,   222,    -1,   222,    -1,    10,   223,    -1,   223,    -1,
+     224,    19,   224,     9,   224,    -1,   224,    10,    19,   224,
+       9,   224,    -1,   241,    20,    10,    92,    -1,   241,    20,
+      92,    -1,   224,   243,   224,    -1,   141,   220,   142,    -1,
+      63,   192,    -1,   224,    77,   192,    -1,   224,    77,   141,
+     237,   142,    -1,   224,    10,    77,   192,    -1,   224,    10,
+      77,   141,   237,   142,    -1,   224,   245,   225,    -1,   225,
+      -1,   225,   246,   226,    -1,   226,    -1,   244,   227,    -1,
+     227,    -1,   241,    -1,   238,    -1,   228,    -1,   228,    98,
+     248,    -1,   228,    98,   141,   207,   142,    -1,   229,    -1,
+     230,    -1,   231,    -1,   141,   224,   142,    -1,   192,    -1,
+     248,   141,   142,    -1,   248,   141,    23,   142,    -1,   248,
+     141,   237,   142,    -1,   248,   141,    57,   237,   142,    -1,
+      64,   141,   239,    71,   224,   142,    -1,   118,   141,   224,
+      71,     7,   142,    -1,   118,   141,   224,    71,     7,    69,
+       7,   142,    -1,    41,   224,   232,   236,    61,    -1,    41,
+     234,   236,    61,    -1,   233,    -1,   232,   233,    -1,   131,
+     224,   120,   224,    -1,   235,    -1,   234,   235,    -1,   131,
+     220,   120,   224,    -1,    -1,    60,   224,    -1,   224,    -1,
+     237,   143,   224,    -1,    92,    -1,     7,    -1,    21,     7,
+      -1,    22,     7,    -1,     5,    -1,    82,     5,    -1,    82,
+       5,   239,    -1,   155,     5,    -1,   135,    -1,    91,    -1,
+      51,    -1,    76,    -1,    90,    -1,   113,    -1,   238,    -1,
+     240,   143,   238,    -1,   248,    -1,   248,    27,   248,    -1,
+     241,    -1,   242,   143,   241,    -1,    11,    -1,    12,    -1,
+      16,    -1,    15,    -1,    14,    -1,    13,    -1,    18,    -1,
+      10,    18,    -1,    17,    -1,    10,    17,    -1,    22,    -1,
+      21,    -1,    22,    -1,   144,    -1,    23,    -1,    24,    -1,
+     248,    -1,   247,   143,   248,    -1,     4,    -1,     6,    -1,
+     123,    -1,    95,    -1,    65,    -1,    94,    -1,     3,   251,
+      -1,   251,     3,    -1,    -1
+};
+
+/* YYRLINE[YYN] -- source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   622,   622,   626,   630,   634,   638,   641,   648,   651,
-     654,   657,   660,   663,   666,   669,   672,   675,   681,   687,
-     694,   700,   707,   716,   721,   730,   735,   740,   744,   750,
-     755,   758,   761,   766,   769,   772,   775,   778,   781,   784,
-     787,   790,   793,   805,   808,   811,   829,   849,   852,   855,
-     860,   865,   871,   877,   886,   890,   896,   899,   904,   909,
-     914,   921,   928,   932,   938,   941,   946,   949,   954,   957,
-     962,   965,   984,   988,   994,   998,  1004,  1007,  1010,  1015,
-    1018,  1025,  1030,  1041,  1046,  1050,  1054,  1060,  1063,  1069,
-    1077,  1080,  1083,  1089,  1094,  1097,  1102,  1106,  1110,  1114,
-    1120,  1125,  1130,  1134,  1140,  1146,  1149,  1154,  1159,  1163,
-    1169,  1175,  1181,  1184,  1188,  1194,  1197,  1202,  1206,  1212,
-    1215,  1218,  1223,  1228,  1233,  1236,  1239,  1244,  1247,  1250,
-    1253,  1256,  1259,  1262,  1265,  1270,  1273,  1278,  1282,  1286,
-    1289,  1293,  1296,  1301,  1304,  1309,  1312,  1317,  1321,  1327,
-    1330,  1335,  1338,  1343,  1346,  1351,  1354,  1373,  1376,  1379,
-    1383,  1389,  1395,  1400,  1403,  1408,  1411,  1416,  1419,  1424,
-    1427,  1432,  1433,  1436,  1441,  1442,  1445,  1450,  1454,  1460,
-    1467,  1470,  1473,  1478,  1481,  1484,  1490,  1493,  1498,  1503,
-    1512,  1517,  1526,  1531,  1534,  1539,  1542,  1547,  1553,  1559,
-    1562,  1565,  1568,  1571,  1574,  1580,  1589,  1592,  1597,  1600,
-    1605,  1608,  1613,  1616,  1619,  1622,  1626,  1630,  1633,  1636,
-    1639,  1642,  1647,  1651,  1655,  1658,  1663,  1668,  1672,  1678,
-    1681,  1686,  1690,  1696,  1701,  1705,  1711,  1716,  1719,  1724,
-    1728,  1734,  1737,  1740,  1743,  1755,  1759,  1778,  1791,  1806,
-    1809,  1812,  1815,  1818,  1821,  1826,  1830,  1836,  1839,  1844,
-    1848,  1855,  1858,  1861,  1864,  1867,  1870,  1873,  1876,  1879,
-    1882,  1887,  1898,  1901,  1906,  1909,  1912,  1918,  1922,  1928,
-    1931,  1939,  1942,  1945,  1948,  1954,  1959,  1964
+       0,   629,   629,   633,   637,   641,   645,   648,   655,   658,
+     661,   664,   667,   670,   673,   676,   679,   682,   688,   694,
+     701,   707,   714,   723,   728,   737,   742,   747,   751,   757,
+     762,   765,   768,   773,   776,   779,   782,   785,   788,   791,
+     794,   797,   800,   812,   815,   818,   836,   856,   859,   862,
+     867,   872,   878,   884,   893,   897,   903,   906,   911,   916,
+     921,   928,   935,   939,   945,   948,   953,   956,   961,   964,
+     969,   972,   991,   995,  1001,  1005,  1011,  1014,  1017,  1022,
+    1025,  1032,  1037,  1048,  1053,  1057,  1061,  1067,  1070,  1076,
+    1084,  1087,  1090,  1096,  1101,  1104,  1109,  1113,  1117,  1121,
+    1127,  1132,  1137,  1141,  1147,  1153,  1156,  1161,  1164,  1183,
+    1188,  1192,  1198,  1204,  1210,  1213,  1217,  1223,  1226,  1231,
+    1235,  1241,  1244,  1247,  1252,  1257,  1262,  1265,  1268,  1273,
+    1276,  1279,  1282,  1285,  1288,  1291,  1294,  1299,  1302,  1307,
+    1311,  1315,  1318,  1322,  1325,  1330,  1333,  1338,  1341,  1346,
+    1350,  1356,  1359,  1364,  1367,  1372,  1375,  1380,  1383,  1402,
+    1405,  1410,  1414,  1420,  1426,  1431,  1434,  1439,  1442,  1447,
+    1450,  1455,  1458,  1463,  1464,  1467,  1472,  1473,  1476,  1481,
+    1485,  1491,  1498,  1501,  1504,  1509,  1512,  1515,  1521,  1524,
+    1529,  1534,  1543,  1548,  1557,  1562,  1565,  1570,  1573,  1578,
+    1584,  1590,  1593,  1596,  1599,  1602,  1605,  1611,  1620,  1623,
+    1628,  1631,  1636,  1639,  1644,  1647,  1650,  1653,  1657,  1661,
+    1664,  1667,  1670,  1673,  1678,  1682,  1686,  1689,  1694,  1699,
+    1703,  1709,  1712,  1717,  1721,  1727,  1732,  1736,  1742,  1747,
+    1750,  1755,  1759,  1765,  1768,  1771,  1774,  1786,  1790,  1809,
+    1822,  1837,  1840,  1843,  1846,  1849,  1852,  1857,  1861,  1867,
+    1870,  1875,  1879,  1886,  1889,  1892,  1895,  1898,  1901,  1904,
+    1907,  1910,  1913,  1918,  1929,  1932,  1937,  1940,  1943,  1949,
+    1953,  1959,  1962,  1970,  1973,  1976,  1979,  1985,  1990,  1995
 };
 #endif
 
@@ -809,9 +940,9 @@ static const char *const yytname[] =
   "$end", "error", "$undefined", "TOKEN_COMMAND", "TOKEN_NAME",
   "TOKEN_STRING_SINGLE_QUOTED", "TOKEN_STRING_DOUBLE_QUOTED",
   "TOKEN_UNSIGNED_NUMVAL", "TOKEN_OR", "TOKEN_AND", "TOKEN_NOT",
-  "TOKEN_EQ", "TOKEN_LT", "TOKEN_LEQ", "TOKEN_GT", "TOKEN_GEQ",
-  "TOKEN_NEQ", "TOKEN_LIKE", "TOKEN_REGEXP", "TOKEN_BETWEEN", "TOKEN_IS",
-  "'+'", "'-'", "'*'", "'/'", "UNARY_PLUS", "UNARY_MINUS", "'.'",
+  "TOKEN_EQ", "TOKEN_NEQ", "TOKEN_GEQ", "TOKEN_GT", "TOKEN_LEQ",
+  "TOKEN_LT", "TOKEN_REGEXP", "TOKEN_LIKE", "TOKEN_BETWEEN", "TOKEN_IS",
+  "'+'", "'-'", "'*'", "'/'", "UNARY_MINUS", "UNARY_PLUS", "'.'",
   "TOKEN_ADD", "TOKEN_ALL", "TOKEN_ALTER", "TOKEN_AS", "TOKEN_ASC",
   "TOKEN_BIGINT", "TOKEN_BIT", "TOKEN_BITWEAVING", "TOKEN_BLOCKPROPERTIES",
   "TOKEN_BLOCKSAMPLE", "TOKEN_BLOOM_FILTER", "TOKEN_CSB_TREE", "TOKEN_BY",
@@ -829,30 +960,31 @@ static const char *const yytname[] =
   "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_QUIT", "TOKEN_RANGE",
-  "TOKEN_REAL", "TOKEN_REFERENCES", "TOKEN_RIGHT", "TOKEN_ROW",
-  "TOKEN_ROW_DELIMITER", "TOKEN_ROWS", "TOKEN_SECOND", "TOKEN_SELECT",
-  "TOKEN_SET", "TOKEN_SMA", "TOKEN_SMALLINT", "TOKEN_SUBSTRING",
-  "TOKEN_TABLE", "TOKEN_THEN", "TOKEN_TIME", "TOKEN_TIMESTAMP",
-  "TOKEN_TRUE", "TOKEN_TUPLESAMPLE", "TOKEN_UNBOUNDED", "TOKEN_UNIQUE",
-  "TOKEN_UPDATE", "TOKEN_USING", "TOKEN_VALUES", "TOKEN_VARCHAR",
-  "TOKEN_WHEN", "TOKEN_WHERE", "TOKEN_WINDOW", "TOKEN_WITH", "TOKEN_YEAR",
-  "TOKEN_YEARMONTH", "TOKEN_EOF", "TOKEN_LEX_ERROR", "';'", "'\\n'", "'('",
-  "')'", "','", "'%'", "$accept", "start", "sql_statement",
-  "quit_statement", "alter_table_statement", "create_table_statement",
-  "create_index_statement", "drop_table_statement", "column_def",
-  "column_def_commalist", "data_type", "column_constraint_def",
-  "column_constraint_def_list", "opt_column_constraint_def_list",
-  "table_constraint_def", "table_constraint_def_commalist",
-  "opt_table_constraint_def_commalist", "opt_column_list",
-  "opt_block_properties", "opt_partition_clause", "partition_type",
-  "key_value_list", "key_value", "key_string_value", "key_string_list",
-  "key_integer_value", "index_type", "opt_index_properties",
-  "insert_statement", "copy_from_statement", "opt_copy_from_params",
-  "copy_from_params", "update_statement", "delete_statement",
-  "assignment_list", "assignment_item", "select_statement", "with_clause",
-  "with_list", "with_list_element", "select_query", "opt_all_distinct",
-  "selection", "selection_item_commalist", "selection_item", "from_clause",
+  "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",
+  "TOKEN_SELECT", "TOKEN_SET", "TOKEN_SMA", "TOKEN_SMALLINT",
+  "TOKEN_SUBSTRING", "TOKEN_TABLE", "TOKEN_THEN", "TOKEN_TIME",
+  "TOKEN_TIMESTAMP", "TOKEN_TRUE", "TOKEN_TUPLESAMPLE", "TOKEN_UNBOUNDED",
+  "TOKEN_UNIQUE", "TOKEN_UPDATE", "TOKEN_USING", "TOKEN_VALUES",
+  "TOKEN_VARCHAR", "TOKEN_WHEN", "TOKEN_WHERE", "TOKEN_WINDOW",
+  "TOKEN_WITH", "TOKEN_YEAR", "TOKEN_YEARMONTH", "TOKEN_EOF",
+  "TOKEN_LEX_ERROR", "';'", "'\\n'", "'('", "')'", "','", "'%'", "$accept",
+  "start", "sql_statement", "quit_statement", "alter_table_statement",
+  "create_table_statement", "create_index_statement",
+  "drop_table_statement", "column_def", "column_def_commalist",
+  "data_type", "column_constraint_def", "column_constraint_def_list",
+  "opt_column_constraint_def_list", "table_constraint_def",
+  "table_constraint_def_commalist", "opt_table_constraint_def_commalist",
+  "opt_column_list", "opt_block_properties", "opt_partition_clause",
+  "partition_type", "key_value_list", "key_value", "key_string_value",
+  "key_string_list", "key_integer_value", "index_type",
+  "opt_index_properties", "insert_statement", "copy_from_statement",
+  "opt_copy_from_params", "copy_from_params", "update_statement",
+  "delete_statement", "assignment_list", "assignment_item",
+  "select_statement", "opt_priority_clause", "with_clause", "with_list",
+  "with_list_element", "select_query", "opt_all_distinct", "selection",
+  "selection_item_commalist", "selection_item", "from_clause",
   "subquery_expression", "opt_sample_clause", "join_type",
   "joined_table_reference", "table_reference", "table_reference_signature",
   "table_reference_signature_primary", "joined_table_reference_commalist",
@@ -872,13 +1004,13 @@ static const char *const yytname[] =
   "literal_value_commalist", "attribute_ref", "attribute_ref_list",
   "comparison_operation", "unary_operation", "add_operation",
   "multiply_operation", "name_commalist", "any_name", "boolean_value",
-  "command", "command_argument_list", YY_NULLPTR
+  "command", "command_argument_list", YY_NULL
 };
 #endif
 
 # ifdef YYPRINT
-/* YYTOKNUM[NUM] -- (External) token number corresponding to the
-   (internal) symbol number NUM (which must be that of a token).  */
+/* YYTOKNUM[YYLEX-NUM] -- Internal token number corresponding to
+   token YYLEX-NUM.  */
 static const yytype_uint16 yytoknum[] =
 {
        0,   256,   257,   258,   259,   260,   261,   262,   263,   264,
@@ -894,593 +1026,605 @@ static const yytype_uint16 yytoknum[] =
      350,   351,   352,   353,   354,   355,   356,   357,   358,   359,
      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,    59,    10,
-      40,    41,    44,    37
+     380,   381,   382,   383,   384,   385,   386,   387,   388,    59,
+      10,    40,    41,    44,    37
 };
 # endif
 
-#define YYPACT_NINF -239
-
-#define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-239)))
-
-#define YYTABLE_NINF -128
-
-#define yytable_value_is_error(Yytable_value) \
-  0
+/* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
+static const yytype_uint8 yyr1[] =
+{
+       0,   145,   146,   146,   146,   146,   146,   146,   147,   147,
+     147,   147,   147,   147,   147,   147,   147,   147,   148,   149,
+     149,   149,   149,   150,   151,   152,   153,   154,   154,   155,
+     155,   155,   155,   155,   155,   155,   155,   155,   155,   155,
+     155,   155,   155,   155,   155,   155,   155,   156,   156,   156,
+     156,   156,   156,   156,   157,   157,   158,   158,   159,   159,
+     159,   159,   160,   160,   161,   161,   162,   162,   163,   163,
+     164,   164,   165,   165,   166,   166,   167,   167,   167,   168,
+     168,   169,   170,   171,   171,   171,   171,   172,   172,   173,
+     173,   173,   173,   174,   175,   175,   176,   176,   176,   176,
+     177,   178,   179,   179,   180,   181,   181,   182,   182,   183,
+     184,   184,   185,   186,   187,   187,   187,   188,   188,   189,
+     189,   190,   190,   190,   191,   192,   193,   193,   193,   194,
+     194,   194,   194,   194,   194,   194,   194,   195,   195,   196,
+     196,   196,   196,   196,   196,   197,   197,   198,   198,   199,
+     199,   200,   200,   201,   201,   202,   202,   203,   203,   204,
+     204,   205,   205,   206,   207,   208,   208,   209,   209,   210,
+     210,   211,   211,   212,   212,   212,   213,   213,   213,   214,
+     214,   215,   216,   216,   216,   217,   217,   217,   218,   218,
+     219,   220,   220,   221,   221,   222,   222,   223,   223,   223,
+     223,   223,   223,   223,   223,   223,   223,   223,   224,   224,
+     225,   225,   226,   226,   227,   227,   227,   227,   227,   227,
+     227,   227,   227,   227,   228,   228,   228,   228,   229,   230,
+     230,   231,   231,   232,   232,   233,   234,   234,   235,   236,
+     236,   237,   237,   238,   238,   238,   238,   238,   238,   238,
+     238,   239,   239,   239,   239,   239,   239,   240,   240,   241,
+     241,   242,   242,   243,   243,   243,   243,   243,   243,   243,
+     243,   243,   243,   244,   245,   245,   246,   246,   246,   247,
+     247,   248,   248,   249,   249,   249,   249,   250,   251,   251
+};
 
-  /* YYPACT[STATE-NUM] -- Index in YYTABLE of the portion describing
-     STATE-NUM.  */
-static const yytype_int16 yypact[] =
+/* YYR2[YYN] -- Number of symbols composing right hand side of rule YYN.  */
+static const yytype_uint8 yyr2[] =
 {
-      71,  -239,  -239,   -52,   270,   -19,    22,   -43,    31,  -239,
-      16,   270,   270,  -239,    97,   162,  -239,  -239,  -239,  -239,
-    -239,  -239,  -239,  -239,  -239,  -239,   -24,  -239,    98,   117,
-     270,  -239,  -239,    58,   270,   270,   270,   270,   270,  -239,
-    -239,   619,    69,    29,  -239,   155,    52,  -239,  -239,  -239,
-     126,  -239,  -239,  -239,  -239,    36,   233,   151,    60,   127,
-    -239,    68,  -239,  -239,   255,   260,  -239,  -239,  -239,   700,
-     131,  -239,   217,  -239,  -239,   137,  -239,  -239,   288,  -239,
-    -239,  -239,  -239,   172,  -239,  -239,   176,   241,   792,   320,
-     297,   189,  -239,  -239,   286,    -1,  -239,  -239,   271,  -239,
-    -239,  -239,  -239,  -239,   954,     0,   270,   270,   234,   270,
-     270,   184,   210,   240,   270,   270,   538,  -239,  -239,   239,
-     270,  -239,  -239,  -239,   538,    47,    -9,  -239,   373,  -239,
-     154,   154,  1035,   374,  -239,   242,    26,  -239,    33,   127,
-    1035,  -239,  -239,   270,  1035,  -239,  -239,  -239,  -239,  1035,
-      38,   260,  -239,   270,   354,    48,  -239,   371,  -239,   272,
-    -239,   140,  -239,   272,   270,   -18,   270,   270,   244,  -239,
-     246,  -239,   163,  1144,   873,   234,   457,   382,   383,  -239,
-    -239,  1270,   377,  1116,   169,    46,  1035,    -2,  -239,  1035,
-    -239,   332,   253,  -239,  -239,  -239,  -239,  -239,  -239,   327,
-    -239,    65,   259,  -239,  -239,    34,   220,   177,  -239,   263,
-     220,    -8,   328,  -239,  -239,    -1,  -239,   303,  -239,  -239,
-     266,  1035,  -239,   293,   180,   270,  -239,  1035,  -239,   270,
-    -239,  -239,   268,   325,   329,   273,  -239,  -239,  -239,   125,
-     270,   290,   -18,   270,  -239,   188,  -239,  -239,     2,   201,
-     538,   538,   105,  -239,  -239,  -239,  -239,  -239,  -239,  -239,
-    -239,  1035,   279,  1035,    43,  -239,   185,   287,  1035,    57,
-    -239,   359,   293,  -239,  -239,  1035,   415,  -239,   178,   270,
-    -239,  -239,   326,  -239,   330,   331,   340,    33,  -239,   419,
-     422,   220,   390,   356,   393,   296,   338,  -239,   187,  -239,
-    1035,  -239,   293,  -239,   538,   298,   299,   270,   435,    -3,
-     192,  -239,   195,   414,    42,  -239,   300,   311,  -239,   353,
-     307,  1116,  -239,   364,   270,  -239,  -239,   188,  -239,  -239,
-     383,  -239,  -239,  -239,  1035,   310,   275,   792,  -239,   293,
-     360,  -239,  -239,  1116,   313,   293,  1035,  -239,    28,   -33,
-    -239,  -239,  -239,  -239,  -239,    33,   177,   350,   357,  -239,
-    1035,   538,   358,  1035,  -239,   425,   -17,  -239,   293,     4,
-     270,   270,   197,  -239,  -239,  -239,  -239,  -239,  -239,  -239,
-     148,  -239,   270,  -239,  -239,  -239,  -239,   315,   -18,   423,
-     367,  -239,   538,  -239,  -239,   333,  -239,   280,   792,  -239,
-    1035,   199,  -239,  -239,  1116,   293,  -239,   462,  -239,   381,
-    -239,  -239,   335,   382,   432,   396,   335,  1035,  -239,  -239,
-    -239,   467,  -239,   202,   204,  -239,   476,    -3,  -239,   270,
-    -239,  -239,   347,   448,  -239,    11,   270,  1035,   206,   293,
-    -239,   208,   351,   538,  1035,   486,   365,   361,  -239,   238,
-       7,   389,  -239,  -239,  -239,   211,  -239,  -239,  -239,  -239,
-      14,   270,    -7,  -239,   363,   293,  -239,  -239,  -239,   382,
-     361,  -239,   270,  -239,   365,  -239,  1035,  -239,  -239,   407,
-     399,   401,   400,   499,   270,  -239,   270,  -239,  -239,   270,
-    -239,   213,  -239,  -239,   372,  -239,   480,  -239,  -239,   102,
-    -239,  -239,  -239,  -239,    17,   376,  -239,   215,  -239,   270,
-     378,  -239,  -239,   446,   408,   451,  -239,   270,  -239,   221,
-     303,  -239,  -239,  -239,   223,   426,   384,  -239,   515,  -239,
-    -239
+       0,     2,     2,     2,     2,     2,     1,     1,     1,     1,
+       1,     1,     1,     1,     1,     1,     1,     1,     1,     6,
+       6,     6,     6,     9,     9,     3,     3,     1,     3,     1,
+       1,     1,     1,     1,     1,     1,     1,     1,     1,     1,
+       1,     1,     1,     2,     2,     4,     4,     1,     2,     1,
+       2,     2,     4,     5,     2,     1,     0,     1,     4,     5,
+      10,     4,     3,     1,     0,     1,     0,     3,     0,     5,
+       0,     8,     1,     1,     1,     3,     1,     1,     1,     2,
+       2,     4,     2,     1,     1,     1,     1,     0,     3,    10,
+       7,     4,     5,     5,     0,     4,     2,     2,     4,     4,
+       5,     4,     3,     1,     3,     2,     3,     0,     3,     2,
+       1,     3,     3,    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,     1,
+       3,     1,     3,     1,     1,     1,     1,     1,     1,     1,
+       2,     1,     2,     1,     1,     1,     1,     1,     1,     1,
+       3,     1,     1,     1,     1,     1,     1,     2,     2,     0
 };
 
-  /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
-     Performed when YYTABLE does not specify something else to do.  Zero
-     means the default is an error.  */
+/* YYDEFACT[STATE-NAME] -- Default reduction number in state STATE-NUM.
+   Performed when YYTABLE doesn't specify something else to do.  Zero
+   means the default is an error.  */
 static const yytype_uint16 yydefact[] =
 {
-       0,     6,   287,     0,     0,     0,     0,     0,     0,    18,
-     112,     0,     0,     7,     0,     0,    15,     8,    10,    11,
-      13,    14,     9,    17,    12,    16,     0,   105,     0,   285,
-       0,   279,   280,     0,     0,     0,     0,     0,     0,   113,
-     114,     0,     0,   107,   108,     0,   145,     1,     3,     2,
-       0,   106,     5,     4,   286,     0,     0,     0,     0,   186,
-      25,     0,   245,   242,     0,   271,   115,    40,    29,     0,
-       0,    30,    31,    34,    36,     0,    37,    39,     0,    41,
-     241,    35,    38,     0,    32,    33,     0,     0,     0,     0,
-       0,   116,   117,   221,   121,   207,   209,   211,   214,   217,
-     218,   219,   213,   212,     0,   257,     0,     0,     0,     0,
-       0,     0,     0,    94,     0,     0,     0,   101,   187,     0,
-       0,    91,   243,   244,     0,     0,   237,   234,     0,    43,
-       0,   246,     0,     0,    44,     0,     0,   248,     0,   186,
-       0,   272,   273,     0,     0,   120,   275,   276,   274,     0,
-       0,     0,   210,     0,     0,   186,   103,     0,   109,     0,
-     110,     0,   277,     0,     0,     0,     0,     0,     0,    93,
-      66,    27,     0,     0,     0,     0,     0,   188,   190,   192,
-     194,     0,   212,     0,     0,     0,     0,   237,   231,     0,
-     235,     0,     0,   251,   252,   253,   250,   254,   249,     0,
-     247,     0,     0,   123,   220,     0,     0,   147,   136,   122,
-     141,   124,   149,   118,   119,   206,   208,   163,   215,   258,
-       0,     0,   222,   239,     0,     0,   100,     0,   146,     0,
-      92,    19,     0,     0,     0,     0,    20,    21,    22,     0,
-       0,     0,    64,     0,    42,    56,   193,   201,     0,     0,
-       0,     0,     0,   261,   263,   264,   265,   266,   262,   267,
-     269,     0,     0,     0,     0,   255,     0,     0,     0,     0,
-     232,     0,   238,   230,    45,     0,     0,    46,   127,     0,
-     137,   143,   133,   128,   129,   131,     0,     0,   140,     0,
-       0,   139,     0,   151,     0,     0,   165,   223,     0,   224,
-       0,   102,   104,   278,     0,     0,     0,     0,     0,     0,
-       0,   259,     0,   257,     0,    63,    65,    68,    28,     0,
-       0,     0,    47,     0,     0,    49,    55,    57,    26,   200,
-     189,   191,   268,   270,     0,     0,     0,     0,   202,   199,
-       0,   198,    90,     0,     0,   236,     0,   229,     0,     0,
-     142,   144,   134,   130,   132,     0,   148,     0,     0,   138,
-       0,     0,   153,     0,   216,     0,   167,   225,   240,     0,
-       0,     0,     0,    96,   283,   284,   282,   281,    97,    95,
-       0,    67,     0,    83,    84,    85,    86,    87,     0,     0,
-      70,    48,     0,    51,    50,     0,    54,     0,     0,   204,
-       0,     0,   197,   256,     0,   233,   226,     0,   227,     0,
-     125,   126,   150,   152,     0,   155,   164,     0,   170,   169,
-     162,     0,    61,     0,     0,    58,     0,     0,   260,     0,
-      24,    62,     0,     0,    23,     0,     0,     0,     0,   195,
-     203,     0,     0,     0,     0,     0,   157,   166,   177,   180,
-       0,     0,    59,    98,    99,     0,    74,    76,    77,    78,
-       0,     0,     0,    52,     0,   196,   205,    89,   228,   135,
-     154,   156,     0,   111,   158,   159,     0,   181,   182,   183,
-       0,     0,     0,     0,     0,    88,     0,    82,    80,     0,
-      79,     0,    72,    73,     0,    53,     0,   160,   178,     0,
-     179,   171,   173,   172,     0,     0,    75,     0,    69,     0,
-       0,   184,   185,     0,     0,     0,   168,     0,    81,     0,
-     163,   174,   176,   175,     0,     0,     0,    60,     0,   161,
-      71
+       0,     6,   289,     0,     0,     0,     0,     0,     0,    18,
+     114,     0,     0,     7,     0,     0,    15,     8,    10,    11,
+      13,    14,     9,    17,    12,    16,     0,   107,     0,   287,
+       0,   281,   282,     0,     0,     0,     0,     0,     0,   115,
+     116,     0,     0,   109,   110,     0,   147,     1,     3,     2,
+       0,   107,     0,   105,     5,     4,   288,     0,     0,     0,
+       0,   188,    25,     0,   247,   244,     0,   273,   117,    40,
+      29,     0,     0,    30,    31,    34,    36,     0,    37,    39,
+       0,    41,   243,    35,    38,     0,    32,    33,     0,     0,
+       0,     0,     0,   118,   119,   223,   123,   209,   211,   213,
+     216,   219,   220,   221,   215,   214,     0,   259,     0,     0,
+       0,     0,     0,   106,     0,     0,     0,    94,     0,     0,
+       0,   101,   189,     0,     0,    91,   245,   246,     0,     0,
+     239,   236,     0,    43,     0,   248,     0,     0,    44,     0,
+       0,   250,     0,   188,     0,   274,   275,     0,     0,   122,
+     277,   278,   276,     0,     0,     0,   212,     0,     0,   188,
+     103,     0,   111,     0,   112,     0,   279,     0,   108,     0,
+       0,     0,     0,     0,    93,    66,    27,     0,     0,     0,
+       0,     0,   190,   192,   194,   196,     0,   214,     0,     0,
+       0,     0,   239,   233,     0,   237,     0,     0,   253,   254,
+     255,   252,   256,   251,     0,   249,     0,     0,   125,   222,
+       0,     0,   149,   138,   124,   143,   126,   151,   120,   121,
+     208,   210,   165,   217,   260,     0,     0,   224,   241,     0,
+       0,   100,     0,   148,     0,    92,    19,     0,     0,     0,
+       0,    20,    21,    22,     0,     0,     0,    64,     0,    42,
+      56,   195,   203,     0,     0,     0,     0,     0,   263,   264,
+     268,   267,   266,   265,   271,   269,     0,     0,     0,     0,
+     257,     0,     0,     0,     0,   234,     0,   240,   232,    45,
+       0,     0,    46,   129,     0,   139,   145,   135,   130,   131,
+     133,     0,     0,   142,     0,     0,   141,     0,   153,     0,
+       0,   167,   225,     0,   226,     0,   102,   104,   280,     0,
+       0,     0,     0,     0,     0,     0,   261,     0,   259,     0,
+      63,    65,    68,    28,     0,     0,     0,    47,     0,     0,
+      49,    55,    57,    26,   202,   191,   193,   272,   270,     0,
+       0,     0,     0,   204,   201,     0,   200,    90,     0,     0,
+     238,     0,   231,     0,     0,   144,   146,   136,   132,   134,
+       0,   150,     0,     0,   140,     0,     0,   155,     0,   218,
+       0,   169,   227,   242,     0,     0,     0,     0,    96,   285,
+     286,   284,   283,    97,    95,     0,    67,     0,    83,    84,
+      85,    86,    87,     0,     0,    70,    48,     0,    51,    50,
+       0,    54,     0,     0,   206,     0,     0,   199,   258,     0,
+     235,   228,     0,   229,     0,   127,   128,   152,   154,     0,
+     157,   166,     0,   172,   171,   164,     0,    61,     0,     0,
+      58,     0,     0,   262,     0,    24,    62,     0,     0,    23,
+       0,     0,     0,     0,   197,   205,     0,     0,     0,     0,
+       0,   159,   168,   179,   182,     0,     0,    59,    98,    99,
+       0,    74,    76,    77,    78,     0,     0,     0,    52,     0,
+     198,   207,    89,   230,   137,   156,   158,     0,   113,   160,
+     161,     0,   183,   184,   185,     0,     0,     0,     0,     0,
+      88,     0,    82,    80,     0,    79,     0,    72,    73,     0,
+      53,     0,   162,   180,     0,   181,   173,   175,   174,     0,
+       0,    75,     0,    69,     0,     0,   186,   187,     0,     0,
+       0,   170,     0,    81,     0,   165,   176,   178,   177,     0,
+       0,     0,    60,     0,   163,    71
 };
 
-  /* YYPGOTO[NTERM-NUM].  */
-static const yytype_int16 yypgoto[] =
+/* YYDEFGOTO[NTERM-NUM].  */
+static const yytype_int16 yydefgoto[] =
 {
-    -239,  -239,  -239,  -239,  -239,  -239,  -239,  -239,   -88,  -239,
-     355,   196,  -239,  -239,  -238,  -239,  -239,  -239,  -239,  -239,
-    -239,    66,    44,  -239,  -239,  -239,  -239,  -239,  -239,  -239,
-    -239,  -239,  -239,  -239,  -239,   304,  -239,  -239,  -239,   424,
-       9,  -239,  -239,  -239,   392,  -239,   -92,  -239,  -239,  -134,
-     179,  -182,   -11,  -239,  -239,  -239,  -239,  -239,  -239,  -239,
-      59,    15,  -239,  -239,  -239,  -239,  -239,  -239,    92,    61,
-    -239,  -239,   -55,  -239,  -119,   291,   289,   379,   -28,   403,
-     402,   450,  -123,  -239,  -239,  -239,  -239,   368,  -239,   430,
-     370,  -204,  -177,   421,   146,  -113,  -239,  -239,  -239,  -239,
-    -239,  -118,    -4,   134,  -239,  -239
+      -1,    14,    15,    16,    17,    18,    19,    20,   176,   177,
+      91,   331,   332,   333,   241,   321,   322,   246,   395,   439,
+     499,   460,   461,   462,   463,   464,   392,   435,    21,    22,
+     174,   315,    23,    24,   159,   160,    25,    53,    26,    43,
+      44,   139,    41,    92,    93,    94,   143,    95,   296,   291,
+     212,   213,   285,   286,   214,   298,   367,   420,   451,   478,
+     479,   480,   300,   301,   371,   425,   426,   488,   521,   452,
+     453,   484,   505,   121,   122,   182,   183,   184,   185,   186,
+      97,    98,    99,   100,   101,   102,   103,   192,   193,   130,
+     131,   196,   229,   104,   204,   271,   105,   317,   268,   106,
+     148,   153,   165,   107,   383,    28,    29
 };
 
-  /* YYDEFGOTO[NTERM-NUM].  */
-static const yytype_int16 yydefgoto[] =
+/* YYPACT[STATE-NUM] -- Index in YYTABLE of the portion describing
+   STATE-NUM.  */
+#define YYPACT_NINF -234
+static const yytype_int16 yypact[] =
+{
+     168,  -234,  -234,   -58,   181,   -19,    40,   -37,    59,  -234,
+     123,   181,   181,  -234,   208,   127,  -234,  -234,  -234,  -234,
+    -234,  -234,  -234,  -234,  -234,  -234,    -2,    95,   -46,   233,
+     181,  -234,  -234,   113,   181,   181,   181,   181,   181,  -234,
+    -234,   667,   139,   114,  -234,   236,   128,  -234,  -234,  -234,
+     189,    95,   183,  -234,  -234,  -234,  -234,    60,   272,   196,
+     138,   191,  -234,    80,  -234,  -234,   296,   299,  -234,  -234,
+    -234,   757,   187,  -234,   242,  -234,  -234,   190,  -234,  -234,
+     310,  -234,  -234,  -234,  -234,   205,  -234,  -234,   214,   277,
+     850,   355,   290,   219,  -234,  -234,   182,    20,  -234,  -234,
+     269,  -234,  -234,  -234,  -234,  -234,  1030,    -9,   181,   181,
+     228,   181,   181,  -234,   366,    89,   174,   241,   181,   181,
+     577,  -234,  -234,   235,   181,  -234,  -234,  -234,   577,    42,
+      17,  -234,   370,  -234,   126,   126,  1120,   371,  -234,   243,
+      30,  -234,    34,   191,  1120,  -234,  -234,   181,  1120,  -234,
+    -234,  -234,  -234,  1120,    35,   299,  -234,   181,   359,   -56,
+    -234,   368,  -234,   270,  -234,   110,  -234,   270,  -234,   181,
+      -1,   181,   181,   247,  -234,   248,  -234,   141,  1158,   940,
+     228,   487,   378,   381,  -234,  -234,   545,   375,  1210,   143,
+       8,  1120,    58,  -234,  1120,  -234,   330,   254,  -234,  -234,
+    -234,  -234,  -234,  -234,   327,  -234,    50,   255,  -234,  -234,
+      16,   245,   153,  -234,   256,   245,   -14,   331,  -234,  -234,
+      20,  -234,   306,  -234,  -234,   264,  1120,  -234,   268,   151,
+     181,  -234,  1120,  -234,   181,  -234,  -234,   271,   325,   328,
+     274,  -234,  -234,  -234,    31,   181,   291,    -1,   181,  -234,
+      14,  -234,  -234,     2,   326,   577,   577,    62,  -234,  -234,
+    -234,  -234,  -234,  -234,  -234,  -234,  1120,   279,  1120,    39,
+    -234,   154,   289,  1120,    25,  -234,   360,   268,  -234,  -234,
+    1120,   415,  -234,   133,   181,  -234,  -234,   332,  -234,   334,
+     335,   340,    34,  -234,   418,   420,   245,   388,   358,   394,
+     293,   342,  -234,   156,  -234,  1120,  -234,   268,  -234,   577,
+     295,   298,   181,   437,     1,   158,  -234,   167,   416,   144,
+    -234,   301,   311,  -234,   357,   309,  1210,  -234,   362,   181,
+    -234,  -234,    14,  -234,  -234,   381,  -234,  -234,  -234,  1120,
+     312,    76,   850,  -234,   268,   363,  -234,  -234,  1210,   313,
+     268,  1120,  -234,    32,   -13,  -234,  -234,  -234,  -234,  -234,
+      34,   153,   351,   356,  -234,  1120,   577,   364,  1120,  -234,
+     419,   159,  -234,   268,     7,   181,   181,   169,  -234,  -234,
+    -234,  -234,  -234,  -234,  -234,   173,  -234,   181,  -234,  -234,
+    -234,  -234,   315,    -1,   422,   365,  -234,   577,  -234,  -234,
+     322,  -234,   237,   850,  -234,  1120,   171,  -234,  -234,  1210,
+     268,  -234,   458,  -234,   372,  -234,  -234,   329,   378,   429,
+     382,   329,  1120,  -234,  -234,  -234,   452,  -234,   175,   177,
+    -234,   469,     1,  -234,   181,  -234,  -234,   337,   435,  -234,
+       9,   181,  1120,   179,   268,  -234,   184,   341,   577,  1120,
+     472,   349,   343,  -234,   178,    22,   376,  -234,  -234,  -234,
+     192,  -234,  -234,  -234,  -234,    21,   181,   -16,  -234,   346,
+     268,  -234,  -234,  -234,   378,   343,  -234,   181,  -234,   349,
+    -234,  1120,  -234,  -234,   403,   396,   389,   400,   494,   181,
+    -234,   181,  -234,  -234,   181,  -234,   207,  -234,  -234,   373,
+    -234,   473,  -234,  -234,   115,  -234,  -234,  -234,  -234,    41,
+     374,  -234,   209,  -234,   181,   383,  -234,  -234,   438,   395,
+     439,  -234,   181,  -234,   211,   306,  -234,  -234,  -234,   215,
+     411,   377,  -234,   505,  -234,  -234
+};
+
+/* YYPGOTO[NTERM-NUM].  */
+static const yytype_int16 yypgoto[] =
 {
-      -1,    14,    15,    16,    17,    18,    19,    20,   171,   172,
-      89,   326,   327,   328,   236,   316,   317,   241,   390,   434,
-     494,   455,   456,   457,   458,   459,   387,   430,    21,    22,
-     169,   310,    23,    24,   155,   156,    25,    26,    43,    44,
-     135,    41,    90,    91,    92,   139,    93,   291,   286,   207,
-     208,   280,   281,   209,   293,   362,   415,   446,   473,   474,
-     475,   295,   296,   366,   420,   421,   483,   516,   447,   448,
-     479,   500,   117,   118,   177,   178,   179,   180,   181,    95,
-      96,    97,    98,    99,   100,   101,   187,   188,   126,   127,
-     191,   224,   102,   199,   266,   103,   312,   263,   104,   144,
-     149,   161,   105,   378,    28,    29
+    -234,  -234,  -234,  -234,  -234,  -234,  -234,  -234,   -94,  -234,
+     338,   185,  -234,  -234,  -233,  -234,  -234,  -234,  -234,  -234,
+    -234,    47,    36,  -234,  -234,  -234,  -234,  -234,  -234,  -234,
+    -234,  -234,  -234,  -234,  -234,   292,  -234,   474,  -234,  -234,
+     414,    11,  -234,  -234,  -234,   386,  -234,   -97,  -234,  -234,
+    -189,   166,  -180,   -10,  -234,  -234,  -234,  -234,  -234,  -234,
+    -234,    52,    10,  -234,  -234,  -234,  -234,  -234,  -234,    83,
+      53,  -234,  -234,   -75,  -234,  -116,   278,   282,   361,   -35,
+     393,   390,   436,  -137,  -234,  -234,  -234,  -234,   352,  -234,
+     417,   354,  -207,  -185,   413,   140,  -119,  -234,  -234,  -234,
+    -234,  -234,  -120,    -4,   120,  -234,  -234
 };
 
-  /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
-     positive, shift that token.  If negative, reduce the rule whose
-     number is the opposite.  If YYTABLE_NINF, syntax error.  */
+/* YYTABLE[YYPACT[STATE-NUM]].  What to do in state STATE-NUM.  If
+   positive, shift that token.  If negative, reduce the rule which
+   number is the opposite.  If YYTABLE_NINF, syntax error.  */
+#define YYTABLE_NINF -130
 static const yytype_int16 yytable[] =
 {
-      33,    45,   184,   182,   315,   185,   265,    42,    46,    27,
-     250,   182,   250,    94,   480,   210,   160,   298,    31,   250,
-      32,   487,   146,   147,   513,   232,    55,   153,   288,   289,
-      57,    58,    59,    60,    61,    51,   407,    31,    31,    32,
-      32,   125,    31,   488,    32,    39,   206,   141,   142,   141,
-     142,   189,   233,   340,   250,   481,    50,   248,   189,    34,
-     136,   182,   374,   182,   111,   514,    30,   492,   141,   142,
-     121,   278,     1,    40,     2,    37,   231,   383,   141,   142,
-     384,   385,   210,   247,   212,   234,   141,   142,   418,    10,
-     145,   375,   376,    36,   419,   112,    45,    47,   493,    35,
-     226,     3,   157,    46,   201,   162,   163,   235,   408,   359,
-     170,   173,    94,   206,    38,   290,   162,     4,     5,   377,
-      54,   124,   332,   333,   334,     6,   223,   311,   186,    56,
-       7,   482,   154,   401,   211,   341,   276,   182,   182,   214,
-     154,   515,   148,   329,   393,   422,   218,    10,   249,   219,
-     431,     8,   463,   356,   489,   318,   412,   386,   269,   416,
-     173,   272,   237,   238,   210,   268,   403,   204,   511,   406,
-     338,   107,   230,   205,   205,     9,   346,   186,   217,   116,
-     308,    10,   335,   106,    10,   369,   108,   309,   512,   372,
-     225,   182,   109,   223,   438,   206,   119,    11,   319,   302,
-     115,   211,    46,   426,    12,   193,    46,    13,   120,   110,
-     427,   252,   253,   254,   255,   256,   257,   258,   259,   260,
-     261,   157,   141,   142,    31,   303,    32,   265,   164,   165,
-     194,   320,   210,   336,    52,   339,   313,    53,   113,   173,
-     345,   321,   413,   399,   195,   196,   114,   348,   182,   282,
-     282,   279,   423,   424,   166,   167,   283,   283,   116,   141,
-     142,  -127,   122,   206,   284,   284,   197,   123,   351,   428,
-     477,   128,   368,   435,    31,    46,    32,   130,   262,   182,
-     322,   228,   229,   211,   400,   285,   285,    46,   198,   437,
-      31,   323,    32,   131,   478,   324,   141,   142,    48,   129,
-      49,   141,   142,   162,   242,   243,   397,   141,   142,   223,
-     267,   229,   132,   325,   141,   142,   133,   143,   405,   350,
-     395,   299,   300,   134,   469,   137,   342,   343,   367,   300,
-     182,   140,   223,   379,   380,   223,   381,   382,   425,   229,
-     440,   300,   204,   451,   229,   452,   229,   466,   300,   467,
-     343,   211,   485,   486,   508,   486,   518,   229,    31,    62,
-      32,    63,   525,   229,   527,   229,   162,   162,   138,   150,
-     223,   507,   439,   168,   159,    64,    65,   220,   313,   183,
-     192,   202,   227,   203,   239,    10,   240,    67,    68,   449,
-     250,   519,   251,   273,   274,    69,    70,   264,   275,   524,
-     277,   292,   294,    71,    72,   287,    73,   297,   304,   465,
-     305,   221,    74,   307,   306,   344,   449,   314,    75,   337,
-     347,    76,   349,   352,   355,   460,   357,   353,   354,   358,
-     360,   361,   464,   363,   365,    77,    78,   364,   370,   371,
-     373,   153,   388,    79,   389,   391,    80,   392,   449,   394,
-     398,   410,   402,   404,   414,   429,   490,   460,   411,   432,
-      81,    31,    62,    32,    63,   417,   433,   174,   496,   442,
-      82,    83,   444,   436,    84,    85,   443,   300,    64,    65,
-     505,   453,   460,    86,   445,   162,   450,   461,   462,    87,
-      67,    68,   468,   471,    88,   222,   484,   472,    69,    70,
-     499,   501,   503,   476,   495,   162,    71,    72,   504,    73,
-     502,   510,   509,   162,   521,    74,   517,   522,   520,   523,
-     175,    75,   530,   396,    76,   529,   528,   491,   245,   301,
-     506,   158,   213,   497,   409,   526,   470,   498,    77,    78,
-     331,   330,    31,    62,    32,    63,    79,   215,   174,    80,
-     441,   216,   200,   246,   152,   270,   190,   271,     0,    64,
-      65,   454,     0,    81,     0,     0,     0,     0,     0,     0,
-      10,    67,    68,    82,    83,     0,     0,    84,    85,    69,
-      70,     0,     0,     0,     0,     0,    86,    71,    72,     0,
-      73,     0,    87,     0,     0,     0,    74,   176,     0,     0,
-       0,   175,    75,     0,     0,    76,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,     0,    77,
-      78,     0,     0,    31,    62,    32,    63,    79,     0,     0,
-      80,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-      64,    65,    66,     0,    81,     0,     0,     0,     0,     0,
-       0,     0,    67,    68,    82,    83,     0,     0,    84,    85,
-      69,    70,     0,     0,     0,     0,     0,    86,    71,    72,
-       0,    73,     0,    87,     0,     0,     0,    74,   176,     0,
-       0,     0,     0,    75,     0,     0,    76,     0,     0,     0,
+      33,   187,    45,   270,   189,   215,    96,    42,    46,   187,
+     255,    27,   190,   164,   320,   255,   255,   255,   157,   303,
+      31,   283,    32,   294,   324,    31,    57,    32,   492,   485,
+      59,    60,    61,    62,    63,   293,   129,    51,    31,    31,
+      32,    32,   237,   150,   151,   211,   145,   146,   518,   345,
+     493,   145,   146,   145,   146,   140,   412,   325,   497,    34,
+     187,    30,   187,   145,   146,   253,   379,   326,   217,   238,
+     486,   145,   146,   215,   125,   236,   120,   194,    50,   337,
+     338,   339,    37,   252,   231,   405,   313,   230,   115,   519,
+     498,    54,   149,   314,    55,   380,   381,   145,   146,    45,
+      35,   206,   239,   361,   161,    46,   327,   166,   167,    96,
+     295,    36,    10,   211,   175,   178,   364,   328,   194,   116,
+     166,   281,   329,   228,   382,   240,   316,   158,   273,   413,
+      10,   346,   158,   169,   170,   406,   187,   187,   216,   340,
+     330,   398,    38,   219,   334,   351,   254,   487,   128,   427,
+     223,   468,    39,   224,   323,   215,   274,   210,   417,   277,
+     436,   421,   494,   408,   152,   178,   520,   242,   243,     1,
+     343,     2,   209,   191,   411,   210,   222,   198,   235,   388,
+      40,   516,   389,   390,    58,    31,    31,    32,    32,   191,
+     187,   228,   377,   374,    10,   211,   443,   307,     3,   145,
+     146,   517,   199,   145,   146,   287,   216,    46,    47,   123,
+     482,    46,   288,   147,     4,     5,   200,   201,   171,   172,
+     289,   124,     6,   215,   270,   287,   161,     7,   431,    52,
+     308,   341,   288,   344,   483,   432,    56,  -129,   350,   202,
+     289,   318,   290,   404,   178,   353,   442,   187,     8,    31,
+     418,    32,   233,   234,   108,   428,   429,   109,   145,   146,
+     391,   203,   290,   211,    48,   423,    49,   110,   433,   111,
+     373,   424,   112,     9,   356,   355,   284,   117,   187,   119,
+      46,   440,    10,   247,   248,   272,   234,   114,   216,   145,
+     146,   118,    46,   304,   305,    11,   347,   348,   372,   305,
+     384,   385,    12,   126,   402,    13,   127,   228,   166,   386,
+     387,   430,   234,   445,   305,   135,   410,   456,   234,   457,
+     234,   471,   305,   120,   133,   400,   472,   348,   132,   187,
+     228,   134,   474,   228,   490,   491,   257,   258,   259,   260,
+     261,   262,   263,   264,   265,   266,   136,   145,   146,   513,
+     491,   523,   234,   530,   234,   137,   216,   532,   234,   138,
+     141,   142,   144,    31,    64,    32,    65,   154,   228,   163,
+     444,   166,   166,   168,   512,   173,   188,   197,   207,   232,
+      66,    67,   225,   318,    10,   208,   255,   454,   244,   245,
+     256,   278,    69,    70,   524,   269,   279,   282,   280,   292,
+      71,    72,   529,   267,   297,   299,   302,   470,    73,    74,
+     310,    75,   309,   311,   454,   312,   226,    76,   349,   319,
+     342,   352,   354,    77,   360,   362,    78,   363,   365,   357,
+     465,   358,   359,   366,   368,   369,   375,   469,   370,   376,
+      79,    80,   378,   157,   393,   394,   454,   399,    81,   396,
+     397,    82,   415,   403,   409,   407,   434,   416,   437,   422,
+     419,   495,   465,   441,   438,   447,    83,   448,   209,   449,
+     450,   455,   305,   501,   458,   467,    84,    85,   466,   476,
+      86,    87,   477,   473,   489,   510,   481,   465,   500,    88,
+     166,    31,    64,    32,    65,    89,   504,   179,   506,   507,
+      90,   227,   508,   509,   515,   527,   526,   528,    66,    67,
+     166,   533,   535,   496,   514,   522,   250,   401,   166,   534,
+      69,    70,   306,   162,   525,   113,   414,   511,    71,    72,
+     218,   502,   475,   335,   503,   531,    73,    74,   336,    75,
+     251,   220,   156,   221,   275,    76,   276,   195,   205,   446,
+     180,    77,   459,     0,    78,   257,   258,   259,   260,   261,
+     262,   263,   264,   265,   266,     0,   145,   146,    79,    80,
+       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
+       0,    31,    64,    32,    65,     0,     0,   179,     0,     0,
+       0,     0,     0,     0,    83,     0,     0,     0,    66,    67,
+       0,    10,     0,     0,    84,    85,     0,     0,    86,    87,
+      69,    70,     0,     0,     0,     0,     0,    88,    71,    72,
+       0,     0,   267,    89,     0,     0,    73,    74,   181,    75,
+       0,     0,     0,     0,     0,    76,     0,     0,     0,     0,
+     180,    77,     0,     0,    78,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,    79,    80,
+       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
+       0,    31,    64,    32,    65,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,    83,     0,     0,     0,    66,    67,
+      68,     0,     0,     0,    84,    85,     0,     0,    86,    87,
+      69,    70,     0,     0,     0,     0,     0,    88,    71,    72,
+       0,     0,     0,    89,     0,     0,    73,    74,   181,    75,
+       0,     0,     0,     0,     0,    76,     0,     0,     0,     0,
+       0,    77,     0,     0,    78,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,    79,    80,
+       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
+       0,    31,    64,    32,    65,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,    83,     0,     0,     0,    66,    67,
+       0,     0,     0,     0,    84,    85,     0,     0,    86,    87,
+      69,    70,     0,     0,     0,     0,     0,    88,    71,    72,
+       0,     0,     0,    89,     0,     0,    73,    74,    90,    75,
+       0,     0,     0,     0,     0,    76,     0,     0,     0,     0,
+       0,    77,     0,     0,    78,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,    79,    80,
+       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
+       0,     0,     0,     0,    31,    64,    32,    65,     0,     0,
+       0,     0,     0,     0,    83,     0,     0,     0,     0,     0,
+       0,    66,    67,     0,    84,    85,     0,     0,    86,    87,
+       0,     0,     0,    69,    70,     0,     0,    88,   128,     0,
+       0,    71,    72,    89,     0,     0,     0,     0,    90,    73,
+      74,     0,    75,     0,     0,     0,     0,     0,    76,     0,
+       0,     0,     0,     0,    77,     0,     0,    78,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-      77,    78,     0,     0,    31,    62,    32,    63,    79,     0,
-       0,    80,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    64,    65,     0,     0,    81,     0,     0,     0,     0,
-       0,     0,     0,    67,    68,    82,    83,     0,     0,    84,
-      85,    69,    70,     0,     0,     0,     0,     0,    86,    71,
-      72,     0,    73,     0,    87,     0,     0,     0,    74,    88,
-       0,     0,     0,     0,    75,     0,     0,    76,     0,     0,
+       0,    79,    80,     0,     0,     0,     0,     0,     0,    81,
+       0,     0,    82,     0,    31,    64,    32,    65,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
+       0,    66,    67,     0,    10,     0,     0,    84,    85,     0,
+       0,    86,    87,    69,    70,     0,     0,     0,     0,     0,
+      88,    71,    72,     0,     0,     0,    89,     0,     0,    73,
+      74,    90,    75,     0,     0,     0,     0,     0,    76,     0,
+       0,     0,     0,   180,    77,     0,     0,    78,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    77,    78,     0,     0,     0,     0,     0,     0,    79,
-       0,     0,    80,     0,     0,     0,    31,    62,    32,    63,
-       0,     0,     0,     0,     0,     0,    81,     0,     0,     0,
-       0,     0,     0,    64,    65,     0,    82,    83,     0,     0,
-      84,    85,     0,     0,     0,    67,    68,     0,     0,    86,
-     124,     0,     0,    69,    70,    87,     0,     0,     0,     0,
-      88,    71,    72,     0,    73,     0,     0,     0,     0,     0,
-      74,     0,     0,     0,     0,     0,    75,     0,     0,    76,
+       0,    79,    80,     0,     0,     0,     0,     0,     0,    81,
+       0,     0,    82,     0,    31,    64,    32,    65,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
+       0,    66,   155,     0,     0,     0,     0,    84,    85,     0,
+       0,    86,    87,    69,    70,     0,     0,     0,     0,     0,
+      88,    71,    72,     0,     0,     0,    89,     0,     0,    73,
+      74,   181,    75,     0,     0,     0,     0,     0,    76,     0,
+       0,     0,     0,     0,    77,     0,     0,    78,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,    77,    78,     0,     0,    31,    62,    32,
-      63,    79,     0,     0,    80,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    64,    65,     0,     0,    81,     0,
-       0,     0,     0,     0,     0,    10,    67,    68,    82,    83,
-       0,     0,    84,    85,    69,    70,     0,     0,     0,     0,
-       0,    86,    71,    72,     0,    73,     0,    87,     0,     0,
-       0,    74,    88,     0,     0,     0,   175,    75,     0,     0,
-      76,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    77,    78,     0,     0,    31,    62,
-      32,    63,    79,     0,     0,    80,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,    64,   151,     0,     0,    81,
-       0,     0,     0,     0,     0,     0,     0,    67,    68,    82,
-      83,     0,     0,    84,    85,    69,    70,     0,     0,     0,
-       0,     0,    86,    71,    72,     0,    73,     0,    87,     0,
-       0,     0,    74,   176,     0,     0,     0,     0,    75,     0,
-       0,    76,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,    77,    78,     0,     0,    31,
-      62,    32,    63,    79,     0,     0,    80,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,    64,    65,     0,     0,
-      81,     0,     0,     0,     0,     0,     0,     0,    67,    68,
-      82,    83,     0,     0,    84,    85,    69,    70,     0,     0,
-       0,     0,     0,    86,    71,    72,     0,    73,     0,    87,
-       0,     0,     0,    74,    88,     0,     0,     0,     0,    75,
-       0,     0,    76,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,    77,    78,     0,     0,
-       0,    62,     0,    63,    79,     0,     0,    80,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    64,   151,     0,
-       0,    81,     0,     0,     0,     0,     0,     0,     0,    67,
-      68,    82,    83,     0,     0,    84,    85,     0,    70,     0,
-       0,     0,     0,     0,    86,    71,    72,     0,    73,     0,
-      87,     0,     0,     0,    74,    88,     0,    67,    68,     0,
-       0,     0,     0,    76,     0,     0,    70,     0,     0,     0,
-       0,     0,     0,    71,    72,     0,    73,    77,    78,     0,
-       0,     0,    74,     0,     0,    79,     0,     0,    80,     0,
-       0,    76,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,    81,     0,     0,    77,   244,     0,     0,     0,
-       0,     0,    82,    79,     0,     0,    84,    85,     0,     0,
-       0,     0,     0,     0,     0,    86,     0,     0,     0,     0,
-      81,    87,     0,     0,     0,     0,     0,     0,     0,     0,
-      82,     0,     0,     0,    84,    85,     0,     0,     0,     0,
-       0,     0,     0,    86,     0,     0,     0,     0,     0,    87,
-     252,   253,   254,   255,   256,   257,   258,   259,   260,   261,
-       0,   141,   142,     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,   262
+       0,    79,    80,     0,     0,     0,     0,     0,     0,    81,
+       0,     0,    82,     0,    31,    64,    32,    65,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
+       0,    66,    67,     0,     0,     0,     0,    84,    85,     0,
+       0,    86,    87,    69,    70,     0,     0,     0,     0,     0,
+      88,    71,    72,     0,     0,     0,    89,     0,     0,    73,
+      74,    90,    75,     0,     0,     0,     0,     0,    76,     0,
+       0,     0,     0,     0,    77,     0,     0,    78,     0,     0,
+       0,    69,    70,     0,     0,     0,     0,     0,     0,     0,
+      72,    79,    80,     0,     0,     0,     0,    73,    74,    81,
+      75,     0,    82,     0,     0,    64,    76,    65,     0,     0,
+       0,     0,     0,     0,     0,    78,     0,    83,     0,     0,
+       0,    66,   155,     0,     0,     0,     0,    84,    85,    79,
+     249,    86,    87,    69,    70,     0,     0,    81,     0,     0,
+      88,     0,    72,     0,     0,     0,    89,     0,     0,    73,
+      74,    90,    75,     0,     0,    83,     0,     0,    76,     0,
+       0,     0,     0,     0,     0,    84,     0,    78,     0,    86,
+      87,     0,     0,     0,     0,     0,     0,     0,    88,     0,
+       0,    79,    80,     0,    89,     0,     0,     0,     0,    81,
+       0,     0,    82,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    84,     0,     0,
+       0,    86,    87,     0,     0,     0,     0,     0,     0,     0,
+      88,     0,     0,     0,     0,     0,    89
 };
 
+#define yypact_value_is_default(Yystate) \
+  (!!((Yystate) == (-234)))
+
+#define yytable_value_is_error(Yytable_value) \
+  YYID (0)
+
 static const yytype_int16 yycheck[] =
 {
-       4,    12,   120,   116,   242,   124,   183,    11,    12,     0,
-       8,   124,     8,    41,     7,   138,   108,   221,     4,     8,
-       6,     7,    23,    24,     7,    43,    30,    27,   210,    37,
-      34,    35,    36,    37,    38,    26,    69,     4,     4,     6,
-       6,    69,     4,    29,     6,    29,   138,    21,    22,    21,
-      22,    60,    70,    10,     8,    48,    80,   176,    60,    78,
-      88,   174,    65,   176,    28,    48,   118,    74,    21,    22,
-      61,   205,     1,    57,     3,   118,   164,    35,    21,    22,
-      38,    39,   205,   175,   139,   103,    21,    22,   105,   113,
-      94,    94,    95,    71,   111,    59,   107,     0,   105,   118,
-     155,    30,   106,   107,   132,   109,   110,   125,   141,   291,
-     114,   115,   140,   205,    83,   123,   120,    46,    47,   122,
-       3,   130,    17,    18,    19,    54,   154,   240,   130,    71,
-      59,   124,   140,   337,   138,    92,    71,   250,   251,   143,
-     140,   124,   143,   141,   321,   141,   150,   113,   176,   153,
-     388,    80,   141,   287,   140,   243,   360,   115,   186,   363,
-     164,   189,   166,   167,   287,   119,   343,   141,    66,   141,
-     262,   142,   163,   140,   140,   104,   119,   130,   140,   131,
-      55,   113,    77,   114,   113,   304,    31,    62,    86,   307,
-     142,   304,   140,   221,   398,   287,   128,   126,    10,   227,
-     140,   205,   206,    55,   133,    51,   210,   136,   140,    83,
-      62,    10,    11,    12,    13,    14,    15,    16,    17,    18,
-      19,   225,    21,    22,     4,   229,     6,   404,    44,    45,
-      76,    43,   355,   261,   136,   263,   240,   139,     5,   243,
-     268,    53,   361,   335,    90,    91,    95,   275,   361,    72,
-      72,    31,   370,   371,    44,    45,    79,    79,   131,    21,
-      22,    84,     7,   355,    87,    87,   112,     7,   279,   382,
-      32,   140,   300,   392,     4,   279,     6,   140,    77,   392,
-      92,   141,   142,   287,     9,   108,   

<TRUNCATED>


[18/50] [abbrv] incubator-quickstep git commit: Long lived Foreman thread

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
index 8352d55..9204073 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
@@ -61,9 +61,11 @@ class ExecutionGeneratorTestRunner : public TextBasedTestRunner {
 
     bus_.Initialize();
 
-    foreman_.reset(new Foreman(&bus_,
-                               test_database_loader_.catalog_database(),
-                               test_database_loader_.storage_manager()));
+    main_thread_client_id_ = bus_.Connect();
+    bus_.RegisterClientAsSender(main_thread_client_id_, kAdmitRequestMessage);
+    bus_.RegisterClientAsSender(main_thread_client_id_, kPoisonMessage);
+    bus_.RegisterClientAsReceiver(main_thread_client_id_, kWorkloadCompletionMessage);
+
     worker_.reset(new Worker(0, &bus_));
 
     std::vector<client_id> worker_client_ids;
@@ -75,27 +77,20 @@ class ExecutionGeneratorTestRunner : public TextBasedTestRunner {
 
     workers_.reset(new WorkerDirectory(1 /* number of workers */,
                                        worker_client_ids, numa_nodes));
-    foreman_->setWorkerDirectory(workers_.get());
+    foreman_.reset(new Foreman(main_thread_client_id_,
+                               workers_.get(),
+                               &bus_,
+                               test_database_loader_.catalog_database(),
+                               test_database_loader_.storage_manager()));
 
+    foreman_->start();
     worker_->start();
   }
 
   ~ExecutionGeneratorTestRunner() {
-    std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
-    TaggedMessage poison_tagged_message(poison_message.get(),
-                                        sizeof(*poison_message),
-                                        quickstep::kPoisonMessage);
-
-    Address worker_address;
-    MessageStyle single_receiver_style;
-
-    worker_address.AddRecipient(worker_->getBusClientID());
-    bus_.Send(foreman_->getBusClientID(),
-              worker_address,
-              single_receiver_style,
-              std::move(poison_tagged_message));
-
+    QueryExecutionUtil::BroadcastPoisonMessage(main_thread_client_id_, &bus_);
     worker_->join();
+    foreman_->join();
   }
 
   void runTestCase(const std::string &input,
@@ -112,6 +107,8 @@ class ExecutionGeneratorTestRunner : public TextBasedTestRunner {
 
   std::unique_ptr<WorkerDirectory> workers_;
 
+  tmb::client_id main_thread_client_id_;
+
   // This map is needed for InsertDestination and some operators that send
   // messages to Foreman directly. To know the reason behind the design of this
   // map, see the note in InsertDestination.hpp.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/tests/TestDatabaseLoader.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/TestDatabaseLoader.cpp b/query_optimizer/tests/TestDatabaseLoader.cpp
index 2de69b6..764ff2f 100644
--- a/query_optimizer/tests/TestDatabaseLoader.cpp
+++ b/query_optimizer/tests/TestDatabaseLoader.cpp
@@ -122,6 +122,7 @@ void TestDatabaseLoader::loadTestRelation() {
                                          nullptr,
                                          &storage_manager_,
                                          0 /* dummy op index */,
+                                         0,  // dummy query ID.
                                          scheduler_client_id_,
                                          &bus_);
   int sign = 1;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 47e36e9..933918b 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -132,6 +132,7 @@ void DeleteWorkOrder::execute() {
   proto.set_operator_index(delete_operator_index_);
   proto.set_block_id(input_block_id_);
   proto.set_relation_id(input_relation_.getID());
+  proto.set_query_id(query_id_);
 
   // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
   const std::size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index aa8a688..74da8c1 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -174,6 +174,7 @@ class DeleteWorkOrder : public WorkOrder {
   StorageManager *storage_manager_;
 
   const std::size_t delete_operator_index_;
+
   const tmb::client_id scheduler_client_id_;
   MessageBus *bus_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 6f4271d..9762f04 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -644,14 +644,14 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param hash_table The JoinHashTable to use.
+   * @param lookup_block_id The block id of the probe_relation.
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param is_selection_on_build Whether each Scalar in the \p selection vector
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
-   * @param lookup_block_id The block id of the probe_relation.
+   * @param hash_table The JoinHashTable to use.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index 86f8eaf..3125447 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -85,6 +85,7 @@ class RebuildWorkOrder : public WorkOrder {
     proto.set_operator_index(input_operator_index_);
     proto.set_block_id(block_ref_->getID());
     proto.set_relation_id(input_relation_id_);
+    proto.set_query_id(query_id_);
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const std::size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.cpp b/relational_operators/SortMergeRunOperator.cpp
index e398d62..1603b78 100644
--- a/relational_operators/SortMergeRunOperator.cpp
+++ b/relational_operators/SortMergeRunOperator.cpp
@@ -327,6 +327,7 @@ void SortMergeRunWorkOrder::execute() {
 
   // Send completion message to operator.
   FeedbackMessage msg(SortMergeRunOperator::kRunOutputMessage,
+                      getQueryID(),
                       operator_index_,
                       serialized_output.first,
                       serialized_output.second);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index 1b2979e..f103b0e 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -114,6 +114,7 @@ void UpdateWorkOrder::execute() {
   proto.set_operator_index(update_operator_index_);
   proto.set_block_id(input_block_id_);
   proto.set_relation_id(relation_.getID());
+  proto.set_query_id(query_id_);
 
   // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
   const std::size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index f6c5053..4471a17 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -184,6 +184,7 @@ class UpdateWorkOrder : public WorkOrder {
   StorageManager *storage_manager_;
 
   const std::size_t update_operator_index_;
+
   const tmb::client_id scheduler_client_id_;
   MessageBus *bus_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index 059865d..df195cc 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -65,20 +65,25 @@ class WorkOrder {
    *       relational operator.
    */
   struct FeedbackMessageHeader {
+    std::size_t query_id;
     std::size_t rel_op_index;
     std::size_t payload_size;
     FeedbackMessageType payload_type;
 
     /**
      * @brief Header constructor.
+     *
+     * @param query_id The ID of the query.
      * @param relational_op_index Index of the relation operator.
      * @param payload_size Size of the payload of the message.
      * @param payload_type Type of payload.
      */
-    FeedbackMessageHeader(const std::size_t relational_op_index,
+    FeedbackMessageHeader(const std::size_t query_id,
+                          const std::size_t relational_op_index,
                           const std::size_t payload_size,
                           const FeedbackMessageType payload_type)
-        : rel_op_index(relational_op_index),
+        : query_id(query_id),
+          rel_op_index(relational_op_index),
           payload_size(payload_size),
           payload_type(payload_type) {}
   };
@@ -93,17 +98,19 @@ class WorkOrder {
      * @brief Feedback message constructor.
      *
      * @param type Type of the message.
+     * @param query_id The ID of the query.
      * @param rel_op_index Relational operator index.
      * @param payload Blob of payload.
      * @param payload_size Size of the payload blob.
      * @param ownership Whether to take ownership of the payload blob.
      */
     FeedbackMessage(const FeedbackMessageType type,
+                    const std::size_t query_id,
                     const std::size_t rel_op_index,
                     void *payload,
                     const std::size_t payload_size,
                     const bool ownership = true)
-        : header_(rel_op_index, payload_size, type),
+        : header_(query_id, rel_op_index, payload_size, type),
           payload_(payload),
           ownership_(ownership) {}
 
@@ -285,6 +292,13 @@ class WorkOrder {
         " receiver thread with TMB client ID " << receiver_id;
   }
 
+  /**
+   * @brief Get the ID of the query which this WorkOder belongs to.
+   **/
+  inline const std::size_t getQueryID() const {
+    return query_id_;
+  }
+
  protected:
   /**
    * @brief Constructor.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index fdcc54f..fd4692a 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -228,6 +228,8 @@ class AggregationOperatorTest : public ::testing::Test {
 
     // Setup the aggregation state proto in the query context proto.
     serialization::QueryContext query_context_proto;
+    query_context_proto.set_query_id(0);  // dummy query ID.
+
     const QueryContext::aggregation_state_id aggr_state_index = query_context_proto.aggregation_states_size();
     serialization::AggregationOperationState *aggr_state_proto = query_context_proto.add_aggregation_states();
     aggr_state_proto->set_relation_id(table_->getID());
@@ -319,6 +321,8 @@ class AggregationOperatorTest : public ::testing::Test {
 
     // Setup the aggregation state proto in the query context proto.
     serialization::QueryContext query_context_proto;
+    query_context_proto.set_query_id(0);  // dummy query ID.
+
     const QueryContext::aggregation_state_id aggr_state_index = query_context_proto.aggregation_states_size();
     serialization::AggregationOperationState *aggr_state_proto = query_context_proto.add_aggregation_states();
     aggr_state_proto->set_relation_id(table_->getID());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 074b603..9c34170 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -294,6 +294,7 @@ class HashJoinOperatorTest : public ::testing::TestWithParam<HashTableImplType>
 TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
+  query_context_proto.set_query_id(0);  // dummy query ID.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -434,6 +435,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
 TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
+  query_context_proto.set_query_id(0);  // dummy query ID.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -604,6 +606,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
 TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
+  query_context_proto.set_query_id(0);  // dummy query ID.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -739,6 +742,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
 TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
+  query_context_proto.set_query_id(0);  // dummy query ID.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -906,6 +910,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
 TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
+  query_context_proto.set_query_id(0);  // dummy query ID.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -1083,6 +1088,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
 TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
+  query_context_proto.set_query_id(0);  // dummy query ID.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index fc10671..46bce5b 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -189,6 +189,7 @@ class RunTest : public ::testing::Test {
                                        nullptr,
                                        storage_manager_.get(),
                                        kOpIndex,
+                                       0,  // dummy query ID.
                                        foreman_client_id_,
                                        &bus_));
   }
@@ -433,6 +434,7 @@ class RunMergerTest : public ::testing::Test {
                                        nullptr,
                                        storage_manager_.get(),
                                        kOpIndex,
+                                       0,  // dummy query ID.
                                        foreman_client_id_,
                                        &bus_));
   }
@@ -1269,6 +1271,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
     ASSERT_EQ(null_col3_, result_table_->getAttributeByName("null-col-3")->getID());
     ASSERT_EQ(tid_col_, result_table_->getAttributeByName("tid")->getID());
 
+    query_context_proto_.set_query_id(0);  // dummy query ID.
+
     // Setup the InsertDestination proto in the query context proto.
     insert_destination_index_ = query_context_proto_.insert_destinations_size();
     serialization::InsertDestination *insert_destination_proto = query_context_proto_.add_insert_destinations();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index 71a80e4..bd682c2 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -328,6 +328,7 @@ class SortRunGenerationOperatorTest : public ::testing::Test {
                    const std::vector<bool> &null_ordering) {
     // Setup the InsertDestination proto in the query context proto.
     serialization::QueryContext query_context_proto;
+    query_context_proto.set_query_id(0);  // dummy query ID.
 
     const QueryContext::insert_destination_id insert_destination_index =
         query_context_proto.insert_destinations_size();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index 5860745..5bcbee5 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -180,6 +180,7 @@ TEST_F(TextScanOperatorTest, ScanTest) {
 
   // Setup the InsertDestination proto in the query context proto.
   serialization::QueryContext query_context_proto;
+  query_context_proto.set_query_id(0);  // dummy query ID.
 
   QueryContext::insert_destination_id output_destination_index = query_context_proto.insert_destinations_size();
   serialization::InsertDestination *output_destination_proto = query_context_proto.add_insert_destinations();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 354bed4..2866c5f 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -60,6 +60,7 @@ InsertDestination::InsertDestination(const CatalogRelationSchema &relation,
                                      const StorageBlockLayout *layout,
                                      StorageManager *storage_manager,
                                      const std::size_t relational_op_index,
+                                     const std::size_t query_id,
                                      const tmb::client_id scheduler_client_id,
                                      tmb::MessageBus *bus)
     : thread_id_map_(*ClientIDMap::Instance()),
@@ -67,6 +68,7 @@ InsertDestination::InsertDestination(const CatalogRelationSchema &relation,
       relation_(relation),
       layout_(layout),
       relational_op_index_(relational_op_index),
+      query_id_(query_id),
       scheduler_client_id_(scheduler_client_id),
       bus_(DCHECK_NOTNULL(bus)) {
   if (layout_ == nullptr) {
@@ -74,11 +76,13 @@ InsertDestination::InsertDestination(const CatalogRelationSchema &relation,
   }
 }
 
-InsertDestination* InsertDestination::ReconstructFromProto(const serialization::InsertDestination &proto,
-                                                           const CatalogRelationSchema &relation,
-                                                           StorageManager *storage_manager,
-                                                           const tmb::client_id scheduler_client_id,
-                                                           tmb::MessageBus *bus) {
+InsertDestination* InsertDestination::ReconstructFromProto(
+    const std::size_t query_id,
+    const serialization::InsertDestination &proto,
+    const CatalogRelationSchema &relation,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
   DCHECK(ProtoIsValid(proto, relation));
 
   StorageBlockLayout *layout = nullptr;
@@ -93,6 +97,7 @@ InsertDestination* InsertDestination::ReconstructFromProto(const serialization::
                                                     layout,
                                                     storage_manager,
                                                     proto.relational_op_index(),
+                                                    query_id,
                                                     scheduler_client_id,
                                                     bus);
     }
@@ -107,6 +112,7 @@ InsertDestination* InsertDestination::ReconstructFromProto(const serialization::
                                             storage_manager,
                                             move(blocks),
                                             proto.relational_op_index(),
+                                            query_id,
                                             scheduler_client_id,
                                             bus);
     }
@@ -134,6 +140,7 @@ InsertDestination* InsertDestination::ReconstructFromProto(const serialization::
           storage_manager,
           move(partitions),
           proto.relational_op_index(),
+          query_id,
           scheduler_client_id,
           bus);
     }
@@ -262,6 +269,7 @@ MutableBlockReference AlwaysCreateBlockInsertDestination::createNewBlock() {
   serialization::CatalogRelationNewBlockMessage proto;
   proto.set_relation_id(relation_.getID());
   proto.set_block_id(new_id);
+  proto.set_query_id(getQueryID());
 
   const size_t proto_length = proto.ByteSize();
   char *proto_bytes = static_cast<char*>(malloc(proto_length));
@@ -309,6 +317,7 @@ MutableBlockReference BlockPoolInsertDestination::createNewBlock() {
   serialization::CatalogRelationNewBlockMessage proto;
   proto.set_relation_id(relation_.getID());
   proto.set_block_id(new_id);
+  proto.set_query_id(getQueryID());
 
   const size_t proto_length = proto.ByteSize();
   char *proto_bytes = static_cast<char*>(malloc(proto_length));
@@ -385,21 +394,29 @@ const std::vector<block_id>& BlockPoolInsertDestination::getTouchedBlocksInterna
   return done_block_ids_;
 }
 
-PartitionAwareInsertDestination::PartitionAwareInsertDestination(PartitionSchemeHeader *partition_scheme_header,
-                                                                 const CatalogRelationSchema &relation,
-                                                                 const StorageBlockLayout *layout,
-                                                                 StorageManager *storage_manager,
-                                                                 vector<vector<block_id>> &&partitions,
-                                                                 const std::size_t relational_op_index,
-                                                                 const tmb::client_id scheduler_client_id,
-                                                                 tmb::MessageBus *bus)
-    : InsertDestination(relation, layout, storage_manager, relational_op_index, scheduler_client_id, bus),
+PartitionAwareInsertDestination::PartitionAwareInsertDestination(
+    PartitionSchemeHeader *partition_scheme_header,
+    const CatalogRelationSchema &relation,
+    const StorageBlockLayout *layout,
+    StorageManager *storage_manager,
+    vector<vector<block_id>> &&partitions,
+    const std::size_t relational_op_index,
+    const std::size_t query_id,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus)
+    : InsertDestination(relation,
+                        layout,
+                        storage_manager,
+                        relational_op_index,
+                        query_id,
+                        scheduler_client_id,
+                        bus),
       partition_scheme_header_(DCHECK_NOTNULL(partition_scheme_header)),
       available_block_refs_(partition_scheme_header_->getNumPartitions()),
       available_block_ids_(move(partitions)),
       done_block_ids_(partition_scheme_header_->getNumPartitions()),
-      mutexes_for_partition_(new SpinMutex[partition_scheme_header_->getNumPartitions()]) {
-}
+      mutexes_for_partition_(
+          new SpinMutex[partition_scheme_header_->getNumPartitions()]) {}
 
 MutableBlockReference PartitionAwareInsertDestination::createNewBlock() {
   FATAL_ERROR("PartitionAwareInsertDestination::createNewBlock needs a partition id as an argument.");
@@ -415,6 +432,7 @@ MutableBlockReference PartitionAwareInsertDestination::createNewBlockInPartition
   proto.set_relation_id(relation_.getID());
   proto.set_block_id(new_id);
   proto.set_partition_id(part_id);
+  proto.set_query_id(getQueryID());
 
   const size_t proto_length = proto.ByteSize();
   char *proto_bytes = static_cast<char*>(malloc(proto_length));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index 670cd6c..5ff33f5 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -78,6 +78,7 @@ class InsertDestination : public InsertDestinationInterface {
    * @param storage_manager The StorageManager to use.
    * @param relational_op_index The index of the relational operator in the
    *        QueryPlan DAG that has outputs.
+   * @param query_id The ID of this query.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
    * @param bus A pointer to the TMB.
    **/
@@ -85,6 +86,7 @@ class InsertDestination : public InsertDestinationInterface {
                     const StorageBlockLayout *layout,
                     StorageManager *storage_manager,
                     const std::size_t relational_op_index,
+                    const std::size_t query_id,
                     const tmb::client_id scheduler_client_id,
                     tmb::MessageBus *bus);
 
@@ -98,6 +100,7 @@ class InsertDestination : public InsertDestinationInterface {
    * @brief A factory method to generate the InsertDestination from the
    *        serialized Protocol Buffer representation.
    *
+   * @param query_id The ID of this query.
    * @param proto A serialized Protocol Buffer representation of an
    *        InsertDestination, originally generated by the optimizer.
    * @param relation The relation to insert tuples into.
@@ -107,11 +110,13 @@ class InsertDestination : public InsertDestinationInterface {
    *
    * @return The constructed InsertDestination.
    */
-  static InsertDestination* ReconstructFromProto(const serialization::InsertDestination &proto,
-                                                 const CatalogRelationSchema &relation,
-                                                 StorageManager *storage_manager,
-                                                 const tmb::client_id scheduler_client_id,
-                                                 tmb::MessageBus *bus);
+  static InsertDestination* ReconstructFromProto(
+      const std::size_t query_id,
+      const serialization::InsertDestination &proto,
+      const CatalogRelationSchema &relation,
+      StorageManager *storage_manager,
+      const tmb::client_id scheduler_client_id,
+      tmb::MessageBus *bus);
 
   /**
    * @brief Check whether a serialized InsertDestination is fully-formed and
@@ -211,6 +216,7 @@ class InsertDestination : public InsertDestinationInterface {
     proto.set_operator_index(relational_op_index_);
     proto.set_block_id(id);
     proto.set_relation_id(relation_.getID());
+    proto.set_query_id(query_id_);
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const std::size_t proto_length = proto.ByteSize();
@@ -253,6 +259,10 @@ class InsertDestination : public InsertDestinationInterface {
         " ID " << scheduler_client_id_;
   }
 
+  inline const std::size_t getQueryID() const {
+    return query_id_;
+  }
+
   const ClientIDMap &thread_id_map_;
 
   StorageManager *storage_manager_;
@@ -260,6 +270,7 @@ class InsertDestination : public InsertDestinationInterface {
 
   std::unique_ptr<const StorageBlockLayout> layout_;
   const std::size_t relational_op_index_;
+  const std::size_t query_id_;
 
   tmb::client_id scheduler_client_id_;
   tmb::MessageBus *bus_;
@@ -288,10 +299,16 @@ class AlwaysCreateBlockInsertDestination : public InsertDestination {
                                      const StorageBlockLayout *layout,
                                      StorageManager *storage_manager,
                                      const std::size_t relational_op_index,
+                                     const std::size_t query_id,
                                      const tmb::client_id scheduler_client_id,
                                      tmb::MessageBus *bus)
-      : InsertDestination(relation, layout, storage_manager, relational_op_index, scheduler_client_id, bus) {
-  }
+      : InsertDestination(relation,
+                          layout,
+                          storage_manager,
+                          relational_op_index,
+                          query_id,
+                          scheduler_client_id,
+                          bus) {}
 
   ~AlwaysCreateBlockInsertDestination() override {
   }
@@ -334,16 +351,23 @@ class BlockPoolInsertDestination : public InsertDestination {
    * @param relational_op_index The index of the relational operator in the
    *        QueryPlan DAG that has outputs.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
+   * @param query_id The ID of the query.
    * @param bus A pointer to the TMB.
    **/
   BlockPoolInsertDestination(const CatalogRelationSchema &relation,
                              const StorageBlockLayout *layout,
                              StorageManager *storage_manager,
                              const std::size_t relational_op_index,
+                             const std::size_t query_id,
                              const tmb::client_id scheduler_client_id,
                              tmb::MessageBus *bus)
-      : InsertDestination(relation, layout, storage_manager, relational_op_index, scheduler_client_id, bus) {
-  }
+      : InsertDestination(relation,
+                          layout,
+                          storage_manager,
+                          relational_op_index,
+                          query_id,
+                          scheduler_client_id,
+                          bus) {}
 
   /**
    * @brief Constructor.
@@ -363,9 +387,16 @@ class BlockPoolInsertDestination : public InsertDestination {
                              StorageManager *storage_manager,
                              std::vector<block_id> &&blocks,
                              const std::size_t relational_op_index,
+                             const std::size_t query_id,
                              const tmb::client_id scheduler_client_id,
                              tmb::MessageBus *bus)
-      : InsertDestination(relation, layout, storage_manager, relational_op_index, scheduler_client_id, bus),
+      : InsertDestination(relation,
+                          layout,
+                          storage_manager,
+                          relational_op_index,
+                          query_id,
+                          scheduler_client_id,
+                          bus),
         available_block_ids_(std::move(blocks)) {
     // TODO(chasseur): Once block fill statistics are available, replace this
     // with something smarter.
@@ -386,7 +417,6 @@ class BlockPoolInsertDestination : public InsertDestination {
   MutableBlockReference createNewBlock() override;
 
  private:
-  FRIEND_TEST(ForemanTest, TwoNodesDAGPartiallyFilledBlocksTest);
   FRIEND_TEST(QueryManagerTest, TwoNodesDAGPartiallyFilledBlocksTest);
 
   // A vector of references to blocks which are loaded in memory.
@@ -416,17 +446,20 @@ class PartitionAwareInsertDestination : public InsertDestination {
    * @param partitions The blocks in partitions.
    * @param relational_op_index The index of the relational operator in the
    *        QueryPlan DAG that has outputs.
+   * @param query_id The ID of the query.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
    * @param bus A pointer to the TMB.
    **/
-  PartitionAwareInsertDestination(PartitionSchemeHeader *partition_scheme_header,
-                                  const CatalogRelationSchema &relation,
-                                  const StorageBlockLayout *layout,
-                                  StorageManager *storage_manager,
-                                  std::vector<std::vector<block_id>> &&partitions,
-                                  const std::size_t relational_op_index,
-                                  const tmb::client_id scheduler_client_id,
-                                  tmb::MessageBus *bus);
+  PartitionAwareInsertDestination(
+      PartitionSchemeHeader *partition_scheme_header,
+      const CatalogRelationSchema &relation,
+      const StorageBlockLayout *layout,
+      StorageManager *storage_manager,
+      std::vector<std::vector<block_id>> &&partitions,
+      const std::size_t relational_op_index,
+      const std::size_t query_id,
+      const tmb::client_id scheduler_client_id,
+      tmb::MessageBus *bus);
 
   ~PartitionAwareInsertDestination() override {
     delete[] mutexes_for_partition_;


[08/50] [abbrv] incubator-quickstep git commit: Disabled Clang Temporarily.

Posted by zu...@apache.org.
Disabled Clang Temporarily.


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

Branch: refs/heads/tmp-relation-col-store
Commit: eab1c9a4f80553d500afdf8cb385822b03aaad0d
Parents: fd75e17
Author: Zuyu Zhang <zz...@pivotal.io>
Authored: Wed Jun 8 18:45:37 2016 -0700
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed Jun 8 20:23:45 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/eab1c9a4/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 483a02b..08d6f38 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -12,7 +12,7 @@ cache: ccache
 
 compiler:
   - gcc
-  - clang
+  # clang
 
 env:
   - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
@@ -21,7 +21,7 @@ env:
   - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=none
 
 install:
-  - if [ "$VECTOR_COPY_ELISION_LEVEL" = "joinwithbinaryexpressions" ] && [ "$CC" = "gcc" ] && [ "$BUILD_TYPE" = "Debug" ]; then
+  - if [ "$VECTOR_COPY_ELISION_LEVEL" = "joinwithbinaryexpressions" ] && [ "$CC" = "gcc" ]; then
       export MAKE_JOBS=1;
     else
       export MAKE_JOBS=2;
@@ -77,11 +77,9 @@ addons:
   apt:
     sources:
       - ubuntu-toolchain-r-test
-      - llvm-toolchain-precise-3.7
     packages:
       - gcc-5
       - g++-5
-      - clang-3.7
       - binutils-gold
       - libprotobuf-dev
       - protobuf-compiler


[35/50] [abbrv] incubator-quickstep git commit: Basic support to report individual work order profiling results

Posted by zu...@apache.org.
Basic support to report individual work order profiling results

- A flag to enable work order profiling report generation.
- At the end of each query, a report is generated which includes worker
  ID, its NUMA socket, the operator that produced the WorkOrder
  and the execution time in microseconds for the latest query.
- The output is printed on stdout in CSV format as of now.
- As this is a rudimentary support for the functionality, there is a lot of
  future work in this regards, which includes printing of CPU core information,
  printing operator name, allowing user to specify a file where the output can
  be written etc.
- Fixed a bug in constructing Foreman thread.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 07435a430776c0b8b6381a4c0f0470250814c14b
Parents: c1476d1
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Thu Jun 16 14:03:34 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Mon Jun 20 09:56:52 2016 -0500

----------------------------------------------------------------------
 cli/QuickstepCli.cpp               | 12 +++++++-
 query_execution/CMakeLists.txt     |  2 ++
 query_execution/Foreman.cpp        | 26 +++++++++++++++--
 query_execution/Foreman.hpp        | 22 ++++++++++++++-
 query_execution/PolicyEnforcer.cpp | 15 ++++++++++
 query_execution/PolicyEnforcer.hpp | 50 +++++++++++++++++++++++++++++++--
 6 files changed, 121 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/07435a43/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 35bd16e..3f99130 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -137,6 +137,9 @@ static constexpr char kPathSeparator = '/';
 static constexpr char kDefaultStoragePath[] = "qsstor/";
 #endif
 
+DEFINE_bool(profile_and_report_workorder_perf, false,
+    "If true, Quickstep will record the exceution time of all the individual "
+    "normal work orders and report it at the end of query execution.");
 DEFINE_int32(num_workers, 0, "Number of worker threads. If this value is "
                              "specified and is greater than 0, then this "
                              "user-supplied value is used. Else (i.e. the"
@@ -356,7 +359,9 @@ int main(int argc, char* argv[]) {
                   &bus,
                   query_processor->getDefaultDatabase(),
                   query_processor->getStorageManager(),
-                  num_numa_nodes_system);
+                  -1,  // Don't pin the Foreman thread.
+                  num_numa_nodes_system,
+                  quickstep::FLAGS_profile_and_report_workorder_perf);
 
   // Start the worker threads.
   for (Worker &worker : workers) {
@@ -461,6 +466,11 @@ int main(int argc, char* argv[]) {
           printf("Time: %s ms\n",
                  quickstep::DoubleToStringWithSignificantDigits(
                      time_ms.count(), 3).c_str());
+          if (quickstep::FLAGS_profile_and_report_workorder_perf) {
+            // TODO(harshad) - Allow user specified file instead of stdout.
+            foreman.printWorkOrderProfilingResults(query_handle->query_id(),
+                                                   stdout);
+          }
         } catch (const std::exception &e) {
           fprintf(stderr, "QUERY EXECUTION ERROR: %s\n", e.what());
           break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/07435a43/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 501166e..b031a44 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -1,5 +1,7 @@
 #   Copyright 2011-2015 Quickstep Technologies LLC.
 #   Copyright 2015-2016 Pivotal Software, Inc.
+#   Copyright 2016, Quickstep Research Group, Computer Sciences Department, 
+#     University of Wisconsin\u2014Madison.
 #
 #   Licensed under the Apache License, Version 2.0 (the "License");
 #   you may not use this file except in compliance with the License.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/07435a43/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index 828834d..f9f2e7a 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -18,7 +18,9 @@
 #include "query_execution/Foreman.hpp"
 
 #include <cstddef>
+#include <cstdio>
 #include <memory>
+#include <tuple>
 #include <utility>
 #include <vector>
 
@@ -54,7 +56,8 @@ Foreman::Foreman(const tmb::client_id main_thread_client_id,
                  CatalogDatabaseLite *catalog_database,
                  StorageManager *storage_manager,
                  const int cpu_id,
-                 const size_t num_numa_nodes)
+                 const size_t num_numa_nodes,
+                 const bool profile_individual_workorders)
     : ForemanLite(bus, cpu_id),
       main_thread_client_id_(main_thread_client_id),
       worker_directory_(DCHECK_NOTNULL(worker_directory)),
@@ -90,7 +93,8 @@ Foreman::Foreman(const tmb::client_id main_thread_client_id,
       catalog_database_,
       storage_manager_,
       worker_directory_,
-      bus_));
+      bus_,
+      profile_individual_workorders));
 }
 
 void Foreman::run() {
@@ -229,4 +233,22 @@ void Foreman::sendWorkerMessage(const size_t worker_thread_index,
       << worker_directory_->getClientID(worker_thread_index);
 }
 
+void Foreman::printWorkOrderProfilingResults(const std::size_t query_id,
+                                             std::FILE *out) const {
+  const std::vector<
+      std::tuple<std::size_t, std::size_t, std::size_t>>
+      &recorded_times = policy_enforcer_->getProfilingResults(query_id);
+  fputs("Worker ID, NUMA Socket, Operator ID, Time (microseconds)\n", out);
+  for (auto workorder_entry : recorded_times) {
+    // Note: Index of the "worker thread index" in the tuple is 0.
+    const std::size_t worker_id = std::get<0>(workorder_entry);
+    fprintf(out,
+            "%lu, %d, %lu, %lu\n",
+            worker_id,
+            worker_directory_->getNUMANode(worker_id),
+            std::get<1>(workorder_entry),
+            std::get<2>(workorder_entry));
+  }
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/07435a43/query_execution/Foreman.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.hpp b/query_execution/Foreman.hpp
index 94cb9fc..7be57e7 100644
--- a/query_execution/Foreman.hpp
+++ b/query_execution/Foreman.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_QUERY_EXECUTION_FOREMAN_HPP_
 
 #include <cstddef>
+#include <cstdio>
 #include <memory>
 #include <vector>
 
@@ -57,6 +58,8 @@ class Foreman final : public ForemanLite {
    * @param storage_manager The StorageManager to use.
    * @param cpu_id The ID of the CPU to which the Foreman thread can be pinned.
    * @param num_numa_nodes The number of NUMA nodes in the system.
+   * @param profile_individual_workorders Whether every workorder's execution
+   *        be profiled or not.
    *
    * @note If cpu_id is not specified, Foreman thread can be possibly moved
    *       around on different CPUs by the OS.
@@ -67,10 +70,27 @@ class Foreman final : public ForemanLite {
           CatalogDatabaseLite *catalog_database,
           StorageManager *storage_manager,
           const int cpu_id = -1,
-          const std::size_t num_numa_nodes = 1);
+          const std::size_t num_numa_nodes = 1,
+          const bool profile_individual_workorders = false);
 
   ~Foreman() override {}
 
+  /**
+   * @brief Print the results of profiling individual work orders for a given
+   *        query.
+   *
+   * TODO(harshad) - Add the name of the operator to the output.
+   * TODO(harshad) - Add the CPU core ID of the operator to the output. This
+   * will require modifying the WorkerDirectory to remember worker affinities.
+   * Until then, the users can refer to the worker_affinities provided to the
+   * cli to infer the CPU core ID where a given worker is pinned.
+   *
+   * @param query_id The ID of the query for which the results are to be printed.
+   * @param out The file stream.
+   **/
+  void printWorkOrderProfilingResults(const std::size_t query_id,
+                                      std::FILE *out) const;
+
  protected:
   void run() override;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/07435a43/query_execution/PolicyEnforcer.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcer.cpp b/query_execution/PolicyEnforcer.cpp
index 9f0502d..84aa86a 100644
--- a/query_execution/PolicyEnforcer.cpp
+++ b/query_execution/PolicyEnforcer.cpp
@@ -76,6 +76,9 @@ void PolicyEnforcer::processMessage(const TaggedMessage &tagged_message) {
       query_id = proto.query_id();
       worker_directory_->decrementNumQueuedWorkOrders(
           proto.worker_thread_index());
+      if (profile_individual_workorders_) {
+        recordTimeForWorkOrder(proto);
+      }
       break;
     }
     case kRebuildWorkOrderCompleteMessage: {
@@ -197,4 +200,16 @@ bool PolicyEnforcer::admitQueries(
   return true;
 }
 
+void PolicyEnforcer::recordTimeForWorkOrder(
+    const serialization::NormalWorkOrderCompletionMessage &proto) {
+  const std::size_t query_id = proto.query_id();
+  if (workorder_time_recorder_.find(query_id) == workorder_time_recorder_.end()) {
+    workorder_time_recorder_[query_id];
+  }
+  workorder_time_recorder_[query_id].emplace_back(
+      proto.worker_thread_index(),
+      proto.operator_index(),
+      proto.execution_time_in_microseconds());
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/07435a43/query_execution/PolicyEnforcer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcer.hpp b/query_execution/PolicyEnforcer.hpp
index 9f87056..470ff2a 100644
--- a/query_execution/PolicyEnforcer.hpp
+++ b/query_execution/PolicyEnforcer.hpp
@@ -21,6 +21,7 @@
 #include <cstddef>
 #include <memory>
 #include <queue>
+#include <tuple>
 #include <unordered_map>
 #include <vector>
 
@@ -62,13 +63,15 @@ class PolicyEnforcer {
                  CatalogDatabaseLite *catalog_database,
                  StorageManager *storage_manager,
                  WorkerDirectory *worker_directory,
-                 tmb::MessageBus *bus)
+                 tmb::MessageBus *bus,
+                 const bool profile_individual_workorders = false)
       : foreman_client_id_(foreman_client_id),
         num_numa_nodes_(num_numa_nodes),
         catalog_database_(catalog_database),
         storage_manager_(storage_manager),
         worker_directory_(worker_directory),
-        bus_(bus) {}
+        bus_(bus),
+        profile_individual_workorders_(profile_individual_workorders) {}
 
   /**
    * @brief Destructor.
@@ -143,9 +146,40 @@ class PolicyEnforcer {
     return !(admitted_queries_.empty() && waiting_queries_.empty());
   }
 
+  /**
+   * @brief Get the profiling results for individual work order execution for a
+   *        given query.
+   *
+   * @note This function should only be called if profiling individual work
+   *       orders option is enabled.
+   *
+   * @param query_id The ID of the query for which the profiling results are
+   *        requested.
+   *
+   * @return A vector of tuples, each being a single profiling entry.
+   **/
+  inline const std::vector<std::tuple<std::size_t, std::size_t, std::size_t>>&
+      getProfilingResults(const std::size_t query_id) const {
+    DCHECK(profile_individual_workorders_);
+    DCHECK(workorder_time_recorder_.find(query_id) !=
+           workorder_time_recorder_.end());
+    return workorder_time_recorder_.at(query_id);
+  }
+
  private:
   static constexpr std::size_t kMaxConcurrentQueries = 1;
 
+  /**
+   * @brief Record the execution time for a finished WorkOrder.
+   *
+   * TODO(harshad) - Extend the functionality to rebuild work orders.
+   *
+   * @param proto The completion message proto sent after the WorkOrder
+   *        execution.
+   **/
+  void recordTimeForWorkOrder(
+      const serialization::NormalWorkOrderCompletionMessage &proto);
+
   const tmb::client_id foreman_client_id_;
   const std::size_t num_numa_nodes_;
 
@@ -154,6 +188,7 @@ class PolicyEnforcer {
   WorkerDirectory *worker_directory_;
 
   tmb::MessageBus *bus_;
+  const bool profile_individual_workorders_;
 
   // Key = query ID, value = QueryManager* for the key query.
   std::unordered_map<std::size_t, std::unique_ptr<QueryManager>> admitted_queries_;
@@ -161,6 +196,17 @@ class PolicyEnforcer {
   // The queries which haven't been admitted yet.
   std::queue<QueryHandle*> waiting_queries_;
 
+  // Key = Query ID.
+  // Value = A tuple indicating a record of executing a work order.
+  // Within a tuple ...
+  // 1st element: Logical worker ID.
+  // 2nd element: Operator ID.
+  // 3rd element: Time in microseconds to execute the work order.
+  std::unordered_map<
+      std::size_t,
+      std::vector<std::tuple<std::size_t, std::size_t, std::size_t>>>
+      workorder_time_recorder_;
+
   DISALLOW_COPY_AND_ASSIGN(PolicyEnforcer);
 };
 


[13/50] [abbrv] incubator-quickstep git commit: Use shared libraries when building on Travis

Posted by zu...@apache.org.
Use shared libraries when building on Travis


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

Branch: refs/heads/tmp-relation-col-store
Commit: ccd11c0858362e4af45822181dc3a8d0c4db539b
Parents: 4f8fdbe
Author: navsan <na...@node-2.qs-prof.quickstep-pg0.wisc.cloudlab.us>
Authored: Thu Jun 9 11:03:42 2016 -0500
Committer: Navneet Potti <na...@apache.org>
Committed: Thu Jun 9 15:22:22 2016 -0500

----------------------------------------------------------------------
 .travis.yml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ccd11c08/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index df39fb0..142dfce 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -58,6 +58,7 @@ before_script:
   - $CLINKER --version
   - (cd build &&
      cmake -D CMAKE_BUILD_TYPE=$BUILD_TYPE
+           -D BUILD_SHARED_LIBS=On
            -D CMAKE_C_FLAGS_DEBUG="$DEBUG_FLAGS"
            -D CMAKE_CXX_FLAGS_DEBUG="$DEBUG_FLAGS"
            -D CMAKE_C_FLAGS_RELEASE="$RELEASE_FLAGS"


[10/50] [abbrv] incubator-quickstep git commit: Add options to build gflag as a shared library.

Posted by zu...@apache.org.
Add options to build gflag as a shared library.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 096abe29ab8f8510b177168f9e492f59b10e49f5
Parents: 2d39b8e
Author: Navneet Potti <na...@gmail.com>
Authored: Wed Jun 8 16:59:07 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed Jun 8 20:30:29 2016 -0700

----------------------------------------------------------------------
 CMakeLists.txt                       |  8 +++++++-
 cli/CMakeLists.txt                   |  8 +++++++-
 query_execution/CMakeLists.txt       |  8 +++++++-
 query_optimizer/CMakeLists.txt       |  8 +++++++-
 query_optimizer/tests/CMakeLists.txt | 10 ++++++++--
 relational_operators/CMakeLists.txt  | 20 +++++++++++++-------
 storage/CMakeLists.txt               | 12 +++++++++---
 transaction/CMakeLists.txt           |  8 +++++++-
 8 files changed, 65 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/096abe29/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index ef7fd50..2d10a78 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -139,6 +139,12 @@ endif()
 
 option(ENABLE_DISTRIBUTED "Use the distributed version of Quickstep" OFF)
 
+if (BUILD_SHARED_LIBS)
+  set(GFLAGS_LIB_NAME gflags_nothreads-shared)
+else()
+  set(GFLAGS_LIB_NAME gflags_nothreads-static)
+endif()
+
 # Turn on the QUICKSTEP_DEBUG flag in the source if this is a debug build.
 if (CMAKE_MAJOR_VERSION GREATER 2)
   cmake_policy(SET CMP0043 NEW)
@@ -700,7 +706,7 @@ add_subdirectory(yarn)
 add_executable (quickstep_cli_shell cli/QuickstepCli.cpp)
 # Link against direct deps (will transitively pull in everything needed).
 target_link_libraries(quickstep_cli_shell
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_cli_CommandExecutor

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/096abe29/cli/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/CMakeLists.txt b/cli/CMakeLists.txt
index 8fee7a4..faf5040 100644
--- a/cli/CMakeLists.txt
+++ b/cli/CMakeLists.txt
@@ -30,6 +30,12 @@ if(LIBNUMA_FOUND)
   set(QUICKSTEP_HAVE_LIBNUMA TRUE)
 endif()
 
+if (BUILD_SHARED_LIBS)
+  set(GFLAGS_LIB_NAME gflags_nothreads-shared)
+else()
+  set(GFLAGS_LIB_NAME gflags_nothreads-static)
+endif()
+
 configure_file (
   "${CMAKE_CURRENT_SOURCE_DIR}/CliConfig.h.in"
   "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
@@ -110,7 +116,7 @@ target_link_libraries(quickstep_cli_InputParserUtil
                       ${LIBNUMA_LIBRARY})
 endif()
 target_link_libraries(quickstep_cli_PrintToScreen
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelation
                       quickstep_storage_StorageBlock

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/096abe29/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 7d9d601..8306f78 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -19,6 +19,12 @@ QS_PROTOBUF_GENERATE_CPP(queryexecution_QueryExecutionMessages_proto_srcs
                          queryexecution_QueryExecutionMessages_proto_hdrs
                          QueryExecutionMessages.proto)
 
+if (BUILD_SHARED_LIBS)
+  set(GFLAGS_LIB_NAME gflags_nothreads-shared)
+else()
+  set(GFLAGS_LIB_NAME gflags_nothreads-static)
+endif()
+
 # Declare micro-libs:
 if (ENABLE_DISTRIBUTED)
   add_library(quickstep_queryexecution_BlockLocator BlockLocator.cpp BlockLocator.hpp)
@@ -202,7 +208,7 @@ if (ENABLE_DISTRIBUTED)
   add_executable(BlockLocator_unittest
                  "${CMAKE_CURRENT_SOURCE_DIR}/tests/BlockLocator_unittest.cpp")
   target_link_libraries(BlockLocator_unittest
-                        gflags_nothreads-static
+                        ${GFLAGS_LIB_NAME}
                         glog
                         gtest
                         quickstep_catalog_CatalogAttribute

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/096abe29/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 5c9438d..8f08130 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -24,6 +24,12 @@ configure_file (
   "${CMAKE_CURRENT_BINARY_DIR}/QueryOptimizerConfig.h"
 )
 
+if (BUILD_SHARED_LIBS)
+  set(GFLAGS_LIB_NAME gflags_nothreads-shared)
+else()
+  set(GFLAGS_LIB_NAME gflags_nothreads-static)
+endif()
+
 add_subdirectory(cost_model)
 add_subdirectory(expressions)
 add_subdirectory(logical)
@@ -182,7 +188,7 @@ target_link_libraries(quickstep_queryoptimizer_OptimizerTree
                       quickstep_utility_Macros
                       quickstep_utility_TreeStringSerializable)
 target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       quickstep_queryoptimizer_LogicalToPhysicalMapper
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_physical_Physical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/096abe29/query_optimizer/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/CMakeLists.txt b/query_optimizer/tests/CMakeLists.txt
index 07af404..6ef2a03 100644
--- a/query_optimizer/tests/CMakeLists.txt
+++ b/query_optimizer/tests/CMakeLists.txt
@@ -18,6 +18,12 @@ add_subdirectory(logical_generator)
 add_subdirectory(physical_generator)
 add_subdirectory(resolver)
 
+if (BUILD_SHARED_LIBS)
+  set(GFLAGS_LIB_NAME gflags_nothreads-shared)
+else()
+  set(GFLAGS_LIB_NAME gflags_nothreads-static)
+endif()
+
 add_library(quickstep_queryoptimizer_tests_OptimizerTest OptimizerTest.cpp OptimizerTest.hpp)
 add_library(quickstep_queryoptimizer_tests_TestDatabaseLoader TestDatabaseLoader.cpp TestDatabaseLoader.hpp)
 
@@ -102,7 +108,7 @@ add_executable(quickstep_queryoptimizer_tests_OptimizerTextTest
                "${PROJECT_SOURCE_DIR}/utility/textbased_test/TextBasedTest.hpp")
 
 target_link_libraries(quickstep_queryoptimizer_tests_ExecutionGeneratorTest
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       gtest
                       quickstep_catalog_CatalogDatabase
@@ -132,7 +138,7 @@ target_link_libraries(quickstep_queryoptimizer_tests_ExecutionGeneratorTest
                       tmb
                       ${LIBS})
 target_link_libraries(quickstep_queryoptimizer_tests_OptimizerTextTest
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       gtest
                       gtest_main

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/096abe29/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index eec5300..d2693eb 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -23,6 +23,12 @@ QS_PROTOBUF_GENERATE_CPP(relationaloperators_WorkOrder_proto_srcs
                          relationaloperators_WorkOrder_proto_hdrs
                          WorkOrder.proto)
 
+if (BUILD_SHARED_LIBS)
+  set(GFLAGS_LIB_NAME gflags_nothreads-shared)
+else()
+  set(GFLAGS_LIB_NAME gflags_nothreads-static)
+endif()
+
 # Declare micro-libs:
 add_library(quickstep_relationaloperators_AggregationOperator AggregationOperator.cpp AggregationOperator.hpp)
 add_library(quickstep_relationaloperators_BuildHashOperator BuildHashOperator.cpp BuildHashOperator.hpp)
@@ -160,7 +166,7 @@ target_link_libraries(quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_relationaloperators_HashJoinOperator
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationSchema
@@ -350,7 +356,7 @@ target_link_libraries(quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_relationaloperators_TextScanOperator
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelation
@@ -470,7 +476,7 @@ target_link_libraries(quickstep_relationaloperators
 add_executable(AggregationOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationOperator_unittest.cpp")
 target_link_libraries(AggregationOperator_unittest
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       gtest
                       quickstep_catalog_CatalogAttribute
@@ -523,7 +529,7 @@ add_test(AggregationOperator_unittest AggregationOperator_unittest)
 add_executable(HashJoinOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/HashJoinOperator_unittest.cpp")
 target_link_libraries(HashJoinOperator_unittest
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       gtest
                       quickstep_catalog_CatalogAttribute
@@ -573,7 +579,7 @@ add_test(HashJoinOperator_unittest HashJoinOperator_unittest)
 add_executable(SortMergeRunOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/SortMergeRunOperator_unittest.cpp")
 target_link_libraries(SortMergeRunOperator_unittest
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       gtest
                       quickstep_catalog_CatalogAttribute
@@ -624,7 +630,7 @@ add_test(SortMergeRunOperator_unittest SortMergeRunOperator_unittest)
 add_executable(SortRunGenerationOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/SortRunGenerationOperator_unittest.cpp")
 target_link_libraries(SortRunGenerationOperator_unittest
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       gtest
                       quickstep_catalog_CatalogAttribute
@@ -672,7 +678,7 @@ add_test(SortRunGenerationOperator_unittest SortRunGenerationOperator_unittest)
 add_executable(TextScanOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/TextScanOperator_unittest.cpp")
 target_link_libraries(TextScanOperator_unittest
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       gtest
                       quickstep_catalog_CatalogAttribute

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/096abe29/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index a77976a..b536411 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -21,6 +21,12 @@ if (REBUILD_INDEX_ON_UPDATE_OVERFLOW)
   set(QUICKSTEP_REBUILD_INDEX_ON_UPDATE_OVERFLOW TRUE)
 endif()
 
+if (BUILD_SHARED_LIBS)
+  set(GFLAGS_LIB_NAME gflags_nothreads-shared)
+else()
+  set(GFLAGS_LIB_NAME gflags_nothreads-static)
+endif()
+
 include(CheckIncludeFileCXX)
 check_include_files("fcntl.h;glob.h;unistd.h;sys/stat.h;sys/types.h" QUICKSTEP_HAVE_FILE_MANAGER_POSIX)
 if (NOT QUICKSTEP_HAVE_FILE_MANAGER_POSIX)
@@ -617,7 +623,7 @@ target_link_libraries(quickstep_storage_FileManager
 if (QUICKSTEP_HAVE_FILE_MANAGER_HDFS)
   target_link_libraries(quickstep_storage_FileManagerHdfs
                         glog
-                        gflags_nothreads-static
+                        ${GFLAGS_LIB_NAME}
                         quickstep_storage_FileManager
                         quickstep_storage_StorageBlockInfo
                         quickstep_storage_StorageConstants
@@ -950,7 +956,7 @@ target_link_libraries(quickstep_storage_StorageBlockLayout
 target_link_libraries(quickstep_storage_StorageBlockLayout_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_storage_StorageManager
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       gtest
                       quickstep_catalog_CatalogTypedefs
@@ -1380,7 +1386,7 @@ if (ENABLE_DISTRIBUTED)
   add_executable(DataExchange_unittest
                  "${CMAKE_CURRENT_SOURCE_DIR}/tests/DataExchange_unittest.cpp")
   target_link_libraries(DataExchange_unittest
-                        gflags_nothreads-static
+                        ${GFLAGS_LIB_NAME}
                         glog
                         gtest
                         quickstep_catalog_CatalogAttribute

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/096abe29/transaction/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/transaction/CMakeLists.txt b/transaction/CMakeLists.txt
index c6c87b6..430b4bc 100644
--- a/transaction/CMakeLists.txt
+++ b/transaction/CMakeLists.txt
@@ -13,6 +13,12 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License.
 
+if (BUILD_SHARED_LIBS)
+  set(GFLAGS_LIB_NAME gflags_nothreads-shared)
+else()
+  set(GFLAGS_LIB_NAME gflags_nothreads-static)
+endif()
+
 add_library(quickstep_transaction_AccessMode
             AccessMode.cpp
             AccessMode.hpp)
@@ -68,7 +74,7 @@ target_link_libraries(quickstep_transaction_Lock
                       quickstep_transaction_AccessMode
                       quickstep_transaction_ResourceId)
 target_link_libraries(quickstep_transaction_LockManager
-                      gflags_nothreads-static
+                      ${GFLAGS_LIB_NAME}
                       glog
                       quickstep_utility_ThreadSafeQueue
                       quickstep_threading_Thread


[27/50] [abbrv] incubator-quickstep git commit: Bug fixed in \analyze command and reuse code.

Posted by zu...@apache.org.
Bug fixed in \analyze command and reuse code.

- The \analyze command issues SQL queries. Due to a recent change in the
  execution engine, there was a bug in issuing the queries, which is
  fixed in this branch.
- Reuse code to receive feedback from Foreman upon query completion.
- Minor inclusion fixes in touched files.


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

Branch: refs/heads/tmp-relation-col-store
Commit: a7013627ecfab78aa80b483ee7f910ef7ea014d9
Parents: 4931623
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Wed Jun 15 14:10:43 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Jun 15 14:10:43 2016 -0500

----------------------------------------------------------------------
 cli/CommandExecutor.cpp                         |  2 ++
 cli/QuickstepCli.cpp                            |  9 ++------
 query_execution/QueryExecutionUtil.hpp          | 23 ++++++++++++++++++++
 .../tests/ExecutionGeneratorTestRunner.cpp      | 10 ++++-----
 .../tests/ExecutionGeneratorTestRunner.hpp      |  3 +++
 5 files changed, 34 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a7013627/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index dc14741..7083ef5 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -220,6 +220,8 @@ inline TypedValue executeQueryForSingleResult(
   QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
       main_thread_client_id, foreman_client_id, query_handle.get(), bus);
 
+  QueryExecutionUtil::ReceiveQueryCompletionMessage(main_thread_client_id, bus);
+
   // Retrieve the scalar result from the result relation.
   const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
   DCHECK(query_result_relation != nullptr);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a7013627/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 0b64fda..35bd16e 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -88,7 +88,6 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include "tmb/id_typedefs.h"
 #include "tmb/message_bus.h"
 #include "tmb/message_style.h"
-#include "tmb/tagged_message.h"
 
 namespace quickstep {
 class CatalogRelation;
@@ -119,7 +118,6 @@ using quickstep::QueryHandle;
 using quickstep::QueryPlan;
 using quickstep::QueryProcessor;
 using quickstep::SqlParserWrapper;
-using quickstep::TaggedMessage;
 using quickstep::Worker;
 using quickstep::WorkerDirectory;
 using quickstep::WorkerMessage;
@@ -128,7 +126,6 @@ using quickstep::kPoisonMessage;
 using quickstep::kWorkloadCompletionMessage;
 
 using tmb::client_id;
-using tmb::AnnotatedMessage;
 
 namespace quickstep {
 
@@ -440,10 +437,8 @@ int main(int argc, char* argv[]) {
             &bus);
 
         try {
-          const AnnotatedMessage annotated_msg =
-              bus.Receive(main_thread_client_id, 0, true);
-          const TaggedMessage &tagged_message = annotated_msg.tagged_message;
-          DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
+          QueryExecutionUtil::ReceiveQueryCompletionMessage(
+              main_thread_client_id, &bus);
           end = std::chrono::steady_clock::now();
 
           const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a7013627/query_execution/QueryExecutionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionUtil.hpp b/query_execution/QueryExecutionUtil.hpp
index 78fd159..6ea4a29 100644
--- a/query_execution/QueryExecutionUtil.hpp
+++ b/query_execution/QueryExecutionUtil.hpp
@@ -25,13 +25,18 @@
 #include "query_execution/WorkerMessage.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 #include "tmb/address.h"
+#include "tmb/id_typedefs.h"
 #include "tmb/message_style.h"
 #include "tmb/message_bus.h"
 #include "tmb/tagged_message.h"
 
 namespace quickstep {
 
+class QueryHandle;
+
 /**
  * @brief A static class for reusable methods in query_execution module.
  **/
@@ -91,6 +96,24 @@ class QueryExecutionUtil {
         bus, sender_id, receiver_id, std::move(admit_tagged_message));
   }
 
+  /**
+   * @brief Receive a query completion message.
+   *
+   * @param receiver_id The TMB client ID of the receiver thread.
+   * @param bus A pointer to the TMB.
+   *
+   * @note Right now the query completion message is of no interest to the
+   *       caller. In the future, if this message needs to be fetched, make this
+   *       function return the TaggedMessage.
+   **/
+  static void ReceiveQueryCompletionMessage(const tmb::client_id receiver_id,
+                                            tmb::MessageBus *bus) {
+    const AnnotatedMessage annotated_msg =
+        bus->Receive(receiver_id, 0, true);
+    const TaggedMessage &tagged_message = annotated_msg.tagged_message;
+    DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
+  }
+
   static void BroadcastPoisonMessage(const tmb::client_id sender_id, tmb::MessageBus *bus) {
     // Terminate all threads.
     // The sender thread broadcasts poison message to the workers and foreman.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a7013627/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
index ea871d0..8c1d306 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
@@ -42,7 +42,8 @@
 
 #include "glog/logging.h"
 
-#include "tmb/tagged_message.h"
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
 
 namespace quickstep {
 
@@ -101,11 +102,8 @@ void ExecutionGeneratorTestRunner::runTestCase(
             &query_handle,
             &bus_);
 
-        // Receive workload completion message from Foreman.
-        const AnnotatedMessage annotated_msg =
-            bus_.Receive(main_thread_client_id_, 0, true);
-        const TaggedMessage &tagged_message = annotated_msg.tagged_message;
-        DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
+        QueryExecutionUtil::ReceiveQueryCompletionMessage(
+            main_thread_client_id_, &bus_);
 
         const CatalogRelation *query_result_relation = query_handle.getQueryResultRelation();
         if (query_result_relation) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a7013627/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
index 9204073..bb2a26f 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
@@ -35,6 +35,9 @@
 #include "utility/Macros.hpp"
 #include "utility/textbased_test/TextBasedTestDriver.hpp"
 
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+
 namespace quickstep {
 
 namespace optimizer {


[12/50] [abbrv] incubator-quickstep git commit: Improved TextScanOperator.

Posted by zu...@apache.org.
Improved TextScanOperator.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 4f8fdbe8451aed1ad1c07a8badb5be85bee1ff57
Parents: eebb464
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Jun 9 03:18:37 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Jun 9 10:52:40 2016 -0700

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp          |   1 -
 relational_operators/CMakeLists.txt             |  23 +-
 relational_operators/TextScanOperator.cpp       | 818 ++++++-------------
 relational_operators/TextScanOperator.hpp       | 286 +++----
 relational_operators/WorkOrder.proto            |  15 +-
 relational_operators/WorkOrderFactory.cpp       |  72 +-
 .../tests/TextScanOperator_unittest.cpp         |   1 -
 relational_operators/tests/text_scan_input.txt  |   8 +-
 8 files changed, 384 insertions(+), 840 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4f8fdbe8/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 99c2a21..f9fd742 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -945,7 +945,6 @@ void ExecutionGenerator::convertCopyFrom(
               physical_plan->file_name(),
               physical_plan->column_delimiter(),
               physical_plan->escape_strings(),
-              FLAGS_parallelize_load,
               *output_relation,
               insert_destination_index));
   insert_destination_proto->set_relational_op_index(scan_operator_index);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4f8fdbe8/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index d2693eb..eb73c07 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -1,5 +1,7 @@
 #   Copyright 2011-2015 Quickstep Technologies LLC.
 #   Copyright 2015-2016 Pivotal Software, Inc.
+#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+#     University of Wisconsin\u2014Madison.
 #
 #   Licensed under the Apache License, Version 2.0 (the "License");
 #   you may not use this file except in compliance with the License.
@@ -16,9 +18,6 @@
 QS_PROTOBUF_GENERATE_CPP(relationaloperators_SortMergeRunOperator_proto_srcs
                          relationaloperators_SortMergeRunOperator_proto_hdrs
                          SortMergeRunOperator.proto)
-QS_PROTOBUF_GENERATE_CPP(relationaloperators_TextScanOperator_proto_srcs
-                         relationaloperators_TextScanOperator_proto_hdrs
-                         TextScanOperator.proto)
 QS_PROTOBUF_GENERATE_CPP(relationaloperators_WorkOrder_proto_srcs
                          relationaloperators_WorkOrder_proto_hdrs
                          WorkOrder.proto)
@@ -61,9 +60,6 @@ add_library(quickstep_relationaloperators_SortRunGenerationOperator SortRunGener
             SortRunGenerationOperator.hpp)
 add_library(quickstep_relationaloperators_TableGeneratorOperator TableGeneratorOperator.cpp TableGeneratorOperator.hpp)
 add_library(quickstep_relationaloperators_TextScanOperator TextScanOperator.cpp TextScanOperator.hpp)
-add_library(quickstep_relationaloperators_TextScanOperator_proto
-            ${relationaloperators_TextScanOperator_proto_srcs}
-            ${relationaloperators_TextScanOperator_proto_hdrs})
 add_library(quickstep_relationaloperators_UpdateOperator UpdateOperator.cpp UpdateOperator.hpp)
 add_library(quickstep_relationaloperators_WorkOrder ../empty_src.cpp WorkOrder.hpp)
 add_library(quickstep_relationaloperators_WorkOrderFactory WorkOrderFactory.cpp WorkOrderFactory.hpp)
@@ -360,27 +356,19 @@ target_link_libraries(quickstep_relationaloperators_TextScanOperator
                       glog
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelation
-                      quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
-                      quickstep_queryexecution_QueryExecutionMessages_proto
-                      quickstep_queryexecution_QueryExecutionTypedefs
-                      quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
-                      quickstep_relationaloperators_TextScanOperator_proto
                       quickstep_relationaloperators_WorkOrder
                       quickstep_storage_InsertDestination
-                      quickstep_storage_StorageBlob
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageManager
-                      quickstep_threading_ThreadIDBasedMap
                       quickstep_types_Type
                       quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_types_containers_Tuple
                       quickstep_utility_Glob
                       quickstep_utility_Macros
-                      quickstep_utility_ThreadSafeQueue
                       tmb)
 target_link_libraries(quickstep_relationaloperators_UpdateOperator
                       glog
@@ -430,7 +418,6 @@ target_link_libraries(quickstep_relationaloperators_WorkOrderFactory
                       quickstep_relationaloperators_SortRunGenerationOperator
                       quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_relationaloperators_TextScanOperator
-                      quickstep_relationaloperators_TextScanOperator_proto
                       quickstep_relationaloperators_UpdateOperator
                       quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
@@ -438,7 +425,6 @@ target_link_libraries(quickstep_relationaloperators_WorkOrderFactory
                       tmb)
 target_link_libraries(quickstep_relationaloperators_WorkOrder_proto
                       quickstep_relationaloperators_SortMergeRunOperator_proto
-                      quickstep_relationaloperators_TextScanOperator_proto
                       ${PROTOBUF_LIBRARY})
 
 # Module all-in-one library:
@@ -466,7 +452,6 @@ target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_SortRunGenerationOperator
                       quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_relationaloperators_TextScanOperator
-                      quickstep_relationaloperators_TextScanOperator_proto
                       quickstep_relationaloperators_UpdateOperator
                       quickstep_relationaloperators_WorkOrder
                       quickstep_relationaloperators_WorkOrderFactory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4f8fdbe8/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index 5acecbf..d2fd0cd 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015-2016 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -20,124 +22,30 @@
 #include <algorithm>
 #include <cctype>
 #include <cstddef>
-#include <cstdint>
 #include <cstdio>
 #include <cstdlib>
-#include <cstring>
+#include <memory>
 #include <string>
 #include <utility>
 #include <vector>
 
 #include "catalog/CatalogAttribute.hpp"
-#include "catalog/CatalogRelationSchema.hpp"
 #include "query_execution/QueryContext.hpp"
-#include "query_execution/QueryExecutionMessages.pb.h"
-#include "query_execution/QueryExecutionUtil.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
-#include "relational_operators/TextScanOperator.pb.h"
 #include "storage/InsertDestination.hpp"
-#include "storage/StorageBlob.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageManager.hpp"
-#include "threading/ThreadIDBasedMap.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/Tuple.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
 #include "utility/Glob.hpp"
 
-#include "gflags/gflags.h"
 #include "glog/logging.h"
 
 #include "tmb/id_typedefs.h"
-#include "tmb/message_bus.h"
-#include "tmb/tagged_message.h"
-
-using std::isxdigit;
-using std::size_t;
-using std::sscanf;
-using std::string;
 
 namespace quickstep {
 
-DEFINE_uint64(textscan_split_blob_size, 2,
-              "Size of blobs in number of slots the input text files "
-              "are split into in the TextScanOperator.");
-
-// Check if blob size is positive.
-static bool ValidateTextScanSplitBlobSize(const char *flagname,
-                                          std::uint64_t blob_size) {
-  if (blob_size == 0) {
-    LOG(ERROR) << "--" << flagname << " must be greater than 0";
-    return false;
-  }
-
-  return true;
-}
-
-static const volatile bool text_scan_split_blob_size_dummy = gflags::RegisterFlagValidator(
-    &FLAGS_textscan_split_blob_size, &ValidateTextScanSplitBlobSize);
-
-namespace {
-
-// Detect whether '*search_string' contains a row-terminator (either line-feed
-// or carriage-return + line-feed) immediately before 'end_pos'. If
-// 'process_escape_sequences' is true, this function will also eliminate
-// false-positives from an escaped row-terminator. Returns the number of
-// characters in the row-terminator, or 0 if no terminator is detected.
-inline unsigned DetectRowTerminator(const char *search_string,
-                                    std::size_t end_pos,
-                                    const bool process_escape_sequences) {
-  if (end_pos == 0) {
-    // Empty string.
-    return 0;
-  }
-
-  if (search_string[end_pos - 1] != '\n') {
-    // String doesn't end in newline.
-    return 0;
-  }
-
-  if (end_pos == 1) {
-    // String is the single newline character.
-    return 1;
-  }
-
-  const bool have_carriage_return = (search_string[end_pos - 2] == '\r');
-  if (have_carriage_return && (end_pos == 2)) {
-    // String is CR-LF and nothing else.
-    return 2;
-  }
-
-  std::size_t backslashes = 0;
-  // Count consecutive backslashes preceding the terminator. If there is an odd
-  // number of backslashes, then the terminator is escaped and doesn't count as
-  // a real terminator. If there is an even number of backslashes, then each
-  // pair is an escaped backslash literal and the terminator still counts.
-  if (process_escape_sequences) {
-    end_pos = end_pos - 2 - have_carriage_return;
-    while (end_pos != 0) {
-      if (search_string[end_pos] == '\\') {
-        ++backslashes;
-        --end_pos;
-        if ((end_pos == 0) && (search_string[0] == '\\')) {
-          // Don't forget to count a backslash at the very beginning of a string.
-          ++backslashes;
-        }
-      } else {
-        break;
-      }
-    }
-  }
-
-  if (backslashes & 0x1) {
-    return 0;
-  } else {
-    return 1 + have_carriage_return;
-  }
-}
-
-}  // namespace
-
 bool TextScanOperator::getAllWorkOrders(
     WorkOrdersContainer *container,
     QueryContext *query_context,
@@ -155,116 +63,50 @@ bool TextScanOperator::getAllWorkOrders(
   InsertDestination *output_destination =
       query_context->getInsertDestination(output_destination_index_);
 
-  if (parallelize_load_) {
-    // Parallel implementation: Split work orders are generated for each file
-    // being bulk-loaded. (More than one file can be loaded, because we support
-    // glob() semantics in file name.) These work orders read the input file,
-    // and split them in the blobs that can be parsed independently.
-    if (blocking_dependencies_met_) {
-      if (!work_generated_) {
-        // First, generate text-split work orders.
-        for (const auto &file : files) {
-          container->addNormalWorkOrder(
-              new TextSplitWorkOrder(query_id_,
-                                     file,
-                                     process_escape_sequences_,
-                                     storage_manager,
-                                     op_index_,
-                                     scheduler_client_id,
-                                     bus),
-              op_index_);
-          ++num_split_work_orders_;
-        }
-        work_generated_ = true;
-        return false;
-      } else {
-        // Check if there are blobs to parse.
-        while (!text_blob_queue_.empty()) {
-          const TextBlob blob_work = text_blob_queue_.popOne();
-          container->addNormalWorkOrder(
-              new TextScanWorkOrder(query_id_,
-                                    blob_work.blob_id,
-                                    blob_work.size,
-                                    field_terminator_,
-                                    process_escape_sequences_,
-                                    output_destination,
-                                    storage_manager),
-              op_index_);
-        }
-        // Done if all split work orders are completed, and no blobs are left to
-        // process.
-        return num_done_split_work_orders_.load(std::memory_order_acquire) == num_split_work_orders_ &&
-               text_blob_queue_.empty();
-      }
-    }
-    return false;
-  } else {
-    // Serial implementation.
-    if (blocking_dependencies_met_ && !work_generated_) {
-      for (const auto &file : files) {
+  // Text segment size set to 256KB.
+  constexpr std::size_t kTextSegmentSize = 0x40000u;
+
+  if (blocking_dependencies_met_ && !work_generated_) {
+    for (const std::string &file : files) {
+      // Use standard C libary to retrieve the file size.
+      FILE *fp = std::fopen(file.c_str(), "rb");
+      std::fseek(fp, 0, SEEK_END);
+      const std::size_t file_size = std::ftell(fp);
+      std::fclose(fp);
+
+      std::size_t text_offset = 0;
+      while (text_offset < file_size) {
         container->addNormalWorkOrder(
             new TextScanWorkOrder(query_id_,
                                   file,
+                                  text_offset,
+                                  std::min(kTextSegmentSize, file_size - text_offset),
                                   field_terminator_,
                                   process_escape_sequences_,
                                   output_destination,
                                   storage_manager),
             op_index_);
+        text_offset += kTextSegmentSize;
       }
-      work_generated_ = true;
     }
-    return work_generated_;
-  }
-}
-
-void TextScanOperator::receiveFeedbackMessage(const WorkOrder::FeedbackMessage &msg) {
-  switch (msg.type()) {
-    case kSplitWorkOrderCompletionMessage: {
-      num_done_split_work_orders_.fetch_add(1, std::memory_order_release);
-      break;
-    }
-    case kNewTextBlobMessage: {
-      serialization::TextBlob proto;
-      CHECK(proto.ParseFromArray(msg.payload(), msg.payload_size()));
-      text_blob_queue_.push(TextBlob(proto.blob_id(), proto.size()));
-      break;
-    }
-    default:
-      LOG(ERROR) << "Unknown feedback message type for TextScanOperator";
+    work_generated_ = true;
   }
+  return work_generated_;
 }
 
 TextScanWorkOrder::TextScanWorkOrder(const std::size_t query_id,
                                      const std::string &filename,
+                                     const std::size_t text_offset,
+                                     const std::size_t text_segment_size,
                                      const char field_terminator,
                                      const bool process_escape_sequences,
                                      InsertDestination *output_destination,
                                      StorageManager *storage_manager)
     : WorkOrder(query_id),
-      is_file_(true),
       filename_(filename),
+      text_offset_(text_offset),
+      text_segment_size_(text_segment_size),
       field_terminator_(field_terminator),
-      text_blob_(0),
-      text_size_(0),
-      process_escape_sequences_(process_escape_sequences),
-      output_destination_(output_destination),
-      storage_manager_(storage_manager) {
-  DCHECK(output_destination_ != nullptr);
-  DCHECK(storage_manager_ != nullptr);
-}
-
-TextScanWorkOrder::TextScanWorkOrder(const std::size_t query_id,
-                                     const block_id text_blob,
-                                     const std::size_t text_size,
-                                     const char field_terminator,
-                                     const bool process_escape_sequences,
-                                     InsertDestination *output_destination,
-                                     StorageManager *storage_manager)
-    : WorkOrder(query_id),
-      is_file_(false),
-      field_terminator_(field_terminator),
-      text_blob_(text_blob),
-      text_size_(text_size),
       process_escape_sequences_(process_escape_sequences),
       output_destination_(output_destination),
       storage_manager_(storage_manager) {
@@ -274,439 +116,293 @@ TextScanWorkOrder::TextScanWorkOrder(const std::size_t query_id,
 
 void TextScanWorkOrder::execute() {
   const CatalogRelationSchema &relation = output_destination_->getRelation();
+  std::vector<Tuple> tuples;
 
-  string current_row_string;
-  if (is_file_) {
-    FILE *file = std::fopen(filename_.c_str(), "r");
-    if (file == nullptr) {
-      throw TextScanReadError(filename_);
-    }
+  constexpr std::size_t kSmallBufferSize = 0x4000;
+  char *buffer = reinterpret_cast<char *>(malloc(std::max(text_segment_size_, kSmallBufferSize)));
 
-    bool have_row = false;
-    do {
-      current_row_string.clear();
-      have_row = readRowFromFile(file, &current_row_string);
-      if (have_row) {
-        Tuple tuple = parseRow(current_row_string, relation);
-        output_destination_->insertTupleInBatch(tuple);
-      }
-    } while (have_row);
-
-    std::fclose(file);
-  } else {
-    BlobReference blob = storage_manager_->getBlob(text_blob_);
-    const char *blob_pos = static_cast<const char*>(blob->getMemory());
-    const char *blob_end = blob_pos + text_size_;
-    bool have_row = false;
-    do {
-      current_row_string.clear();
-      have_row = readRowFromBlob(&blob_pos, blob_end, &current_row_string);
-      if (have_row) {
-        Tuple tuple = parseRow(current_row_string, relation);
-        output_destination_->insertTupleInBatch(tuple);
-      }
-    } while (have_row);
-
-    // Drop the consumed blob produced by TextSplitWorkOrder.
-    blob.release();
-    storage_manager_->deleteBlockOrBlobFile(text_blob_);
+  // Read text segment into buffer.
+  FILE *file = std::fopen(filename_.c_str(), "rb");
+  std::fseek(file, text_offset_, SEEK_SET);
+  std::size_t bytes_read = std::fread(buffer, 1, text_segment_size_, file);
+  if (bytes_read != text_segment_size_) {
+    throw TextScanReadError(filename_);
   }
-}
 
-char TextScanWorkOrder::ParseOctalLiteral(const std::string &row_string,
-                                          std::size_t *start_pos) {
-  const std::size_t stop_pos = std::min(row_string.length(), *start_pos + 3);
-
-  int value = 0;
-  for (; *start_pos < stop_pos; ++*start_pos) {
-    int char_value = row_string[*start_pos] - '0';
-    if ((char_value >= 0) && (char_value < 8)) {
-      value = value * 8 + char_value;
-    } else {
-      return value;
+  // Locate the first newline character.
+  const char *buffer_end = buffer + text_segment_size_;
+  const char *row_ptr = buffer;
+  if (text_offset_ != 0) {
+    while (row_ptr < buffer_end && *row_ptr != '\n') {
+      ++row_ptr;
     }
+  } else {
+    --row_ptr;
   }
 
-  return value;
-}
-
-char TextScanWorkOrder::ParseHexLiteral(const std::string &row_string,
-                                        std::size_t *start_pos) {
-  const std::size_t stop_pos = std::min(row_string.length(), *start_pos + 2);
+  if (row_ptr >= buffer_end) {
+    // This block does not even contain a newline character.
+    return;
+  }
 
-  int value = 0;
-  for (; *start_pos < stop_pos; ++*start_pos) {
-    if (!std::isxdigit(row_string[*start_pos])) {
-      break;
-    }
+  // Locate the last newline character.
+  const char *end_ptr = buffer_end - 1;
+  while (end_ptr > row_ptr && *end_ptr != '\n') {
+    --end_ptr;
+  }
 
-    int char_value;
-    if (std::isdigit(row_string[*start_pos])) {
-      char_value = row_string[*start_pos] - '0';
-    } else if (std::islower(row_string[*start_pos])) {
-      char_value = row_string[*start_pos] - 'a' + 10;
+  // Advance both row_ptr and end_ptr by 1.
+  ++row_ptr;
+  ++end_ptr;
+  // Now row_ptr is pointing to the first character RIGHT AFTER the FIRST newline
+  // character in this text segment, and end_ptr is pointing to the first character
+  // RIGHT AFTER the LAST newline character in this text segment.
+
+  // Process the tuples which are between the first newline character and the
+  // last newline character.
+  while (row_ptr < end_ptr) {
+    if (*row_ptr == '\r' || *row_ptr == '\n') {
+      // Skip empty lines.
+      ++row_ptr;
     } else {
-      char_value = row_string[*start_pos] - 'A' + 10;
+      tuples.emplace_back(parseRow(&row_ptr, relation));
     }
-
-    value = value * 16 + char_value;
   }
 
-  return value;
-}
+  // Process the tuple that is right after the last newline character.
+  // NOTE(jianqiao): dynamic_read_size is trying to balance between the cases
+  // that the last tuple is very small / very large.
+  std::size_t dynamic_read_size = 1024;
+  std::string row_string;
+  std::fseek(file, text_offset_ + (end_ptr - buffer), SEEK_SET);
+  bool has_reached_end = false;
+  do {
+    bytes_read = std::fread(buffer, 1, dynamic_read_size, file);
+    std::size_t bytes_to_copy = bytes_read;
 
-bool TextScanWorkOrder::readRowFromFile(FILE *file, std::string *row_string) const {
-  // Read up to 1023 chars + null-terminator at a time.
-  static constexpr std::size_t kRowBufferSize = 1024;
-  char row_buffer[kRowBufferSize];
-  for (;;) {
-    char *read_string = std::fgets(row_buffer, sizeof(row_buffer), file);
-    if (read_string == nullptr) {
-      if (std::feof(file)) {
-        if (row_string->empty()) {
-          return false;
-        } else {
-          throw TextScanFormatError("File ended without delimiter");
-        }
-      } else {
-        throw TextScanReadError(filename_);
+    for (std::size_t i = 0; i < bytes_read; ++i) {
+      if (buffer[i] == '\n') {
+        bytes_to_copy = i + 1;
+        has_reached_end = true;
+        break;
       }
     }
-
-    // Append the contents of the buffer to '*row_string', and see if we've
-    // reached a genuine row-terminator yet.
-    row_string->append(row_buffer);
-    if (removeRowTerminator(row_string)) {
-      row_string->push_back(field_terminator_);
-      return true;
+    if (!has_reached_end && bytes_read != dynamic_read_size) {
+      has_reached_end = true;
     }
-  }
-}
 
-bool TextScanWorkOrder::readRowFromBlob(const char **start_pos,
-                                        const char *end_pos,
-                                        std::string *row_string) const {
-  while (*start_pos != end_pos) {
-    const char *next_newline = static_cast<const char*>(std::memchr(
-        *start_pos,
-        '\n',
-        end_pos - *start_pos));
-
-    if (next_newline == nullptr) {
-      throw TextScanFormatError("File ended without delimiter");
-    }
+    row_string.append(buffer, bytes_to_copy);
+    dynamic_read_size = std::min(dynamic_read_size * 2, kSmallBufferSize);
+  } while (!has_reached_end);
 
-    // Append the blob's contents through the next newline to '*row_string',
-    // and see if we've reached a genuine row-terminator yet.
-    row_string->append(*start_pos, next_newline - *start_pos + 1);
-    *start_pos = next_newline + 1;
-    if (removeRowTerminator(row_string)) {
-      row_string->push_back(field_terminator_);
-      return true;
+  if (!row_string.empty()) {
+    if (row_string.back() != '\n') {
+      row_string.push_back('\n');
     }
+    row_ptr = row_string.c_str();
+    tuples.emplace_back(parseRow(&row_ptr, relation));
   }
 
-  if (row_string->empty()) {
-    return false;
-  } else {
-    throw TextScanFormatError("File ended without delimiter");
-  }
-}
-
-bool TextScanWorkOrder::removeRowTerminator(std::string *row_string) const {
-  unsigned row_term_chars = DetectRowTerminator(row_string->c_str(),
-                                                row_string->length(),
-                                                process_escape_sequences_);
-  if (row_term_chars == 0) {
-    return false;
-  } else {
-    row_string->resize(row_string->length() - row_term_chars);
-    return true;
-  }
-}
-
-bool TextScanWorkOrder::extractFieldString(const std::string &row_string,
-                                           std::size_t *start_pos,
-                                           std::string *field_string) const {
-  // Check for NULL literal string.
-  if (process_escape_sequences_
-      && (row_string.length() - *start_pos >= 3)
-      && (row_string[*start_pos] == '\\')
-      && (row_string[*start_pos + 1] == 'N')
-      && (row_string[*start_pos + 2] == field_terminator_)) {
-    *start_pos += 3;
-    return false;
-  }
-
-  // Scan up until terminator, expanding backslashed escape sequences as we go.
-  std::size_t terminator_pos = row_string.find(field_terminator_, *start_pos);
-  std::size_t scan_pos = *start_pos;
-
-  if (process_escape_sequences_) {
-    for (;;) {
-      std::size_t backslash_pos = row_string.find('\\', scan_pos);
-      if ((backslash_pos == std::string::npos) || (backslash_pos >= terminator_pos)) {
-        // No more backslashes, or the next backslash is beyond the field
-        // terminator.
-        break;
-      }
-
-      // Copy up to the backslash.
-      field_string->append(row_string, scan_pos, backslash_pos - scan_pos);
-
-      if (backslash_pos + 1 == terminator_pos) {
-        // The terminator we found was escaped by a backslash, so append the
-        // literal terminator and re-scan for the next terminator character.
-        field_string->push_back(field_terminator_);
-        scan_pos = terminator_pos + 1;
-        terminator_pos = row_string.find(field_terminator_, scan_pos);
-        continue;
+  std::fclose(file);
+  free(buffer);
+
+  // Store the tuples in a ColumnVectorsValueAccessor for bulk insert.
+  ColumnVectorsValueAccessor column_vectors;
+  std::size_t attr_id = 0;
+  for (const auto &attribute : relation) {
+    const Type &attr_type = attribute.getType();
+    if (attr_type.isVariableLength()) {
+      std::unique_ptr<IndirectColumnVector> column(
+          new IndirectColumnVector(attr_type, tuples.size()));
+      for (const auto &tuple : tuples) {
+        column->appendTypedValue(tuple.getAttributeValue(attr_id));
       }
-
-      // Expand escape sequence.
-      switch (row_string[backslash_pos + 1]) {
-        case '0':  // Fallthrough for octal digits.
-        case '1':
-        case '2':
-        case '3':
-        case '4':
-        case '5':
-        case '6':
-        case '7':
-          // Octal char literal.
-          scan_pos = backslash_pos + 1;
-          field_string->push_back(ParseOctalLiteral(row_string, &scan_pos));
-          break;
-        case 'N': {
-          // Null literal after some other column data.
-          throw TextScanFormatError(
-              "Null indicator '\\N' encountered in text scan mixed in with "
-              "other column data.");
-        }
-        case '\\':
-          // Backslash.
-          field_string->push_back('\\');
-          scan_pos = backslash_pos + 2;
-          break;
-        case 'b':
-          // Backspace.
-          field_string->push_back('\b');
-          scan_pos = backslash_pos + 2;
-          break;
-        case 'f':
-          // Form-feed.
-          field_string->push_back('\f');
-          scan_pos = backslash_pos + 2;
-          break;
-        case 'n':
-          // Newline.
-          field_string->push_back('\n');
-          scan_pos = backslash_pos + 2;
-          break;
-        case 'r':
-          // Carriage return.
-          field_string->push_back('\r');
-          scan_pos = backslash_pos + 2;
-          break;
-        case 't':
-          // Tab.
-          field_string->push_back('\t');
-          scan_pos = backslash_pos + 2;
-          break;
-        case 'v':
-          // Vertical tab.
-          field_string->push_back('\v');
-          scan_pos = backslash_pos + 2;
-          break;
-        case 'x':
-          if ((backslash_pos + 2 < row_string.length()) && std::isxdigit(row_string[backslash_pos + 2])) {
-            // Hexidecimal char literal.
-            scan_pos = backslash_pos + 2;
-            field_string->push_back(ParseHexLiteral(row_string, &scan_pos));
-          } else {
-            // Just an escaped 'x' with no hex digits.
-            field_string->push_back('x');
-            scan_pos = backslash_pos + 2;
-          }
-          break;
-        default:
-          // Append escaped character as-is.
-          field_string->push_back(row_string[backslash_pos + 1]);
-          scan_pos = backslash_pos + 2;
-          break;
+      column_vectors.addColumn(column.release());
+    } else {
+      std::unique_ptr<NativeColumnVector> column(
+          new NativeColumnVector(attr_type, tuples.size()));
+      for (const auto &tuple : tuples) {
+        column->appendTypedValue(tuple.getAttributeValue(attr_id));
       }
+      column_vectors.addColumn(column.release());
     }
+    ++attr_id;
   }
 
-  DCHECK_NE(terminator_pos, std::string::npos);
-  field_string->append(row_string, scan_pos, terminator_pos - scan_pos);
-  *start_pos = terminator_pos + 1;
-  return true;
+  // Bulk insert the tuples.
+  output_destination_->bulkInsertTuples(&column_vectors);
 }
 
-Tuple TextScanWorkOrder::parseRow(const std::string &row_string, const CatalogRelationSchema &relation) const {
+Tuple TextScanWorkOrder::parseRow(const char **row_ptr,
+                                  const CatalogRelationSchema &relation) const {
   std::vector<TypedValue> attribute_values;
 
-  std::size_t pos = 0;
+  bool is_null_literal;
+  bool has_reached_end_of_line = false;
   std::string value_str;
-  CatalogRelationSchema::const_iterator attr_it = relation.begin();
-  while (pos < row_string.length()) {
-    if (attr_it == relation.end()) {
-      throw TextScanFormatError("Row has too many fields");
+  for (const auto &attr : relation) {
+    if (has_reached_end_of_line) {
+      throw TextScanFormatError("Row has too few fields");
     }
 
     value_str.clear();
-    if (extractFieldString(row_string, &pos, &value_str)) {
-      attribute_values.emplace_back();
-      if (!attr_it->getType().parseValueFromString(value_str, &(attribute_values.back()))) {
-        throw TextScanFormatError("Failed to parse value");
-      }
-    } else {
+    extractFieldString(row_ptr,
+                       &is_null_literal,
+                       &has_reached_end_of_line,
+                       &value_str);
+
+    if (is_null_literal) {
       // NULL literal.
-      if (!attr_it->getType().isNullable()) {
+      if (!attr.getType().isNullable()) {
         throw TextScanFormatError(
             "NULL literal '\\N' was specified for a column with a "
             "non-nullable Type");
       }
-
-      attribute_values.emplace_back(attr_it->getType().makeNullValue());
+      attribute_values.emplace_back(attr.getType().makeNullValue());
+    } else {
+      attribute_values.emplace_back();
+      if (!attr.getType().parseValueFromString(value_str, &(attribute_values.back()))) {
+        throw TextScanFormatError("Failed to parse value");
+      }
     }
-
-    ++attr_it;
   }
 
-  if (attr_it != relation.end()) {
-    throw TextScanFormatError("Row has too few fields");
+  if (!has_reached_end_of_line) {
+    throw TextScanFormatError("Row has too many fields");
   }
 
   return Tuple(std::move(attribute_values));
 }
 
-void TextSplitWorkOrder::execute() {
-  std::FILE *file = std::fopen(filename_.c_str(), "r");
-  if (!file) {
-    throw TextScanReadError(filename_);
-  }
-
-  bool eof = false;
-  do {
-    // Allocate new blob, if current is empty.
-    if (0 == remainingBlobBytes()) {
-      allocateBlob();
-    }
-
-    // Read the into the unwritten part of blob.
-    std::size_t bytes =
-        std::fread(writeableBlobAddress(), 1, remainingBlobBytes(), file);
-    eof = bytes < remainingBlobBytes();
-    written_ += bytes;
-
-    // Write the current blob to queue for processing.
-    sendBlobInfoToOperator(!eof /* write_row_aligned */);
-  } while (!eof);
-
-  std::fclose(file);
+void TextScanWorkOrder::extractFieldString(const char **field_ptr,
+                                           bool *is_null_literal,
+                                           bool *has_reached_end_of_line,
+                                           std::string *field_string) const {
+  const char *cur_ptr = *field_ptr;
+  *is_null_literal = false;
 
-  // Notify the operator about the completion of this Work Order.
-  FeedbackMessage msg(TextScanOperator::kSplitWorkOrderCompletionMessage,
-                      operator_index_,
-                      nullptr /* payload */,
-                      0 /* payload_size */,
-                      false /* ownership */);
-  SendFeedbackMessage(bus_, ClientIDMap::Instance()->getValue(), scheduler_client_id_, msg);
-}
+  // Check for NULL literal string.
+  if (process_escape_sequences_ && cur_ptr[0] == '\\' && cur_ptr[1] == 'N') {
+    cur_ptr += 2;
 
-// Allocate new blob.
-void TextSplitWorkOrder::allocateBlob() {
-  text_blob_id_ = storage_manager_->createBlob(FLAGS_textscan_split_blob_size);
-  text_blob_ = storage_manager_->getBlobMutable(text_blob_id_);
-  blob_size_ = text_blob_->size();
-  written_ = 0;
-}
+    // Skip '\r'
+    if (*cur_ptr == '\r') {
+      ++cur_ptr;
+    }
 
-// Find the last row terminator in the blob.
-std::size_t TextSplitWorkOrder::findLastRowTerminator() {
-  std::size_t found = 0;
-  const char *blob = static_cast<const char *>(text_blob_->getMemory());
-
-  for (std::size_t index = written_;
-       index != 0;
-       --index) {
-    if (DetectRowTerminator(blob, index, process_escape_sequences_)) {
-      found = index;
-      break;
+    const char c = *cur_ptr;
+    if (c == field_terminator_ || c == '\n') {
+      *is_null_literal = true;
+      *has_reached_end_of_line = (c == '\n');
+      *field_ptr = cur_ptr + 1;
+      return;
     }
   }
 
-  // TODO(quickstep-team): Design a way to handle long rows that are larger than
-  // the configured blob size.
-  CHECK_NE(0u, found) << "No row terminator found in " << FLAGS_textscan_split_blob_size
-                      << "-slot chunk of " << filename_;
-  return found;
-}
+  // Not a NULL literal string, rewind cur_ptr to the start position for parsing.
+  cur_ptr = *field_ptr;
 
-void TextSplitWorkOrder::sendBlobInfoToOperator(const bool write_row_aligned) {
-  std::size_t text_len = written_;
-  std::string residue;
-  if (write_row_aligned) {
-    // Find last row terminator in current blob.
-    text_len = findLastRowTerminator();
-
-    // Copy the residual bytes after the last row terminator.
-    residue = std::string(
-        static_cast<char *>(text_blob_->getMemoryMutable()) + text_len,
-        written_ - text_len);
-  }
+  if (!process_escape_sequences_) {
+    // Simply copy until field_terminator or '\n'.
+    for (;; ++cur_ptr) {
+      const char c = *cur_ptr;
+      if (c == field_terminator_) {
+        *has_reached_end_of_line = false;
+        break;
+      } else if (c == '\n') {
+        *has_reached_end_of_line = true;
+        break;
+      }
 
-  // Notify the operator for the split-up blob.
-  serialization::TextBlob proto;
-  proto.set_blob_id(text_blob_id_);
-  proto.set_size(text_len);
-
-  const std::size_t payload_size = proto.ByteSize();
-  // NOTE(zuyu): 'payload' gets released by FeedbackMessage's destructor.
-  char *payload = static_cast<char *>(std::malloc(payload_size));
-  CHECK(proto.SerializeToArray(payload, payload_size));
-
-  const tmb::client_id worker_thread_client_id = ClientIDMap::Instance()->getValue();
-  FeedbackMessage feedback_msg(TextScanOperator::kNewTextBlobMessage,
-                               operator_index_,
-                               payload,
-                               payload_size);
-  SendFeedbackMessage(bus_, worker_thread_client_id, scheduler_client_id_, feedback_msg);
-
-  // Notify Foreman for the avaiable work order on the blob.
-  serialization::WorkOrdersAvailableMessage message_proto;
-  message_proto.set_operator_index(operator_index_);
-
-  // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
-  const size_t message_proto_length = message_proto.ByteSize();
-  char *message_proto_bytes = static_cast<char*>(std::malloc(message_proto_length));
-  CHECK(message_proto.SerializeToArray(message_proto_bytes, message_proto_length));
-
-  tmb::TaggedMessage tagged_message(static_cast<const void *>(message_proto_bytes),
-                                    message_proto_length,
-                                    kWorkOrdersAvailableMessage);
-  std::free(message_proto_bytes);
-
-  // Send new work order available message to Foreman.
-  const tmb::MessageBus::SendStatus send_status =
-      QueryExecutionUtil::SendTMBMessage(
-          bus_,
-          worker_thread_client_id,
-          scheduler_client_id_,
-          std::move(tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could not "
-      "be sent from thread with TMB client ID "
-      << worker_thread_client_id << " to Foreman with TMB client "
-      "ID " << scheduler_client_id_;
-
-  if (residue.size()) {
-    // Allocate new blob, and copy residual bytes from last blob.
-    allocateBlob();
-    std::memcpy(writeableBlobAddress(), residue.data(), residue.size());
-    written_ += residue.size();
+      // Ignore '\r'
+      if (c != '\r') {
+        field_string->push_back(c);
+      }
+    }
+  } else {
+    for (;; ++cur_ptr) {
+      const char c = *cur_ptr;
+      if (c == '\\') {
+        ++cur_ptr;
+        const char first_escaped_character = *cur_ptr;
+        switch (first_escaped_character) {
+          case '0':  // Fallthrough for octal digits.
+          case '1':
+          case '2':
+          case '3':
+          case '4':
+          case '5':
+          case '6':
+          case '7':
+            field_string->push_back(ParseOctalLiteral(&cur_ptr));
+            break;
+          case 'N': {
+            // Null literal after some other column data.
+            throw TextScanFormatError(
+                "Null indicator '\\N' encountered in text scan mixed in with "
+                "other column data.");
+          }
+          case '\\':
+            // Backslash.
+            field_string->push_back('\\');
+            break;
+          case 'b':
+            // Backspace.
+            field_string->push_back('\b');
+            break;
+          case 'f':
+            // Form-feed.
+            field_string->push_back('\f');
+            break;
+          case 'n':
+            // Newline.
+            field_string->push_back('\n');
+            break;
+          case 'r':
+            // Carriage return.
+            field_string->push_back('\r');
+            break;
+          case 't':
+            // Tab.
+            field_string->push_back('\t');
+            break;
+          case 'v':
+            // Vertical tab.
+            field_string->push_back('\v');
+            break;
+          case 'x':
+            if (std::isxdigit(cur_ptr[1])) {
+              // Hexidecimal char literal.
+              ++cur_ptr;
+              field_string->push_back(ParseHexLiteral(&cur_ptr));
+            } else {
+              // Just an escaped 'x' with no hex digits.
+              field_string->push_back('x');
+            }
+            break;
+          case '\n':
+            throw TextScanFormatError(
+                "Backslash line splicing is not supported.");
+          default:
+            // Append escaped character as-is.
+            field_string->push_back(first_escaped_character);
+            break;
+        }
+      } else if (c == field_terminator_) {
+        *has_reached_end_of_line = false;
+        break;
+      } else if (c == '\n') {
+        *has_reached_end_of_line = true;
+        break;
+      } else {
+        if (c != '\r') {
+          // Ignore '\r'
+          field_string->push_back(c);
+        }
+      }
+    }
   }
+  *field_ptr = cur_ptr + 1;
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4f8fdbe8/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index 3cda65b..d73e7dd 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015-2016 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -18,26 +20,18 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_TEXT_SCAN_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_TEXT_SCAN_OPERATOR_HPP_
 
-#include <atomic>
+#include <cctype>
 #include <cstddef>
-#include <cstdint>
-#include <cstdio>
 #include <exception>
 #include <string>
 
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryContext.hpp"
-#include "query_execution/QueryExecutionTypedefs.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
-#include "storage/StorageBlob.hpp"
-#include "storage/StorageBlockInfo.hpp"
 #include "types/containers/Tuple.hpp"
 #include "utility/Macros.hpp"
-#include "utility/ThreadSafeQueue.hpp"
-
-#include "glog/logging.h"
 
 #include "tmb/id_typedefs.h"
 
@@ -98,26 +92,11 @@ class TextScanFormatError : public std::exception {
 };
 
 /**
- * @brief A structure for text data blobs.
- */
-struct TextBlob {
-  TextBlob(const block_id text_blob_id, const std::size_t text_size)
-      : blob_id(text_blob_id), size(text_size) {}
-  block_id blob_id;
-  std::size_t size;
-};
-
-/**
  * @brief An operator which reads tuples from a text file and inserts them into
  *        a relation.
  **/
 class TextScanOperator : public RelationalOperator {
  public:
-  enum FeedbackMessageType : WorkOrder::FeedbackMessageType {
-    kNewTextBlobMessage,
-    kSplitWorkOrderCompletionMessage,
-  };
-
   /**
    * @brief Constructor
    *
@@ -130,29 +109,22 @@ class TextScanOperator : public RelationalOperator {
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param parallelize_load Parallelize the load process by th spliting file
-   *        into blobs, and generating separate work-orders for each of them.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert tuples.
    **/
-  TextScanOperator(
-      const std::size_t query_id,
-      const std::string &file_pattern,
-      const char field_terminator,
-      const bool process_escape_sequences,
-      const bool parallelize_load,
-      const CatalogRelation &output_relation,
-      const QueryContext::insert_destination_id output_destination_index)
+  TextScanOperator(const std::size_t query_id,
+                   const std::string &file_pattern,
+                   const char field_terminator,
+                   const bool process_escape_sequences,
+                   const CatalogRelation &output_relation,
+                   const QueryContext::insert_destination_id output_destination_index)
       : RelationalOperator(query_id),
         file_pattern_(file_pattern),
         field_terminator_(field_terminator),
         process_escape_sequences_(process_escape_sequences),
-        parallelize_load_(parallelize_load),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
-        num_done_split_work_orders_(0),
-        num_split_work_orders_(0),
         work_generated_(false) {}
 
   ~TextScanOperator() override {}
@@ -171,23 +143,14 @@ class TextScanOperator : public RelationalOperator {
     return output_relation_.getID();
   }
 
-  void receiveFeedbackMessage(const WorkOrder::FeedbackMessage &msg) override;
-
  private:
   const std::string file_pattern_;
   const char field_terminator_;
   const bool process_escape_sequences_;
-  const bool parallelize_load_;
 
   const CatalogRelation &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;
 
-  ThreadSafeQueue<TextBlob> text_blob_queue_;
-  std::atomic<std::uint32_t> num_done_split_work_orders_;
-  std::uint32_t num_split_work_orders_;
-
-  // Indicates if work order to load file is generated for non-parallel load, and
-  // if work order to split file to blobs is generated for parallel load.
   bool work_generated_;
 
   DISALLOW_COPY_AND_ASSIGN(TextScanOperator);
@@ -203,7 +166,9 @@ class TextScanWorkOrder : public WorkOrder {
    *
    * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param filename The name of the text file to bulk insert.
-   * @param field_terminator The string which separates attribute values in
+   * @param text_offset The start position in the text file to start text scan.
+   * @param text_segment_size The size of text segment to be scanned.
+   * @param field_terminator The character which separates attribute values in
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
@@ -213,28 +178,8 @@ class TextScanWorkOrder : public WorkOrder {
   TextScanWorkOrder(
       const std::size_t query_id,
       const std::string &filename,
-      const char field_terminator,
-      const bool process_escape_sequences,
-      InsertDestination *output_destination,
-      StorageManager *storage_manager);
-
-  /**
-   * @brief Constructor.
-   *
-   * @param query_id The ID of the query to which this WorkOrder belongs.
-   * @param text_blob Blob ID containing the data to be scanned.
-   * @param text_size Size of the data in the blob.
-   * @param field_terminator The character which separates attribute values in
-   *        the text file.
-   * @param process_escape_sequences Whether to decode escape sequences in the
-   *        text file.
-   * @param output_destination The InsertDestination to write the read tuples.
-   * @param storage_manager The StorageManager to use.
-   */
-  TextScanWorkOrder(
-      const std::size_t query_id,
-      const block_id text_blob,
-      const std::size_t text_size,
+      const std::size_t text_offset,
+      const std::size_t text_segment_size,
       const char field_terminator,
       const bool process_escape_sequences,
       InsertDestination *output_destination,
@@ -255,141 +200,106 @@ class TextScanWorkOrder : public WorkOrder {
   void execute() override;
 
  private:
-  // Parse up to three octal digits (0-7) starting at '*start_pos' in
-  // 'row_string' as a char literal. '*start_pos' will be modified to
-  // the first position AFTER the parsed octal digits.
-  static char ParseOctalLiteral(const std::string &row_string,
-                                std::size_t *start_pos);
-
-  // Parse up to two hexadecimal digits (0-F, case insensitive) starting at
-  // '*start_pos' in 'row_string' as a char literal. '*start_pos' will be
-  // modified to the first position AFTER the parsed hexadecimal digits.
-  static char ParseHexLiteral(const std::string &row_string,
-                              std::size_t *start_pos);
-
-  // Read the next text row from the open FILE stream '*file' into
-  // '*row_string'. Returns false if end-of-file is reached and there are no
-  // more rows, true if a row string was successfully read. For ease of
-  // parsing, '*row_string' has the trailing row-terminator removed and
-  // replaced with a field-terminator.
-  bool readRowFromFile(FILE *file, std::string *row_string) const;
-
-  // Read the next text from blob memory starting at '**start_pos' and ending
-  // at '*end_pos' into '*row_string'. Returns false if the end of the blob is
-  // reached and there are no more rows, true if a row was successfully read.
-  // For ease of parsing, '*row_string' has the trailing row-terminator removed
-  // and replaced with a field-terminator. After call '*start_pos' points to
-  // first character AFTER the read row in the blob.
-  bool readRowFromBlob(const char **start_pos,
-                       const char *end_pos,
-                       std::string *row_string) const;
-
-  // Trim a row-terminator (newline or carriage-return + newline) off the end
-  // of '*row_string'. Returns true if the row-terminator was successfully
-  // removed, false if '*row_string' did not end in a row-terminator.
-  bool removeRowTerminator(std::string *row_string) const;
-
-  // Extract a field string starting at '*start_pos' in 'row_string' into
-  // '*field_string'. This method also expands escape sequences if
-  // 'process_escape_sequences_' is true. Returns true if a field string was
-  // successfully extracted, false in the special case where the NULL-literal
-  // string "\N" was found. Throws TextScanFormatError if text was malformed.
-  bool extractFieldString(const std::string &row_string,
-                          std::size_t *start_pos,
-                          std::string *field_string) const;
-
-  // Make a tuple by parsing all of the individual fields specified in
-  // 'row_string'.
-  Tuple parseRow(const std::string &row_string, const CatalogRelationSchema &relation) const;
-
-  const bool is_file_;
-  const std::string filename_;
-  const char field_terminator_;
-  const block_id text_blob_;
-  const std::size_t text_size_;
-  const bool process_escape_sequences_;
-
-  InsertDestination *output_destination_;
-  StorageManager *storage_manager_;
-
-  DISALLOW_COPY_AND_ASSIGN(TextScanWorkOrder);
-};
-
-/**
- * @brief A WorkOrder to split the file into blobs of text that can be processed
- * separately.
- **/
-class TextSplitWorkOrder : public WorkOrder {
- public:
   /**
-   * @brief Constructor.
+   * @brief Extract a field string starting at \p *field_ptr. This method also
+   *        expands escape sequences if \p process_escape_sequences_ is true.
+   *        Throws TextScanFormatError if text was malformed.
    *
-   * @param query_id The ID of the query to which this WorkOrder belongs.
-   * @param filename File to split into row-aligned blobs.
-   * @param process_escape_sequences Whether to decode escape sequences in the
-   *        text file.
-   * @param storage_manager The StorageManager to use.
-   * @param operator_index Operator index of the current operator. This is used
-   *                       to send new-work available message to Foreman.
-   * @param scheduler_client_id The TMB client ID of the scheduler thread.
-   * @param bus A pointer to the TMB.
+   * @param field_ptr \p *field_ptr points to the current position of the input
+   *        char stream for parsing. The overall char stream must end with a
+   *        newline character. After the call, \p *field_ptr will be modified to
+   *        the start position of the NEXT field string.
+   * @param is_null_literal OUTPUT parameter. Set to true if the NULL-literal
+   *        string "\N" was found.
+   * @param has_reached_end_of_line OUTPUT parameter. Set to true if the newline
+   *        character was encountered.
+   * @param field_string OUTPUT parameter. Set to the extracted field string.
    */
-  TextSplitWorkOrder(const std::size_t query_id,
-                     const std::string &filename,
-                     const bool process_escape_sequences,
-                     StorageManager *storage_manager,
-                     const std::size_t operator_index,
-                     const tmb::client_id scheduler_client_id,
-                     MessageBus *bus)
-      : WorkOrder(query_id),
-        filename_(filename),
-        process_escape_sequences_(process_escape_sequences),
-        storage_manager_(DCHECK_NOTNULL(storage_manager)),
-        operator_index_(operator_index),
-        scheduler_client_id_(scheduler_client_id),
-        bus_(DCHECK_NOTNULL(bus)) {}
+  void extractFieldString(const char **field_ptr,
+                          bool *is_null_literal,
+                          bool *has_reached_end_of_line,
+                          std::string *field_string) const;
 
   /**
-   * @exception TextScanReadError The text file could not be opened for
-   *            reading.
+   * @brief Make a tuple by parsing all of the individual fields from a char stream.
+   *
+   * @param \p *row_ptr points to the current position of the input char stream
+   *        for parsing. The overall char stream must end with a newline character.
+   *        After the call, \p *row_ptr will be modified to the start position of
+   *        the NEXT text row.
+   * @param relation The relation schema for the tuple.
+   * @return The tuple parsed from the char stream.
    */
-  void execute() override;
-
- private:
-  // Allocate a new blob.
-  void allocateBlob();
-
-  // Find the last row terminator in current blob.
-  std::size_t findLastRowTerminator();
+  Tuple parseRow(const char **row_ptr,
+                 const CatalogRelationSchema &relation) const;
 
-  // Send the blob info to its operator via TMB.
-  void sendBlobInfoToOperator(const bool write_row_aligned);
 
-  // Get the writeable address (unwritten chunk) in current blob.
-  inline char* writeableBlobAddress() {
-    return static_cast<char*>(text_blob_->getMemoryMutable()) + written_;
+  /**
+   * @brief Parse up to three octal digits (0-7) starting at \p *literal_ptr as
+   *        a char literal. \p *literal_ptr will be modified to the last position
+   *        of the parsed octal digits.
+   *
+   * @param literal_ptr \p *literal_ptr points to the current position of the
+   *        input char stream for parsing. The overall char stream must end with
+   *        a newline character.
+   * @return The char literal from the parsed octal digits.
+   */
+  inline static char ParseOctalLiteral(const char **literal_ptr) {
+    int value = 0;
+    const char *ptr = *literal_ptr;
+    for (int i = 0; i < 3; ++i, ++ptr) {
+      const int char_value = *ptr - '0';
+      if ((char_value >= 0) && (char_value < 8)) {
+        value = value * 8 + char_value;
+      } else {
+        break;
+      }
+    }
+    *literal_ptr = ptr - 1;
+    return value;
   }
 
-  // Number of bytes remaining to be written.
-  inline std::size_t remainingBlobBytes() const {
-    return blob_size_ - written_;
+  /**
+   * @brief Parse up to two hexadecimal digits (0-F, case insensitive) starting
+   *        at \p *literal_ptr as a char literal. \p *literal_ptr will be modified
+   *        to the last position of the parsed octal digits.
+   *
+   * @param literal_ptr \p *literal_ptr points to the current position of the
+   *        input char stream for parsing. The overall char stream must end with
+   *        a newline character.
+   * @return The char literal from the parsed hexadecimal digits.
+   */
+  inline static char ParseHexLiteral(const char **literal_ptr) {
+    int value = 0;
+    const char *ptr = *literal_ptr;
+    for (int i = 0; i < 2; ++i, ++ptr) {
+      const char c = *ptr;
+      int char_value;
+      if (std::isdigit(c)) {
+        char_value = c - '0';
+      } else if (c >= 'a' && c <= 'f') {
+        char_value = c - 'a' + 10;
+      } else if (c >= 'A' && c <= 'F') {
+        char_value = c - 'A' + 10;
+      } else {
+        break;
+      }
+      value = value * 16 + char_value;
+    }
+    *literal_ptr = ptr - 1;
+    return value;
   }
 
-  const std::string filename_;  // File to split.
+  const std::string filename_;
+  const std::size_t text_offset_;
+  const std::size_t text_segment_size_;
+  const char field_terminator_;
   const bool process_escape_sequences_;
 
+  InsertDestination *output_destination_;
   StorageManager *storage_manager_;
 
-  const std::size_t operator_index_;  // Opeartor index.
-  const tmb::client_id scheduler_client_id_;  // The scheduler's TMB client ID.
-  MessageBus *bus_;
-
-  MutableBlobReference text_blob_;  // Mutable reference to current blob.
-  block_id text_blob_id_;  // Current blob ID.
-  std::size_t written_ = 0;  // Bytes written in current blob.
-  std::size_t blob_size_ = 0;  // Size of the current blob.
-
-  DISALLOW_COPY_AND_ASSIGN(TextSplitWorkOrder);
+  DISALLOW_COPY_AND_ASSIGN(TextScanWorkOrder);
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4f8fdbe8/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index fd731f7..60d4c8f 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -1,5 +1,7 @@
 //   Copyright 2011-2015 Quickstep Technologies LLC.
 //   Copyright 2015-2016 Pivotal Software, Inc.
+//   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+//     University of Wisconsin\u2014Madison.
 //
 //   Licensed under the Apache License, Version 2.0 (the "License");
 //   you may not use this file except in compliance with the License.
@@ -18,7 +20,6 @@ syntax = "proto2";
 package quickstep.serialization;
 
 import "relational_operators/SortMergeRunOperator.proto";
-import "relational_operators/TextScanOperator.proto";
 
 enum WorkOrderType {
   AGGREGATION = 1;
@@ -39,8 +40,7 @@ enum WorkOrderType {
   SORT_RUN_GENERATION = 16;
   TABLE_GENERATOR = 17;
   TEXT_SCAN = 18;
-  TEXT_SPLIT = 19;
-  UPDATE = 20;
+  UPDATE = 19;
 }
 
 message WorkOrder {
@@ -223,15 +223,12 @@ message TableGeneratorWorkOrder {
 message TextScanWorkOrder {
   extend WorkOrder {
     // All required.
+    optional string filename = 301;
+    optional uint64 text_offset = 302;
+    optional uint64 text_segment_size = 303;
     optional uint32 field_terminator = 304;  // For one-byte char.
     optional bool process_escape_sequences = 305;
     optional int32 insert_destination_index = 306;
-
-    // Either
-    optional string filename = 307;
-
-    // Or
-    optional TextBlob text_blob = 308;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4f8fdbe8/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 489b666..da42b4d 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -42,7 +42,6 @@
 #include "relational_operators/SortRunGenerationOperator.hpp"
 #include "relational_operators/TableGeneratorOperator.hpp"
 #include "relational_operators/TextScanOperator.hpp"
-#include "relational_operators/TextScanOperator.pb.h"
 #include "relational_operators/UpdateOperator.hpp"
 #include "relational_operators/WorkOrder.pb.h"
 #include "storage/StorageBlockInfo.hpp"
@@ -389,40 +388,16 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     }
     case serialization::TEXT_SCAN: {
       LOG(INFO) << "Creating TextScanWorkOrder";
-      if (proto.HasExtension(serialization::TextScanWorkOrder::filename)) {
-        return new TextScanWorkOrder(
-            proto.query_id(),
-            proto.GetExtension(serialization::TextScanWorkOrder::filename),
-            proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
-            proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
-            query_context->getInsertDestination(
-                proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
-            storage_manager);
-      } else {
-        const serialization::TextBlob &text_blob_proto =
-            proto.GetExtension(serialization::TextScanWorkOrder::text_blob);
-
-        return new TextScanWorkOrder(
-            proto.query_id(),
-            text_blob_proto.blob_id(),
-            text_blob_proto.size(),
-            proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
-            proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
-            query_context->getInsertDestination(
-                proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
-            storage_manager);
-      }
-    }
-    case serialization::TEXT_SPLIT: {
-      LOG(INFO) << "Creating TextSplitWorkOrder";
-      return new TextSplitWorkOrder(
+      return new TextScanWorkOrder(
           proto.query_id(),
-          proto.GetExtension(serialization::TextSplitWorkOrder::filename),
-          proto.GetExtension(serialization::TextSplitWorkOrder::process_escape_sequences),
-          storage_manager,
-          proto.GetExtension(serialization::TextSplitWorkOrder::operator_index),
-          shiftboss_client_id,
-          bus);
+          proto.GetExtension(serialization::TextScanWorkOrder::filename),
+          proto.GetExtension(serialization::TextScanWorkOrder::text_offset),
+          proto.GetExtension(serialization::TextScanWorkOrder::text_segment_size),
+          proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
+          proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
+          query_context->getInsertDestination(
+              proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
+          storage_manager);
     }
     case serialization::UPDATE: {
       LOG(INFO) << "Creating UpdateWorkOrder";
@@ -691,27 +666,14 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
                  proto.GetExtension(serialization::TableGeneratorWorkOrder::insert_destination_index));
     }
     case serialization::TEXT_SCAN: {
-      if (!proto.HasExtension(serialization::TextScanWorkOrder::field_terminator) ||
-          !proto.HasExtension(serialization::TextScanWorkOrder::process_escape_sequences) ||
-          !proto.HasExtension(serialization::TextScanWorkOrder::insert_destination_index) ||
-          !query_context.isValidInsertDestinationId(
-              proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index))) {
-        return false;
-      }
-
-      // Two fields are exclusive.
-      if (proto.HasExtension(serialization::TextScanWorkOrder::filename) ==
-              proto.HasExtension(serialization::TextScanWorkOrder::text_blob)) {
-        return false;
-      }
-
-      return proto.HasExtension(serialization::TextScanWorkOrder::filename) ||
-             proto.GetExtension(serialization::TextScanWorkOrder::text_blob).IsInitialized();
-    }
-    case serialization::TEXT_SPLIT: {
-      return proto.HasExtension(serialization::TextSplitWorkOrder::filename) &&
-             proto.HasExtension(serialization::TextSplitWorkOrder::process_escape_sequences) &&
-             proto.HasExtension(serialization::TextSplitWorkOrder::operator_index);
+      return proto.HasExtension(serialization::TextScanWorkOrder::filename) &&
+             proto.HasExtension(serialization::TextScanWorkOrder::text_offset) &&
+             proto.HasExtension(serialization::TextScanWorkOrder::text_segment_size) &&
+             proto.HasExtension(serialization::TextScanWorkOrder::field_terminator) &&
+             proto.HasExtension(serialization::TextScanWorkOrder::process_escape_sequences) &&
+             proto.HasExtension(serialization::TextScanWorkOrder::insert_destination_index) &&
+             query_context.isValidInsertDestinationId(
+                 proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index));
     }
     case serialization::UPDATE: {
       return proto.HasExtension(serialization::UpdateWorkOrder::relation_id) &&

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4f8fdbe8/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index ef6fc2d..5860745 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -193,7 +193,6 @@ TEST_F(TextScanOperatorTest, ScanTest) {
                            input_filename,
                            '\t',
                            true,
-                           false,
                            *relation_,
                            output_destination_index));
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4f8fdbe8/relational_operators/tests/text_scan_input.txt
----------------------------------------------------------------------
diff --git a/relational_operators/tests/text_scan_input.txt b/relational_operators/tests/text_scan_input.txt
index bcb76bf..51015bd 100644
--- a/relational_operators/tests/text_scan_input.txt
+++ b/relational_operators/tests/text_scan_input.txt
@@ -2,9 +2,5 @@
 -1234567890	-1.2e-200	A twenty char string	1969-07-21 02:56:00	00:00:01.001	Another twenty chars
 \N	\N	\N	\N	\N	\N
 \N	\N	\\N	\N	\N	\\N
-\x34\062	\55\064\x32\56\65	\x7B\
-\t\	\\\e\s\c\a\p\e\d\x\b\n\x7d	1988-07-16\T00:00\:00\x2E0\x30\60\06001	00:00:00	'good\' \"bye"\r\n\
-\r\n\v\n\
-
-0	0.0	\\\\\
-\\\\\n	1970-01-01	0 s	\\\\
+\x34\062	\55\064\x32\56\65	\x7B\n\t\	\\\e\s\c\a\p\e\d\x\b\n\x7d	1988-07-16\T00:00\:00\x2E0\x30\60\06001	00:00:00	'good\' \"bye"\r\n\n\r\n\v\n\n
+0	0.0	\\\\\n\\\\\n	1970-01-01	0 s	\\\\


[16/50] [abbrv] incubator-quickstep git commit: Configure libtcmalloc_minimal based on shared libs setting

Posted by zu...@apache.org.
Configure libtcmalloc_minimal based on shared libs setting

- Added a switch to change the way we build libtcmalloc_minimal i.e.
  either a static library or shared library.


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

Branch: refs/heads/tmp-relation-col-store
Commit: e40e3ba30fccd4230f9255e65dd5e76e84605860
Parents: c9214ec
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Jun 10 10:55:49 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Fri Jun 10 10:19:25 2016 -0700

----------------------------------------------------------------------
 CMakeLists.txt | 10 +++++++---
 1 file changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e40e3ba3/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 2d10a78..207f313 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -500,9 +500,13 @@ if(USE_TCMALLOC)
     #   as some generators, e.g. Ninja, may need it to build properly
     # BUILD_BYPRODUCTS <INSTALL_DIR>/lib/libtcmalloc_minimal.a
   )
-  # Static libtcmalloc_minimal.a
-  add_library(libtcmalloc_minimal STATIC IMPORTED)
-  set_property(TARGET libtcmalloc_minimal PROPERTY IMPORTED_LOCATION ${CMAKE_CURRENT_BINARY_DIR}/third_party/gperftools/lib/libtcmalloc_minimal.a)
+  if (BUILD_SHARED_LIBS)
+    add_library(libtcmalloc_minimal SHARED IMPORTED)
+    set_property(TARGET libtcmalloc_minimal PROPERTY IMPORTED_LOCATION ${CMAKE_CURRENT_BINARY_DIR}/third_party/gperftools/lib/libtcmalloc_minimal.so)
+  else()
+    add_library(libtcmalloc_minimal STATIC IMPORTED)
+    set_property(TARGET libtcmalloc_minimal PROPERTY IMPORTED_LOCATION ${CMAKE_CURRENT_BINARY_DIR}/third_party/gperftools/lib/libtcmalloc_minimal.a)
+  endif()
   # Linking against tcmalloc also requires linking against the system threading
   # library.
   set_property(TARGET libtcmalloc_minimal PROPERTY IMPORTED_LINK_INTERFACE_LIBRARIES ${CMAKE_THREAD_LIBS_INIT})


[17/50] [abbrv] incubator-quickstep git commit: Test the default vector elision level `selection` in CI.

Posted by zu...@apache.org.
Test the default vector elision level `selection` in CI.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 659967a96c597ec6c54d277d79e563e7d736d8a0
Parents: e40e3ba
Author: Jignesh Patel <jm...@hotmail.com>
Authored: Fri Jun 10 12:40:15 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Fri Jun 10 11:00:08 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/659967a9/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 142dfce..fce3ba0 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -17,8 +17,8 @@ compiler:
 env:
   - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
   - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
-  - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=none
-  - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=none
+  - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=selection
+  - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=selection
 
 before_install:
   - LLVM_VERSION=3.7.1
@@ -34,7 +34,7 @@ before_install:
     fi
 
 install:
-  - if [ "$VECTOR_COPY_ELISION_LEVEL" = "joinwithbinaryexpressions" ] && [ "$CC" = "gcc" ]; then
+  - if [ "$CC" = "gcc" ]; then
       export MAKE_JOBS=1;
     else
       export MAKE_JOBS=2;


[26/50] [abbrv] incubator-quickstep git commit: BugFix: Update NumQueuedWorkOrders to fix scheduling

Posted by zu...@apache.org.
BugFix: Update NumQueuedWorkOrders to fix scheduling


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

Branch: refs/heads/tmp-relation-col-store
Commit: 49316237f54cfff6e2ea92fe3c4333f35d46190f
Parents: dd44958
Author: Navneet Potti <na...@apache.org>
Authored: Tue Jun 14 21:52:25 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Jun 15 00:20:34 2016 -0500

----------------------------------------------------------------------
 query_execution/CMakeLists.txt     | 1 +
 query_execution/Foreman.cpp        | 8 ++++++--
 query_execution/PolicyEnforcer.cpp | 2 ++
 query_execution/PolicyEnforcer.hpp | 4 ++++
 4 files changed, 13 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/49316237/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 323e4a9..501166e 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -93,6 +93,7 @@ target_link_libraries(quickstep_queryexecution_PolicyEnforcer
                       quickstep_queryexecution_QueryExecutionMessages_proto
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryManager
+                      quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage
                       quickstep_queryoptimizer_QueryHandle
                       quickstep_relationaloperators_WorkOrder

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/49316237/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index 0577e20..828834d 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -89,6 +89,7 @@ Foreman::Foreman(const tmb::client_id main_thread_client_id,
       num_numa_nodes,
       catalog_database_,
       storage_manager_,
+      worker_directory_,
       bus_));
 }
 
@@ -115,6 +116,7 @@ void Foreman::run() {
         policy_enforcer_->processMessage(tagged_message);
         break;
       }
+
       case kAdmitRequestMessage: {
         const AdmitRequestMessage *msg =
             static_cast<const AdmitRequestMessage *>(tagged_message.message());
@@ -195,9 +197,11 @@ void Foreman::dispatchWorkerMessages(const vector<unique_ptr<WorkerMessage>> &me
     if (recipient_worker_thread_index != WorkerMessage::kInvalidRecipientIndexHint) {
       sendWorkerMessage(static_cast<size_t>(recipient_worker_thread_index),
                         *message);
+      worker_directory_->incrementNumQueuedWorkOrders(recipient_worker_thread_index);
     } else {
-      sendWorkerMessage(worker_directory_->getLeastLoadedWorker().first,
-                        *message);
+      const size_t least_loaded_worker_thread_index = worker_directory_->getLeastLoadedWorker().first;
+      sendWorkerMessage(least_loaded_worker_thread_index, *message);
+      worker_directory_->incrementNumQueuedWorkOrders(least_loaded_worker_thread_index);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/49316237/query_execution/PolicyEnforcer.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcer.cpp b/query_execution/PolicyEnforcer.cpp
index 2145429..4501026 100644
--- a/query_execution/PolicyEnforcer.cpp
+++ b/query_execution/PolicyEnforcer.cpp
@@ -27,6 +27,7 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryManager.hpp"
+#include "query_execution/WorkerDirectory.hpp"
 #include "query_optimizer/QueryHandle.hpp"
 #include "relational_operators/WorkOrder.hpp"
 
@@ -72,6 +73,7 @@ void PolicyEnforcer::processMessage(const TaggedMessage &tagged_message) {
       CHECK(proto.ParseFromArray(tagged_message.message(),
                                  tagged_message.message_bytes()));
       query_id = proto.query_id();
+      worker_directory_->decrementNumQueuedWorkOrders(proto.worker_thread_index());
       break;
     }
     case kCatalogRelationNewBlockMessage: {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/49316237/query_execution/PolicyEnforcer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcer.hpp b/query_execution/PolicyEnforcer.hpp
index 5915b79..9f87056 100644
--- a/query_execution/PolicyEnforcer.hpp
+++ b/query_execution/PolicyEnforcer.hpp
@@ -40,6 +40,7 @@ namespace quickstep {
 class CatalogDatabaseLite;
 class QueryHandle;
 class StorageManager;
+class WorkerDirectory;
 
 /**
  * @brief A class that ensures that a high level policy is maintained
@@ -60,11 +61,13 @@ class PolicyEnforcer {
                  const std::size_t num_numa_nodes,
                  CatalogDatabaseLite *catalog_database,
                  StorageManager *storage_manager,
+                 WorkerDirectory *worker_directory,
                  tmb::MessageBus *bus)
       : foreman_client_id_(foreman_client_id),
         num_numa_nodes_(num_numa_nodes),
         catalog_database_(catalog_database),
         storage_manager_(storage_manager),
+        worker_directory_(worker_directory),
         bus_(bus) {}
 
   /**
@@ -148,6 +151,7 @@ class PolicyEnforcer {
 
   CatalogDatabaseLite *catalog_database_;
   StorageManager *storage_manager_;
+  WorkerDirectory *worker_directory_;
 
   tmb::MessageBus *bus_;
 


[32/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-20: Added parser support for SQL window aggregation function

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index db20491..d02db56 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -8,8 +8,8 @@
 
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
-#define YY_FLEX_MINOR_VERSION 6
-#define YY_FLEX_SUBMINOR_VERSION 0
+#define YY_FLEX_MINOR_VERSION 5
+#define YY_FLEX_SUBMINOR_VERSION 39
 #if YY_FLEX_SUBMINOR_VERSION > 0
 #define FLEX_BETA
 #endif
@@ -354,7 +354,7 @@ void quickstep_yyfree (void * ,yyscan_t yyscanner );
 
 /* Begin user sect3 */
 
-#define quickstep_yywrap(yyscanner) (/*CONSTCOND*/1)
+#define quickstep_yywrap(yyscanner) 1
 #define YY_SKIP_YYWRAP
 
 typedef unsigned char YY_CHAR;
@@ -366,9 +366,6 @@ typedef int yy_state_type;
 static yy_state_type yy_get_previous_state (yyscan_t yyscanner );
 static yy_state_type yy_try_NUL_trans (yy_state_type current_state  ,yyscan_t yyscanner);
 static int yy_get_next_buffer (yyscan_t yyscanner );
-#if defined(__GNUC__) && __GNUC__ >= 3
-__attribute__((__noreturn__))
-#endif
 static void yy_fatal_error (yyconst char msg[] ,yyscan_t yyscanner );
 
 /* Done after the current pattern has been matched and before the
@@ -381,8 +378,8 @@ static void yy_fatal_error (yyconst char msg[] ,yyscan_t yyscanner );
 	*yy_cp = '\0'; \
 	yyg->yy_c_buf_p = yy_cp;
 
-#define YY_NUM_RULES 152
-#define YY_END_OF_BUFFER 153
+#define YY_NUM_RULES 160
+#define YY_END_OF_BUFFER 161
 /* This struct is not used in this scanner,
    but its presence is necessary. */
 struct yy_trans_info
@@ -390,72 +387,75 @@ struct yy_trans_info
 	flex_int32_t yy_verify;
 	flex_int32_t yy_nxt;
 	};
-static yyconst flex_int16_t yy_accept[553] =
+static yyconst flex_int16_t yy_accept[588] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
-        0,    0,  153,    2,    2,  151,  151,  150,  149,  151,
-      128,  124,  127,  124,  124,  147,  120,  117,  121,  146,
-      146,  146,  146,  146,  146,  146,  146,  146,  146,  146,
-      146,  146,  146,  146,  146,  146,  146,  146,  146,  146,
-      146,  146,  146,  146,  125,    4,    5,    5,    3,  143,
-      143,  140,  144,  144,  138,  145,  145,  142,    1,  150,
-      118,  148,  147,  147,  147,    0,  122,  119,  123,  146,
-      146,  146,  146,   10,  146,  146,  146,   22,  146,  146,
-      146,  146,  146,  146,  146,  146,  146,  146,  126,  146,
-
-      146,  146,  146,  146,  146,  146,  146,  146,  146,  146,
-      146,  146,   58,   66,  146,  146,  146,  146,  146,  146,
-      146,  146,  146,  146,  146,   80,   81,  146,  146,  146,
-      146,  146,  146,  146,  146,  146,  146,  146,  146,  146,
-      146,  146,  146,  146,  146,  146,  146,  146,  146,  146,
-      146,    4,    5,    3,  143,  139,  144,  137,  137,  129,
-      131,  132,  133,  134,  135,  136,  137,  145,  141,  148,
-      147,    0,  147,    6,    7,  146,    9,   11,  146,  146,
-       15,  146,  146,  146,  146,  146,  146,  146,  146,  146,
-      146,   32,  146,  146,  146,  146,  146,  146,  146,  146,
-
-       43,  146,  146,  146,  146,  146,  146,   50,  146,  146,
-      146,  146,  146,  146,  146,  146,  146,   62,  146,   68,
-      146,  146,  146,  146,  146,  146,  146,   76,  146,   79,
-      146,  146,  146,  146,  146,  146,  146,  146,  146,  146,
-      146,  146,  146,  146,   97,   98,  146,  146,  146,  146,
-      146,  146,  146,  146,  146,  146,  146,  146,  146,  146,
-      129,  131,  130,  146,  146,  146,  146,  146,  146,  146,
-       20,   23,  146,  146,  146,   28,  146,  146,   30,  146,
-      146,  146,  146,   37,  146,  146,   41,   42,  146,  146,
-      146,  146,  146,  146,  146,   52,   53,  146,   55,  146,
-
-       57,  146,  146,  146,  146,   65,   67,   69,   70,   71,
-      146,   73,  146,  146,   77,  146,  146,  146,  146,  146,
-       88,  146,   90,  146,  146,  146,  146,  146,  146,  146,
-      146,  146,  102,  103,  105,  146,  146,  146,  146,  146,
-      146,  112,  146,  114,  115,  129,  130,    8,  146,  146,
-      146,  146,  146,  146,  146,   25,  146,  146,  146,  146,
-      146,  146,  146,  146,  146,  146,  146,  146,  146,  146,
-      146,   47,   48,   49,  146,   54,  146,   59,   60,  146,
-      146,  146,   72,  146,   75,   78,   82,   83,  146,  146,
-      146,   89,  146,  146,   93,  146,  146,  146,  146,  146,
-
-      101,  146,  146,  146,  146,  109,  146,  146,  113,  146,
-      146,  146,   14,  146,  146,  146,  146,  146,   26,  146,
-       29,  146,  146,  146,  146,   35,  146,  146,  146,   40,
-      146,   45,  146,  146,   56,   61,  146,  146,   74,  146,
-      146,  146,  146,   92,  146,   95,   96,  146,  146,  146,
-      146,  107,  108,  110,  146,  146,  146,   13,  146,  146,
-      146,  146,  146,  146,   21,  146,   33,   34,  146,  146,
-      146,  146,   46,   51,   63,  146,  146,   86,   87,  146,
-      146,  146,  146,  146,  146,  111,  146,  146,  146,  146,
-      146,  146,  146,  146,   31,  146,  146,   39,  146,   64,
-
-      146,  146,  146,   99,  146,  146,  146,  146,   12,  146,
-      146,  146,  146,   24,  146,   36,  146,  146,   84,  146,
-      146,  100,  104,  146,  116,   16,  146,  146,  146,   27,
-       38,  146,   85,   91,  146,  146,  146,   18,   19,  146,
-      146,  106,  146,  146,  146,  146,  146,   94,  146,   44,
-       17,    0
+        0,    0,  161,    2,    2,  159,  159,  158,  157,  159,
+      136,  132,  135,  132,  132,  155,  128,  125,  129,  154,
+      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
+      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
+      154,  154,  154,  154,  133,    4,    5,    5,    3,  151,
+      151,  148,  152,  152,  146,  153,  153,  150,    1,  158,
+      126,  156,  155,  155,  155,    0,  130,  127,  131,  154,
+      154,  154,  154,   10,  154,  154,  154,   22,  154,  154,
+      154,  154,  154,  154,  154,  154,  154,  154,  154,  134,
+
+      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
+      154,  154,  154,   60,   68,  154,  154,  154,  154,  154,
+      154,  154,  154,  154,  154,  154,   82,   83,  154,  154,
+      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
+      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
+      154,  154,  154,    4,    5,    3,  151,  147,  152,  145,
+      145,  137,  139,  140,  141,  142,  143,  144,  145,  153,
+      149,  156,  155,    0,  155,    6,    7,  154,    9,   11,
+      154,  154,   15,  154,  154,  154,  154,  154,  154,  154,
+      154,  154,  154,  154,   33,  154,  154,  154,  154,  154,
+
+      154,  154,  154,   44,  154,  154,  154,  154,  154,  154,
+      154,   52,  154,  154,  154,  154,  154,  154,  154,  154,
+      154,   64,  154,   70,  154,  154,  154,  154,  154,  154,
+      154,   78,  154,   81,  154,  154,  154,  154,  154,  154,
+      154,  154,  154,  154,  154,  154,  154,   98,  154,  154,
+      103,  104,  154,  154,  154,  154,  154,  154,  154,  154,
+      154,  154,  154,  154,  154,  154,  154,  154,  137,  139,
+      138,  154,  154,  154,  154,  154,  154,  154,   20,   23,
+      154,  154,  154,   28,  154,  154,  154,   31,  154,  154,
+      154,  154,   38,  154,  154,   42,   43,  154,  154,  154,
+
+      154,  154,  154,  154,  154,   54,   55,  154,   57,  154,
+       59,  154,  154,  154,  154,   67,   69,   71,   72,   73,
+      154,   75,  154,  154,   79,  154,  154,   86,  154,  154,
+      154,  154,   92,  154,   94,  154,  154,  154,  100,  154,
+      154,  154,  154,  154,  154,  108,  109,  111,  154,  154,
+      154,  154,  154,  154,  154,  119,  154,  154,  122,  123,
+      137,  138,    8,  154,  154,  154,  154,  154,  154,  154,
+       25,  154,  154,  154,  154,  154,  154,  154,  154,  154,
+      154,  154,  154,  154,  154,  154,  154,   48,   49,   50,
+      154,  154,   56,  154,   61,   62,  154,  154,  154,   74,
+
+      154,   77,   80,   84,   85,  154,  154,  154,  154,   93,
+      154,  154,   97,  154,  154,  154,  154,  154,  107,  154,
+      154,  154,  154,  154,  116,  154,  154,  120,  154,  154,
+      154,  154,   14,  154,  154,  154,  154,  154,   26,  154,
+       29,  154,  154,  154,  154,  154,   36,  154,  154,  154,
+       41,  154,   46,  154,  154,  154,   58,   63,  154,  154,
+       76,  154,  154,  154,  154,  154,   96,  154,  101,  102,
+      154,  154,  154,  154,  154,  114,  115,  117,  154,  121,
+      154,  154,   13,  154,  154,  154,  154,  154,  154,   21,
+       30,  154,   34,   35,  154,  154,  154,  154,   47,  154,
+
+       53,   65,  154,  154,   89,  154,   91,  154,  154,  154,
+      154,  154,  154,  154,  118,  154,  154,  154,  154,  154,
+      154,  154,  154,   32,  154,  154,   40,  154,  154,   66,
+      154,  154,  154,  154,  105,  154,  154,  154,  154,  154,
+       12,  154,  154,  154,  154,   24,  154,   37,  154,  154,
+       51,   87,   90,  154,  154,  106,  110,  154,  113,  124,
+       16,  154,  154,  154,   27,   39,  154,   88,   95,  154,
+      154,  154,   18,   19,  154,  154,  112,  154,  154,  154,
+      154,  154,   99,  154,   45,   17,    0
     } ;
 
-static yyconst YY_CHAR yy_ec[256] =
+static yyconst flex_int32_t yy_ec[256] =
     {   0,
         1,    1,    1,    1,    1,    1,    1,    1,    2,    3,
         1,    1,    4,    1,    1,    1,    1,    1,    1,    1,
@@ -487,7 +487,7 @@ static yyconst YY_CHAR yy_ec[256] =
         1,    1,    1,    1,    1
     } ;
 
-static yyconst YY_CHAR yy_meta[72] =
+static yyconst flex_int32_t yy_meta[72] =
     {   0,
         1,    1,    2,    1,    1,    3,    1,    4,    1,    5,
         5,    6,    6,    5,    1,    1,    1,    7,    7,    7,
@@ -499,145 +499,155 @@ static yyconst YY_CHAR yy_meta[72] =
         8
     } ;
 
-static yyconst flex_uint16_t yy_base[568] =
+static yyconst flex_int16_t yy_base[603] =
     {   0,
         0,    1,   46,    0,  117,  163,    2,    3,  128,  132,
-        6,   10,  257, 1219, 1219,    0, 1219,   13, 1219,  233,
-     1219, 1219, 1219,  208,    6,  130,    4, 1219,  195,  124,
-      161,  170,  178,  207,  260,   92,  167,  161,   96,  107,
-      219,  214,  212,  224,  236,   92,  279,  272,  278,  281,
-      128,  168,    0,  125, 1219,  184,    4,   19,    0,    0,
-        0,  146,    0,    0,  343,    0,    0,  145,    0,   22,
-     1219,    0,  307,  337,  341,   18, 1219, 1219, 1219,    0,
-      170,  224,  177,  181,  215,  269,  224,    0,  263,  326,
-      336,  286,  290,  333,  382,  321,  322,  329, 1219,  328,
-
-      347,  351,  346,  350,  346,  353,  352,  374,  386,  385,
-      383,  382,  402,    0,  395,  382,  389,  403,  399,  397,
-      399,  401,  399,  408,  418,    0,  421,  406,  410,  425,
-      437,  438,  438,  455,  451,  436,  457,  462,  462,  463,
-      461,  454,  447,  454,  462,  469,  465,  466,  474,  460,
-      481,  148,   29,    0,    0, 1219,    0, 1219, 1219,   22,
-       24, 1219, 1219, 1219, 1219, 1219,    0,    0, 1219,    0,
-      509,   26,   28,    0,    0,  478,    0,  481,  478,  501,
-      493,  514,  513,  501,  517,  500,  503,  498,  523,  505,
-      521,    0,  518,  527,  526,  529,  513,  532,  520,  532,
-
-        0,  537,  521,  523,  523,  525,  558,  563,  559,  562,
-      554,  568,  568,  560,  574,  575,  576,  577,  569,    0,
-      564,  565,  581,  578,  582,  569,  571,    0,  581,    0,
-      589,  590,  576,  595,  586,  580,  595,  605,  621,  625,
-      624,   98,  618,  629,    0,  623,  617,  625,  624,  634,
-      635,  629,  625,  642,  630,  624,  644,  634,  641,  632,
-       30,  125,    0,  633,  639,  649,  642,  652,  647,  647,
-        0,  674,  673,  675,  670,    0,  671,  674,  673,  681,
-      674,  676,  684,  693,  690,  688,    0,    0,  685,  682,
-      702,  700,  686,  687,  699,    0,    0,  693,    0,  696,
-
-        0,  687,  694,  696,  726,    0,    0,    0,    0,    0,
-      696,    0,  698,  734,  727,  730,  731,  741,  746,  751,
-        0,  748,    0,  736,  731,  736,  753,  744,  756,  748,
-      741,  758,    0,  745,    0,  760,  745,  747,  761,  764,
-      762,    0,  767,    0,  761,  136, 1219,    0,  772,  786,
-      780,  801,  788,  800,  805,    0,  795,  792,  806,  807,
-      804,  813,  803,  811,  808,  804,  805,  816,  817,  804,
-      821,    0,    0,    0,  818,    0,  819,    0,    0,  807,
-      823,  808,    0,  827,    0,    0,    0,    0,  814,  835,
-      840,    0,  855,  846,    0,  859,  861,  846,  858,  850,
-
-        0,  849,  851,  866,  867,    0,  854,  873,    0,  860,
-      867,  863,    0,  857,  862,  880,  873,  863,    0,  883,
-        0,  880,  873,  876,  870,    0,  872,  903,  913,    0,
-       93,    0,  898,  906,    0,    0,  904,  922,    0,  915,
-      905,  901,  913,    0,  916,    0,    0,  915,  921,  930,
-      931,    0,    0,    0,  915,  920,  921,    0,  928,  923,
-      926,  928,  936,  933,    0,  938,    0,    0,  939,  937,
-      928,  931,    0,    0,    0,  953,  959,    0,    0,  973,
-      969,  960,  967,  969,  970,    0,  964,  978,  972,  971,
-      972,  969,  972,  977,    0,  974,  979,    0,  975,    0,
-
-      982,  992,  985,    0,  992,  984,  985,  994,    0,  997,
-     1001,  996, 1018,    0, 1012,    0, 1027, 1018, 1019, 1020,
-     1031,    0,    0, 1029,    0,    0, 1024, 1038, 1026,    0,
-        0, 1036,    0,    0, 1026, 1042, 1028,    0,    0, 1035,
-     1045,    0, 1042, 1046, 1036, 1050, 1037,    0, 1038,    0,
-        0, 1219, 1103, 1113, 1123, 1133, 1143, 1147, 1150, 1156,
-     1164, 1174, 1184, 1194, 1204, 1209, 1211
+        6,   10,  211, 1307, 1307,    0, 1307,   13, 1307,  194,
+     1307, 1307, 1307,  194,    6,  130,    4, 1307,  170,  124,
+      161,  215,  170,  207,  265,   92,  167,  162,   96,  107,
+      223,  165,  162,  221,  274,   92,  284,  277,  315,  206,
+      128,  232,    0,  125, 1307,  152,    4,   19,    0,    0,
+        0,  143,    0,    0,  379,    0,    0,  144,    0,   22,
+     1307,    0,  292,  305,  335,   18, 1307, 1307, 1307,    0,
+      174,  264,  180,  186,  209,  271,  229,    0,  277,  333,
+      335,  312,  330,  317,  323,  338,  323,  333,  340, 1307,
+
+      339,  355,  357,  381,  370,  373,  377,  382,  380,  384,
+      383,  383,  383,  431,    0,  398,  383,  390,  406,  402,
+      403,  404,  425,  420,  431,  442,    0,  445,  432,  448,
+      436,  437,  451,  448,  444,  460,  452,  439,  489,  464,
+      468,  469,  468,  462,  455,  477,  501,  493,  489,  494,
+      502,  494,  510,  142,   29,    0,    0, 1307,    0, 1307,
+     1307,   22,   24, 1307, 1307, 1307, 1307, 1307,    0,    0,
+     1307,    0,  520,   26,   28,    0,    0,  508,    0,  512,
+      495,  510,  497,  530,  518,  506,  525,  508,  512,  509,
+      543,  530,  533,  550,    0,  547,  558,  556,  561,  546,
+
+      565,  552,  564,    0,  569,  552,  554,  556,  557,  576,
+      566,  576,  569,  571,  565,  579,  580,  573,  587,  597,
+      598,  604,  594,    0,  593,  594,  612,  613,  616,  603,
+      605,    0,  614,    0,  622,  623,  611,  610,  630,  631,
+      622,  616,  632,  628,  636,  639,  637,  631,  632,  644,
+        0,  649,  647,  655,  657,  667,  670,  665,  665,  664,
+      681,  669,  663,  682,  673,  682,  680,  671,   30,  125,
+        0,  674,  679,  689,  683,  693,  688,  689,    0,  702,
+      694,  694,  688,    0,  699,  706,  720,  709,  717,  712,
+      715,  725,  734,  731,  729,    0,    0,  726,  723,  743,
+
+      740,  726,  727,  734,  741,    0,    0,  735,    0,  738,
+        0,  731,  738,  739,  759,    0,    0,    0,    0,    0,
+      740,    0,  741,  755,  746,  764,  765,    0,  778,  783,
+      786,  792,    0,  791,    0,  779,  774,  779,    0,  796,
+      787,  799,  791,  784,  800,    0,  788,    0,  803,  788,
+      789,  793,  807,  810,  810,    0,  814,  806,    0,  810,
+      136, 1307,    0,  820,  830,  820,  840,  830,  839,  848,
+        0,  840,  837,  851,  852,  844,  850,  859,  849,  857,
+      854,  851,  852,  862,  863,  852,  869,    0,    0,    0,
+      850,  869,    0,  870,    0,    0,  859,  876,  860,    0,
+
+      888,    0,    0,    0,    0,  878,  885,  899,  886,    0,
+      902,  893,    0,  907,  909,  894,  906,  898,    0,  897,
+      899,  905,  915,  916,    0,  904,  923,    0,  902,  911,
+      920,  916,    0,  909,  916,  934,  928,  919,    0,  939,
+        0,  946,  936,  944,  949,  942,    0,  945,  963,  967,
+        0,   93,    0,  951,  963,  959,    0,    0,  956,  974,
+        0,  967,  957,  969,  954,  967,    0,  970,    0,    0,
+      969,  975,  986,  987,  985,    0,    0,    0,  974,    0,
+      979,  981,    0,  988,  983,  996, 1002, 1010, 1010,    0,
+        0, 1015,    0,    0, 1018, 1016, 1008, 1010,    0, 1016,
+
+        0,    0, 1019, 1017,    0, 1019,    0, 1031, 1026, 1016,
+     1023, 1026, 1027, 1036,    0, 1022, 1038, 1032, 1031, 1034,
+     1031, 1035, 1041,    0, 1038, 1053,    0, 1052, 1066,    0,
+     1063, 1071, 1076, 1070,    0, 1079, 1071, 1072, 1085, 1082,
+        0, 1084, 1087, 1081, 1089,    0, 1075,    0, 1090, 1080,
+        0, 1080,    0, 1081, 1094,    0,    0, 1092,    0,    0,
+        0, 1087, 1103, 1091,    0,    0, 1102,    0,    0, 1093,
+     1109, 1105,    0,    0, 1116, 1126,    0, 1126, 1129, 1121,
+     1136, 1125,    0, 1126,    0,    0, 1307, 1191, 1201, 1211,
+     1221, 1231, 1235, 1238, 1244, 1252, 1262, 1272, 1282, 1292,
+
+     1297, 1299
     } ;
 
-static yyconst flex_int16_t yy_def[568] =
+static yyconst flex_int16_t yy_def[603] =
     {   0,
-      553,  553,  552,    3,  554,  554,  555,  555,  556,  556,
-      557,  557,  552,  552,  552,  558,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  552,  552,  552,  552,  560,  561,
-      561,  552,  562,  562,  563,  564,  564,  552,  558,  552,
-      552,  565,  552,  552,  552,  552,  552,  552,  552,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  552,  559,
-
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  552,  552,  560,  561,  552,  562,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  566,  564,  552,  565,
-      552,  552,  552,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      552,  552,  567,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  552,  552,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,  559,  559,  559,  559,  559,  559,  559,  559,  559,
-      559,    0,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552
+      588,  588,  587,    3,  589,  589,  590,  590,  591,  591,
+      592,  592,  587,  587,  587,  593,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  587,  587,  587,  587,  595,  596,
+      596,  587,  597,  597,  598,  599,  599,  587,  593,  587,
+      587,  600,  587,  587,  587,  587,  587,  587,  587,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  587,
+
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  587,  587,  595,  596,  587,  597,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  601,  599,
+      587,  600,  587,  587,  587,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  587,  587,
+      602,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      587,  587,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      594,  594,  594,  594,  594,  594,    0,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+
+      587,  587
     } ;
 
-static yyconst flex_uint16_t yy_nxt[1291] =
+static yyconst flex_int16_t yy_nxt[1379] =
     {   0,
-      552,  552,   15,   15,   61,   61,  153,  153,   67,   62,
-       62,   68,   67,  552,   70,   68,   70,   73,   73,   77,
-       78,  153,  153,   70,  552,   70,  172,  172,  552,  173,
-      173,  153,  153,  261,  262,  262,  262,  173,  173,  173,
-      173,  346,  262,  552,   16,   16,   17,   18,   19,   18,
+      587,  587,   15,   15,   61,   61,  155,  155,   67,   62,
+       62,   68,   67,  587,   70,   68,   70,   73,   73,   77,
+       78,  155,  155,   70,  587,   70,  174,  174,  587,  175,
+      175,  155,  155,  269,  270,  270,  270,  175,  175,  175,
+      175,  361,  270,  587,   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,
@@ -646,143 +656,153 @@ static yyconst flex_uint16_t yy_nxt[1291] =
 
        38,   39,   40,   41,   42,   43,   44,   45,   46,   47,
        48,   49,   50,   51,   52,   53,   54,   17,   56,   57,
-       58,   17,   17,   17,   17,   17,  110,  115,  116,  132,
-       64,   17,   17,   17,   64,   62,  262,  262,  472,   62,
-       74,   75,   75,  327,   81,  148,  151,  262,  262,  152,
-      169,   76,   82,  156,   83,  110,  115,  116,  132,   84,
+       58,   17,   17,   17,   17,   17,  111,  116,  117,  134,
+       64,   17,   17,   17,   64,   62,  270,  270,  498,   62,
+       74,   75,   75,  154,   81,  150,  153,  270,  270,  171,
+      158,   76,   82,  154,   83,  111,  116,  117,  134,   84,
        17,   17,   17,   17,   56,   57,   58,   17,   17,   17,
-       17,   17,   65,   81,  148,  151,   65,   17,   17,   17,
-       76,   82,   85,   83,  111,  152,   86,   89,   84,   87,
-      174,  113,  149,  150,   90,   94,  114,  177,  112,   95,
-
-      178,   91,   88,   96,   92,   93,   17,   17,   17,   97,
-       79,   85,   98,  111,   99,   86,   89,   72,   87,  174,
-      113,  149,  150,   90,   94,  114,  177,  112,   95,  178,
-       91,   88,   96,   92,   93,  100,  117,  101,   97,  121,
-      118,   98,  102,  123,  119,  122,  125,  103,   71,  124,
-      120,  179,  175,  129,  126,  182,  552,  130,  127,  552,
-      176,  128,  552,  552,  100,  117,  101,  552,  121,  118,
-      131,  102,  123,  119,  122,  125,  103,  104,  124,  120,
-      179,  175,  129,  126,  182,  105,  130,  127,  106,  176,
-      128,  107,  180,  137,  108,  140,  133,  109,  183,  131,
-
-      134,  138,  141,  142,  135,  181,  104,  189,  190,  139,
-      136,  145,  143,  146,  105,  144,  147,  106,   73,   73,
-      107,  180,  137,  108,  140,  133,  109,  183,   76,  134,
-      138,  141,  142,  135,  181,  552,  189,  190,  139,  136,
-      145,  143,  146,  184,  144,  147,  159,  185,  171,  171,
-      552,   74,   75,   75,  160,  161,  197,   76,   76,  198,
-      199,  162,   76,  200,  186,  163,  187,  201,  188,  191,
-      202,  203,  184,  164,  192,  552,  185,  165,  205,  166,
-      206,  552,  204,  167,  207,  197,  208,   76,  198,  199,
-      162,   76,  200,  186,  163,  187,  201,  188,  191,  202,
-
-      203,  193,  164,  192,  194,  209,  165,  205,  166,  206,
-      195,  204,  167,  207,  210,  208,  211,  196,  212,  214,
-      219,  213,  215,  220,  221,  222,  223,  225,  224,  226,
-      193,  227,  216,  194,  209,  228,  229,  217,  218,  195,
-      230,  231,  232,  210,  233,  211,  196,  212,  214,  219,
-      213,  215,  220,  221,  222,  223,  225,  224,  226,  234,
-      227,  216,  235,  236,  228,  229,  217,  218,  237,  230,
-      231,  232,  238,  233,  241,  242,  243,  239,  240,  246,
-      247,  248,  249,  250,  251,  244,  252,  253,  234,  254,
-      255,  235,  236,  245,  256,  258,  259,  237,  260,  264,
-
-      257,  238,  265,  241,  242,  243,  239,  240,  246,  247,
-      248,  249,  250,  251,  244,  252,  253,  266,  254,  255,
-      171,  171,  245,  256,  258,  259,  267,  260,  264,  257,
-       76,  265,  268,  269,  271,  272,  273,  274,  275,  276,
-      277,  278,  279,  280,  281,  270,  266,  282,  284,  285,
-      286,  283,  287,  288,  289,  267,  290,  291,  292,   76,
-      293,  268,  269,  271,  272,  273,  274,  275,  276,  277,
-      278,  279,  280,  281,  270,  294,  282,  284,  285,  286,
-      283,  287,  288,  289,  295,  290,  291,  292,  296,  293,
-      297,  298,  299,  300,  301,  302,  303,  304,  305,  307,
-
-      308,  309,  310,  311,  294,  312,  313,  314,  306,  315,
-      316,  317,  318,  295,  319,  320,  321,  296,  322,  297,
-      298,  299,  300,  301,  302,  303,  304,  305,  307,  308,
-      309,  310,  311,  323,  312,  313,  314,  306,  315,  316,
-      317,  318,  324,  319,  320,  321,  325,  322,  326,  328,
-      329,  330,  331,  332,  333,  334,  335,  336,  337,  338,
-      339,  340,  323,  341,  342,  344,  345,  348,  343,  349,
-      350,  324,  351,  352,  353,  325,  354,  326,  328,  329,
-      330,  331,  332,  333,  334,  335,  336,  337,  338,  339,
-      340,  355,  341,  342,  344,  345,  348,  343,  349,  350,
-
-      356,  351,  352,  353,  357,  354,  358,  359,  360,  361,
-      362,  363,  364,  365,  366,  367,  368,  369,  370,  371,
-      355,  372,  373,  374,  375,  376,  377,  378,  379,  356,
-      380,  552,  383,  357,  384,  358,  359,  360,  361,  362,
-      363,  364,  365,  366,  367,  368,  369,  370,  371,  381,
-      372,  373,  374,  375,  376,  377,  378,  379,  385,  380,
-      382,  383,  386,  384,  387,  388,  389,  390,  391,  392,
-      393,  394,  395,  396,  397,  398,  399,  400,  381,  401,
-      402,  403,  404,  405,  406,  407,  408,  385,  409,  382,
-      410,  386,  411,  387,  388,  389,  390,  391,  392,  393,
-
-      394,  395,  396,  397,  398,  399,  400,  412,  401,  402,
-      403,  404,  405,  406,  407,  408,  413,  409,  414,  410,
-      415,  411,  417,  416,  418,  419,  420,  421,  422,  423,
-      424,  425,  426,  427,  428,  429,  412,  430,  431,  432,
-      433,  434,  435,  436,  437,  413,  438,  414,  439,  415,
-      440,  417,  416,  418,  419,  420,  421,  422,  423,  424,
-      425,  426,  427,  428,  429,  441,  430,  431,  432,  433,
-      434,  435,  436,  437,  442,  438,  443,  439,  444,  440,
-      445,  446,  447,  448,  449,  450,  451,  452,  453,  454,
-      455,  456,  457,  458,  441,  459,  460,  461,  462,  463,
-
-      464,  465,  466,  442,  467,  443,  468,  444,  469,  445,
-      446,  447,  448,  449,  450,  451,  452,  453,  454,  455,
-      456,  457,  458,  470,  459,  460,  461,  462,  463,  464,
-      465,  466,  471,  467,  473,  468,  474,  469,  475,  476,
-      477,  478,  479,  480,  481,  482,  483,  484,  485,  486,
-      487,  488,  470,  489,  490,  491,  492,  493,  494,  495,
-      496,  471,  497,  473,  498,  474,  499,  475,  476,  477,
-      478,  479,  480,  481,  482,  483,  484,  485,  486,  487,
-      488,  500,  489,  490,  491,  492,  493,  494,  495,  496,
-      501,  497,  502,  498,  503,  499,  504,  505,  506,  507,
-
-      508,  509,  510,  511,  512,  513,  514,  515,  516,  517,
-      500,  518,  519,  520,  521,  522,  523,  524,  525,  501,
-      526,  502,  527,  503,  528,  504,  505,  506,  507,  508,
-      509,  510,  511,  512,  513,  514,  515,  516,  517,  529,
-      518,  519,  520,  521,  522,  523,  524,  525,  530,  526,
-      531,  527,  532,  528,  533,  534,  535,  536,  537,  538,
-      539,  540,  541,  542,  543,  544,  545,  546,  529,  547,
-      548,  549,  550,  551,  552,  552,  552,  530,  552,  531,
-      552,  532,  552,  533,  534,  535,  536,  537,  538,  539,
-      540,  541,  542,  543,  544,  545,  546,  552,  547,  548,
-
-      549,  550,  551,   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,  552,   80,
-      154,  154,  154,  154,  155,  155,  155,  552,  155,  155,
-      155,  155,  155,  155,  157,  157,  157,  552,  157,  157,
-      157,  157,  552,  157,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  168,  168,  552,  168,  168,  168,
-
-      168,  168,  168,  168,  170,  552,  170,  170,  170,  170,
-      170,  170,  170,  170,  263,  263,  347,  347,   13,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552
+       17,   17,   65,   81,  150,  153,   65,   17,   17,   17,
+       76,   82,   85,   83,  112,   79,   86,   95,   84,   87,
+      122,   96,  114,  124,  176,   97,  123,  115,  113,  125,
+
+      179,   98,   88,   72,   99,  180,   17,   17,   17,   71,
+      587,   85,  587,  112,  100,   86,   95,  587,   87,  122,
+       96,  114,  124,  176,   97,  123,  115,  113,  125,  179,
+       98,   88,   89,   99,  180,  101,  147,  102,  148,   90,
+      118,  149,  103,  126,  119,  181,   91,  104,  120,   92,
+       93,  127,   94,  587,  121,  128,  151,  152,  129,  130,
+      184,   89,  587,  587,  101,  147,  102,  148,   90,  118,
+      149,  103,  126,  119,  181,   91,  104,  120,   92,   93,
+      127,   94,  105,  121,  128,  151,  152,  129,  130,  184,
+      106,  131,  177,  107,  182,  132,  108,  587,  139,  109,
+
+      178,  135,  110,   73,   73,  136,  140,  183,  133,  137,
+      587,  105,  185,   76,  141,  138,  173,  173,  587,  106,
+      131,  177,  107,  182,  132,  108,   76,  139,  109,  178,
+      135,  110,  142,  191,  136,  140,  183,  133,  137,  143,
+      144,  185,   76,  141,  138,   74,   75,   75,  192,  145,
+      186,  193,  146,  587,  187,   76,   76,  196,  200,  194,
+      197,  142,  191,  188,  195,  189,  198,  190,  143,  144,
+      201,  202,  587,  199,  203,  204,  205,  192,  145,  186,
+      193,  146,  161,  187,  587,   76,  196,  200,  194,  197,
+      162,  163,  188,  195,  189,  198,  190,  164,  208,  201,
+
+      202,  165,  199,  203,  204,  205,  206,  209,  210,  166,
+      211,  213,  214,  167,  215,  168,  212,  207,  216,  169,
+      218,  217,  587,  223,  224,  225,  164,  208,  226,  227,
+      165,  228,  587,  229,  230,  206,  209,  210,  166,  211,
+      213,  214,  167,  215,  168,  212,  207,  216,  169,  218,
+      217,  219,  223,  224,  225,  231,  232,  226,  227,  233,
+      228,  220,  229,  230,  234,  235,  221,  222,  236,  237,
+      238,  239,  240,  242,  243,  247,  241,  244,  248,  587,
+      219,  252,  245,  246,  231,  232,  253,  254,  233,  255,
+      220,  256,  257,  234,  235,  221,  222,  236,  237,  238,
+
+      239,  240,  242,  243,  247,  241,  244,  248,  249,  258,
+      252,  245,  246,  261,  262,  253,  254,  250,  255,  259,
+      256,  257,  263,  265,  266,  251,  260,  268,  264,  272,
+      267,  173,  173,  273,  274,  275,  276,  249,  258,  279,
+      280,   76,  261,  262,  281,  282,  250,  283,  259,  277,
+      284,  263,  265,  266,  251,  260,  268,  264,  272,  267,
+      285,  278,  273,  274,  275,  276,  286,  287,  279,  280,
+       76,  288,  289,  281,  282,  290,  283,  291,  277,  284,
+      293,  292,  294,  295,  296,  297,  298,  299,  300,  285,
+      278,  301,  302,  303,  304,  286,  287,  305,  306,  307,
+
+      288,  289,  308,  309,  290,  310,  291,  311,  312,  293,
+      292,  294,  295,  296,  297,  298,  299,  300,  313,  314,
+      301,  302,  303,  304,  317,  315,  305,  306,  307,  318,
+      319,  308,  309,  320,  310,  316,  311,  312,  321,  322,
+      323,  324,  325,  326,  327,  328,  329,  313,  314,  330,
+      331,  332,  333,  317,  315,  334,  335,  336,  318,  319,
+      337,  338,  320,  341,  316,  342,  339,  321,  322,  323,
+      324,  325,  326,  327,  328,  329,  340,  343,  330,  331,
+      332,  333,  344,  345,  334,  335,  336,  346,  347,  337,
+      338,  348,  341,  349,  342,  339,  350,  351,  352,  353,
+
+      354,  355,  358,  356,  359,  360,  343,  357,  363,  364,
+      365,  344,  345,  366,  367,  368,  346,  347,  369,  370,
+      348,  371,  349,  372,  373,  350,  351,  352,  353,  354,
+      355,  358,  356,  359,  360,  374,  357,  363,  364,  365,
+      375,  376,  366,  367,  368,  377,  378,  369,  370,  379,
+      371,  380,  372,  373,  381,  382,  383,  384,  385,  386,
+      387,  388,  389,  390,  374,  391,  392,  393,  394,  375,
+      376,  395,  396,  397,  377,  378,  400,  401,  379,  402,
+      380,  403,  398,  381,  382,  383,  384,  385,  386,  387,
+      388,  389,  390,  399,  391,  392,  393,  394,  404,  405,
+
+      395,  396,  397,  406,  407,  400,  401,  408,  402,  409,
+      403,  398,  410,  411,  412,  413,  414,  415,  416,  417,
+      418,  419,  399,  420,  421,  422,  423,  404,  405,  424,
+      425,  426,  406,  407,  427,  428,  408,  429,  409,  430,
+      431,  410,  411,  412,  413,  414,  415,  416,  417,  418,
+      419,  432,  420,  421,  422,  423,  433,  434,  424,  425,
+      426,  437,  435,  427,  428,  436,  429,  438,  430,  431,
+      439,  440,  441,  442,  443,  444,  445,  446,  447,  448,
+      432,  449,  450,  451,  452,  433,  434,  453,  454,  455,
+      437,  435,  456,  457,  436,  458,  438,  459,  460,  439,
+
+      440,  441,  442,  443,  444,  445,  446,  447,  448,  461,
+      449,  450,  451,  452,  462,  463,  453,  454,  455,  464,
+      465,  456,  457,  466,  458,  467,  459,  460,  468,  469,
+      470,  471,  472,  473,  474,  475,  476,  477,  461,  478,
+      479,  480,  481,  462,  463,  482,  483,  484,  464,  465,
+      485,  486,  466,  487,  467,  488,  489,  468,  469,  470,
+      471,  472,  473,  474,  475,  476,  477,  490,  478,  479,
+      480,  481,  491,  492,  482,  483,  484,  493,  494,  485,
+      486,  495,  487,  496,  488,  489,  497,  499,  500,  501,
+      502,  503,  504,  505,  506,  507,  490,  508,  509,  510,
+
+      511,  491,  492,  512,  513,  514,  493,  494,  515,  516,
+      495,  517,  496,  518,  519,  497,  499,  500,  501,  502,
+      503,  504,  505,  506,  507,  520,  508,  509,  510,  511,
+      521,  522,  512,  513,  514,  523,  524,  515,  516,  525,
+      517,  526,  518,  519,  527,  528,  529,  530,  531,  532,
+      533,  534,  535,  536,  520,  537,  538,  539,  540,  521,
+      522,  541,  542,  543,  523,  524,  544,  545,  525,  546,
+      526,  547,  548,  527,  528,  529,  530,  531,  532,  533,
+      534,  535,  536,  549,  537,  538,  539,  540,  550,  551,
+      541,  542,  543,  552,  553,  544,  545,  554,  546,  555,
+
+      547,  548,  556,  557,  558,  559,  560,  561,  562,  563,
+      564,  565,  549,  566,  567,  568,  569,  550,  551,  570,
+      571,  572,  552,  553,  573,  574,  554,  575,  555,  576,
+      577,  556,  557,  558,  559,  560,  561,  562,  563,  564,
+      565,  578,  566,  567,  568,  569,  579,  580,  570,  571,
+      572,  581,  582,  573,  574,  583,  575,  584,  576,  577,
+      585,  586,  587,  587,  587,  587,  587,  587,  587,  587,
+      578,  587,  587,  587,  587,  579,  580,  587,  587,  587,
+      581,  582,  587,  587,  583,  587,  584,  587,  587,  585,
+      586,   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,  587,   80,  156,  156,
+      156,  156,  157,  157,  157,  587,  157,  157,  157,  157,
+      157,  157,  159,  159,  159,  587,  159,  159,  159,  159,
+      587,  159,  160,  160,  160,  160,  160,  160,  160,  160,
+      160,  160,  170,  170,  587,  170,  170,  170,  170,  170,
+      170,  170,  172,  587,  172,  172,  172,  172,  172,  172,
+
+      172,  172,  271,  271,  362,  362,   13,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587
     } ;
 
-static yyconst flex_int16_t yy_chk[1291] =
+static yyconst flex_int16_t yy_chk[1379] =
     {   0,
         0,    0,    1,    2,    7,    8,   57,   57,   11,    7,
         8,   11,   12,    0,   18,   12,   18,   25,   25,   27,
        27,   58,   58,   70,    0,   70,   76,   76,    0,   76,
-       76,  153,  153,  160,  160,  161,  161,  172,  172,  173,
-      173,  261,  261,    0,    1,    2,    3,    3,    3,    3,
+       76,  155,  155,  162,  162,  163,  163,  174,  174,  175,
+      175,  269,  269,    0,    1,    2,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
@@ -792,137 +812,147 @@ static yyconst flex_int16_t yy_chk[1291] =
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    5,    5,    5,
         5,    5,    5,    5,    5,    5,   36,   39,   40,   46,
-        9,    5,    5,    5,   10,    9,  262,  262,  431,   10,
-       26,   26,   26,  242,   30,   51,   54,  346,  346,  152,
-       68,   26,   30,   62,   30,   36,   39,   40,   46,   30,
+        9,    5,    5,    5,   10,    9,  270,  270,  452,   10,
+       26,   26,   26,  154,   30,   51,   54,  361,  361,   68,
+       62,   26,   30,   56,   30,   36,   39,   40,   46,   30,
         5,    5,    5,    6,    6,    6,    6,    6,    6,    6,
         6,    6,    9,   30,   51,   54,   10,    6,    6,    6,
-       26,   30,   31,   30,   37,   56,   31,   32,   30,   31,
-       81,   38,   52,   52,   32,   33,   38,   83,   37,   33,
-
-       84,   32,   31,   33,   32,   32,    6,    6,    6,   33,
-       29,   31,   33,   37,   34,   31,   32,   24,   31,   81,
-       38,   52,   52,   32,   33,   38,   83,   37,   33,   84,
-       32,   31,   33,   32,   32,   34,   41,   34,   33,   42,
-       41,   33,   34,   43,   41,   42,   44,   34,   20,   43,
-       41,   85,   82,   45,   44,   87,   13,   45,   44,    0,
-       82,   44,    0,    0,   34,   41,   34,    0,   42,   41,
-       45,   34,   43,   41,   42,   44,   34,   35,   43,   41,
-       85,   82,   45,   44,   87,   35,   45,   44,   35,   82,
-       44,   35,   86,   48,   35,   49,   47,   35,   89,   45,
-
-       47,   48,   49,   49,   47,   86,   35,   92,   93,   48,
-       47,   50,   49,   50,   35,   49,   50,   35,   73,   73,
-       35,   86,   48,   35,   49,   47,   35,   89,   73,   47,
-       48,   49,   49,   47,   86,    0,   92,   93,   48,   47,
-       50,   49,   50,   90,   49,   50,   65,   90,   74,   74,
-        0,   75,   75,   75,   65,   65,   96,   73,   74,   97,
-       98,   65,   75,  100,   91,   65,   91,  101,   91,   94,
-      102,  103,   90,   65,   94,    0,   90,   65,  104,   65,
-      105,    0,  103,   65,  106,   96,  107,   74,   97,   98,
-       65,   75,  100,   91,   65,   91,  101,   91,   94,  102,
-
-      103,   95,   65,   94,   95,  108,   65,  104,   65,  105,
-       95,  103,   65,  106,  109,  107,  110,   95,  111,  112,
-      115,  111,  113,  116,  117,  118,  119,  120,  119,  121,
-       95,  122,  113,   95,  108,  123,  124,  113,  113,   95,
-      125,  127,  128,  109,  129,  110,   95,  111,  112,  115,
-      111,  113,  116,  117,  118,  119,  120,  119,  121,  130,
-      122,  113,  131,  132,  123,  124,  113,  113,  133,  125,
-      127,  128,  134,  129,  135,  136,  137,  134,  134,  138,
-      139,  140,  141,  142,  143,  137,  144,  145,  130,  146,
-      147,  131,  132,  137,  148,  149,  150,  133,  151,  176,
-
-      148,  134,  178,  135,  136,  137,  134,  134,  138,  139,
-      140,  141,  142,  143,  137,  144,  145,  179,  146,  147,
-      171,  171,  137,  148,  149,  150,  180,  151,  176,  148,
-      171,  178,  181,  182,  183,  184,  185,  186,  187,  188,
-      189,  190,  191,  193,  194,  182,  179,  195,  196,  197,
-      198,  195,  199,  200,  202,  180,  203,  204,  205,  171,
-      206,  181,  182,  183,  184,  185,  186,  187,  188,  189,
-      190,  191,  193,  194,  182,  207,  195,  196,  197,  198,
-      195,  199,  200,  202,  208,  203,  204,  205,  209,  206,
-      210,  211,  212,  213,  214,  215,  216,  217,  218,  219,
-
-      221,  222,  223,  224,  207,  225,  226,  227,  218,  229,
-      231,  232,  233,  208,  234,  235,  236,  209,  237,  210,
-      211,  212,  213,  214,  215,  216,  217,  218,  219,  221,
-      222,  223,  224,  238,  225,  226,  227,  218,  229,  231,
-      232,  233,  239,  234,  235,  236,  240,  237,  241,  243,
-      244,  246,  247,  248,  249,  250,  251,  252,  253,  254,
-      255,  256,  238,  257,  258,  259,  260,  264,  258,  265,
-      266,  239,  267,  268,  269,  240,  270,  241,  243,  244,
-      246,  247,  248,  249,  250,  251,  252,  253,  254,  255,
-      256,  272,  257,  258,  259,  260,  264,  258,  265,  266,
-
-      273,  267,  268,  269,  274,  270,  275,  277,  278,  279,
-      280,  281,  282,  283,  284,  285,  286,  289,  290,  291,
-      272,  292,  293,  294,  295,  298,  300,  302,  303,  273,
-      304,    0,  311,  274,  313,  275,  277,  278,  279,  280,
-      281,  282,  283,  284,  285,  286,  289,  290,  291,  305,
-      292,  293,  294,  295,  298,  300,  302,  303,  314,  304,
-      305,  311,  315,  313,  316,  317,  318,  319,  320,  322,
-      324,  325,  326,  327,  328,  329,  330,  331,  305,  332,
-      334,  336,  337,  338,  339,  340,  341,  314,  343,  305,
-      345,  315,  349,  316,  317,  318,  319,  320,  322,  324,
-
-      325,  326,  327,  328,  329,  330,  331,  350,  332,  334,
-      336,  337,  338,  339,  340,  341,  351,  343,  352,  345,
-      353,  349,  354,  353,  355,  357,  358,  359,  360,  361,
-      362,  363,  364,  365,  366,  367,  350,  368,  369,  370,
-      371,  375,  377,  380,  381,  351,  382,  352,  384,  353,
-      389,  354,  353,  355,  357,  358,  359,  360,  361,  362,
-      363,  364,  365,  366,  367,  390,  368,  369,  370,  371,
-      375,  377,  380,  381,  391,  382,  393,  384,  394,  389,
-      396,  397,  398,  399,  400,  402,  403,  404,  405,  407,
-      408,  410,  411,  412,  390,  414,  415,  416,  417,  418,
-
-      420,  422,  423,  391,  424,  393,  425,  394,  427,  396,
-      397,  398,  399,  400,  402,  403,  404,  405,  407,  408,
-      410,  411,  412,  428,  414,  415,  416,  417,  418,  420,
-      422,  423,  429,  424,  433,  425,  434,  427,  437,  438,
-      440,  441,  442,  443,  445,  448,  449,  450,  451,  455,
-      456,  457,  428,  459,  460,  461,  462,  463,  464,  466,
-      469,  429,  470,  433,  471,  434,  472,  437,  438,  440,
-      441,  442,  443,  445,  448,  449,  450,  451,  455,  456,
-      457,  476,  459,  460,  461,  462,  463,  464,  466,  469,
-      477,  470,  480,  471,  481,  472,  482,  483,  484,  485,
-
-      487,  488,  489,  490,  491,  492,  493,  494,  496,  497,
-      476,  499,  501,  502,  503,  505,  506,  507,  508,  477,
-      510,  480,  511,  481,  512,  482,  483,  484,  485,  487,
-      488,  489,  490,  491,  492,  493,  494,  496,  497,  513,
-      499,  501,  502,  503,  505,  506,  507,  508,  515,  510,
-      517,  511,  518,  512,  519,  520,  521,  524,  527,  528,
-      529,  532,  535,  536,  537,  540,  541,  543,  513,  544,
-      545,  546,  547,  549,    0,    0,    0,  515,    0,  517,
-        0,  518,    0,  519,  520,  521,  524,  527,  528,  529,
-      532,  535,  536,  537,  540,  541,  543,    0,  544,  545,
-
-      546,  547,  549,  553,  553,  553,  553,  553,  553,  553,
-      553,  553,  553,  554,  554,  554,  554,  554,  554,  554,
-      554,  554,  554,  555,  555,  555,  555,  555,  555,  555,
-      555,  555,  555,  556,  556,  556,  556,  556,  556,  556,
-      556,  556,  556,  557,  557,  557,  557,  557,  557,  557,
-      557,  557,  557,  558,  558,  559,  559,  559,    0,  559,
-      560,  560,  560,  560,  561,  561,  561,    0,  561,  561,
-      561,  561,  561,  561,  562,  562,  562,    0,  562,  562,
-      562,  562,    0,  562,  563,  563,  563,  563,  563,  563,
-      563,  563,  563,  563,  564,  564,    0,  564,  564,  564,
-
-      564,  564,  564,  564,  565,    0,  565,  565,  565,  565,
-      565,  565,  565,  565,  566,  566,  567,  567,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552,
-      552,  552,  552,  552,  552,  552,  552,  552,  552,  552
+       26,   30,   31,   30,   37,   29,   31,   33,   30,   31,
+       42,   33,   38,   43,   81,   33,   42,   38,   37,   43,
+
+       83,   33,   31,   24,   33,   84,    6,    6,    6,   20,
+       13,   31,    0,   37,   34,   31,   33,    0,   31,   42,
+       33,   38,   43,   81,   33,   42,   38,   37,   43,   83,
+       33,   31,   32,   33,   84,   34,   50,   34,   50,   32,
+       41,   50,   34,   44,   41,   85,   32,   34,   41,   32,
+       32,   44,   32,    0,   41,   44,   52,   52,   44,   44,
+       87,   32,    0,    0,   34,   50,   34,   50,   32,   41,
+       50,   34,   44,   41,   85,   32,   34,   41,   32,   32,
+       44,   32,   35,   41,   44,   52,   52,   44,   44,   87,
+       35,   45,   82,   35,   86,   45,   35,    0,   48,   35,
+
+       82,   47,   35,   73,   73,   47,   48,   86,   45,   47,
+        0,   35,   89,   73,   48,   47,   74,   74,    0,   35,
+       45,   82,   35,   86,   45,   35,   74,   48,   35,   82,
+       47,   35,   49,   92,   47,   48,   86,   45,   47,   49,
+       49,   89,   73,   48,   47,   75,   75,   75,   93,   49,
+       90,   94,   49,    0,   90,   74,   75,   96,   97,   95,
+       96,   49,   92,   91,   95,   91,   96,   91,   49,   49,
+       98,   99,    0,   96,  101,  102,  103,   93,   49,   90,
+       94,   49,   65,   90,    0,   75,   96,   97,   95,   96,
+       65,   65,   91,   95,   91,   96,   91,   65,  105,   98,
+
+       99,   65,   96,  101,  102,  103,  104,  106,  107,   65,
+      108,  109,  110,   65,  111,   65,  108,  104,  112,   65,
+      113,  112,    0,  116,  117,  118,   65,  105,  119,  120,
+       65,  120,    0,  121,  122,  104,  106,  107,   65,  108,
+      109,  110,   65,  111,   65,  108,  104,  112,   65,  113,
+      112,  114,  116,  117,  118,  123,  124,  119,  120,  125,
+      120,  114,  121,  122,  126,  128,  114,  114,  129,  130,
+      131,  132,  133,  134,  135,  137,  133,  136,  138,    0,
+      114,  140,  136,  136,  123,  124,  141,  142,  125,  143,
+      114,  144,  145,  126,  128,  114,  114,  129,  130,  131,
+
+      132,  133,  134,  135,  137,  133,  136,  138,  139,  146,
+      140,  136,  136,  148,  149,  141,  142,  139,  143,  147,
+      144,  145,  150,  151,  152,  139,  147,  153,  150,  178,
+      152,  173,  173,  180,  181,  182,  183,  139,  146,  185,
+      186,  173,  148,  149,  187,  188,  139,  189,  147,  184,
+      190,  150,  151,  152,  139,  147,  153,  150,  178,  152,
+      191,  184,  180,  181,  182,  183,  192,  193,  185,  186,
+      173,  194,  196,  187,  188,  197,  189,  198,  184,  190,
+      199,  198,  200,  201,  202,  203,  205,  206,  207,  191,
+      184,  208,  209,  210,  211,  192,  193,  212,  213,  214,
+
+      194,  196,  215,  216,  197,  217,  198,  218,  219,  199,
+      198,  200,  201,  202,  203,  205,  206,  207,  220,  221,
+      208,  209,  210,  211,  223,  222,  212,  213,  214,  225,
+      226,  215,  216,  227,  217,  222,  218,  219,  228,  229,
+      230,  231,  233,  235,  236,  237,  238,  220,  221,  239,
+      240,  241,  242,  223,  222,  243,  244,  245,  225,  226,
+      246,  247,  227,  249,  222,  250,  248,  228,  229,  230,
+      231,  233,  235,  236,  237,  238,  248,  252,  239,  240,
+      241,  242,  253,  254,  243,  244,  245,  255,  256,  246,
+      247,  257,  249,  258,  250,  248,  259,  260,  261,  262,
+
+      263,  264,  266,  265,  267,  268,  252,  265,  272,  273,
+      274,  253,  254,  275,  276,  277,  255,  256,  278,  280,
+      257,  281,  258,  282,  283,  259,  260,  261,  262,  263,
+      264,  266,  265,  267,  268,  285,  265,  272,  273,  274,
+      286,  287,  275,  276,  277,  288,  289,  278,  280,  290,
+      281,  291,  282,  283,  292,  293,  294,  295,  298,  299,
+      300,  301,  302,  303,  285,  304,  305,  308,  310,  286,
+      287,  312,  313,  314,  288,  289,  321,  323,  290,  324,
+      291,  325,  315,  292,  293,  294,  295,  298,  299,  300,
+      301,  302,  303,  315,  304,  305,  308,  310,  326,  327,
+
+      312,  313,  314,  329,  330,  321,  323,  331,  324,  332,
+      325,  315,  334,  336,  337,  338,  340,  341,  342,  343,
+      344,  345,  315,  347,  349,  350,  351,  326,  327,  352,
+      353,  354,  329,  330,  355,  357,  331,  358,  332,  360,
+      364,  334,  336,  337,  338,  340,  341,  342,  343,  344,
+      345,  365,  347,  349,  350,  351,  366,  367,  352,  353,
+      354,  369,  368,  355,  357,  368,  358,  370,  360,  364,
+      372,  373,  374,  375,  376,  377,  378,  379,  380,  381,
+      365,  382,  383,  384,  385,  366,  367,  386,  387,  391,
+      369,  368,  392,  394,  368,  397,  370,  398,  399,  372,
+
+      373,  374,  375,  376,  377,  378,  379,  380,  381,  401,
+      382,  383,  384,  385,  406,  407,  386,  387,  391,  408,
+      409,  392,  394,  411,  397,  412,  398,  399,  414,  415,
+      416,  417,  418,  420,  421,  422,  423,  424,  401,  426,
+      427,  429,  430,  406,  407,  431,  432,  434,  408,  409,
+      435,  436,  411,  437,  412,  438,  440,  414,  415,  416,
+      417,  418,  420,  421,  422,  423,  424,  442,  426,  427,
+      429,  430,  443,  444,  431,  432,  434,  445,  446,  435,
+      436,  448,  437,  449,  438,  440,  450,  454,  455,  456,
+      459,  460,  462,  463,  464,  465,  442,  466,  468,  471,
+
+      472,  443,  444,  473,  474,  475,  445,  446,  479,  481,
+      448,  482,  449,  484,  485,  450,  454,  455,  456,  459,
+      460,  462,  463,  464,  465,  486,  466,  468,  471,  472,
+      487,  488,  473,  474,  475,  489,  492,  479,  481,  495,
+      482,  496,  484,  485,  497,  498,  500,  503,  504,  506,
+      508,  509,  510,  511,  486,  512,  513,  514,  516,  487,
+      488,  517,  518,  519,  489,  492,  520,  521,  495,  522,
+      496,  523,  525,  497,  498,  500,  503,  504,  506,  508,
+      509,  510,  511,  526,  512,  513,  514,  516,  528,  529,
+      517,  518,  519,  531,  532,  520,  521,  533,  522,  534,
+
+      523,  525,  536,  537,  538,  539,  540,  542,  543,  544,
+      545,  547,  526,  549,  550,  552,  554,  528,  529,  555,
+      558,  562,  531,  532,  563,  564,  533,  567,  534,  570,
+      571,  536,  537,  538,  539,  540,  542,  543,  544,  545,
+      547,  572,  549,  550,  552,  554,  575,  576,  555,  558,
+      562,  578,  579,  563,  564,  580,  567,  581,  570,  571,
+      582,  584,    0,    0,    0,    0,    0,    0,    0,    0,
+      572,    0,    0,    0,    0,  575,  576,    0,    0,    0,
+      578,  579,    0,    0,  580,    0,  581,    0,    0,  582,
+      584,  588,  588,  588,  588,  588,  588,  588,  588,  588,
+
+      588,  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,  594,  594,  594,    0,  594,  595,  595,
+      595,  595,  596,  596,  596,    0,  596,  596,  596,  596,
+      596,  596,  597,  597,  597,    0,  597,  597,  597,  597,
+        0,  597,  598,  598,  598,  598,  598,  598,  598,  598,
+      598,  598,  599,  599,    0,  599,  599,  599,  599,  599,
+      599,  599,  600,    0,  600,  600,  600,  600,  600,  600,
+
+      600,  600,  601,  601,  602,  602,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
+      587,  587,  587,  587,  587,  587,  587,  587
     } ;
 
 /* Table of booleans, true if rule could match eol. */
-static yyconst flex_int32_t yy_rule_can_match_eol[153] =
+static yyconst flex_int32_t yy_rule_can_match_eol[161] =
     {   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, 
@@ -930,8 +960,9 @@ static yyconst flex_int32_t yy_rule_can_match_eol[153] =
     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, 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, 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,     };
 
 /* The intent behind this definition is that it'll catch
  * any uses of REJECT which flex missed.
@@ -984,6 +1015,7 @@ class ParseCommand;
 struct ParseCopyFromParams;
 class ParseDataType;
 class ParseExpression;
+struct ParseFrameInfo;
 class ParseFunctionCall;
 class ParseGroupBy;
 class ParseHaving;
@@ -1022,6 +1054,7 @@ class ParseSubqueryExpression;
 class ParseSubqueryTableReference;
 class ParseTableReference;
 class ParseTableReferenceSignature;
+class ParseWindow;
 class Type;
 class UnaryOperation;
 
@@ -1048,7 +1081,7 @@ class UnaryOperation;
 
 
 
-#line 1052 "SqlLexer_gen.cpp"
+#line 1085 "SqlLexer_gen.cpp"
 
 #define INITIAL 0
 #define CONDITION_SQL 1
@@ -1134,11 +1167,11 @@ void quickstep_yyset_extra (YY_EXTRA_TYPE user_defined ,yyscan_t yyscanner );
 
 FILE *quickstep_yyget_in (yyscan_t yyscanner );
 
-void quickstep_yyset_in  (FILE * _in_str ,yyscan_t yyscanner );
+void quickstep_yyset_in  (FILE * in_str ,yyscan_t yyscanner );
 
 FILE *quickstep_yyget_out (yyscan_t yyscanner );
 
-void quickstep_yyset_out  (FILE * _out_str ,yyscan_t yyscanner );
+void quickstep_yyset_out  (FILE * out_str ,yyscan_t yyscanner );
 
 yy_size_t quickstep_yyget_leng (yyscan_t yyscanner );
 
@@ -1146,11 +1179,11 @@ char *quickstep_yyget_text (yyscan_t yyscanner );
 
 int quickstep_yyget_lineno (yyscan_t yyscanner );
 
-void quickstep_yyset_lineno (int _line_number ,yyscan_t yyscanner );
+void quickstep_yyset_lineno (int line_number ,yyscan_t yyscanner );
 
 int quickstep_yyget_column  (yyscan_t yyscanner );
 
-void quickstep_yyset_column (int _column_no ,yyscan_t yyscanner );
+void quickstep_yyset_column (int column_no ,yyscan_t yyscanner );
 
 YYSTYPE * quickstep_yyget_lval (yyscan_t yyscanner );
 
@@ -1172,10 +1205,6 @@ extern int quickstep_yywrap (yyscan_t yyscanner );
 #endif
 #endif
 
-#ifndef YY_NO_UNPUT
-    
-#endif
-
 #ifndef yytext_ptr
 static void yy_flex_strncpy (char *,yyconst char *,int ,yyscan_t yyscanner);
 #endif
@@ -1290,7 +1319,7 @@ extern int quickstep_yylex \
 
 /* Code executed at the end of each rule. */
 #ifndef YY_BREAK
-#define YY_BREAK /*LINTED*/break;
+#define YY_BREAK break;
 #endif
 
 #define YY_RULE_SETUP \
@@ -1300,9 +1329,9 @@ extern int quickstep_yylex \
  */
 YY_DECL
 {
-	yy_state_type yy_current_state;
-	char *yy_cp, *yy_bp;
-	int yy_act;
+	register yy_state_type yy_current_state;
+	register char *yy_cp, *yy_bp;
+	register int yy_act;
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 
     yylval = yylval_param;
@@ -1336,12 +1365,12 @@ YY_DECL
 		}
 
 	{
-#line 128 "../SqlLexer.lpp"
+#line 130 "../SqlLexer.lpp"
 
 
-#line 1343 "SqlLexer_gen.cpp"
+#line 1372 "SqlLexer_gen.cpp"
 
-	while ( /*CONSTCOND*/1 )		/* loops until end-of-file is reached */
+	while ( 1 )		/* loops until end-of-file is reached */
 		{
 		yy_cp = yyg->yy_c_buf_p;
 
@@ -1357,7 +1386,7 @@ YY_DECL
 yy_match:
 		do
 			{
-			YY_CHAR yy_c = yy_ec[YY_SC_TO_UI(*yy_cp)] ;
+			register YY_CHAR yy_c = yy_ec[YY_SC_TO_UI(*yy_cp)] ;
 			if ( yy_accept[yy_current_state] )
 				{
 				yyg->yy_last_accepting_state = yy_current_state;
@@ -1366,13 +1395,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 >= 553 )
+				if ( yy_current_state >= 588 )
 					yy_c = yy_meta[(unsigned int) yy_c];
 				}
 			yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
 			++yy_cp;
 			}
-		while ( yy_current_state != 552 );
+		while ( yy_current_state != 587 );
 		yy_cp = yyg->yy_last_accepting_cpos;
 		yy_current_state = yyg->yy_last_accepting_state;
 
@@ -1406,7 +1435,7 @@ do_action:	/* This label is used only to access EOF actions. */
 
 case 1:
 YY_RULE_SETUP
-#line 131 "../SqlLexer.lpp"
+#line 133 "../SqlLexer.lpp"
 {
     /* A forward slash character represents a system command. */
     BEGIN(CONDITION_COMMAND);
@@ -1418,7 +1447,7 @@ YY_RULE_SETUP
 case 2:
 /* rule 2 can match eol */
 YY_RULE_SETUP
-#line 139 "../SqlLexer.lpp"
+#line 141 "../SqlLexer.lpp"
 {
     /* This is a SQL command. Place the char back and process normally. */
     yyless(0);
@@ -1430,7 +1459,7 @@ YY_RULE_SETUP
 
 case 3:
 YY_RULE_SETUP
-#line 148 "../SqlLexer.lpp"
+#line 150 "../SqlLexer.lpp"
 {
     /* This is a command argument. */
     yylval->string_value_ = new quickstep::ParseString(
@@ -1440,7 +1469,7 @@ YY_RULE_SETUP
 	YY_BREAK
 case 4:
 YY_RULE_SETUP
-#line 155 "../SqlLexer.lpp"
+#line 157 "../SqlLexer.lpp"
 {
     /* Ignore whitespace. */
   }
@@ -1448,7 +1477,7 @@ YY_RULE_SETUP
 case 5:
 /* rule 5 can match eol */
 YY_RULE_SETUP
-#line 159 "../SqlLexer.lpp"
+#line 161 "../SqlLexer.lpp"
 {
     /* Newline reverts the lexer to the initial state. */
     yycolumn = 0;
@@ -1460,627 +1489,667 @@ YY_RULE_SETUP
 
 case 6:
 YY_RULE_SETUP
-#line 168 "../SqlLexer.lpp"
+#line 170 "../SqlLexer.lpp"
 return TOKEN_ADD;
 	YY_BREAK
 case 7:
 YY_RULE_SETUP
-#line 169 "../SqlLexer.lpp"
+#line 171 "../SqlLexer.lpp"
 return TOKEN_ALL;
 	YY_BREAK
 case 8:
 YY_RULE_SETUP
-#line 170 "../SqlLexer.lpp"
+#line 172 "../SqlLexer.lpp"
 return TOKEN_ALTER;
 	YY_BREAK
 case 9:
 YY_RULE_SETUP
-#line 171 "../SqlLexer.lpp"
+#line 173 "../SqlLexer.lpp"
 return TOKEN_AND;
 	YY_BREAK
 case 10:
 YY_RULE_SETUP
-#line 172 "../SqlLexer.lpp"
+#line 174 "../SqlLexer.lpp"
 return TOKEN_AS;
 	YY_BREAK
 case 11:
 YY_RULE_SETUP
-#line 173 "../SqlLexer.lpp"
+#line 175 "../SqlLexer.lpp"
 return TOKEN_ASC;
 	YY_BREAK
 case 12:
 YY_RULE_SETUP
-#line 174 "../SqlLexer.lpp"
+#line 176 "../SqlLexer.lpp"
 return TOKEN_ASC;
 	YY_BREAK
 case 13:
 YY_RULE_SETUP
-#line 175 "../SqlLexer.lpp"
+#line 177 "../SqlLexer.lpp"
 return TOKEN_BETWEEN;
 	YY_BREAK
 case 14:
 YY_RULE_SETUP
-#line 176 "../SqlLexer.lpp"
+#line 178 "../SqlLexer.lpp"
 return TOKEN_BIGINT;
 	YY_BREAK
 case 15:
 YY_RULE_SETUP
-#line 177 "../SqlLexer.lpp"
+#line 179 "../SqlLexer.lpp"
 return TOKEN_BIT;
 	YY_BREAK
 case 16:
 YY_RULE_SETUP
-#line 178 "../SqlLexer.lpp"
+#line 180 "../SqlLexer.lpp"
 return TOKEN_BITWEAVING;
 	YY_BREAK
 case 17:
 YY_RULE_SETUP
-#line 179 "../SqlLexer.lpp"
+#line 181 "../SqlLexer.lpp"
 return TOKEN_BLOCKPROPERTIES;
 	YY_BREAK
 case 18:
 YY_RULE_SETUP
-#line 180 "../SqlLexer.lpp"
+#line 182 "../SqlLexer.lpp"
 return TOKEN_BLOCKSAMPLE;
 	YY_BREAK
 case 19:
 YY_RULE_SETUP
-#line 181 "../SqlLexer.lpp"
+#line 183 "../SqlLexer.lpp"
 return TOKEN_BLOOM_FILTER;
 	YY_BREAK
 case 20:
 YY_RULE_SETUP
-#line 182 "../SqlLexer.lpp"
+#line 184 "../SqlLexer.lpp"
 return TOKEN_CASE;
 	YY_BREAK
 case 21:
 YY_RULE_SETUP
-#line 183 "../SqlLexer.lpp"
+#line 185 "../SqlLexer.lpp"
 return TOKEN_CSB_TREE;
 	YY_BREAK
 case 22:
 YY_RULE_SETUP
-#line 184 "../SqlLexer.lpp"
+#line 186 "../SqlLexer.lpp"
 return TOKEN_BY;
 	YY_BREAK
 case 23:
 YY_RULE_SETUP
-#line 185 "../SqlLexer.lpp"
+#line 187 "../SqlLexer.lpp"
 return TOKEN_CHARACTER;
 	YY_BREAK
 case 24:
 YY_RULE_SETUP
-#line 186 "../SqlLexer.lpp"
+#line 188 "../SqlLexer.lpp"
 return TOKEN_CHARACTER;
 	YY_BREAK
 case 25:
 YY_RULE_SETUP
-#line 187 "../SqlLexer.lpp"
+#line 189 "../SqlLexer.lpp"
 return TOKEN_CHECK;
 	YY_BREAK
 case 26:
 YY_RULE_SETUP
-#line 188 "../SqlLexer.lpp"
+#line 190 "../SqlLexer.lpp"
 return TOKEN_COLUMN;
 	YY_BREAK
 case 27:
 YY_RULE_SETUP
-#line 189 "../SqlLexer.lpp"
+#line 191 "../SqlLexer.lpp"
 return TOKEN_CONSTRAINT;
 	YY_BREAK
 case 28:
 YY_RULE_SETUP
-#line 190 "../SqlLexer.lpp"
+#line 192 "../SqlLexer.lpp"
 return TOKEN_COPY;
 	YY_BREAK
 case 29:
 YY_RULE_SETUP
-#line 191 "../SqlLexer.lpp"
+#line 193 "../SqlLexer.lpp"
 return TOKEN_CREATE;
 	YY_BREAK
 case 30:
 YY_RULE_SETUP
-#line 192 "../SqlLexer.lpp"
-return TOKEN_DATE;
+#line 194 "../SqlLexer.lpp"
+return TOKEN_CURRENT;
 	YY_BREAK
 case 31:
 YY_RULE_SETUP
-#line 193 "../SqlLexer.lpp"
-return TOKEN_DATETIME;
+#line 195 "../SqlLexer.lpp"
+return TOKEN_DATE;
 	YY_BREAK
 case 32:
 YY_RULE_SETUP
-#line 194 "../SqlLexer.lpp"
-return TOKEN_DAY;
+#line 196 "../SqlLexer.lpp"
+return TOKEN_DATETIME;
 	YY_BREAK
 case 33:
 YY_RULE_SETUP
-#line 195 "../SqlLexer.lpp"
-return TOKEN_DECIMAL;
+#line 197 "../SqlLexer.lpp"
+return TOKEN_DAY;
 	YY_BREAK
 case 34:
 YY_RULE_SETUP
-#line 196 "../SqlLexer.lpp"
-return TOKEN_DEFAULT;
+#line 198 "../SqlLexer.lpp"
+return TOKEN_DECIMAL;
 	YY_BREAK
 case 35:
 YY_RULE_SETUP
-#line 197 "../SqlLexer.lpp"
-return TOKEN_DELETE;
+#line 199 "../SqlLexer.lpp"
+return TOKEN_DEFAULT;
 	YY_BREAK
 case 36:
 YY_RULE_SETUP
-#line 198 "../SqlLexer.lpp"
-return TOKEN_DELIMITER;
+#line 200 "../SqlLexer.lpp"
+return TOKEN_DELETE;
 	YY_BREAK
 case 37:
 YY_RULE_SETUP
-#line 199 "../SqlLexer.lpp"
-return TOKEN_DESC;
+#line 201 "../SqlLexer.lpp"
+return TOKEN_DELIMITER;
 	YY_BREAK
 case 38:
 YY_RULE_SETUP
-#line 200 "../SqlLexer.lpp"
+#line 202 "../SqlLexer.lpp"
 return TOKEN_DESC;
 	YY_BREAK
 case 39:
 YY_RULE_SETUP
-#line 201 "../SqlLexer.lpp"
-return TOKEN_DISTINCT;
+#line 203 "../SqlLexer.lpp"
+return TOKEN_DESC;
 	YY_BREAK
 case 40:
 YY_RULE_SETUP
-#line 202 "../SqlLexer.lpp"
-return TOKEN_DOUBLE;
+#line 204 "../SqlLexer.lpp"
+return TOKEN_DISTINCT;
 	YY_BREAK
 case 41:
 YY_RULE_SETUP
-#line 203 "../SqlLexer.lpp"
-return TOKEN_DROP;
+#line 205 "../SqlLexer.lpp"
+return TOKEN_DOUBLE;
 	YY_BREAK
 case 42:
 YY_RULE_SETUP
-#line 204 "../SqlLexer.lpp"
-return TOKEN_ELSE;
+#line 206 "../SqlLexer.lpp"
+return TOKEN_DROP;
 	YY_BREAK
 case 43:
 YY_RULE_SETUP
-#line 205 "../SqlLexer.lpp"
-return TOKEN_END;
+#line 207 "../SqlLexer.lpp"
+return TOKEN_ELSE;
 	YY_BREAK
 case 44:
 YY_RULE_SETUP
-#line 206 "../SqlLexer.lpp"
-return TOKEN_ESCAPE_STRINGS;
+#line 208 "../SqlLexer.lpp"
+return TOKEN_END;
 	YY_BREAK
 case 45:
 YY_RULE_SETUP
-#line 207 "../SqlLexer.lpp"
-return TOKEN_EXISTS;
+#line 209 "../SqlLexer.lpp"
+return TOKEN_ESCAPE_STRINGS;
 	YY_BREAK
 case 46:
 YY_RULE_SETUP
-#line 208 "../SqlLexer.lpp"
-return TOKEN_EXTRACT;
+#line 210 "../SqlLexer.lpp"
+return TOKEN_EXISTS;
 	YY_BREAK
 case 47:
 YY_RULE_SETUP
-#line 209 "../SqlLexer.lpp"
-return TOKEN_FALSE;
+#line 211 "../SqlLexer.lpp"
+return TOKEN_EXTRACT;
 	YY_BREAK
 case 48:
 YY_RULE_SETUP
-#line 210 "../SqlLexer.lpp"
-return TOKEN_FIRST;
+#line 212 "../SqlLexer.lpp"
+return TOKEN_FALSE;
 	YY_BREAK
 case 49:
 YY_RULE_SETUP
-#line 211 "../SqlLexer.lpp"
-return TOKEN_FLOAT;
+#line 213 "../SqlLexer.lpp"
+return TOKEN_FIRST;
 	YY_BREAK
 case 50:
 YY_RULE_SETUP
-#line 212 "../SqlLexer.lpp"
-return TOKEN_FOR;
+#line 214 "../SqlLexer.lpp"
+return TOKEN_FLOAT;
 	YY_BREAK
 case 51:
 YY_RULE_SETUP
-#line 213 "../SqlLexer.lpp"
-return TOKEN_FOREIGN;
+#line 215 "../SqlLexer.lpp"
+return TOKEN_FOLLOWING;
 	YY_BREAK
 case 52:
 YY_RULE_SETUP
-#line 214 "../SqlLexer.lpp"
-return TOKEN_FROM;
+#line 216 "../SqlLexer.lpp"
+return TOKEN_FOR;
 	YY_BREAK
 case 53:
 YY_RULE_SETUP
-#line 215 "../SqlLexer.lpp"
-return TOKEN_FULL;
+#line 217 "../SqlLexer.lpp"
+return TOKEN_FOREIGN;
 	YY_BREAK
 case 54:
 YY_RULE_SETUP
-#line 216 "../SqlLexer.lpp"
-return TOKEN_GROUP;
+#line 218 "../SqlLexer.lpp"
+return TOKEN_FROM;
 	YY_BREAK
 case 55:
 YY_RULE_SETUP
-#line 217 "../SqlLexer.lpp"
-return TOKEN_HASH;
+#line 219 "../SqlLexer.lpp"
+return TOKEN_FULL;
 	YY_BREAK
 case 56:
 YY_RULE_SETUP
-#line 218 "../SqlLexer.lpp"
-return TOKEN_HAVING;
+#line 220 "../SqlLexer.lpp"
+return TOKEN_GROUP;
 	YY_BREAK
 case 57:
 YY_RULE_SETUP
-#line 219 "../SqlLexer.lpp"
-return TOKEN_HOUR;
+#line 221 "../SqlLexer.lpp"
+return TOKEN_HASH;
 	YY_BREAK
 case 58:
 YY_RULE_SETUP
-#line 220 "../SqlLexer.lpp"
-return TOKEN_IN;
+#line 222 "../SqlLexer.lpp"
+return TOKEN_HAVING;
 	YY_BREAK
 case 59:
 YY_RULE_SETUP
-#line 221 "../SqlLexer.lpp"
-return TOKEN_INDEX;
+#line 223 "../SqlLexer.lpp"
+return TOKEN_HOUR;
 	YY_BREAK
 case 60:
 YY_RULE_SETUP
-#line 222 "../SqlLexer.lpp"
-return TOKEN_INNER;
+#line 224 "../SqlLexer.lpp"
+return TOKEN_IN;
 	YY_BREAK
 case 61:
 YY_RULE_SETUP
-#line 223 "../SqlLexer.lpp"
-return TOKEN_INSERT;
+#line 225 "../SqlLexer.lpp"
+return TOKEN_INDEX;
 	YY_BREAK
 case 62:
 YY_RULE_SETUP
-#line 224 "../SqlLexer.lpp"
-return TOKEN_INTEGER;
+#line 226 "../SqlLexer.lpp"
+return TOKEN_INNER;
 	YY_BREAK
 case 63:
 YY_RULE_SETUP
-#line 225 "../SqlLexer.lpp"
-return TOKEN_INTEGER;
+#line 227 "../SqlLexer.lpp"
+return TOKEN_INSERT;
 	YY_BREAK
 case 64:
 YY_RULE_SETUP
-#line 226 "../SqlLexer.lpp"
-return TOKEN_INTERVAL;
+#line 228 "../SqlLexer.lpp"
+return TOKEN_INTEGER;
 	YY_BREAK
 case 65:
 YY_RULE_SETUP
-#line 227 "../SqlLexer.lpp"
-return TOKEN_INTO;
+#line 229 "../SqlLexer.lpp"
+return TOKEN_INTEGER;
 	YY_BREAK
 case 66:
 YY_RULE_SETUP
-#line 228 "../SqlLexer.lpp"
-return TOKEN_IS;
+#line 230 "../SqlLexer.lpp"
+return TOKEN_INTERVAL;
 	YY_BREAK
 case 67:
 YY_RULE_SETUP
-#line 229 "../SqlLexer.lpp"
-return TOKEN_JOIN;
+#line 231 "../SqlLexer.lpp"
+return TOKEN_INTO;
 	YY_BREAK
 case 68:
 YY_RULE_SETUP
-#line 230 "../SqlLexer.lpp"
-return TOKEN_KEY;
+#line 232 "../SqlLexer.lpp"
+return TOKEN_IS;
 	YY_BREAK
 case 69:
 YY_RULE_SETUP
-#line 231 "../SqlLexer.lpp"
-return TOKEN_LAST;
+#line 233 "../SqlLexer.lpp"
+return TOKEN_JOIN;
 	YY_BREAK
 case 70:
 YY_RULE_SETUP
-#line 232 "../SqlLexer.lpp"
-return TOKEN_LEFT;
+#line 234 "../SqlLexer.lpp"
+return TOKEN_KEY;
 	YY_BREAK
 case 71:
 YY_RULE_SETUP
-#line 233 "../SqlLexer.lpp"
-return TOKEN_LIKE;
+#line 235 "../SqlLexer.lpp"
+return TOKEN_LAST;
 	YY_BREAK
 case 72:
 YY_RULE_SETUP
-#line 234 "../SqlLexer.lpp"
-return TOKEN_LIMIT;
+#line 236 "../SqlLexer.lpp"
+return TOKEN_LEFT;
 	YY_BREAK
 case 73:
 YY_RULE_SETUP
-#line 235 "../SqlLexer.lpp"
-return TOKEN_LONG;
+#line 237 "../SqlLexer.lpp"
+return TOKEN_LIKE;
 	YY_BREAK
 case 74:
 YY_RULE_SETUP
-#line 236 "../SqlLexer.lpp"
-return TOKEN_MINUTE;
+#line 238 "../SqlLexer.lpp"
+return TOKEN_LIMIT;
 	YY_BREAK
 case 75:
 YY_RULE_SETUP
-#line 237 "../SqlLexer.lpp"
-return TOKEN_MONTH;
+#line 239 "../SqlLexer.lpp"
+return TOKEN_LONG;
 	YY_BREAK
 case 76:
 YY_RULE_SETUP
-#line 238 "../SqlLexer.lpp"
-return TOKEN_NOT;
+#line 240 "../SqlLexer.lpp"
+return TOKEN_MINUTE;
 	YY_BREAK
 case 77:
 YY_RULE_SETUP
-#line 239 "../SqlLexer.lpp"
-return TOKEN_NULL;
+#line 241 "../SqlLexer.lpp"
+return TOKEN_MONTH;
 	YY_BREAK
 case 78:
 YY_RULE_SETUP
-#line 240 "../SqlLexer.lpp"
-return TOKEN_NULLS;
+#line 242 "../SqlLexer.lpp"
+return TOKEN_NOT;
 	YY_BREAK
 case 79:
 YY_RULE_SETUP
-#line 241 "../SqlLexer.lpp"
-return TOKEN_OFF;
+#line 243 "../SqlLexer.lpp"
+return TOKEN_NULL;
 	YY_BREAK
 case 80:
 YY_RULE_SETUP
-#line 242 "../SqlLexer.lpp"
-return TOKEN_ON;
+#line 244 "../SqlLexer.lpp"
+return TOKEN_NULLS;
 	YY_BREAK
 case 81:
 YY_RULE_SETUP
-#line 243 "../SqlLexer.lpp"
-return TOKEN_OR;
+#line 245 "../SqlLexer.lpp"
+return TOKEN_OFF;
 	YY_BREAK
 case 82:
 YY_RULE_SETUP
-#line 244 "../SqlLexer.lpp"
+#line 246 "../SqlLexer.lpp"
+return TOKEN_ON;
+	YY_BREAK
+case 83:
+YY_RULE_SETUP
+#line 247 "../SqlLexer.lpp"
+return TOKEN_OR;
+	YY_BREAK
+case 84:
+YY_RULE_SETUP
+#line 248 "../SqlLexer.lpp"
 return TOKEN_ORDER;
 	YY_BREAK
-case 83:
+case 85:
 YY_RULE_SETUP
-#line 245 "../SqlLexer.lpp"
+#line 249 "../SqlLexer.lpp"
 return TOKEN_OUTER;
 	YY_BREAK
-case 84:
+case 86:
 YY_RULE_SETUP
-#line 246 "../SqlLexer.lpp"
+#line 250 "../SqlLexer.lpp"
+return TOKEN_OVER;
+	YY_BREAK
+case 87:
+YY_RULE_SETUP
+#line 251 "../SqlLexer.lpp"
 return TOKEN_PARTITION;
 	YY_BREAK
-case 85:
+case 88:
 YY_RULE_SETUP
-#line 247 "../SqlLexer.lpp"
+#line 252 "../SqlLexer.lpp"
 return TOKEN_PARTITIONS;
 	YY_BREAK
-case 86:
+case 89:
 YY_RULE_SETUP
-#line 248 "../SqlLexer.lpp"
+#line 253 "../SqlLexer.lpp"
 return TOKEN_PERCENT;
 	YY_BREAK
-case 87:
+case 90:
 YY_RULE_SETUP
-#line 249 "../SqlLexer.lpp"
+#line 254 "../SqlLexer.lpp"
+return TOKEN_PRECEDING;
+	YY_BREAK
+case 91:
+YY_RULE_SETUP
+#line 255 "../SqlLexer.lpp"
 return TOKEN_PRIMARY;
 	YY_BREAK
-case 88:
+case 92:
 YY_RULE_SETUP
-#line 250 "../SqlLexer.lpp"
+#line 256 "../SqlLexer.lpp"
 return TOKEN_QUIT;
 	YY_BREAK
-case 89:
+case 93:
 YY_RULE_SETUP
-#line 251 "../SqlLexer.lpp"
+#line 257 "../SqlLexer.lpp"
 return TOKEN_RANGE;
 	YY_BREAK
-case 90:
+case 94:
 YY_RULE_SETUP
-#line 252 "../SqlLexer.lpp"
+#line 258 "../SqlLexer.lpp"
 return TOKEN_REAL;
 	YY_BREAK
-case 91:
+case 95:
 YY_RULE_SETUP
-#line 253 "../SqlLexer.lpp"
+#line 259 "../SqlLexer.lpp"
 return TOKEN_REFERENCES;
 	YY_BREAK
-case 92:
+case 96:
 YY_RULE_SETUP
-#line 254 "../SqlLexer.lpp"
+#line 260 "../SqlLexer.lpp"
 return TOKEN_REGEXP;
 	YY_BREAK
-case 93:
+case 97:
 YY_RULE_SETUP
-#line 255 "../SqlLexer.lpp"
+#line 261 "../SqlLexer.lpp"
 return TOKEN_RIGHT;
 	YY_BREAK
-case 94:
+case 98:
 YY_RULE_SETUP
-#line 256 "../SqlLexer.lpp"
+#line 262 "../SqlLexer.lpp"
+return TOKEN_ROW;
+	YY_BREAK
+case 99:
+YY_RULE_SETUP
+#line 263 "../SqlLexer.lpp"
 return TOKEN_ROW_DELIMITER;
 	YY_BREAK
-case 95:
+case 100:
 YY_RULE_SETUP
-#line 257 "../SqlLexer.lpp"
+#line 264 "../SqlLexer.lpp"
+return TOKEN_ROWS;
+	YY_BREAK
+case 101:
+YY_RULE_SETUP
+#line 265 "../SqlLexer.lpp"
 return TOKEN_SECOND;
 	YY_BREAK
-case 96:
+case 102:
 YY_RULE_SETUP
-#line 258 "../SqlLexer.lpp"
+#line 266 "../SqlLexer.lpp"
 return TOKEN_SELECT;
 	YY_BREAK
-case 97:
+case 103:
 YY_RULE_SETUP
-#line 259 "../SqlLexer.lpp"
+#line 267 "../SqlLexer.lpp"
 return TOKEN_SET;
 	YY_BREAK
-case 98:
+case 104:
 YY_RULE_SETUP
-#line 260 "../SqlLexer.lpp"
+#line 268 "../SqlLexer.lpp"
 return TOKEN_SMA;
 	YY_BREAK
-case 99:
+case 105:
 YY_RULE_SETUP
-#line 261 "../SqlLexer.lpp"
+#line 269 "../SqlLexer.lpp"
 return TOKEN_SMALLINT;
 	YY_BREAK
-case 100:
+case 106:
 YY_RULE_SETUP
-#line 262 "../SqlLexer.lpp"
+#line 270 "../SqlLexer.lpp"
 return TOKEN_SUBSTRING;
 	YY_BREAK
-case 101:
+case 107:
 YY_RULE_SETUP
-#line 263 "../SqlLexer.lpp"
+#line 271 "../SqlLexer.lpp"
 return TOKEN_TABLE;
 	YY_BREAK
-case 102:
+case 108:
 YY_RULE_SETUP
-#line 264 "../SqlLexer.lpp"
+#line 272 "../SqlLexer.lpp"
 return TOKEN_THEN;
 	YY_BREAK
-case 103:
+case 109:
 YY_RULE_SETUP
-#line 265 "../SqlLexer.lpp"
+#line 273 "../SqlLexer.lpp"
 return TOKEN_TIME;
 	YY_BREAK
-case 104:
+case 110:
 YY_RULE_SETUP
-#line 266 "../SqlLexer.lpp"
+#line 274 "../SqlLexer.lpp"
 return TOKEN_TIMESTAMP;
 	YY_BREAK
-case 105:
+case 111:
 YY_RULE_SETUP
-#line 267 "../SqlLexer.lpp"
+#line 275 "../SqlLexer.lpp"
 return TOKEN_TRUE;
 	YY_BREAK
-case 106:
+case 112:
 YY_RULE_SETUP
-#line 268 "../SqlLexer.lpp"
+#line 276 "../SqlLexer.lpp"
 return TOKEN_TUPLESAMPLE;
 	YY_BREAK
-case 107:
+case 113:
 YY_RULE_SETUP
-#line 269 "../SqlLexer.lpp"
+#line 277 "../SqlLexer.lpp"
+return TOKEN_UNBOUNDED;
+	YY_BREAK
+case 114:
+YY_RULE_SETUP
+#line 278 "../SqlLexer.lpp"
 return TOKEN_UNIQUE;
 	YY_BREAK
-case 108:
+case 115:
 YY_RULE_SETUP
-#line 270 "../SqlLexer.lpp"
+#line 279 "../SqlLexer.lpp"
 return TOKEN_UPDATE;
 	YY_BREAK
-case 109:
+case 116:
 YY_RULE_SETUP
-#line 271 "../SqlLexer.lpp"
+#line 280 "../SqlLexer.lpp"
 return TOKEN_USING;
 	YY_BREAK
-case 110:
+case 117:
 YY_RULE_SETUP
-#line 272 "../SqlLexer.lpp"
+#line 281 "../SqlLexer.lpp"
 return TOKEN_VALUES;
 	YY_BREAK
-case 111:
+case 118:
 YY_RULE_SETUP
-#line 273 "../SqlLexer.lpp"
+#line 282 "../SqlLexer.lpp"
 return TOKEN_VARCHAR;
 	YY_BREAK
-case 112:
+case 119:
 YY_RULE_SETUP
-#line 274 "../SqlLexer.lpp"
+#line 283 "../SqlLexer.lpp"
 return TOKEN_WHEN;
 	YY_BREAK
-case 113:
+case 120:
 YY_RULE_SETUP
-#line 275 "../SqlLexer.lpp"
+#line 284 "../SqlLexer.lpp"
 return TOKEN_WHERE;
 	YY_BREAK
-case 114:
+case 121:
 YY_RULE_SETUP
-#line 276 "../SqlLexer.lpp"
+#line 285 "../SqlLexer.lpp"
+return TOKEN_WINDOW;
+	YY_BREAK
+case 122:
+YY_RULE_SETUP
+#line 286 "../SqlLexer.lpp"
 return TOKEN_WITH;
 	YY_BREAK
-case 115:
+case 123:
 YY_RULE_SETUP
-#line 277 "../SqlLexer.lpp"
+#line 287 "../SqlLexer.lpp"
 return TOKEN_YEAR;
 	YY_BREAK
-case 116:
+case 124:
 YY_RULE_SETUP
-#line 278 "../SqlLexer.lpp"
+#line 288 "../SqlLexer.lpp"
 return TOKEN_YEARMONTH;
 	YY_BREAK
-case 117:
+case 125:
 YY_RULE_SETUP
-#line 280 "../SqlLexer.lpp"
+#line 290 "../SqlLexer.lpp"
 return TOKEN_EQ;
 	YY_BREAK
-case 118:
+case 126:
 YY_RULE_SETUP
-#line 281 "../SqlLexer.lpp"
+#line 291 "../SqlLexer.lpp"
 return TOKEN_NEQ;
 	YY_BREAK
-case 119:
+case 127:
 YY_RULE_SETUP
-#line 282 "../SqlLexer.lpp"
+#line 292 "../SqlLexer.lpp"
 return TOKEN_NEQ;
 	YY_BREAK
-case 120:
+case 128:
 YY_RULE_SETUP
-#line 283 "../SqlLexer.lpp"
+#line 293 "../SqlLexer.lpp"
 return TOKEN_LT;
 	YY_BREAK
-case 121:
+case 129:
 YY_RULE_SETUP
-#line 284 "../SqlLexer.lpp"
+#line 294 "../SqlLexer.lpp"
 return TOKEN_GT;
 	YY_BREAK
-case 122:
+case 130:
 YY_RULE_SETUP
-#line 285 "../SqlLexer.lpp"
+#line 295 "../SqlLexer.lpp"
 return TOKEN_LEQ;
 	YY_BREAK
-case 123:
+case 131:
 YY_RULE_SETUP
-#line 286 "../SqlLexer.lpp"
+#line 296 "../SqlLexer.lpp"
 return TOKEN_GEQ;
 	YY_BREAK
-case 124:
+case 132:
 YY_RULE_SETUP
-#line 288 "../SqlLexer.lpp"
+#line 298 "../SqlLexer.lpp"
 return yytext[0];
 	YY_BREAK
-case 125:
+case 133:
 YY_RULE_SETUP
-#line 289 "../SqlLexer.lpp"
+#line 299 "../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 126:
+case 134:
 YY_RULE_SETUP
-#line 295 "../SqlLexer.lpp"
+#line 305 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED_ESCAPED);
   }
 	YY_BREAK
-case 127:
+case 135:
 YY_RULE_SETUP
-#line 300 "../SqlLexer.lpp"
+#line 310 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED);
   }
 	YY_BREAK
-case 128:
+case 136:
 YY_RULE_SETUP
-#line 305 "../SqlLexer.lpp"
+#line 315 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_DOUBLE_QUOTED);
@@ -2092,7 +2161,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 314 "../SqlLexer.lpp"
+#line 324 "../SqlLexer.lpp"
 {
     delete yylval->string_value_;
     BEGIN(INITIAL);
@@ -2103,9 +2172,9 @@ case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
 
 /* Process escape sequences. */
 
-case 129:
+case 137:
 YY_RULE_SETUP
-#line 324 "../SqlLexer.lpp"
+#line 334 "../SqlLexer.lpp"
 {
     /* Octal code */
     unsigned int code;
@@ -2119,9 +2188,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 130:
+case 138:
 YY_RULE_SETUP
-#line 336 "../SqlLexer.lpp"
+#line 346 "../SqlLexer.lpp"
 {
     /* Hexadecimal code */
     unsigned int code;
@@ -2129,9 +2198,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 131:
+case 139:
 YY_RULE_SETUP
-#line 342 "../SqlLexer.lpp"
+#line 352 "../SqlLexer.lpp"
 {
     /* A numeric escape sequence that isn't correctly specified. */
     delete yylval->string_value_;
@@ -2140,58 +2209,58 @@ YY_RULE_SETUP
     return TOKEN_LEX_ERROR;
   }
 	YY_BREAK
-case 132:
+case 140:
 YY_RULE_SETUP
-#line 349 "../SqlLexer.lpp"
+#line 359 "../SqlLexer.lpp"
 {
     /* Backspace */
     yylval->string_value_->push_back('\b');
   }
 	YY_BREAK
-case 133:
+case 141:
 YY_RULE_SETUP
-#line 353 "../SqlLexer.lpp"
+#line 363 "../SqlLexer.lpp"
 {
     /* Form-feed */
     yylval->string_value_->push_back('\f');
   }
 	YY_BREAK
-case 134:
+case 142:
 YY_RULE_SETUP
-#line 357 "../SqlLexer.lpp"
+#line 367 "../SqlLexer.lpp"
 {
     /* Newline */
     yylval->string_value_->push_back('\n');
   }
 	YY_BREAK
-case 135:
+case 143:
 YY_RULE_SETUP
-#line 361 "../SqlLexer.lpp"
+#line 371 "../SqlLexer.lpp"
 {
     /* Carriage-return */
     yylval->string_value_->push_back('\r');
   }
 	YY_BREAK
-case 136:
+case 144:
 YY_RULE_SETUP
-#line 365 "../SqlLexer.lpp"
+#line 375 "../SqlLexer.lpp"
 {
     /* Horizontal Tab */
     yylval->string_value_->push_back('\t');
   }
 	YY_BREAK
-case 137:
-/* rule 137 can match eol */
+case 145:
+/* rule 145 can match eol */
 YY_RULE_SETUP
-#line 369 "../SqlLexer.lpp"
+#line 379 "../SqlLexer.lpp"
 {
     /* Any other character (including actual newline or carriage return) */
     yylval->string_value_->push_back(yytext[1]);
   }
 	YY_BREAK
-case 138:
+case 146:
 YY_RULE_SETUP
-#line 373 "../SqlLexer.lpp"
+#line 383 "../SqlLexer.lpp"
 {
     /* This should only be encountered right before an EOF. */
     delete yylval->string_value_;
@@ -2202,17 +2271,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 139:
+case 147:
 YY_RULE_SETUP
-#line 383 "../SqlLexer.lpp"
+#line 393 "../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 140:
+case 148:
 YY_RULE_SETUP
-#line 387 "../SqlLexer.lpp"
+#line 397 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2221,17 +2290,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 141:
+case 149:
 YY_RULE_SETUP
-#line 395 "../SqlLexer.lpp"
+#line 405 "../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 142:
+case 150:
 YY_RULE_SETUP
-#line 399 "../SqlLexer.lpp"
+#line 409 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2239,94 +2308,94 @@ YY_RULE_SETUP
   }
 	YY_BREAK
 
-case 143:
-/* rule 143 can match eol */
+case 151:
+/* rule 151 can match eol */
 YY_RULE_SETUP
-#line 406 "../SqlLexer.lpp"
+#line 416 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 144:
-/* rule 144 can match eol */
+case 152:
+/* rule 152 can match eol */
 YY_RULE_SETUP
-#line 411 "../SqlLexer.lpp"
+#line 421 "../SqlLexer.lpp"
 {
   /* Scan up to a quote or escape sequence. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 145:
-/* rule 145 can match eol */
+case 153:
+/* rule 153 can match eol */
 YY_RULE_SETUP
-#line 416 "../SqlLexer.lpp"
+#line 426 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
 
-case 146:
+case 154:
 YY_RULE_SETUP
-#line 422 "../SqlLexer.lpp"
+#line 432 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(
         yylloc->first_line, yylloc->first_column, std::string(yytext, yyleng));
     return TOKEN_NAME;
   }
 	YY_BREAK
-case 147:
+case 155:
 YY_RULE_SETUP
-#line 428 "../SqlLexer.lpp"
+#line 438 "../SqlLexer.lpp"
 {
     yylval->numeric_literal_value_ = new quickstep::NumericParseLiteralValue(
         yylloc->first_line, yylloc->first_column, yytext);
     return TOKEN_UNSIGNED_NUMVAL;
   }
 	YY_BREAK
-case 148:
+case 156:
 YY_RULE_SETUP
-#line 434 "../SqlLexer.lpp"
+#line 444 "../SqlLexer.lpp"
 /* comment */
 	YY_BREAK
-case 149:
-/* rule 149 can match eol */
+case 157:
+/* rule 157 can match eol */
 YY_RULE_SETUP
-#line 436 "../SqlLexer.lpp"
+#line 446 "../SqlLexer.lpp"
 { yycolumn = 0; }
 	YY_BREAK
-case 150:
+case 158:
 YY_RULE_SETUP
-#line 438 "../SqlLexer.lpp"
+#line 448 "../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 442 "../SqlLexer.lpp"
+#line 452 "../SqlLexer.lpp"
 {
   /* All conditions except for mutli-state string extracting conditions. */
   BEGIN(INITIAL);
   return TOKEN_EOF;
 }
 	YY_BREAK
-case 151:
+case 159:
 YY_RULE_SETUP
-#line 448 "../SqlLexer.lpp"
+#line 458 "../SqlLexer.lpp"
 {
   BEGIN(INITIAL);
   quickstep_yyerror(NULL, yyscanner, NULL, "illegal character");
   return TOKEN_LEX_ERROR;
 }
 	YY_BREAK
-case 152:
+case 160:
 YY_RULE_SETUP
-#line 454 "../SqlLexer.lpp"
+#line 464 "../SqlLexer.lpp"
 YY_FATAL_ERROR( "flex scanner jammed" );
 	YY_BREAK
-#line 2330 "SqlLexer_gen.cpp"
+#line 2399 "SqlLexer_gen.cpp"
 
 	case YY_END_OF_BUFFER:
 		{
@@ -2469,9 +2538,9 @@ YY_FATAL_ERROR( "flex scanner jammed" );
 static int yy_get_next_buffer (yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
-	char *dest = YY_CURRENT_BUFFER_LVALUE->yy_ch_buf;
-	char *source = yyg->yytext_ptr;
-	yy_size_t number_to_move, i;
+	register char *dest = YY_CURRENT_BUFFER_LVALUE->yy_ch_buf;
+	register char *source = yyg->yytext_ptr;
+	register int number_to_move, i;
 	int ret_val;
 
 	if ( yyg->yy_c_buf_p > &YY_CURRENT_BUFFER_LVALUE->yy_ch_buf[yyg->yy_n_chars + 1] )
@@ -2500,7 +2569,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 	/* Try to read more data. */
 
 	/* First move last chars to start of buffer. */
-	number_to_move = (yy_size_t) (yyg->yy_c_buf_p - yyg->yytext_ptr) - 1;
+	number_to_move = (int) (yyg->yy_c_buf_p - yyg->yytext_ptr) - 1;
 
 	for ( i = 0; i < number_to_move; ++i )
 		*(dest++) = *(source++);
@@ -2603,15 +2672,15 @@ static int yy_get_next_buffer (yyscan_t 

<TRUNCATED>


[43/50] [abbrv] incubator-quickstep git commit: Added move semantic in the constructor of window plan.

Posted by zu...@apache.org.
Added move semantic in the constructor of window plan.


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

Branch: refs/heads/tmp-relation-col-store
Commit: f6f0c8b8dbebbf9eba1a80fbc9462707090715a4
Parents: f66f4bd
Author: shixuan-fan <sh...@apache.org>
Authored: Tue Jun 28 15:42:09 2016 +0000
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jun 28 22:08:49 2016 -0700

----------------------------------------------------------------------
 query_optimizer/resolver/Resolver.cpp | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f6f0c8b8/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 1eb6d86..c07751a 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -228,13 +228,13 @@ struct Resolver::QueryAggregationInfo {
 };
 
 struct Resolver::WindowPlan {
-  WindowPlan(const E::WindowInfo &window_info_in,
-             const L::LogicalPtr &logical_plan_in)
-      : window_info(window_info_in),
-        logical_plan(logical_plan_in) {}
+  WindowPlan(const L::LogicalPtr &logical_plan_in,
+             E::WindowInfo &&window_info_in)  // NOLINT(whitespace/operators)
+      : logical_plan(logical_plan_in),
+        window_info(std::move(window_info_in)) {}
 
-  const E::WindowInfo window_info;
   const L::LogicalPtr logical_plan;
+  const E::WindowInfo window_info;
 };
 
 struct Resolver::WindowAggregationInfo {
@@ -1032,9 +1032,9 @@ L::LogicalPtr Resolver::resolveSelect(
       L::LogicalPtr sorted_logical_plan = resolveSortInWindow(logical_plan,
                                                               resolved_window);
 
-      WindowPlan window_plan(resolved_window, sorted_logical_plan);
+      WindowPlan window_plan(sorted_logical_plan, std::move(resolved_window));
 
-      sorted_window_map.emplace(window.name()->value(), window_plan);
+      sorted_window_map.emplace(window.name()->value(), std::move(window_plan));
     }
   }
 


[23/50] [abbrv] incubator-quickstep git commit: Fix bug: specify log file name correctly

Posted by zu...@apache.org.
Fix bug: specify log file name correctly


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

Branch: refs/heads/tmp-relation-col-store
Commit: 31e9e4cefb2a9fffd42138fcd40543f915ed07bb
Parents: 07c011d
Author: Navneet <na...@cs.wisc.edu>
Authored: Mon Jun 13 18:34:40 2016 -0500
Committer: Navneet <na...@cs.wisc.edu>
Committed: Mon Jun 13 18:39:29 2016 -0500

----------------------------------------------------------------------
 build/profile_build.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/31e9e4ce/build/profile_build.sh
----------------------------------------------------------------------
diff --git a/build/profile_build.sh b/build/profile_build.sh
index 5f2fc33..b7164ad 100755
--- a/build/profile_build.sh
+++ b/build/profile_build.sh
@@ -37,7 +37,7 @@ CMAKE_COMMAND="cmake \
 -D CMAKE_BUILD_TYPE=Debug \
 .. "
 MAKE_COMMAND="make VERBOSE=1"
-LOG_FILENAME=$LOG_FILENAME
+LOG_FILENAME=build.log
 
 # Continuously dump memory usage and cpu load info to files for later analysis
 function start_stat_collectors {


[39/50] [abbrv] incubator-quickstep git commit: Added PRIORITY clause in parser.

Posted by zu...@apache.org.
Added PRIORITY clause in parser.

- SQL statements (right now only SELECT statements) support "WITH
  PRIORITY N" clause.
- The priority value should be a non-zero unsigned integer.
- The priority value is a part of the QueryHandle class.


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

Branch: refs/heads/tmp-relation-col-store
Commit: d64289148bee499e2232637602b4197da6791fc4
Parents: 07435a4
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Jun 17 16:01:19 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Mon Jun 20 12:30:20 2016 -0500

----------------------------------------------------------------------
 parser/CMakeLists.txt                 |    8 +
 parser/ParsePriority.hpp              |   94 +
 parser/ParseStatement.hpp             |   33 +-
 parser/SqlLexer.lpp                   |    2 +
 parser/SqlParser.ypp                  |   37 +-
 parser/preprocessed/SqlLexer_gen.cpp  | 1263 +++---
 parser/preprocessed/SqlLexer_gen.hpp  |   16 +-
 parser/preprocessed/SqlParser_gen.cpp | 5699 +++++++++++++++-------------
 parser/preprocessed/SqlParser_gen.hpp |  323 +-
 parser/tests/Select.test              |   43 +
 query_optimizer/QueryHandle.hpp       |   13 +-
 query_optimizer/QueryProcessor.cpp    |    4 +-
 12 files changed, 4120 insertions(+), 3415 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index 114ad14..32ea1a9 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -103,6 +103,7 @@ add_library(quickstep_parser_ParsePartitionClause ../empty_src.cpp ParsePartitio
 add_library(quickstep_parser_ParsePredicate ParsePredicate.cpp ParsePredicate.hpp)
 add_library(quickstep_parser_ParsePredicateExists ../empty_src.cpp ParsePredicateExists.hpp)
 add_library(quickstep_parser_ParsePredicateInTableQuery ../empty_src.cpp ParsePredicateInTableQuery.hpp)
+add_library(quickstep_parser_ParsePriority ../empty_src.cpp ParsePriority.hpp)
 add_library(quickstep_parser_ParseSample ParseSample.cpp ParseSample.hpp)
 add_library(quickstep_parser_ParseSelect ../empty_src.cpp ParseSelect.hpp)
 add_library(quickstep_parser_ParseSelectionClause ParseSelectionClause.cpp ParseSelectionClause.hpp)
@@ -229,6 +230,10 @@ target_link_libraries(quickstep_parser_ParsePredicateInTableQuery
                       quickstep_parser_ParsePredicate
                       quickstep_parser_ParseSubqueryExpression
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_parser_ParsePriority
+                      quickstep_parser_ParseLiteralValue
+                      quickstep_parser_ParseTreeNode
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_parser_ParseSample
                       quickstep_parser_ParseLiteralValue
                       quickstep_parser_ParseTreeNode
@@ -267,6 +272,7 @@ target_link_libraries(quickstep_parser_ParseStatement
                       quickstep_parser_ParseKeyValue
                       quickstep_parser_ParsePartitionClause
                       quickstep_parser_ParsePredicate
+                      quickstep_parser_ParsePriority
                       quickstep_parser_ParseSelect
                       quickstep_parser_ParseString
                       quickstep_parser_ParseSubqueryTableReference
@@ -337,6 +343,7 @@ target_link_libraries(quickstep_parser_SqlParser
                       quickstep_parser_ParsePredicate
                       quickstep_parser_ParsePredicateExists
                       quickstep_parser_ParsePredicateInTableQuery
+                      quickstep_parser_ParsePriority
                       quickstep_parser_ParseSample
                       quickstep_parser_ParseSelect
                       quickstep_parser_ParseSelectionClause
@@ -414,6 +421,7 @@ target_link_libraries(quickstep_parser
                       quickstep_parser_ParsePredicate
                       quickstep_parser_ParsePredicateExists
                       quickstep_parser_ParsePredicateInTableQuery
+                      quickstep_parser_ParsePriority
                       quickstep_parser_ParserUtil
                       quickstep_parser_ParseSample
                       quickstep_parser_ParseSelect

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/ParsePriority.hpp
----------------------------------------------------------------------
diff --git a/parser/ParsePriority.hpp b/parser/ParsePriority.hpp
new file mode 100644
index 0000000..89806d4
--- /dev/null
+++ b/parser/ParsePriority.hpp
@@ -0,0 +1,94 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_PARSER_PARSE_PRIORITY_HPP_
+#define QUICKSTEP_PARSER_PARSE_PRIORITY_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parser/ParseLiteralValue.hpp"
+#include "parser/ParseTreeNode.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Parser
+ *  @{
+ */
+
+/**
+ * @brief A parsed representation of PRIORITY.
+ **/
+class ParsePriority : public ParseTreeNode {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param line_number The line number of "PRIORITY" in the SQL statement.
+   * @param column_number The column number of "PRIORITY" in the SQL statement.
+   * @param priority_expression The PRIORITY value expression.
+   **/
+  ParsePriority(const int line_number,
+                const int column_number,
+                NumericParseLiteralValue *priority_expression)
+      : ParseTreeNode(line_number, column_number),
+        priority_expression_(priority_expression) {}
+
+  /**
+   * @brief Destructor.
+   */
+  ~ParsePriority() override {}
+
+  /**
+   * @brief Gets the PRIORITY expression.
+   *
+   * @return PRIORITY expression
+   */
+  const NumericParseLiteralValue* priority_expression() const {
+    return priority_expression_.get();
+  }
+
+  std::string getName() const override {
+    return "PRIORITY";
+  }
+
+ 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 {
+    non_container_child_field_names->push_back("");
+    non_container_child_fields->push_back(priority_expression_.get());
+  }
+
+ private:
+  std::unique_ptr<NumericParseLiteralValue> priority_expression_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParsePriority);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_PARSER_PARSE_PRIORITY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/ParseStatement.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseStatement.hpp b/parser/ParseStatement.hpp
index 65acc68..61475a9 100644
--- a/parser/ParseStatement.hpp
+++ b/parser/ParseStatement.hpp
@@ -32,6 +32,7 @@
 #include "parser/ParseKeyValue.hpp"
 #include "parser/ParsePartitionClause.hpp"
 #include "parser/ParsePredicate.hpp"
+#include "parser/ParsePriority.hpp"
 #include "parser/ParseSelect.hpp"
 #include "parser/ParseString.hpp"
 #include "parser/ParseSubqueryTableReference.hpp"
@@ -83,6 +84,16 @@ class ParseStatement : public ParseTreeNode {
    **/
   virtual StatementType getStatementType() const = 0;
 
+  /**
+   * @brief Get the priority of the SQL statement. Note that the priority is
+   *        an unsigned non-zero integer.
+   *
+   * @return The priority of the SQL statement. The default priority is 1.
+   **/
+  virtual const std::uint64_t getPriority() const {
+    return 1;
+  }
+
  protected:
   ParseStatement(const int line_number, const int column_number)
       : ParseTreeNode(line_number, column_number) {
@@ -480,14 +491,18 @@ class ParseStatementSelect : public ParseStatement {
    * @param column_number Column number of the first token of this node in the SQL statement.
    * @param select_query The top-level SELECT query.
    * @param with_clause The WITH clause of common table query expressions.
+   * @param priority_clause The PRIORITY clause of this query. If not valid or
+   *        not present, this is NULL.
    **/
   ParseStatementSelect(const int line_number,
                        const int column_number,
                        ParseSelect *select_query,
-                       PtrVector<ParseSubqueryTableReference> *with_clause)
+                       PtrVector<ParseSubqueryTableReference> *with_clause,
+                       ParsePriority *priority_clause)
       : ParseStatement(line_number, column_number),
         select_query_(select_query),
-        with_clause_(with_clause) {
+        with_clause_(with_clause),
+        priority_clause_(priority_clause) {
   }
 
   /**
@@ -518,6 +533,14 @@ class ParseStatementSelect : public ParseStatement {
     return with_clause_.get();
   }
 
+  const std::uint64_t getPriority() const override {
+    if (priority_clause_ != nullptr) {
+      DCHECK(priority_clause_->priority_expression() != nullptr);
+      return priority_clause_->priority_expression()->long_value();
+    }
+    return 1;
+  }
+
  protected:
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
@@ -536,11 +559,17 @@ class ParseStatementSelect : public ParseStatement {
         container_child_fields->back().push_back(&common_subquery);
       }
     }
+
+    if (priority_clause_ != nullptr) {
+      non_container_child_field_names->push_back("priority");
+      non_container_child_fields->push_back(priority_clause_.get());
+    }
   }
 
  private:
   std::unique_ptr<ParseSelect> select_query_;
   std::unique_ptr<PtrVector<ParseSubqueryTableReference>> with_clause_;
+  std::unique_ptr<ParsePriority> priority_clause_;
 
   DISALLOW_COPY_AND_ASSIGN(ParseStatementSelect);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index ee34400..ec18f38 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -61,6 +61,7 @@ class ParseOrderBy;
 class ParseOrderByItem;
 class ParsePartitionClause;
 class ParsePredicate;
+class ParsePriority;
 class ParseSample;
 class ParseScalarLiteral;
 class ParseSearchedWhenClause;
@@ -253,6 +254,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "percent"          return TOKEN_PERCENT;
   "preceding"        return TOKEN_PRECEDING;
   "primary"          return TOKEN_PRIMARY;
+  "priority"         return TOKEN_PRIORITY;
   "quit"             return TOKEN_QUIT;
   "range"            return TOKEN_RANGE;
   "real"             return TOKEN_REAL;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index 81fa3ae..382ea44 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -85,6 +85,7 @@ typedef struct YYLTYPE {
 #include "parser/ParsePredicate.hpp"
 #include "parser/ParsePredicateExists.hpp"
 #include "parser/ParsePredicateInTableQuery.hpp"
+#include "parser/ParsePriority.hpp"
 #include "parser/ParserUtil.hpp"
 #include "parser/ParseSample.hpp"
 #include "parser/ParseSelect.hpp"
@@ -212,6 +213,8 @@ typedef void* yyscan_t;
 
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
+
+  quickstep::ParsePriority *opt_priority_clause_;
 }
 
 %{
@@ -319,6 +322,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_PERCENT;
 %token TOKEN_PRECEDING;
 %token TOKEN_PRIMARY;
+%token TOKEN_PRIORITY;
 %token TOKEN_QUIT;
 %token TOKEN_RANGE;
 %token TOKEN_REAL;
@@ -583,6 +587,9 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %type <window_frame_info_>
   opt_window_frame
 
+%type <opt_priority_clause_>
+  opt_priority_clause
+
 %type <with_list_>
   with_clause
   with_list
@@ -1143,13 +1150,35 @@ assignment_item:
 
 /* Select Queries */
 select_statement:
-  select_query {
-    $$ = new quickstep::ParseStatementSelect(@1.first_line, @1.first_column, $1, nullptr);
+  select_query opt_priority_clause {
+    $$ = new quickstep::ParseStatementSelect(@1.first_line, @1.first_column, $1, nullptr, $2);
   }
-  | with_clause select_query {
-    $$ = new quickstep::ParseStatementSelect(@1.first_line, @1.first_column, $2, $1);
+  | with_clause select_query opt_priority_clause {
+    $$ = new quickstep::ParseStatementSelect(@1.first_line, @1.first_column, $2, $1, $3);
   };
 
+opt_priority_clause:
+  {
+    $$ = nullptr;
+  }
+  | TOKEN_WITH TOKEN_PRIORITY TOKEN_UNSIGNED_NUMVAL {
+    if ($3->float_like()) {
+      delete $3;
+      $$ = nullptr;
+      quickstep_yyerror(&@3, yyscanner, nullptr, "PRIORITY value must be an integer");
+      YYERROR;
+    } else {
+      if ($3->long_value() <= 0) {
+        delete $3;
+        $$ = nullptr;
+        quickstep_yyerror(&@3, yyscanner, nullptr, "PRIORITY value must be positive");
+        YYERROR;
+      } else {
+        $$ = new quickstep::ParsePriority(@1.first_line, @1.first_column, $3);
+      }
+    }
+  };
+  
 with_clause:
   TOKEN_WITH with_list {
     $$ = $2;


[04/50] [abbrv] incubator-quickstep git commit: Reordered Query ID in operators and work orders.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 4ef5a5c..074b603 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -92,6 +92,7 @@ constexpr tuple_id kNumDimTuples = 200;
 constexpr tuple_id kNumFactTuples = 300;
 constexpr tuple_id kBlockSize = 10;
 
+constexpr std::size_t kQueryId = 0;
 constexpr int kOpIndex = 0;
 
 }  // namespace
@@ -332,12 +333,12 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
 
   // Create the builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_long.getID()),
                             dim_col_long.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID
+                            join_hash_table_index));
 
   // Create the prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -359,18 +360,18 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_long.getID()),
-                           fact_col_long.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_long.getID()),
+      fact_col_long.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -423,7 +424,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -475,12 +476,12 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
 
   // Create the builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_int.getID()),
                             dim_col_int.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID
+                            join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -507,18 +508,19 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_int.getID()),
-                           fact_col_int.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_int.getID()),
+      fact_col_int.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
+
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -592,7 +594,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -636,12 +638,12 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
 
   // Create builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_char.getID()),
                             dim_col_char.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -663,18 +665,18 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_char.getID()),
-                           fact_col_char.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_char.getID()),
+      fact_col_char.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -727,7 +729,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -772,12 +774,12 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
 
   // Create builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_varchar.getID()),
                             dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -804,18 +806,19 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_varchar.getID()),
-                           fact_col_varchar.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_varchar.getID()),
+      fact_col_varchar.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
+
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -893,7 +896,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   }
 
   // Create the cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -943,12 +946,12 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   dim_key_attrs.push_back(dim_col_varchar.getID());
 
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -979,18 +982,19 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   fact_key_attrs.push_back(fact_col_long.getID());
   fact_key_attrs.push_back(fact_col_varchar.getID());
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           fact_key_attrs,
-                           fact_col_long.getType().isNullable() || fact_col_varchar.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      fact_key_attrs,
+      fact_col_long.getType().isNullable() ||
+          fact_col_varchar.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -1068,7 +1072,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -1119,12 +1123,12 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   dim_key_attrs.push_back(dim_col_varchar.getID());
 
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -1166,17 +1170,18 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   query_context_proto.add_predicates()->CopyFrom(residual_pred->getProto());
 
   unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
+      new HashJoinOperator(kQueryId,
+                           *dim_table_,
                            *fact_table_,
                            true /* is_stored */,
                            fact_key_attrs,
-                           fact_col_long.getType().isNullable() || fact_col_varchar.getType().isNullable(),
+                           fact_col_long.getType().isNullable() ||
+                               fact_col_varchar.getType().isNullable(),
                            *result_table,
                            output_destination_index,
                            join_hash_table_index,
                            residual_pred_index,
-                           selection_index,
-                           0  /* dummy query ID */));
+                           selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -1254,7 +1259,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index 244091f..fc10671 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -84,6 +84,7 @@ namespace quickstep {
 
 namespace {
 
+constexpr std::size_t kQueryId = 0;
 constexpr const std::size_t kOpIndex = 0;
 
 // Helper struct for test tuple that will that will be inserted and sorted.
@@ -1565,7 +1566,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                            const std::size_t top_k = 0) {
     const QueryContext::sort_config_id sort_config_index = createSortConfigProto(attrs, ordering, null_ordering);
 
-    merge_op_.reset(new SortMergeRunOperator(*input_table_,
+    merge_op_.reset(new SortMergeRunOperator(kQueryId,
+                                             *input_table_,
                                              *result_table_,
                                              insert_destination_index_,
                                              *run_table_,
@@ -1573,8 +1575,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             true,
-                                             0  /* dummy query ID */));
+                                             true));
+
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.
@@ -1609,7 +1611,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                         const std::size_t top_k = 0) {
     const QueryContext::sort_config_id sort_config_index = createSortConfigProto(attrs, ordering, null_ordering);
 
-    merge_op_.reset(new SortMergeRunOperator(*input_table_,
+    merge_op_.reset(new SortMergeRunOperator(kQueryId,
+                                             *input_table_,
                                              *result_table_,
                                              insert_destination_index_,
                                              *run_table_,
@@ -1617,8 +1620,7 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             false,
-                                             0  /* dummy query ID */));
+                                             false));
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index 6f24b92..71a80e4 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -77,6 +77,7 @@ namespace quickstep {
 
 namespace {
 
+constexpr std::size_t kQueryId = 0;
 constexpr int kOpIndex = 0;
 
 // Helper struct for test tuple that will that will be inserted and sorted.
@@ -353,13 +354,14 @@ class SortRunGenerationOperatorTest : public ::testing::Test {
       order_by_proto->set_null_first(null_ordering[i]);
     }
 
-    std::unique_ptr<RelationalOperator> run_gen(
-        new SortRunGenerationOperator(*input_table_,
-                                      *result_table_,
-                                      insert_destination_index,
-                                      sort_config_index,
-                                      true /* is_stored */,
-                                      0  /* dummy query ID */));
+    std::unique_ptr<RelationalOperator> run_gen(new SortRunGenerationOperator(
+        kQueryId,
+        *input_table_,
+        *result_table_,
+        insert_destination_index,
+        sort_config_index,
+        true /* is_stored */));
+
     run_gen->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index 7626686..ef6fc2d 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -55,6 +55,7 @@ const char *failure_output_filename;
 namespace quickstep {
 
 namespace {
+constexpr std::size_t kQueryId = 0;
 constexpr int kOpIndex = 0;
 }  // namespace
 
@@ -188,13 +189,13 @@ TEST_F(TextScanOperatorTest, ScanTest) {
   output_destination_proto->set_relational_op_index(kOpIndex);
 
   std::unique_ptr<TextScanOperator> text_scan_op(
-      new TextScanOperator(input_filename,
+      new TextScanOperator(kQueryId,
+                           input_filename,
                            '\t',
                            true,
                            false,
                            *relation_,
-                           output_destination_index,
-                           0  /* dummy query ID */));
+                           output_destination_index));
 
   // Setup query_context_.
   query_context_.reset(new QueryContext(query_context_proto,


[44/50] [abbrv] incubator-quickstep git commit: Added PhysicalGenerator support for Window Aggregation.

Posted by zu...@apache.org.
Added PhysicalGenerator support for Window Aggregation.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 5c4e8dbf19dd604fdb728367031dcf33d345240c
Parents: f6f0c8b
Author: shixuan-fan <sh...@apache.org>
Authored: Tue Jun 28 16:25:52 2016 +0000
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jun 28 22:50:17 2016 -0700

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/ExecutionGenerator.cpp          |   4 +
 query_optimizer/physical/CMakeLists.txt         |  15 +-
 query_optimizer/physical/PhysicalType.hpp       |   3 +-
 query_optimizer/physical/WindowAggregate.cpp    |  66 +++++
 query_optimizer/physical/WindowAggregate.hpp    | 133 ++++++++++
 query_optimizer/strategy/CMakeLists.txt         |   5 +-
 query_optimizer/strategy/OneToOne.cpp           |  11 +-
 .../tests/physical_generator/Select.test        | 264 +++++++++++++++++++
 9 files changed, 495 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 8f08130..8912414 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -109,6 +109,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_UpdateTable
+                      quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_relationaloperators_AggregationOperator
                       quickstep_relationaloperators_BuildHashOperator
                       quickstep_relationaloperators_CreateIndexOperator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index f9fd742..45f5f78 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -83,6 +83,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 #include "query_optimizer/physical/UpdateTable.hpp"
+#include "query_optimizer/physical/WindowAggregate.hpp"
 #include "relational_operators/AggregationOperator.hpp"
 #include "relational_operators/BuildHashOperator.hpp"
 #include "relational_operators/CreateIndexOperator.hpp"
@@ -282,6 +283,9 @@ void ExecutionGenerator::generatePlanInternal(
     case P::PhysicalType::kUpdateTable:
       return convertUpdateTable(
           std::static_pointer_cast<const P::UpdateTable>(physical_plan));
+    case P::PhysicalType::kWindowAggregate:
+      THROW_SQL_ERROR()
+          << "Window aggregate function is not supported yet :(";
     default:
       LOG(FATAL) << "Unknown physical plan node "
                  << physical_plan->getShortString();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/physical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CMakeLists.txt b/query_optimizer/physical/CMakeLists.txt
index ea3752d..1ad30e4 100644
--- a/query_optimizer/physical/CMakeLists.txt
+++ b/query_optimizer/physical/CMakeLists.txt
@@ -39,6 +39,7 @@ add_library(quickstep_queryoptimizer_physical_TableGenerator ../../empty_src.cpp
 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_UpdateTable UpdateTable.cpp UpdateTable.hpp)
+add_library(quickstep_queryoptimizer_physical_WindowAggregate WindowAggregate.cpp WindowAggregate.hpp)
 
 # Link dependencies:
 target_link_libraries(quickstep_queryoptimizer_physical_Aggregate
@@ -250,6 +251,17 @@ target_link_libraries(quickstep_queryoptimizer_physical_UpdateTable
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_utility_Cast
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_physical_WindowAggregate
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_Alias
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_utility_Cast
+                      quickstep_utility_Macros)
 
 # Module all-in-one library:
 add_library(quickstep_queryoptimizer_physical ../../empty_src.cpp OptimizerPhysicalModule.hpp)
@@ -276,4 +288,5 @@ target_link_libraries(quickstep_queryoptimizer_physical
                       quickstep_queryoptimizer_physical_TableGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
-                      quickstep_queryoptimizer_physical_UpdateTable)
+                      quickstep_queryoptimizer_physical_UpdateTable
+                      quickstep_queryoptimizer_physical_WindowAggregate)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/physical/PhysicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PhysicalType.hpp b/query_optimizer/physical/PhysicalType.hpp
index b036557..14f8e1a 100644
--- a/query_optimizer/physical/PhysicalType.hpp
+++ b/query_optimizer/physical/PhysicalType.hpp
@@ -47,7 +47,8 @@ enum class PhysicalType {
   kTableGenerator,
   kTableReference,
   kTopLevelPlan,
-  kUpdateTable
+  kUpdateTable,
+  kWindowAggregate
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/physical/WindowAggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/WindowAggregate.cpp b/query_optimizer/physical/WindowAggregate.cpp
new file mode 100644
index 0000000..2927107
--- /dev/null
+++ b/query_optimizer/physical/WindowAggregate.cpp
@@ -0,0 +1,66 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "query_optimizer/physical/WindowAggregate.hpp"
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "utility/Cast.hpp"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+namespace E = ::quickstep::optimizer::expressions;
+
+std::vector<E::AttributeReferencePtr> WindowAggregate::getOutputAttributes() const {
+  std::vector<E::AttributeReferencePtr> output_attributes(
+      input_->getOutputAttributes());
+  output_attributes.push_back(E::ToRef(window_aggregate_expression_));
+  return output_attributes;
+}
+
+std::vector<E::AttributeReferencePtr> WindowAggregate::getReferencedAttributes()
+    const {
+  return window_aggregate_expression_->getReferencedAttributes();
+}
+
+void WindowAggregate::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("input");
+  non_container_child_fields->push_back(input_);
+
+  non_container_child_field_names->push_back("window_aggregate_expression");
+  non_container_child_fields->push_back(window_aggregate_expression_);
+}
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/physical/WindowAggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/WindowAggregate.hpp b/query_optimizer/physical/WindowAggregate.hpp
new file mode 100644
index 0000000..4b17d07
--- /dev/null
+++ b/query_optimizer/physical/WindowAggregate.hpp
@@ -0,0 +1,133 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_WINDOW_AGGREGATE_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_WINDOW_AGGREGATE_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/Alias.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+/** \addtogroup OptimizerLogical
+ *  @{
+ */
+
+class WindowAggregate;
+typedef std::shared_ptr<const WindowAggregate> WindowAggregatePtr;
+
+/**
+ * @brief Window Aggregate operator that computes window aggregate expressions.
+ */
+class WindowAggregate : public Physical {
+ public:
+  PhysicalType getPhysicalType() const override {
+    return PhysicalType::kWindowAggregate;
+  }
+
+  std::string getName() const override { return "WindowAggregate"; }
+
+  /**
+   * @return The input physical node.
+   */
+  const PhysicalPtr input() const { return input_; }
+
+  /**
+   * @return Window aggregate expression.
+   */
+  inline const expressions::AliasPtr window_aggregate_expression() const {
+    return window_aggregate_expression_;
+  }
+
+  PhysicalPtr copyWithNewChildren(
+      const std::vector<PhysicalPtr> &new_children) const override {
+    DCHECK_EQ(getNumChildren(), new_children.size());
+    return Create(new_children[0], window_aggregate_expression_);
+  }
+
+  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 {
+    // The project expressions for an WindowAggregate cannot be changed.
+    return false;
+  }
+
+  /**
+   * @brief Creates an WindowAggregate physical node.
+   *
+   * @param input The input node.
+   * @param window_aggregate_expression The window aggregate expression.
+   * @return An immutable WindowAggregate node.
+   */
+  static WindowAggregatePtr Create(
+      const PhysicalPtr &input,
+      const expressions::AliasPtr &window_aggregate_expression) {
+    return WindowAggregatePtr(new WindowAggregate(input,
+                                                  window_aggregate_expression));
+  }
+
+ 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:
+  WindowAggregate(
+      const PhysicalPtr &input,
+      const expressions::AliasPtr &window_aggregate_expression)
+      : input_(input),
+        window_aggregate_expression_(window_aggregate_expression) {
+    addChild(input_);
+  }
+
+  const PhysicalPtr input_;
+  const expressions::AliasPtr window_aggregate_expression_;
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregate);
+};
+
+/** @} */
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_WINDOW_AGGREGATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/strategy/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/CMakeLists.txt b/query_optimizer/strategy/CMakeLists.txt
index 84e151e..517bea3 100644
--- a/query_optimizer/strategy/CMakeLists.txt
+++ b/query_optimizer/strategy/CMakeLists.txt
@@ -89,6 +89,7 @@ target_link_libraries(quickstep_queryoptimizer_strategy_OneToOne
                       quickstep_queryoptimizer_logical_TableReference
                       quickstep_queryoptimizer_logical_TopLevelPlan
                       quickstep_queryoptimizer_logical_UpdateTable
+                      quickstep_queryoptimizer_logical_WindowAggregate
                       quickstep_queryoptimizer_physical_CopyFrom
                       quickstep_queryoptimizer_physical_CreateIndex
                       quickstep_queryoptimizer_physical_CreateTable
@@ -104,9 +105,9 @@ target_link_libraries(quickstep_queryoptimizer_strategy_OneToOne
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_UpdateTable
+                      quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_queryoptimizer_strategy_Strategy
-                      quickstep_utility_Macros
-                      quickstep_utility_SqlError)
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_strategy_Selection
                       glog
                       quickstep_queryoptimizer_LogicalToPhysicalMapper

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/strategy/OneToOne.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/OneToOne.cpp b/query_optimizer/strategy/OneToOne.cpp
index f49a25c..e9a8897 100644
--- a/query_optimizer/strategy/OneToOne.cpp
+++ b/query_optimizer/strategy/OneToOne.cpp
@@ -41,6 +41,7 @@
 #include "query_optimizer/logical/TableReference.hpp"
 #include "query_optimizer/logical/TopLevelPlan.hpp"
 #include "query_optimizer/logical/UpdateTable.hpp"
+#include "query_optimizer/logical/WindowAggregate.hpp"
 #include "query_optimizer/physical/CopyFrom.hpp"
 #include "query_optimizer/physical/CreateIndex.hpp"
 #include "query_optimizer/physical/CreateTable.hpp"
@@ -55,7 +56,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 #include "query_optimizer/physical/UpdateTable.hpp"
-#include "utility/SqlError.hpp"
+#include "query_optimizer/physical/WindowAggregate.hpp"
 
 namespace quickstep {
 namespace optimizer {
@@ -210,8 +211,12 @@ bool OneToOne::generatePlan(const L::LogicalPtr &logical_input,
       return true;
     }
     case L::LogicalType::kWindowAggregate: {
-      THROW_SQL_ERROR()
-          << "Window aggregate function is not supported currently :(";
+      const L::WindowAggregatePtr window_aggregate =
+          std::static_pointer_cast<const L::WindowAggregate>(logical_input);
+      *physical_output = P::WindowAggregate::Create(
+          physical_mapper_->createOrGetPhysicalFromLogical(window_aggregate->input()),
+          window_aggregate->window_aggregate_expression());
+      return true;
     }
     default:
       return false;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/tests/physical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index 3365206..d99916c 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -2876,3 +2876,267 @@ TopLevelPlan
 +-output_attributes=
   +-AttributeReference[id=5,name=x,relation=,type=Int]
   +-AttributeReference[id=6,name=y,relation=,type=Int]
+==
+
+# Window Aggregate Function Test.
+SELECT avg(int_col) OVER w FROM test
+WINDOW w AS
+(PARTITION BY char_col
+ ORDER BY long_col DESC NULLS LAST
+ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW);
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,false],nulls_first=[false,false]]
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_expressions=
+| | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=AVG,window_name=w,is_ascending=[false],
+| |     nulls_first=[false],frame_mode=row,num_preceding=-1,num_following=0]
+| |     +-arguments=
+| |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| |     +-order_by=
+| |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,false],nulls_first=[false,false]]
+| | | +-input=TableReference[relation=Test,alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_attributes=
+| | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=AVG,window_name=w,is_ascending=[false],
+| |     nulls_first=[false],frame_mode=row,num_preceding=-1,num_following=0]
+| |     +-arguments=
+| |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| |     +-order_by=
+| |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| +-project_expressions=
+|   +-Alias[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+==
+
+SELECT int_col, sum(float_col) OVER
+(PARTITION BY vchar_col, long_col
+ ORDER BY double_col DESC NULLS LAST, int_col ASC NULLS FIRST
+ RANGE BETWEEN 3 PRECEDING AND 3 FOLLOWING)
+FROM test;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,true,false,true],
+| | | nulls_first=[false,false,false,true]]
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_expressions=
+| | |   +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   | type=VarChar(20) NULL]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | |   +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=SUM,window_name=,
+| |     is_ascending=[false,true],nulls_first=[false,true],frame_mode=range,
+| |     num_preceding=3,num_following=3]
+| |     +-arguments=
+| |     | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| |     | | type=VarChar(20) NULL]
+| |     | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |     +-order_by=
+| |       +-AttributeReference[id=3,name=double_col,relation=test,
+| |       | type=Double NULL]
+| |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| +-project_list=
+|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   +-Alias[id=6,name=,alias=sum(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+  +-AttributeReference[id=6,name=,alias=sum(float_col),relation=,
+    type=Double NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,true,false,true],
+| | | nulls_first=[false,false,false,true]]
+| | | +-input=TableReference[relation=Test,alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_attributes=
+| | |   +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   | type=VarChar(20) NULL]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | |   +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=SUM,window_name=,
+| |     is_ascending=[false,true],nulls_first=[false,true],frame_mode=range,
+| |     num_preceding=3,num_following=3]
+| |     +-arguments=
+| |     | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| |     | | type=VarChar(20) NULL]
+| |     | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |     +-order_by=
+| |       +-AttributeReference[id=3,name=double_col,relation=test,
+| |       | type=Double NULL]
+| |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| +-project_expressions=
+|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   +-Alias[id=6,name=,alias=sum(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+  +-AttributeReference[id=6,name=,alias=sum(float_col),relation=,
+    type=Double NULL]
+==
+
+SELECT sum(avg(int_col) OVER w) FROM test
+WINDOW w AS
+(PARTITION BY char_col
+ ORDER BY long_col
+ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW);
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=Aggregate
+| | +-input=WindowAggregate
+| | | +-input=Sort[is_ascending=[true,true],nulls_first=[false,false]]
+| | | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | | +-AttributeReference[id=3,name=double_col,relation=test,
+| | | | | | type=Double NULL]
+| | | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | | |   type=VarChar(20) NULL]
+| | | | +-sort_expressions=
+| | | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| | |   relation=$window_aggregate,type=Double NULL]
+| | |   +-WindowAggregateFunction[function=AVG,window_name=w,
+| | |     is_ascending=[true],nulls_first=[false],frame_mode=row,
+| | |     num_preceding=-1,num_following=0]
+| | |     +-arguments=
+| | |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |     +-partition_by=
+| | |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |     +-order_by=
+| | |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,type=Double NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+| |         relation=$window_aggregate,type=Double NULL]
+| +-project_list=
+|   +-Alias[id=7,name=,alias=sum(avg(int_col)),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=7,name=,alias=sum(avg(int_col)),relation=,
+    type=Double NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=Aggregate
+| | +-input=WindowAggregate
+| | | +-input=Sort[is_ascending=[true,true],nulls_first=[false,false]]
+| | | | +-input=TableReference[relation=Test,alias=test]
+| | | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | | +-AttributeReference[id=3,name=double_col,relation=test,
+| | | | | | type=Double NULL]
+| | | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | | |   type=VarChar(20) NULL]
+| | | | +-sort_attributes=
+| | | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| | |   relation=$window_aggregate,type=Double NULL]
+| | |   +-WindowAggregateFunction[function=AVG,window_name=w,
+| | |     is_ascending=[true],nulls_first=[false],frame_mode=row,
+| | |     num_preceding=-1,num_following=0]
+| | |     +-arguments=
+| | |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |     +-partition_by=
+| | |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |     +-order_by=
+| | |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,type=Double NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+| |         relation=$window_aggregate,type=Double NULL]
+| +-project_expressions=
+|   +-Alias[id=7,name=,alias=sum(avg(int_col)),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=7,name=,alias=sum(avg(int_col)),relation=,
+    type=Double NULL]
+==


[21/50] [abbrv] incubator-quickstep git commit: Add option to enable Google Profiler.

Posted by zu...@apache.org.
Add option to enable Google Profiler.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 540c09e623744c5ebf24f845372baaddca4265d1
Parents: 8230b12
Author: Navneet Potti <na...@apache.org>
Authored: Mon Jun 13 15:17:06 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jun 13 14:57:16 2016 -0700

----------------------------------------------------------------------
 CMakeLists.txt       | 26 ++++++++++++++++++++++++++
 cli/CMakeLists.txt   |  6 +++++-
 cli/CliConfig.h.in   |  1 +
 cli/QuickstepCli.cpp | 47 ++++++++++++++++++++++++++++++++++++++++++++++-
 4 files changed, 78 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/540c09e6/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 9e445f0..20e1fb9 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -159,6 +159,32 @@ else()
   )
 endif()
 
+# Include Google Perftools CPU Profiler. You probably want to
+# use this option with CMAKE_BUILD_TYPE=RelWithDebInfo.
+# Pass profiler_file_name to quickstep_cli_shell to actually
+# run the profiler. The profiler only starts collecting
+# samples after the first query, so that it runs against a
+# warm buffer pool and caches. If you want to profile everything,
+# including the first query run, set the environment variable
+# CPUPROFILE instead of passing the flag profile_file_name
+# Use google-pprof on the output file to convert it into a useful
+# format like graphviz (dot).
+option(ENABLE_GOOGLE_PROFILER "Include Google Perftools CPU Profiler." OFF)
+
+if (ENABLE_GOOGLE_PROFILER)
+  set_property(
+    DIRECTORY
+    APPEND PROPERTY COMPILE_DEFINITIONS QUICKSTEP_ENABLE_PROFILER
+  )
+
+  # TODO(navsan) Add a FindGperftools.cmake module and use that here.
+  check_include_files("gperftools/profiler.h" HAVE_GPERFTOOLS_PROFILER)
+  if (NOT HAVE_GPERFTOOLS_PROFILER)
+    message(FATAL_ERROR "Could not find gperftools. Ensure that it is installed.")
+  endif()
+  set(LIBS ${LIBS} profiler)
+endif()
+
 # Link against the system's threading library.
 find_package(Threads REQUIRED)
 set(LIBS ${LIBS} ${CMAKE_THREAD_LIBS_INIT})

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/540c09e6/cli/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/CMakeLists.txt b/cli/CMakeLists.txt
index faf5040..44ec223 100644
--- a/cli/CMakeLists.txt
+++ b/cli/CMakeLists.txt
@@ -36,6 +36,10 @@ else()
   set(GFLAGS_LIB_NAME gflags_nothreads-static)
 endif()
 
+if (ENABLE_GOOGLE_PROFILER)
+  set(QUICKSTEP_ENABLE_GOOGLE_PROFILER TRUE)
+endif()
+
 configure_file (
   "${CMAKE_CURRENT_SOURCE_DIR}/CliConfig.h.in"
   "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
@@ -106,7 +110,7 @@ target_link_libraries(quickstep_cli_DefaultsConfigurator
 if(QUICKSTEP_HAVE_LIBNUMA)
   target_link_libraries(quickstep_cli_DefaultsConfigurator
                       ${LIBNUMA_LIBRARY})
-endif() 
+endif()
 target_link_libraries(quickstep_cli_InputParserUtil
                       glog
                       quickstep_utility_Macros

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/540c09e6/cli/CliConfig.h.in
----------------------------------------------------------------------
diff --git a/cli/CliConfig.h.in b/cli/CliConfig.h.in
index b44dfb7..69f4d8a 100644
--- a/cli/CliConfig.h.in
+++ b/cli/CliConfig.h.in
@@ -17,3 +17,4 @@
 
 #cmakedefine QUICKSTEP_USE_LINENOISE
 #cmakedefine QUICKSTEP_OS_WINDOWS
+#cmakedefine QUICKSTEP_ENABLE_GOOGLE_PROFILER

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/540c09e6/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index fbe7e3b..0b64fda 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -36,7 +36,7 @@
 #include <stdlib.h>
 #endif
 
-#include "cli/CliConfig.h"  // For QUICKSTEP_USE_LINENOISE.
+#include "cli/CliConfig.h"  // For QUICKSTEP_USE_LINENOISE, QUICKSTEP_ENABLE_GOOGLE_PROFILER.
 #include "cli/CommandExecutor.hpp"
 #include "cli/DropRelation.hpp"
 
@@ -48,6 +48,10 @@ typedef quickstep::LineReaderLineNoise LineReaderImpl;
 typedef quickstep::LineReaderDumb LineReaderImpl;
 #endif
 
+#ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
+#include <gperftools/profiler.h>
+#endif
+
 #include "cli/DefaultsConfigurator.hpp"
 #include "cli/InputParserUtil.hpp"
 #include "cli/PrintToScreen.hpp"
@@ -157,6 +161,30 @@ DEFINE_bool(initialize_db, false, "If true, initialize a database.");
 DEFINE_bool(print_query, false,
             "Print each input query statement. This is useful when running a "
             "large number of queries in a batch.");
+DEFINE_string(profile_file_name, "",
+              "If nonempty, enable profiling using GOOGLE CPU Profiler, and write "
+              "its output to the given file name. This flag has no effect if "
+              "ENABLE_GOOGLE_PROFILER CMake flag was not set during build. "
+              "The profiler only starts collecting samples after the first query, "
+              "so that it runs against a warm buffer pool and caches. If you want to profile "
+              "everything, including the first query run, set the "
+              "environment variable CPUPROFILE instead of passing this flag.");
+              // Here's a detailed explanation of why we skip the first query run
+              // during profiling:
+              // Unless you\u2019ve preloaded the buffer pool (which is not always a good
+              // idea), the first run of the query results in disk I/O and other overhead
+              // that significantly skews the profiling results. It\u2019s the same reason we don\u2019t
+              // include the first run time in our benchmarking: when profiling query
+              // execution, it makes more sense to get numbers using a warm buffer pool and
+              // warm caches. This is not *always* the right thing to do: it\u2019s obviously
+              // wrong for profiling the TextScan operator. In those cases, you might want
+              // to put in your own Profiler probes (just follow the start/stop pattern used
+              // in this file) or just run quickstep with the CPUPROFILE environment variable
+              // set (as per gperftools documentation) to get the full profile for the
+              // entire execution.
+              // To put things in perspective, the first run is, in my experiments, about 5-10
+              // times more expensive than the average run. That means the query needs to be
+              // run at least a hundred times to make the impact of the first run small (< 5 %).
 
 }  // namespace quickstep
 
@@ -345,6 +373,9 @@ int main(int argc, char* argv[]) {
   std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
   std::chrono::time_point<std::chrono::steady_clock> start, end;
 
+#ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
+  bool started_profiling = false;
+#endif
   for (;;) {
     string *command_string = new string();
     *command_string = line_reader.getNextCommand();
@@ -446,6 +477,13 @@ int main(int argc, char* argv[]) {
         reset_parser = true;
         break;
       }
+#ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
+      // Profile only if profile_file_name flag is set
+      if (!started_profiling && !quickstep::FLAGS_profile_file_name.empty()) {
+        started_profiling = true;
+        ProfilerStart(quickstep::FLAGS_profile_file_name.c_str());
+      }
+#endif
     }
 
     if (quitting) {
@@ -456,6 +494,13 @@ int main(int argc, char* argv[]) {
     }
   }
 
+#ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
+  if (started_profiling) {
+    ProfilerStop();
+    ProfilerFlush();
+  }
+#endif
+
   // Kill the foreman and workers.
   QueryExecutionUtil::BroadcastPoisonMessage(main_thread_client_id, &bus);
 


[22/50] [abbrv] incubator-quickstep git commit: Add a shell script that logs the build process for later profiling.

Posted by zu...@apache.org.
Add a shell script that logs the build process for later profiling.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 07c011d27da2e8b14551b439a11e6002fd33c5fa
Parents: 540c09e
Author: Navneet Potti <na...@gmail.com>
Authored: Thu Jun 9 11:18:26 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jun 13 15:26:34 2016 -0700

----------------------------------------------------------------------
 build/profile_build.sh | 104 ++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 104 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/07c011d2/build/profile_build.sh
----------------------------------------------------------------------
diff --git a/build/profile_build.sh b/build/profile_build.sh
new file mode 100755
index 0000000..5f2fc33
--- /dev/null
+++ b/build/profile_build.sh
@@ -0,0 +1,104 @@
+#!/bin/bash
+# This script may be useful for developers to profile the build process itself.
+#
+# This shell script runs CMake and make, dumping all output into a log file.
+# It also logs the CPU usage and memory information during the build.
+# All log messages are timestamped to enable profiling.
+#
+# Dependencies:
+# - ts
+# sudo apt-get install -y libtime-duration-perl moreutils
+# - vmstat and mpstat
+# sudo apt-get install -y sysstat
+#
+# Usage: ./profile_build.sh
+# Set the CMake and make command you want to use below.
+# If CMakeLists.txt is detected, the script exits because in-source-tree build
+# is not supported.
+# If CMakeCache.txt is detected, the script skips cmake and runs make only.
+
+#   Licensed under the Apache License, Version 2.0 (the "License");
+#   you may not use this file except in compliance with the License.
+#   You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#   Unless required by applicable law or agreed to in writing, software
+#   distributed under the License is distributed on an "AS IS" BASIS,
+#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#   See the License for the specific language governing permissions and
+#   limitations under the License.
+
+set -e
+
+CMAKE_COMMAND="cmake \
+-D BUILD_SHARED_LIBS=On \
+-D USE_TCMALLOC=0 \
+-D CMAKE_BUILD_TYPE=Debug \
+.. "
+MAKE_COMMAND="make VERBOSE=1"
+LOG_FILENAME=$LOG_FILENAME
+
+# Continuously dump memory usage and cpu load info to files for later analysis
+function start_stat_collectors {
+  rm -f stats_*.txt
+  vmstat -SM 3 | ts "%.s (%H:%M:%S)" > stats_mem.txt 3>&1 &
+  PID_vmstat=$!
+  mpstat 3 | ts "%.s (%H:%M:%S)" > stats_cpu.txt 2>&1  &
+  PID_mpstat=$!
+}
+
+function kill_stat_collectors {
+  kill $PID_vmstat
+  kill $PID_mpstat
+  exit
+}
+
+function check_directory {
+  if [[ -f CMakeLists.txt ]]; then
+    echo "Running the build in the source tree is not supported."
+    exit 1
+  fi
+}
+
+function log_repo_version_info {
+  git log master... >> $LOG_FILENAME
+  git diff master >> $LOG_FILENAME
+}
+
+function run_cmake {
+  if [[ ! -f CMakeCache.txt ]]; then
+    echo "$CMAKE_COMMAND" | tee -a $LOG_FILENAME
+    $CMAKE_COMMAND 2>&1 | ts "%.s (%H:%M:%S)" | tee -a $LOG_FILENAME
+  else
+    echo "CMakeCache.txt detected. Not running CMake again."
+  fi
+}
+
+function run_make {
+  echo "$MAKE_COMMAND" | tee -a $LOG_FILENAME
+  $MAKE_COMMAND 2>&1 | ts "%.s (%H:%M:%S)" | tee -a $LOG_FILENAME
+}
+
+function print_stats {
+  avg_mem=`grep -v r stats_mem.txt | tr -s ' ' | awk -F " " '{s+= $6; c++} END {print s/c/1024}'`
+  echo -e "\n\n"
+  echo "Average memory used was $avg_mem GB"  | tee -a $LOG_FILENAME
+
+  time_taken=`expr $END_TIME - $START_TIME`
+  mins=`expr $time_taken / 60`
+  secs=`expr $time_taken % 60`
+  echo "Time taken was ${mins}m ${secs}s" | tee -a $LOG_FILENAME
+}
+
+check_directory
+echo "Starting build in " `pwd`  >> $LOG_FILENAME
+start_stat_collectors
+trap kill_stat_collectors SIGHUP SIGINT SIGTERM
+
+START_TIME=`date +"%s"`
+run_cmake
+run_make
+END_TIME=`date +"%s"`
+kill_stat_collectors
+print_stats


[29/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-20: Added parser support for SQL window aggregation function

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index 71e4332..fea31d6 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -85,88 +85,96 @@ extern int quickstep_yydebug;
     TOKEN_CONSTRAINT = 295,
     TOKEN_COPY = 296,
     TOKEN_CREATE = 297,
-    TOKEN_DATE = 298,
-    TOKEN_DATETIME = 299,
-    TOKEN_DAY = 300,
-    TOKEN_DECIMAL = 301,
-    TOKEN_DEFAULT = 302,
-    TOKEN_DELETE = 303,
-    TOKEN_DELIMITER = 304,
-    TOKEN_DESC = 305,
-    TOKEN_DISTINCT = 306,
-    TOKEN_DOUBLE = 307,
-    TOKEN_DROP = 308,
-    TOKEN_ELSE = 309,
-    TOKEN_END = 310,
-    TOKEN_ESCAPE_STRINGS = 311,
-    TOKEN_EXISTS = 312,
-    TOKEN_EXTRACT = 313,
-    TOKEN_FALSE = 314,
-    TOKEN_FIRST = 315,
-    TOKEN_FLOAT = 316,
-    TOKEN_FOR = 317,
-    TOKEN_FOREIGN = 318,
-    TOKEN_FROM = 319,
-    TOKEN_FULL = 320,
-    TOKEN_GROUP = 321,
-    TOKEN_HASH = 322,
-    TOKEN_HAVING = 323,
-    TOKEN_HOUR = 324,
-    TOKEN_IN = 325,
-    TOKEN_INDEX = 326,
-    TOKEN_INNER = 327,
-    TOKEN_INSERT = 328,
-    TOKEN_INTEGER = 329,
-    TOKEN_INTERVAL = 330,
-    TOKEN_INTO = 331,
-    TOKEN_JOIN = 332,
-    TOKEN_KEY = 333,
-    TOKEN_LAST = 334,
-    TOKEN_LEFT = 335,
-    TOKEN_LIMIT = 336,
-    TOKEN_LONG = 337,
-    TOKEN_MINUTE = 338,
-    TOKEN_MONTH = 339,
-    TOKEN_NULL = 340,
-    TOKEN_NULLS = 341,
-    TOKEN_OFF = 342,
-    TOKEN_ON = 343,
-    TOKEN_ORDER = 344,
-    TOKEN_OUTER = 345,
-    TOKEN_PARTITION = 346,
-    TOKEN_PARTITIONS = 347,
-    TOKEN_PERCENT = 348,
-    TOKEN_PRIMARY = 349,
-    TOKEN_QUIT = 350,
-    TOKEN_RANGE = 351,
-    TOKEN_REAL = 352,
-    TOKEN_REFERENCES = 353,
-    TOKEN_RIGHT = 354,
-    TOKEN_ROW_DELIMITER = 355,
-    TOKEN_SECOND = 356,
-    TOKEN_SELECT = 357,
-    TOKEN_SET = 358,
-    TOKEN_SMA = 359,
-    TOKEN_SMALLINT = 360,
-    TOKEN_SUBSTRING = 361,
-    TOKEN_TABLE = 362,
-    TOKEN_THEN = 363,
-    TOKEN_TIME = 364,
-    TOKEN_TIMESTAMP = 365,
-    TOKEN_TRUE = 366,
-    TOKEN_TUPLESAMPLE = 367,
-    TOKEN_UNIQUE = 368,
-    TOKEN_UPDATE = 369,
-    TOKEN_USING = 370,
-    TOKEN_VALUES = 371,
-    TOKEN_VARCHAR = 372,
-    TOKEN_WHEN = 373,
-    TOKEN_WHERE = 374,
-    TOKEN_WITH = 375,
-    TOKEN_YEAR = 376,
-    TOKEN_YEARMONTH = 377,
-    TOKEN_EOF = 378,
-    TOKEN_LEX_ERROR = 379
+    TOKEN_CURRENT = 298,
+    TOKEN_DATE = 299,
+    TOKEN_DATETIME = 300,
+    TOKEN_DAY = 301,
+    TOKEN_DECIMAL = 302,
+    TOKEN_DEFAULT = 303,
+    TOKEN_DELETE = 304,
+    TOKEN_DELIMITER = 305,
+    TOKEN_DESC = 306,
+    TOKEN_DISTINCT = 307,
+    TOKEN_DOUBLE = 308,
+    TOKEN_DROP = 309,
+    TOKEN_ELSE = 310,
+    TOKEN_END = 311,
+    TOKEN_ESCAPE_STRINGS = 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_QUIT = 354,
+    TOKEN_RANGE = 355,
+    TOKEN_REAL = 356,
+    TOKEN_REFERENCES = 357,
+    TOKEN_RIGHT = 358,
+    TOKEN_ROW = 359,
+    TOKEN_ROW_DELIMITER = 360,
+    TOKEN_ROWS = 361,
+    TOKEN_SECOND = 362,
+    TOKEN_SELECT = 363,
+    TOKEN_SET = 364,
+    TOKEN_SMA = 365,
+    TOKEN_SMALLINT = 366,
+    TOKEN_SUBSTRING = 367,
+    TOKEN_TABLE = 368,
+    TOKEN_THEN = 369,
+    TOKEN_TIME = 370,
+    TOKEN_TIMESTAMP = 371,
+    TOKEN_TRUE = 372,
+    TOKEN_TUPLESAMPLE = 373,
+    TOKEN_UNBOUNDED = 374,
+    TOKEN_UNIQUE = 375,
+    TOKEN_UPDATE = 376,
+    TOKEN_USING = 377,
+    TOKEN_VALUES = 378,
+    TOKEN_VARCHAR = 379,
+    TOKEN_WHEN = 380,
+    TOKEN_WHERE = 381,
+    TOKEN_WINDOW = 382,
+    TOKEN_WITH = 383,
+    TOKEN_YEAR = 384,
+    TOKEN_YEARMONTH = 385,
+    TOKEN_EOF = 386,
+    TOKEN_LEX_ERROR = 387
   };
 #endif
 
@@ -175,7 +183,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 118 "../SqlParser.ypp" /* yacc.c:1915  */
+#line 119 "../SqlParser.ypp" /* yacc.c:1909  */
 
   quickstep::ParseString *string_value_;
 
@@ -261,13 +269,19 @@ union YYSTYPE
 
   quickstep::ParseSample *opt_sample_clause_;
 
+  quickstep::PtrList<quickstep::ParseWindow> *opt_window_clause_;
+  quickstep::ParseWindow *window_definition_;
+  quickstep::PtrList<quickstep::ParseExpression> *window_partition_by_list_;
+  quickstep::PtrList<quickstep::ParseOrderByItem> *window_order_by_list_;
+  quickstep::ParseFrameInfo *window_frame_info_;
+
   quickstep::PtrList<quickstep::ParseOrderByItem> *order_commalist_;
   quickstep::ParseOrderByItem *order_item_;
 
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
 
-#line 271 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 285 "SqlParser_gen.hpp" /* yacc.c:1909  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/tests/Select.test
----------------------------------------------------------------------
diff --git a/parser/tests/Select.test b/parser/tests/Select.test
index e70ee5c..79da7ee 100644
--- a/parser/tests/Select.test
+++ b/parser/tests/Select.test
@@ -1,5 +1,7 @@
 #   Copyright 2011-2015 Quickstep Technologies LLC.
 #   Copyright 2015 Pivotal Software, Inc.
+#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+#     University of Wisconsin\u2014Madison.
 #
 #   Licensed under the Apache License, Version 2.0 (the "License");
 #   you may not use this file except in compliance with the License.
@@ -1709,3 +1711,121 @@ SelectStatement
   |         +-TableReference[table=bar]
   +-from_clause=
     +-TableReference[table=test]
+==
+
+SELECT avg(attr1) OVER w FROM test
+WINDOW w AS
+(PARTITION BY attr2, attr3
+ ORDER BY attr4
+ ROWS BETWEEN 3 PRECEDING AND 3 FOLLOWING)
+--
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  |   +-FunctionCall[name=avg,window_name=w]
+  |     +-AttributeReference[attribute_name=attr1]
+  +-from_clause=
+  | +-TableReference[table=test]
+  +-window_list=
+    +-window[window_name=w]
+      +-frame_info=FrameInfo[frame_mode=row,num_preceding=3,num_following=3]
+      +-partition_by=
+      | +-AttributeReference[attribute_name=attr2]
+      | +-AttributeReference[attribute_name=attr3]
+      +-order_by=
+        +-OrderByItem[is_asc=true,nulls_first=false]
+          +-AttributeReference[attribute_name=attr4]
+==
+
+SELECT avg(attr1) OVER w FROM test 
+WINDOW w AS
+(PARTITION BY attr2, attr3
+ ORDER BY attr4 DESC NULLS FIRST, attr5 ASC NULLS LAST
+ RANGE BETWEEN 3 PRECEDING AND CURRENT ROW)
+--
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  |   +-FunctionCall[name=avg,window_name=w]
+  |     +-AttributeReference[attribute_name=attr1]
+  +-from_clause=
+  | +-TableReference[table=test]
+  +-window_list=
+    +-window[window_name=w]
+      +-frame_info=FrameInfo[frame_mode=range,num_preceding=3,num_following=0]
+      +-partition_by=
+      | +-AttributeReference[attribute_name=attr2]
+      | +-AttributeReference[attribute_name=attr3]
+      +-order_by=
+        +-OrderByItem[is_asc=false,nulls_first=true]
+        | +-AttributeReference[attribute_name=attr4]
+        +-OrderByItem[is_asc=true,nulls_first=false]
+          +-AttributeReference[attribute_name=attr5]
+==
+
+SELECT avg(attr1) OVER w1 AS avg1, sum(attr5) OVER w2 AS sum5 FROM test 
+WINDOW w1 AS
+(PARTITION BY attr2, attr3
+ ORDER BY attr4
+ ROWS BETWEEN 3 PRECEDING AND 3 FOLLOWING)
+WINDOW w2 AS
+(PARTITION BY attr1
+ ORDER BY attr6
+ RANGE BETWEEN 2 PRECEDING AND CURRENT ROW)
+--
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem[alias=avg1]
+  | | +-FunctionCall[name=avg,window_name=w1]
+  | |   +-AttributeReference[attribute_name=attr1]
+  | +-SelectListItem[alias=sum5]
+  |   +-FunctionCall[name=sum,window_name=w2]
+  |     +-AttributeReference[attribute_name=attr5]
+  +-from_clause=
+  | +-TableReference[table=test]
+  +-window_list=
+    +-window[window_name=w1]
+    | +-frame_info=FrameInfo[frame_mode=row,num_preceding=3,num_following=3]
+    | +-partition_by=
+    | | +-AttributeReference[attribute_name=attr2]
+    | | +-AttributeReference[attribute_name=attr3]
+    | +-order_by=
+    |   +-OrderByItem[is_asc=true,nulls_first=false]
+    |     +-AttributeReference[attribute_name=attr4]
+    +-window[window_name=w2]
+      +-frame_info=FrameInfo[frame_mode=range,num_preceding=2,num_following=0]
+      +-partition_by=
+      | +-AttributeReference[attribute_name=attr1]
+      +-order_by=
+        +-OrderByItem[is_asc=true,nulls_first=false]
+          +-AttributeReference[attribute_name=attr6]
+==
+
+SELECT avg(attr1) OVER 
+(PARTITION BY attr2, attr3
+ ORDER BY attr4 DESC NULLS FIRST, attr5 ASC NULLS LAST
+ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)
+FROM test
+--
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  |   +-FunctionCall[name=avg]
+  |     +-AttributeReference[attribute_name=attr1]
+  |     +-window=window
+  |       +-frame_info=FrameInfo[frame_mode=range,num_preceding=-1,
+  |       | num_following=0]
+  |       +-partition_by=
+  |       | +-AttributeReference[attribute_name=attr2]
+  |       | +-AttributeReference[attribute_name=attr3]
+  |       +-order_by=
+  |         +-OrderByItem[is_asc=false,nulls_first=true]
+  |         | +-AttributeReference[attribute_name=attr4]
+  |         +-OrderByItem[is_asc=true,nulls_first=false]
+  |           +-AttributeReference[attribute_name=attr5]
+  +-from_clause=
+    +-TableReference[table=test]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index 5959879..dc7eac0 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -62,6 +62,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_parser_ParseSubqueryExpression
                       quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseTableReference
+                      quickstep_parser_ParseWindow
                       quickstep_queryoptimizer_OptimizerContext
                       quickstep_queryoptimizer_Validator
                       quickstep_queryoptimizer_expressions_AggregateFunction

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 78985a0..ffc173a 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -58,6 +58,7 @@
 #include "parser/ParseSubqueryExpression.hpp"
 #include "parser/ParseSubqueryTableReference.hpp"
 #include "parser/ParseTableReference.hpp"
+#include "parser/ParseWindow.hpp"
 #include "query_optimizer/OptimizerContext.hpp"
 #include "query_optimizer/Validator.hpp"
 #include "query_optimizer/expressions/AggregateFunction.hpp"
@@ -2366,6 +2367,12 @@ E::ScalarPtr Resolver::resolveFunctionCall(
     ExpressionResolutionInfo *expression_resolution_info) {
   std::string function_name = ToLower(parse_function_call.name()->value());
 
+  // TODO(Shixuan): Add support for window aggregation function.
+  if (parse_function_call.isWindow()) {
+    THROW_SQL_ERROR_AT(&parse_function_call)
+        << "Window Aggregation Function is not supported currently";
+  }
+
   // First check for the special case COUNT(*).
   bool count_star = false;
   if (parse_function_call.star() != nullptr) {


[42/50] [abbrv] incubator-quickstep git commit: Added the move semantic in optimizer::logical::Sort

Posted by zu...@apache.org.
Added the move semantic in optimizer::logical::Sort


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

Branch: refs/heads/tmp-relation-col-store
Commit: f66f4bd19cf1a34c3ee2e6acb91f75b92c037927
Parents: c0c13f6
Author: shixuan-fan <sh...@apache.org>
Authored: Mon Jun 27 15:50:56 2016 +0000
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jun 27 14:14:48 2016 -0700

----------------------------------------------------------------------
 query_optimizer/logical/Sort.hpp      | 26 ++++++++++++++++++++++++++
 query_optimizer/resolver/Resolver.cpp | 18 +++++++++---------
 2 files changed, 35 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f66f4bd1/query_optimizer/logical/Sort.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Sort.hpp b/query_optimizer/logical/Sort.hpp
index c20ce0f..2233151 100644
--- a/query_optimizer/logical/Sort.hpp
+++ b/query_optimizer/logical/Sort.hpp
@@ -113,6 +113,19 @@ class Sort : public Logical {
                             limit));
   }
 
+  static SortPtr Create(
+      const LogicalPtr &input,
+      std::vector<expressions::AttributeReferencePtr> &&sort_attributes,
+      std::vector<bool> &&sort_ascending,
+      std::vector<bool> &&nulls_first_flags,
+      const int limit) {
+    return SortPtr(new Sort(input,
+                            std::move(sort_attributes),
+                            std::move(sort_ascending),
+                            std::move(nulls_first_flags),
+                            limit));
+  }
+
  protected:
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
@@ -137,6 +150,19 @@ class Sort : public Logical {
     addChild(input_);
   }
 
+  Sort(const LogicalPtr &input,
+       std::vector<expressions::AttributeReferencePtr> &&sort_attributes,
+       std::vector<bool> &&sort_ascending,
+       std::vector<bool> &&nulls_first_flags,
+       const int limit)
+      : input_(input),
+        sort_attributes_(std::move(sort_attributes)),
+        sort_ascending_(std::move(sort_ascending)),
+        nulls_first_flags_(std::move(nulls_first_flags)),
+        limit_(limit) {
+    addChild(input_);
+  }
+
   LogicalPtr input_;
   std::vector<expressions::AttributeReferencePtr> sort_attributes_;
   // Has 1:1 matching with <sort_expressions_>.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f66f4bd1/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index f880ce7..1eb6d86 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -1270,16 +1270,16 @@ L::LogicalPtr Resolver::resolveSelect(
     if (select_query.limit() != nullptr) {
       logical_plan =
           L::Sort::Create(logical_plan,
-                          order_by_attributes,
-                          order_by_directions,
-                          nulls_first,
+                          std::move(order_by_attributes),
+                          std::move(order_by_directions),
+                          std::move(nulls_first),
                           select_query.limit()->limit_expression()->long_value());
     } else {
       logical_plan =
           L::Sort::Create(logical_plan,
-                          order_by_attributes,
-                          order_by_directions,
-                          nulls_first,
+                          std::move(order_by_attributes),
+                          std::move(order_by_directions),
+                          std::move(nulls_first),
                           -1 /* limit */);
     }
   } else if (select_query.limit() != nullptr) {
@@ -1880,9 +1880,9 @@ L::LogicalPtr Resolver::resolveSortInWindow(
 
   L::LogicalPtr sorted_logical_plan =
       L::Sort::Create(logical_plan,
-                      sort_attributes,
-                      sort_directions,
-                      sort_nulls_first,
+                      std::move(sort_attributes),
+                      std::move(sort_directions),
+                      std::move(sort_nulls_first),
                       -1 /* limit */);
 
   return sorted_logical_plan;


[38/50] [abbrv] incubator-quickstep git commit: Added PRIORITY clause in parser.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index d02db56..b6c747c 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -8,8 +8,8 @@
 
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
-#define YY_FLEX_MINOR_VERSION 5
-#define YY_FLEX_SUBMINOR_VERSION 39
+#define YY_FLEX_MINOR_VERSION 6
+#define YY_FLEX_SUBMINOR_VERSION 0
 #if YY_FLEX_SUBMINOR_VERSION > 0
 #define FLEX_BETA
 #endif
@@ -354,7 +354,7 @@ void quickstep_yyfree (void * ,yyscan_t yyscanner );
 
 /* Begin user sect3 */
 
-#define quickstep_yywrap(yyscanner) 1
+#define quickstep_yywrap(yyscanner) (/*CONSTCOND*/1)
 #define YY_SKIP_YYWRAP
 
 typedef unsigned char YY_CHAR;
@@ -366,6 +366,9 @@ typedef int yy_state_type;
 static yy_state_type yy_get_previous_state (yyscan_t yyscanner );
 static yy_state_type yy_try_NUL_trans (yy_state_type current_state  ,yyscan_t yyscanner);
 static int yy_get_next_buffer (yyscan_t yyscanner );
+#if defined(__GNUC__) && __GNUC__ >= 3
+__attribute__((__noreturn__))
+#endif
 static void yy_fatal_error (yyconst char msg[] ,yyscan_t yyscanner );
 
 /* Done after the current pattern has been matched and before the
@@ -378,8 +381,8 @@ static void yy_fatal_error (yyconst char msg[] ,yyscan_t yyscanner );
 	*yy_cp = '\0'; \
 	yyg->yy_c_buf_p = yy_cp;
 
-#define YY_NUM_RULES 160
-#define YY_END_OF_BUFFER 161
+#define YY_NUM_RULES 161
+#define YY_END_OF_BUFFER 162
 /* This struct is not used in this scanner,
    but its presence is necessary. */
 struct yy_trans_info
@@ -387,75 +390,77 @@ struct yy_trans_info
 	flex_int32_t yy_verify;
 	flex_int32_t yy_nxt;
 	};
-static yyconst flex_int16_t yy_accept[588] =
+static yyconst flex_int16_t yy_accept[593] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
-        0,    0,  161,    2,    2,  159,  159,  158,  157,  159,
-      136,  132,  135,  132,  132,  155,  128,  125,  129,  154,
-      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
-      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
-      154,  154,  154,  154,  133,    4,    5,    5,    3,  151,
-      151,  148,  152,  152,  146,  153,  153,  150,    1,  158,
-      126,  156,  155,  155,  155,    0,  130,  127,  131,  154,
-      154,  154,  154,   10,  154,  154,  154,   22,  154,  154,
-      154,  154,  154,  154,  154,  154,  154,  154,  154,  134,
-
-      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
-      154,  154,  154,   60,   68,  154,  154,  154,  154,  154,
-      154,  154,  154,  154,  154,  154,   82,   83,  154,  154,
-      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
-      154,  154,  154,  154,  154,  154,  154,  154,  154,  154,
-      154,  154,  154,    4,    5,    3,  151,  147,  152,  145,
-      145,  137,  139,  140,  141,  142,  143,  144,  145,  153,
-      149,  156,  155,    0,  155,    6,    7,  154,    9,   11,
-      154,  154,   15,  154,  154,  154,  154,  154,  154,  154,
-      154,  154,  154,  154,   33,  154,  154,  154,  154,  154,
-
-      154,  154,  154,   44,  154,  154,  154,  154,  154,  154,
-      154,   52,  154,  154,  154,  154,  154,  154,  154,  154,
-      154,   64,  154,   70,  154,  154,  154,  154,  154,  154,
-      154,   78,  154,   81,  154,  154,  154,  154,  154,  154,
-      154,  154,  154,  154,  154,  154,  154,   98,  154,  154,
-      103,  104,  154,  154,  154,  154,  154,  154,  154,  154,
-      154,  154,  154,  154,  154,  154,  154,  154,  137,  139,
-      138,  154,  154,  154,  154,  154,  154,  154,   20,   23,
-      154,  154,  154,   28,  154,  154,  154,   31,  154,  154,
-      154,  154,   38,  154,  154,   42,   43,  154,  154,  154,
-
-      154,  154,  154,  154,  154,   54,   55,  154,   57,  154,
-       59,  154,  154,  154,  154,   67,   69,   71,   72,   73,
-      154,   75,  154,  154,   79,  154,  154,   86,  154,  154,
-      154,  154,   92,  154,   94,  154,  154,  154,  100,  154,
-      154,  154,  154,  154,  154,  108,  109,  111,  154,  154,
-      154,  154,  154,  154,  154,  119,  154,  154,  122,  123,
-      137,  138,    8,  154,  154,  154,  154,  154,  154,  154,
-       25,  154,  154,  154,  154,  154,  154,  154,  154,  154,
-      154,  154,  154,  154,  154,  154,  154,   48,   49,   50,
-      154,  154,   56,  154,   61,   62,  154,  154,  154,   74,
-
-      154,   77,   80,   84,   85,  154,  154,  154,  154,   93,
-      154,  154,   97,  154,  154,  154,  154,  154,  107,  154,
-      154,  154,  154,  154,  116,  154,  154,  120,  154,  154,
-      154,  154,   14,  154,  154,  154,  154,  154,   26,  154,
-       29,  154,  154,  154,  154,  154,   36,  154,  154,  154,
-       41,  154,   46,  154,  154,  154,   58,   63,  154,  154,
-       76,  154,  154,  154,  154,  154,   96,  154,  101,  102,
-      154,  154,  154,  154,  154,  114,  115,  117,  154,  121,
-      154,  154,   13,  154,  154,  154,  154,  154,  154,   21,
-       30,  154,   34,   35,  154,  154,  154,  154,   47,  154,
-
-       53,   65,  154,  154,   89,  154,   91,  154,  154,  154,
-      154,  154,  154,  154,  118,  154,  154,  154,  154,  154,
-      154,  154,  154,   32,  154,  154,   40,  154,  154,   66,
-      154,  154,  154,  154,  105,  154,  154,  154,  154,  154,
-       12,  154,  154,  154,  154,   24,  154,   37,  154,  154,
-       51,   87,   90,  154,  154,  106,  110,  154,  113,  124,
-       16,  154,  154,  154,   27,   39,  154,   88,   95,  154,
-      154,  154,   18,   19,  154,  154,  112,  154,  154,  154,
-      154,  154,   99,  154,   45,   17,    0
+        0,    0,  162,    2,    2,  160,  160,  159,  158,  160,
+      137,  133,  136,  133,  133,  156,  129,  126,  130,  155,
+      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
+      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
+      155,  155,  155,  155,  134,    4,    5,    5,    3,  152,
+      152,  149,  153,  153,  147,  154,  154,  151,    1,  159,
+      127,  157,  156,  156,  156,    0,  131,  128,  132,  155,
+      155,  155,  155,   10,  155,  155,  155,   22,  155,  155,
+      155,  155,  155,  155,  155,  155,  155,  155,  155,  135,
+
+      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
+      155,  155,  155,   60,   68,  155,  155,  155,  155,  155,
+      155,  155,  155,  155,  155,  155,   82,   83,  155,  155,
+      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
+      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
+      155,  155,  155,    4,    5,    3,  152,  148,  153,  146,
+      146,  138,  140,  141,  142,  143,  144,  145,  146,  154,
+      150,  157,  156,    0,  156,    6,    7,  155,    9,   11,
+      155,  155,   15,  155,  155,  155,  155,  155,  155,  155,
+      155,  155,  155,  155,   33,  155,  155,  155,  155,  155,
+
+      155,  155,  155,   44,  155,  155,  155,  155,  155,  155,
+      155,   52,  155,  155,  155,  155,  155,  155,  155,  155,
+      155,   64,  155,   70,  155,  155,  155,  155,  155,  155,
+      155,   78,  155,   81,  155,  155,  155,  155,  155,  155,
+      155,  155,  155,  155,  155,  155,  155,   99,  155,  155,
+      104,  105,  155,  155,  155,  155,  155,  155,  155,  155,
+      155,  155,  155,  155,  155,  155,  155,  155,  138,  140,
+      139,  155,  155,  155,  155,  155,  155,  155,   20,   23,
+      155,  155,  155,   28,  155,  155,  155,   31,  155,  155,
+      155,  155,   38,  155,  155,   42,   43,  155,  155,  155,
+
+      155,  155,  155,  155,  155,   54,   55,  155,   57,  155,
+       59,  155,  155,  155,  155,   67,   69,   71,   72,   73,
+      155,   75,  155,  155,   79,  155,  155,   86,  155,  155,
+      155,  155,  155,   93,  155,   95,  155,  155,  155,  101,
+      155,  155,  155,  155,  155,  155,  109,  110,  112,  155,
+      155,  155,  155,  155,  155,  155,  120,  155,  155,  123,
+      124,  138,  139,    8,  155,  155,  155,  155,  155,  155,
+      155,   25,  155,  155,  155,  155,  155,  155,  155,  155,
+      155,  155,  155,  155,  155,  155,  155,  155,   48,   49,
+       50,  155,  155,   56,  155,   61,   62,  155,  155,  155,
+
+       74,  155,   77,   80,   84,   85,  155,  155,  155,  155,
+      155,   94,  155,  155,   98,  155,  155,  155,  155,  155,
+      108,  155,  155,  155,  155,  155,  117,  155,  155,  121,
+      155,  155,  155,  155,   14,  155,  155,  155,  155,  155,
+       26,  155,   29,  155,  155,  155,  155,  155,   36,  155,
+      155,  155,   41,  155,   46,  155,  155,  155,   58,   63,
+      155,  155,   76,  155,  155,  155,  155,  155,  155,   97,
+      155,  102,  103,  155,  155,  155,  155,  155,  115,  116,
+      118,  155,  122,  155,  155,   13,  155,  155,  155,  155,
+      155,  155,   21,   30,  155,   34,   35,  155,  155,  155,
+
+      155,   47,  155,   53,   65,  155,  155,   89,  155,   91,
+      155,  155,  155,  155,  155,  155,  155,  155,  119,  155,
+      155,  155,  155,  155,  155,  155,  155,   32,  155,  155,
+       40,  155,  155,   66,  155,  155,   92,  155,  155,  106,
+      155,  155,  155,  155,  155,   12,  155,  155,  155,  155,
+       24,  155,   37,  155,  155,   51,   87,   90,  155,  155,
+      107,  111,  155,  114,  125,   16,  155,  155,  155,   27,
+       39,  155,   88,   96,  155,  155,  155,   18,   19,  155,
+      155,  113,  155,  155,  155,  155,  155,  100,  155,   45,
+       17,    0
+
     } ;
 
-static yyconst flex_int32_t yy_ec[256] =
+static yyconst YY_CHAR yy_ec[256] =
     {   0,
         1,    1,    1,    1,    1,    1,    1,    1,    2,    3,
         1,    1,    4,    1,    1,    1,    1,    1,    1,    1,
@@ -487,7 +492,7 @@ static yyconst flex_int32_t yy_ec[256] =
         1,    1,    1,    1,    1
     } ;
 
-static yyconst flex_int32_t yy_meta[72] =
+static yyconst YY_CHAR yy_meta[72] =
     {   0,
         1,    1,    2,    1,    1,    3,    1,    4,    1,    5,
         5,    6,    6,    5,    1,    1,    1,    7,    7,    7,
@@ -499,27 +504,27 @@ static yyconst flex_int32_t yy_meta[72] =
         8
     } ;
 
-static yyconst flex_int16_t yy_base[603] =
+static yyconst flex_uint16_t yy_base[608] =
     {   0,
         0,    1,   46,    0,  117,  163,    2,    3,  128,  132,
-        6,   10,  211, 1307, 1307,    0, 1307,   13, 1307,  194,
-     1307, 1307, 1307,  194,    6,  130,    4, 1307,  170,  124,
+        6,   10,  211, 1312, 1312,    0, 1312,   13, 1312,  194,
+     1312, 1312, 1312,  194,    6,  130,    4, 1312,  170,  124,
       161,  215,  170,  207,  265,   92,  167,  162,   96,  107,
       223,  165,  162,  221,  274,   92,  284,  277,  315,  206,
-      128,  232,    0,  125, 1307,  152,    4,   19,    0,    0,
+      128,  232,    0,  125, 1312,  152,    4,   19,    0,    0,
         0,  143,    0,    0,  379,    0,    0,  144,    0,   22,
-     1307,    0,  292,  305,  335,   18, 1307, 1307, 1307,    0,
+     1312,    0,  292,  305,  335,   18, 1312, 1312, 1312,    0,
       174,  264,  180,  186,  209,  271,  229,    0,  277,  333,
-      335,  312,  330,  317,  323,  338,  323,  333,  340, 1307,
+      335,  312,  330,  317,  323,  338,  323,  333,  340, 1312,
 
       339,  355,  357,  381,  370,  373,  377,  382,  380,  384,
       383,  383,  383,  431,    0,  398,  383,  390,  406,  402,
       403,  404,  425,  420,  431,  442,    0,  445,  432,  448,
       436,  437,  451,  448,  444,  460,  452,  439,  489,  464,
       468,  469,  468,  462,  455,  477,  501,  493,  489,  494,
-      502,  494,  510,  142,   29,    0,    0, 1307,    0, 1307,
-     1307,   22,   24, 1307, 1307, 1307, 1307, 1307,    0,    0,
-     1307,    0,  520,   26,   28,    0,    0,  508,    0,  512,
+      502,  494,  510,  142,   29,    0,    0, 1312,    0, 1312,
+     1312,   22,   24, 1312, 1312, 1312, 1312, 1312,    0,    0,
+     1312,    0,  520,   26,   28,    0,    0,  508,    0,  512,
       495,  510,  497,  530,  518,  506,  525,  508,  512,  509,
       543,  530,  533,  550,    0,  547,  558,  556,  561,  546,
 
@@ -527,127 +532,127 @@ static yyconst flex_int16_t yy_base[603] =
       566,  576,  569,  571,  565,  579,  580,  573,  587,  597,
       598,  604,  594,    0,  593,  594,  612,  613,  616,  603,
       605,    0,  614,    0,  622,  623,  611,  610,  630,  631,
-      622,  616,  632,  628,  636,  639,  637,  631,  632,  644,
-        0,  649,  647,  655,  657,  667,  670,  665,  665,  664,
-      681,  669,  663,  682,  673,  682,  680,  671,   30,  125,
-        0,  674,  679,  689,  683,  693,  688,  689,    0,  702,
-      694,  694,  688,    0,  699,  706,  720,  709,  717,  712,
-      715,  725,  734,  731,  729,    0,    0,  726,  723,  743,
-
-      740,  726,  727,  734,  741,    0,    0,  735,    0,  738,
-        0,  731,  738,  739,  759,    0,    0,    0,    0,    0,
-      740,    0,  741,  755,  746,  764,  765,    0,  778,  783,
-      786,  792,    0,  791,    0,  779,  774,  779,    0,  796,
-      787,  799,  791,  784,  800,    0,  788,    0,  803,  788,
-      789,  793,  807,  810,  810,    0,  814,  806,    0,  810,
-      136, 1307,    0,  820,  830,  820,  840,  830,  839,  848,
-        0,  840,  837,  851,  852,  844,  850,  859,  849,  857,
-      854,  851,  852,  862,  863,  852,  869,    0,    0,    0,
-      850,  869,    0,  870,    0,    0,  859,  876,  860,    0,
-
-      888,    0,    0,    0,    0,  878,  885,  899,  886,    0,
-      902,  893,    0,  907,  909,  894,  906,  898,    0,  897,
-      899,  905,  915,  916,    0,  904,  923,    0,  902,  911,
-      920,  916,    0,  909,  916,  934,  928,  919,    0,  939,
-        0,  946,  936,  944,  949,  942,    0,  945,  963,  967,
-        0,   93,    0,  951,  963,  959,    0,    0,  956,  974,
-        0,  967,  957,  969,  954,  967,    0,  970,    0,    0,
-      969,  975,  986,  987,  985,    0,    0,    0,  974,    0,
-      979,  981,    0,  988,  983,  996, 1002, 1010, 1010,    0,
-        0, 1015,    0,    0, 1018, 1016, 1008, 1010,    0, 1016,
-
-        0,    0, 1019, 1017,    0, 1019,    0, 1031, 1026, 1016,
-     1023, 1026, 1027, 1036,    0, 1022, 1038, 1032, 1031, 1034,
-     1031, 1035, 1041,    0, 1038, 1053,    0, 1052, 1066,    0,
-     1063, 1071, 1076, 1070,    0, 1079, 1071, 1072, 1085, 1082,
-        0, 1084, 1087, 1081, 1089,    0, 1075,    0, 1090, 1080,
-        0, 1080,    0, 1081, 1094,    0,    0, 1092,    0,    0,
-        0, 1087, 1103, 1091,    0,    0, 1102,    0,    0, 1093,
-     1109, 1105,    0,    0, 1116, 1126,    0, 1126, 1129, 1121,
-     1136, 1125,    0, 1126,    0,    0, 1307, 1191, 1201, 1211,
-     1221, 1231, 1235, 1238, 1244, 1252, 1262, 1272, 1282, 1292,
-
-     1297, 1299
+      626,  615,  629,  628,  639,  640,  639,  631,  634,  656,
+        0,  654,  648,  659,  658,  670,  672,  668,  666,  665,
+      682,  670,  664,  683,  673,  684,  681,  674,   30,  125,
+        0,  675,  680,  692,  684,  694,  691,  690,    0,  704,
+      696,  695,  699,    0,  704,  707,  724,  710,  720,  714,
+      718,  726,  735,  732,  730,    0,    0,  727,  724,  744,
+
+      741,  727,  729,  735,  742,    0,    0,  736,    0,  741,
+        0,  732,  739,  742,  759,    0,    0,    0,    0,    0,
+      741,    0,  743,  757,  763,  765,  769,    0,  779,  786,
+      788,  795,  779,    0,  793,    0,  781,  776,  781,    0,
+      798,  789,  801,  793,  787,  803,    0,  790,    0,  805,
+      792,  793,  795,  811,  814,  813,    0,  818,  809,    0,
+      822,  136, 1312,    0,  836,  836,  825,  845,  835,  843,
+      852,    0,  842,  839,  853,  854,  846,  852,  861,  851,
+      860,  857,  853,  854,  866,  867,  854,  873,    0,    0,
+        0,  854,  872,    0,  874,    0,    0,  862,  888,  876,
+
+        0,  894,    0,    0,    0,    0,  883,  890,  903,  891,
+      903,    0,  908,  898,    0,  910,  912,  897,  909,  901,
+        0,  900,  902,  909,  919,  920,    0,  907,  928,    0,
+      907,  916,  925,  921,    0,  915,  921,  939,  942,  936,
+        0,  956,    0,  956,  942,  952,  955,  950,    0,  951,
+      968,  970,    0,   93,    0,  954,  966,  962,    0,    0,
+      959,  977,    0,  970,  961,  973,  958,  964,  973,    0,
+      976,    0,    0,  975,  983,  992,  994,  993,    0,    0,
+        0,  980,    0,  995, 1000,    0, 1006, 1004, 1007, 1011,
+     1020, 1019,    0,    0, 1024,    0,    0, 1025, 1022, 1012,
+
+     1014,    0, 1020,    0,    0, 1023, 1021,    0, 1023,    0,
+     1014, 1037, 1032, 1022, 1031, 1033, 1034, 1045,    0, 1031,
+     1046, 1041, 1040, 1051, 1052, 1055, 1063,    0, 1060, 1067,
+        0, 1063, 1079,    0, 1073, 1081,    0, 1084, 1077,    0,
+     1084, 1076, 1077, 1090, 1087,    0, 1090, 1093, 1087, 1095,
+        0, 1083,    0, 1097, 1087,    0, 1089,    0, 1090, 1102,
+        0,    0, 1101,    0,    0,    0, 1096, 1120, 1112,    0,
+        0, 1122,    0,    0, 1115, 1131, 1119,    0,    0, 1127,
+     1139,    0, 1136, 1139, 1129, 1143, 1130,    0, 1131,    0,
+        0, 1312, 1196, 1206, 1216, 1226, 1236, 1240, 1243, 1249,
+
+     1257, 1267, 1277, 1287, 1297, 1302, 1304
     } ;
 
-static yyconst flex_int16_t yy_def[603] =
+static yyconst flex_int16_t yy_def[608] =
     {   0,
-      588,  588,  587,    3,  589,  589,  590,  590,  591,  591,
-      592,  592,  587,  587,  587,  593,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  587,  587,  587,  587,  595,  596,
-      596,  587,  597,  597,  598,  599,  599,  587,  593,  587,
-      587,  600,  587,  587,  587,  587,  587,  587,  587,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  587,
-
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  587,  587,  595,  596,  587,  597,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  601,  599,
-      587,  600,  587,  587,  587,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  587,  587,
-      602,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      587,  587,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,    0,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-
-      587,  587
+      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,  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,  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,  599,  599,
+      599,    0,  592,  592,  592,  592,  592,  592,  592,  592,
+
+      592,  592,  592,  592,  592,  592,  592
     } ;
 
-static yyconst flex_int16_t yy_nxt[1379] =
+static yyconst flex_uint16_t yy_nxt[1384] =
     {   0,
-      587,  587,   15,   15,   61,   61,  155,  155,   67,   62,
-       62,   68,   67,  587,   70,   68,   70,   73,   73,   77,
-       78,  155,  155,   70,  587,   70,  174,  174,  587,  175,
+      592,  592,   15,   15,   61,   61,  155,  155,   67,   62,
+       62,   68,   67,  592,   70,   68,   70,   73,   73,   77,
+       78,  155,  155,   70,  592,   70,  174,  174,  592,  175,
       175,  155,  155,  269,  270,  270,  270,  175,  175,  175,
-      175,  361,  270,  587,   16,   16,   17,   18,   19,   18,
+      175,  362,  270,  592,   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,
@@ -657,7 +662,7 @@ static yyconst flex_int16_t yy_nxt[1379] =
        38,   39,   40,   41,   42,   43,   44,   45,   46,   47,
        48,   49,   50,   51,   52,   53,   54,   17,   56,   57,
        58,   17,   17,   17,   17,   17,  111,  116,  117,  134,
-       64,   17,   17,   17,   64,   62,  270,  270,  498,   62,
+       64,   17,   17,   17,   64,   62,  270,  270,  501,   62,
        74,   75,   75,  154,   81,  150,  153,  270,  270,  171,
       158,   76,   82,  154,   83,  111,  116,  117,  134,   84,
        17,   17,   17,   17,   56,   57,   58,   17,   17,   17,
@@ -666,35 +671,35 @@ static yyconst flex_int16_t yy_nxt[1379] =
       122,   96,  114,  124,  176,   97,  123,  115,  113,  125,
 
       179,   98,   88,   72,   99,  180,   17,   17,   17,   71,
-      587,   85,  587,  112,  100,   86,   95,  587,   87,  122,
+      592,   85,  592,  112,  100,   86,   95,  592,   87,  122,
        96,  114,  124,  176,   97,  123,  115,  113,  125,  179,
        98,   88,   89,   99,  180,  101,  147,  102,  148,   90,
       118,  149,  103,  126,  119,  181,   91,  104,  120,   92,
-       93,  127,   94,  587,  121,  128,  151,  152,  129,  130,
-      184,   89,  587,  587,  101,  147,  102,  148,   90,  118,
+       93,  127,   94,  592,  121,  128,  151,  152,  129,  130,
+      184,   89,  592,  592,  101,  147,  102,  148,   90,  118,
       149,  103,  126,  119,  181,   91,  104,  120,   92,   93,
       127,   94,  105,  121,  128,  151,  152,  129,  130,  184,
-      106,  131,  177,  107,  182,  132,  108,  587,  139,  109,
+      106,  131,  177,  107,  182,  132,  108,  592,  139,  109,
 
       178,  135,  110,   73,   73,  136,  140,  183,  133,  137,
-      587,  105,  185,   76,  141,  138,  173,  173,  587,  106,
+      592,  105,  185,   76,  141,  138,  173,  173,  592,  106,
       131,  177,  107,  182,  132,  108,   76,  139,  109,  178,
       135,  110,  142,  191,  136,  140,  183,  133,  137,  143,
       144,  185,   76,  141,  138,   74,   75,   75,  192,  145,
-      186,  193,  146,  587,  187,   76,   76,  196,  200,  194,
+      186,  193,  146,  592,  187,   76,   76,  196,  200,  194,
       197,  142,  191,  188,  195,  189,  198,  190,  143,  144,
-      201,  202,  587,  199,  203,  204,  205,  192,  145,  186,
-      193,  146,  161,  187,  587,   76,  196,  200,  194,  197,
+      201,  202,  592,  199,  203,  204,  205,  192,  145,  186,
+      193,  146,  161,  187,  592,   76,  196,  200,  194,  197,
       162,  163,  188,  195,  189,  198,  190,  164,  208,  201,
 
       202,  165,  199,  203,  204,  205,  206,  209,  210,  166,
       211,  213,  214,  167,  215,  168,  212,  207,  216,  169,
-      218,  217,  587,  223,  224,  225,  164,  208,  226,  227,
-      165,  228,  587,  229,  230,  206,  209,  210,  166,  211,
+      218,  217,  592,  223,  224,  225,  164,  208,  226,  227,
+      165,  228,  592,  229,  230,  206,  209,  210,  166,  211,
       213,  214,  167,  215,  168,  212,  207,  216,  169,  218,
       217,  219,  223,  224,  225,  231,  232,  226,  227,  233,
       228,  220,  229,  230,  234,  235,  221,  222,  236,  237,
-      238,  239,  240,  242,  243,  247,  241,  244,  248,  587,
+      238,  239,  240,  242,  243,  247,  241,  244,  248,  592,
       219,  252,  245,  246,  231,  232,  253,  254,  233,  255,
       220,  256,  257,  234,  235,  221,  222,  236,  237,  238,
 
@@ -714,33 +719,33 @@ static yyconst flex_int16_t yy_nxt[1379] =
       301,  302,  303,  304,  317,  315,  305,  306,  307,  318,
       319,  308,  309,  320,  310,  316,  311,  312,  321,  322,
       323,  324,  325,  326,  327,  328,  329,  313,  314,  330,
-      331,  332,  333,  317,  315,  334,  335,  336,  318,  319,
-      337,  338,  320,  341,  316,  342,  339,  321,  322,  323,
-      324,  325,  326,  327,  328,  329,  340,  343,  330,  331,
-      332,  333,  344,  345,  334,  335,  336,  346,  347,  337,
-      338,  348,  341,  349,  342,  339,  350,  351,  352,  353,
+      331,  334,  335,  317,  315,  332,  336,  333,  318,  319,
+      337,  338,  320,  339,  316,  342,  340,  321,  322,  323,
+      324,  325,  326,  327,  328,  329,  341,  343,  330,  331,
+      334,  335,  344,  345,  332,  336,  333,  346,  347,  337,
+      338,  348,  339,  349,  342,  340,  350,  351,  352,  353,
 
-      354,  355,  358,  356,  359,  360,  343,  357,  363,  364,
+      354,  355,  356,  357,  359,  360,  343,  358,  361,  364,
       365,  344,  345,  366,  367,  368,  346,  347,  369,  370,
       348,  371,  349,  372,  373,  350,  351,  352,  353,  354,
-      355,  358,  356,  359,  360,  374,  357,  363,  364,  365,
+      355,  356,  357,  359,  360,  374,  358,  361,  364,  365,
       375,  376,  366,  367,  368,  377,  378,  369,  370,  379,
       371,  380,  372,  373,  381,  382,  383,  384,  385,  386,
       387,  388,  389,  390,  374,  391,  392,  393,  394,  375,
-      376,  395,  396,  397,  377,  378,  400,  401,  379,  402,
-      380,  403,  398,  381,  382,  383,  384,  385,  386,  387,
-      388,  389,  390,  399,  391,  392,  393,  394,  404,  405,
+      376,  395,  396,  397,  377,  378,  398,  401,  379,  402,
+      380,  403,  399,  381,  382,  383,  384,  385,  386,  387,
+      388,  389,  390,  400,  391,  392,  393,  394,  404,  405,
 
-      395,  396,  397,  406,  407,  400,  401,  408,  402,  409,
-      403,  398,  410,  411,  412,  413,  414,  415,  416,  417,
-      418,  419,  399,  420,  421,  422,  423,  404,  405,  424,
+      395,  396,  397,  406,  407,  398,  401,  408,  402,  409,
+      403,  399,  410,  411,  412,  413,  414,  415,  416,  417,
+      418,  419,  400,  420,  421,  422,  423,  404,  405,  424,
       425,  426,  406,  407,  427,  428,  408,  429,  409,  430,
       431,  410,  411,  412,  413,  414,  415,  416,  417,  418,
       419,  432,  420,  421,  422,  423,  433,  434,  424,  425,
-      426,  437,  435,  427,  428,  436,  429,  438,  430,  431,
-      439,  440,  441,  442,  443,  444,  445,  446,  447,  448,
+      426,  435,  436,  427,  428,  439,  429,  437,  430,  431,
+      438,  440,  441,  442,  443,  444,  445,  446,  447,  448,
       432,  449,  450,  451,  452,  433,  434,  453,  454,  455,
-      437,  435,  456,  457,  436,  458,  438,  459,  460,  439,
+      435,  436,  456,  457,  439,  458,  437,  459,  460,  438,
 
       440,  441,  442,  443,  444,  445,  446,  447,  448,  461,
       449,  450,  451,  452,  462,  463,  453,  454,  455,  464,
@@ -750,11 +755,11 @@ static yyconst flex_int16_t yy_nxt[1379] =
       485,  486,  466,  487,  467,  488,  489,  468,  469,  470,
       471,  472,  473,  474,  475,  476,  477,  490,  478,  479,
       480,  481,  491,  492,  482,  483,  484,  493,  494,  485,
-      486,  495,  487,  496,  488,  489,  497,  499,  500,  501,
+      486,  495,  487,  496,  488,  489,  497,  498,  499,  500,
       502,  503,  504,  505,  506,  507,  490,  508,  509,  510,
 
       511,  491,  492,  512,  513,  514,  493,  494,  515,  516,
-      495,  517,  496,  518,  519,  497,  499,  500,  501,  502,
+      495,  517,  496,  518,  519,  497,  498,  499,  500,  502,
       503,  504,  505,  506,  507,  520,  508,  509,  510,  511,
       521,  522,  512,  513,  514,  523,  524,  515,  516,  525,
       517,  526,  518,  519,  527,  528,  529,  530,  531,  532,
@@ -770,33 +775,34 @@ static yyconst flex_int16_t yy_nxt[1379] =
       577,  556,  557,  558,  559,  560,  561,  562,  563,  564,
       565,  578,  566,  567,  568,  569,  579,  580,  570,  571,
       572,  581,  582,  573,  574,  583,  575,  584,  576,  577,
-      585,  586,  587,  587,  587,  587,  587,  587,  587,  587,
-      578,  587,  587,  587,  587,  579,  580,  587,  587,  587,
-      581,  582,  587,  587,  583,  587,  584,  587,  587,  585,
-      586,   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,  587,   80,  156,  156,
-      156,  156,  157,  157,  157,  587,  157,  157,  157,  157,
-      157,  157,  159,  159,  159,  587,  159,  159,  159,  159,
-      587,  159,  160,  160,  160,  160,  160,  160,  160,  160,
-      160,  160,  170,  170,  587,  170,  170,  170,  170,  170,
-      170,  170,  172,  587,  172,  172,  172,  172,  172,  172,
-
-      172,  172,  271,  271,  362,  362,   13,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587
+      585,  586,  587,  588,  589,  590,  591,  592,  592,  592,
+      578,  592,  592,  592,  592,  579,  580,  592,  592,  592,
+      581,  582,  592,  592,  583,  592,  584,  592,  592,  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,  156,  156,  156,  156,  157,  157,  157,
+      592,  157,  157,  157,  157,  157,  157,  159,  159,  159,
+      592,  159,  159,  159,  159,  592,  159,  160,  160,  160,
+      160,  160,  160,  160,  160,  160,  160,  170,  170,  592,
+      170,  170,  170,  170,  170,  170,  170,  172,  592,  172,
+
+      172,  172,  172,  172,  172,  172,  172,  271,  271,  363,
+      363,   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 yyconst flex_int16_t yy_chk[1379] =
+static yyconst flex_int16_t yy_chk[1384] =
     {   0,
         0,    0,    1,    2,    7,    8,   57,   57,   11,    7,
         8,   11,   12,    0,   18,   12,   18,   25,   25,   27,
@@ -812,8 +818,8 @@ static yyconst flex_int16_t yy_chk[1379] =
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    5,    5,    5,
         5,    5,    5,    5,    5,    5,   36,   39,   40,   46,
-        9,    5,    5,    5,   10,    9,  270,  270,  452,   10,
-       26,   26,   26,  154,   30,   51,   54,  361,  361,   68,
+        9,    5,    5,    5,   10,    9,  270,  270,  454,   10,
+       26,   26,   26,  154,   30,   51,   54,  362,  362,   68,
        62,   26,   30,   56,   30,   36,   39,   40,   46,   30,
         5,    5,    5,    6,    6,    6,    6,    6,    6,    6,
         6,    6,    9,   30,   51,   54,   10,    6,    6,    6,
@@ -869,90 +875,91 @@ static yyconst flex_int16_t yy_chk[1379] =
       208,  209,  210,  211,  223,  222,  212,  213,  214,  225,
       226,  215,  216,  227,  217,  222,  218,  219,  228,  229,
       230,  231,  233,  235,  236,  237,  238,  220,  221,  239,
-      240,  241,  242,  223,  222,  243,  244,  245,  225,  226,
-      246,  247,  227,  249,  222,  250,  248,  228,  229,  230,
-      231,  233,  235,  236,  237,  238,  248,  252,  239,  240,
-      241,  242,  253,  254,  243,  244,  245,  255,  256,  246,
-      247,  257,  249,  258,  250,  248,  259,  260,  261,  262,
-
-      263,  264,  266,  265,  267,  268,  252,  265,  272,  273,
-      274,  253,  254,  275,  276,  277,  255,  256,  278,  280,
-      257,  281,  258,  282,  283,  259,  260,  261,  262,  263,
-      264,  266,  265,  267,  268,  285,  265,  272,  273,  274,
-      286,  287,  275,  276,  277,  288,  289,  278,  280,  290,
-      281,  291,  282,  283,  292,  293,  294,  295,  298,  299,
-      300,  301,  302,  303,  285,  304,  305,  308,  310,  286,
-      287,  312,  313,  314,  288,  289,  321,  323,  290,  324,
-      291,  325,  315,  292,  293,  294,  295,  298,  299,  300,
-      301,  302,  303,  315,  304,  305,  308,  310,  326,  327,
-
-      312,  313,  314,  329,  330,  321,  323,  331,  324,  332,
-      325,  315,  334,  336,  337,  338,  340,  341,  342,  343,
-      344,  345,  315,  347,  349,  350,  351,  326,  327,  352,
-      353,  354,  329,  330,  355,  357,  331,  358,  332,  360,
-      364,  334,  336,  337,  338,  340,  341,  342,  343,  344,
-      345,  365,  347,  349,  350,  351,  366,  367,  352,  353,
-      354,  369,  368,  355,  357,  368,  358,  370,  360,  364,
-      372,  373,  374,  375,  376,  377,  378,  379,  380,  381,
-      365,  382,  383,  384,  385,  366,  367,  386,  387,  391,
-      369,  368,  392,  394,  368,  397,  370,  398,  399,  372,
-
-      373,  374,  375,  376,  377,  378,  379,  380,  381,  401,
-      382,  383,  384,  385,  406,  407,  386,  387,  391,  408,
-      409,  392,  394,  411,  397,  412,  398,  399,  414,  415,
-      416,  417,  418,  420,  421,  422,  423,  424,  401,  426,
-      427,  429,  430,  406,  407,  431,  432,  434,  408,  409,
-      435,  436,  411,  437,  412,  438,  440,  414,  415,  416,
-      417,  418,  420,  421,  422,  423,  424,  442,  426,  427,
-      429,  430,  443,  444,  431,  432,  434,  445,  446,  435,
-      436,  448,  437,  449,  438,  440,  450,  454,  455,  456,
-      459,  460,  462,  463,  464,  465,  442,  466,  468,  471,
-
-      472,  443,  444,  473,  474,  475,  445,  446,  479,  481,
-      448,  482,  449,  484,  485,  450,  454,  455,  456,  459,
-      460,  462,  463,  464,  465,  486,  466,  468,  471,  472,
-      487,  488,  473,  474,  475,  489,  492,  479,  481,  495,
-      482,  496,  484,  485,  497,  498,  500,  503,  504,  506,
-      508,  509,  510,  511,  486,  512,  513,  514,  516,  487,
-      488,  517,  518,  519,  489,  492,  520,  521,  495,  522,
-      496,  523,  525,  497,  498,  500,  503,  504,  506,  508,
-      509,  510,  511,  526,  512,  513,  514,  516,  528,  529,
-      517,  518,  519,  531,  532,  520,  521,  533,  522,  534,
-
-      523,  525,  536,  537,  538,  539,  540,  542,  543,  544,
-      545,  547,  526,  549,  550,  552,  554,  528,  529,  555,
-      558,  562,  531,  532,  563,  564,  533,  567,  534,  570,
-      571,  536,  537,  538,  539,  540,  542,  543,  544,  545,
-      547,  572,  549,  550,  552,  554,  575,  576,  555,  558,
-      562,  578,  579,  563,  564,  580,  567,  581,  570,  571,
-      582,  584,    0,    0,    0,    0,    0,    0,    0,    0,
-      572,    0,    0,    0,    0,  575,  576,    0,    0,    0,
-      578,  579,    0,    0,  580,    0,  581,    0,    0,  582,
-      584,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-
-      588,  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,  594,  594,  594,    0,  594,  595,  595,
-      595,  595,  596,  596,  596,    0,  596,  596,  596,  596,
-      596,  596,  597,  597,  597,    0,  597,  597,  597,  597,
-        0,  597,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  599,  599,    0,  599,  599,  599,  599,  599,
-      599,  599,  600,    0,  600,  600,  600,  600,  600,  600,
-
-      600,  600,  601,  601,  602,  602,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587,  587,  587,
-      587,  587,  587,  587,  587,  587,  587,  587
+      240,  242,  243,  223,  222,  241,  244,  241,  225,  226,
+      245,  246,  227,  247,  222,  249,  248,  228,  229,  230,
+      231,  233,  235,  236,  237,  238,  248,  250,  239,  240,
+      242,  243,  252,  253,  241,  244,  241,  254,  255,  245,
+      246,  256,  247,  257,  249,  248,  258,  259,  260,  261,
+
+      262,  263,  264,  265,  266,  267,  250,  265,  268,  272,
+      273,  252,  253,  274,  275,  276,  254,  255,  277,  278,
+      256,  280,  257,  281,  282,  258,  259,  260,  261,  262,
+      263,  264,  265,  266,  267,  283,  265,  268,  272,  273,
+      285,  286,  274,  275,  276,  287,  288,  277,  278,  289,
+      280,  290,  281,  282,  291,  292,  293,  294,  295,  298,
+      299,  300,  301,  302,  283,  303,  304,  305,  308,  285,
+      286,  310,  312,  313,  287,  288,  314,  321,  289,  323,
+      290,  324,  315,  291,  292,  293,  294,  295,  298,  299,
+      300,  301,  302,  315,  303,  304,  305,  308,  325,  326,
+
+      310,  312,  313,  327,  329,  314,  321,  330,  323,  331,
+      324,  315,  332,  333,  335,  337,  338,  339,  341,  342,
+      343,  344,  315,  345,  346,  348,  350,  325,  326,  351,
+      352,  353,  327,  329,  354,  355,  330,  356,  331,  358,
+      359,  332,  333,  335,  337,  338,  339,  341,  342,  343,
+      344,  361,  345,  346,  348,  350,  365,  366,  351,  352,
+      353,  367,  368,  354,  355,  370,  356,  369,  358,  359,
+      369,  371,  373,  374,  375,  376,  377,  378,  379,  380,
+      361,  381,  382,  383,  384,  365,  366,  385,  386,  387,
+      367,  368,  388,  392,  370,  393,  369,  395,  398,  369,
+
+      371,  373,  374,  375,  376,  377,  378,  379,  380,  399,
+      381,  382,  383,  384,  400,  402,  385,  386,  387,  407,
+      408,  388,  392,  409,  393,  410,  395,  398,  411,  413,
+      414,  416,  417,  418,  419,  420,  422,  423,  399,  424,
+      425,  426,  428,  400,  402,  429,  431,  432,  407,  408,
+      433,  434,  409,  436,  410,  437,  438,  411,  413,  414,
+      416,  417,  418,  419,  420,  422,  423,  439,  424,  425,
+      426,  428,  440,  442,  429,  431,  432,  444,  445,  433,
+      434,  446,  436,  447,  437,  438,  448,  450,  451,  452,
+      456,  457,  458,  461,  462,  464,  439,  465,  466,  467,
+
+      468,  440,  442,  469,  471,  474,  444,  445,  475,  476,
+      446,  477,  447,  478,  482,  448,  450,  451,  452,  456,
+      457,  458,  461,  462,  464,  484,  465,  466,  467,  468,
+      485,  487,  469,  471,  474,  488,  489,  475,  476,  490,
+      477,  491,  478,  482,  492,  495,  498,  499,  500,  501,
+      503,  506,  507,  509,  484,  511,  512,  513,  514,  485,
+      487,  515,  516,  517,  488,  489,  518,  520,  490,  521,
+      491,  522,  523,  492,  495,  498,  499,  500,  501,  503,
+      506,  507,  509,  524,  511,  512,  513,  514,  525,  526,
+      515,  516,  517,  527,  529,  518,  520,  530,  521,  532,
+
+      522,  523,  533,  535,  536,  538,  539,  541,  542,  543,
+      544,  545,  524,  547,  548,  549,  550,  525,  526,  552,
+      554,  555,  527,  529,  557,  559,  530,  560,  532,  563,
+      567,  533,  535,  536,  538,  539,  541,  542,  543,  544,
+      545,  568,  547,  548,  549,  550,  569,  572,  552,  554,
+      555,  575,  576,  557,  559,  577,  560,  580,  563,  567,
+      581,  583,  584,  585,  586,  587,  589,    0,    0,    0,
+      568,    0,    0,    0,    0,  569,  572,    0,    0,    0,
+      575,  576,    0,    0,  577,    0,  580,    0,    0,  581,
+      583,  584,  585,  586,  587,  589,  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,  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 yyconst flex_int32_t yy_rule_can_match_eol[161] =
+static yyconst flex_int32_t yy_rule_can_match_eol[162] =
     {   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, 
@@ -961,8 +968,8 @@ static yyconst flex_int32_t yy_rule_can_match_eol[161] =
     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, 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.
@@ -1028,6 +1035,7 @@ class ParseOrderBy;
 class ParseOrderByItem;
 class ParsePartitionClause;
 class ParsePredicate;
+class ParsePriority;
 class ParseSample;
 class ParseScalarLiteral;
 class ParseSearchedWhenClause;
@@ -1081,7 +1089,7 @@ class UnaryOperation;
 
 
 
-#line 1085 "SqlLexer_gen.cpp"
+#line 1093 "SqlLexer_gen.cpp"
 
 #define INITIAL 0
 #define CONDITION_SQL 1
@@ -1167,11 +1175,11 @@ void quickstep_yyset_extra (YY_EXTRA_TYPE user_defined ,yyscan_t yyscanner );
 
 FILE *quickstep_yyget_in (yyscan_t yyscanner );
 
-void quickstep_yyset_in  (FILE * in_str ,yyscan_t yyscanner );
+void quickstep_yyset_in  (FILE * _in_str ,yyscan_t yyscanner );
 
 FILE *quickstep_yyget_out (yyscan_t yyscanner );
 
-void quickstep_yyset_out  (FILE * out_str ,yyscan_t yyscanner );
+void quickstep_yyset_out  (FILE * _out_str ,yyscan_t yyscanner );
 
 yy_size_t quickstep_yyget_leng (yyscan_t yyscanner );
 
@@ -1179,11 +1187,11 @@ char *quickstep_yyget_text (yyscan_t yyscanner );
 
 int quickstep_yyget_lineno (yyscan_t yyscanner );
 
-void quickstep_yyset_lineno (int line_number ,yyscan_t yyscanner );
+void quickstep_yyset_lineno (int _line_number ,yyscan_t yyscanner );
 
 int quickstep_yyget_column  (yyscan_t yyscanner );
 
-void quickstep_yyset_column (int column_no ,yyscan_t yyscanner );
+void quickstep_yyset_column (int _column_no ,yyscan_t yyscanner );
 
 YYSTYPE * quickstep_yyget_lval (yyscan_t yyscanner );
 
@@ -1205,6 +1213,10 @@ extern int quickstep_yywrap (yyscan_t yyscanner );
 #endif
 #endif
 
+#ifndef YY_NO_UNPUT
+    
+#endif
+
 #ifndef yytext_ptr
 static void yy_flex_strncpy (char *,yyconst char *,int ,yyscan_t yyscanner);
 #endif
@@ -1319,7 +1331,7 @@ extern int quickstep_yylex \
 
 /* Code executed at the end of each rule. */
 #ifndef YY_BREAK
-#define YY_BREAK break;
+#define YY_BREAK /*LINTED*/break;
 #endif
 
 #define YY_RULE_SETUP \
@@ -1329,9 +1341,9 @@ extern int quickstep_yylex \
  */
 YY_DECL
 {
-	register yy_state_type yy_current_state;
-	register char *yy_cp, *yy_bp;
-	register int yy_act;
+	yy_state_type yy_current_state;
+	char *yy_cp, *yy_bp;
+	int yy_act;
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 
     yylval = yylval_param;
@@ -1365,12 +1377,12 @@ YY_DECL
 		}
 
 	{
-#line 130 "../SqlLexer.lpp"
+#line 131 "../SqlLexer.lpp"
 
 
-#line 1372 "SqlLexer_gen.cpp"
+#line 1384 "SqlLexer_gen.cpp"
 
-	while ( 1 )		/* loops until end-of-file is reached */
+	while ( /*CONSTCOND*/1 )		/* loops until end-of-file is reached */
 		{
 		yy_cp = yyg->yy_c_buf_p;
 
@@ -1386,7 +1398,7 @@ YY_DECL
 yy_match:
 		do
 			{
-			register YY_CHAR yy_c = yy_ec[YY_SC_TO_UI(*yy_cp)] ;
+			YY_CHAR yy_c = yy_ec[YY_SC_TO_UI(*yy_cp)] ;
 			if ( yy_accept[yy_current_state] )
 				{
 				yyg->yy_last_accepting_state = yy_current_state;
@@ -1395,13 +1407,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 >= 588 )
+				if ( yy_current_state >= 593 )
 					yy_c = yy_meta[(unsigned int) yy_c];
 				}
 			yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
 			++yy_cp;
 			}
-		while ( yy_current_state != 587 );
+		while ( yy_current_state != 592 );
 		yy_cp = yyg->yy_last_accepting_cpos;
 		yy_current_state = yyg->yy_last_accepting_state;
 
@@ -1435,7 +1447,7 @@ do_action:	/* This label is used only to access EOF actions. */
 
 case 1:
 YY_RULE_SETUP
-#line 133 "../SqlLexer.lpp"
+#line 134 "../SqlLexer.lpp"
 {
     /* A forward slash character represents a system command. */
     BEGIN(CONDITION_COMMAND);
@@ -1447,7 +1459,7 @@ YY_RULE_SETUP
 case 2:
 /* rule 2 can match eol */
 YY_RULE_SETUP
-#line 141 "../SqlLexer.lpp"
+#line 142 "../SqlLexer.lpp"
 {
     /* This is a SQL command. Place the char back and process normally. */
     yyless(0);
@@ -1459,7 +1471,7 @@ YY_RULE_SETUP
 
 case 3:
 YY_RULE_SETUP
-#line 150 "../SqlLexer.lpp"
+#line 151 "../SqlLexer.lpp"
 {
     /* This is a command argument. */
     yylval->string_value_ = new quickstep::ParseString(
@@ -1469,7 +1481,7 @@ YY_RULE_SETUP
 	YY_BREAK
 case 4:
 YY_RULE_SETUP
-#line 157 "../SqlLexer.lpp"
+#line 158 "../SqlLexer.lpp"
 {
     /* Ignore whitespace. */
   }
@@ -1477,7 +1489,7 @@ YY_RULE_SETUP
 case 5:
 /* rule 5 can match eol */
 YY_RULE_SETUP
-#line 161 "../SqlLexer.lpp"
+#line 162 "../SqlLexer.lpp"
 {
     /* Newline reverts the lexer to the initial state. */
     yycolumn = 0;
@@ -1489,667 +1501,672 @@ YY_RULE_SETUP
 
 case 6:
 YY_RULE_SETUP
-#line 170 "../SqlLexer.lpp"
+#line 171 "../SqlLexer.lpp"
 return TOKEN_ADD;
 	YY_BREAK
 case 7:
 YY_RULE_SETUP
-#line 171 "../SqlLexer.lpp"
+#line 172 "../SqlLexer.lpp"
 return TOKEN_ALL;
 	YY_BREAK
 case 8:
 YY_RULE_SETUP
-#line 172 "../SqlLexer.lpp"
+#line 173 "../SqlLexer.lpp"
 return TOKEN_ALTER;
 	YY_BREAK
 case 9:
 YY_RULE_SETUP
-#line 173 "../SqlLexer.lpp"
+#line 174 "../SqlLexer.lpp"
 return TOKEN_AND;
 	YY_BREAK
 case 10:
 YY_RULE_SETUP
-#line 174 "../SqlLexer.lpp"
+#line 175 "../SqlLexer.lpp"
 return TOKEN_AS;
 	YY_BREAK
 case 11:
 YY_RULE_SETUP
-#line 175 "../SqlLexer.lpp"
+#line 176 "../SqlLexer.lpp"
 return TOKEN_ASC;
 	YY_BREAK
 case 12:
 YY_RULE_SETUP
-#line 176 "../SqlLexer.lpp"
+#line 177 "../SqlLexer.lpp"
 return TOKEN_ASC;
 	YY_BREAK
 case 13:
 YY_RULE_SETUP
-#line 177 "../SqlLexer.lpp"
+#line 178 "../SqlLexer.lpp"
 return TOKEN_BETWEEN;
 	YY_BREAK
 case 14:
 YY_RULE_SETUP
-#line 178 "../SqlLexer.lpp"
+#line 179 "../SqlLexer.lpp"
 return TOKEN_BIGINT;
 	YY_BREAK
 case 15:
 YY_RULE_SETUP
-#line 179 "../SqlLexer.lpp"
+#line 180 "../SqlLexer.lpp"
 return TOKEN_BIT;
 	YY_BREAK
 case 16:
 YY_RULE_SETUP
-#line 180 "../SqlLexer.lpp"
+#line 181 "../SqlLexer.lpp"
 return TOKEN_BITWEAVING;
 	YY_BREAK
 case 17:
 YY_RULE_SETUP
-#line 181 "../SqlLexer.lpp"
+#line 182 "../SqlLexer.lpp"
 return TOKEN_BLOCKPROPERTIES;
 	YY_BREAK
 case 18:
 YY_RULE_SETUP
-#line 182 "../SqlLexer.lpp"
+#line 183 "../SqlLexer.lpp"
 return TOKEN_BLOCKSAMPLE;
 	YY_BREAK
 case 19:
 YY_RULE_SETUP
-#line 183 "../SqlLexer.lpp"
+#line 184 "../SqlLexer.lpp"
 return TOKEN_BLOOM_FILTER;
 	YY_BREAK
 case 20:
 YY_RULE_SETUP
-#line 184 "../SqlLexer.lpp"
+#line 185 "../SqlLexer.lpp"
 return TOKEN_CASE;
 	YY_BREAK
 case 21:
 YY_RULE_SETUP
-#line 185 "../SqlLexer.lpp"
+#line 186 "../SqlLexer.lpp"
 return TOKEN_CSB_TREE;
 	YY_BREAK
 case 22:
 YY_RULE_SETUP
-#line 186 "../SqlLexer.lpp"
+#line 187 "../SqlLexer.lpp"
 return TOKEN_BY;
 	YY_BREAK
 case 23:
 YY_RULE_SETUP
-#line 187 "../SqlLexer.lpp"
+#line 188 "../SqlLexer.lpp"
 return TOKEN_CHARACTER;
 	YY_BREAK
 case 24:
 YY_RULE_SETUP
-#line 188 "../SqlLexer.lpp"
+#line 189 "../SqlLexer.lpp"
 return TOKEN_CHARACTER;
 	YY_BREAK
 case 25:
 YY_RULE_SETUP
-#line 189 "../SqlLexer.lpp"
+#line 190 "../SqlLexer.lpp"
 return TOKEN_CHECK;
 	YY_BREAK
 case 26:
 YY_RULE_SETUP
-#line 190 "../SqlLexer.lpp"
+#line 191 "../SqlLexer.lpp"
 return TOKEN_COLUMN;
 	YY_BREAK
 case 27:
 YY_RULE_SETUP
-#line 191 "../SqlLexer.lpp"
+#line 192 "../SqlLexer.lpp"
 return TOKEN_CONSTRAINT;
 	YY_BREAK
 case 28:
 YY_RULE_SETUP
-#line 192 "../SqlLexer.lpp"
+#line 193 "../SqlLexer.lpp"
 return TOKEN_COPY;
 	YY_BREAK
 case 29:
 YY_RULE_SETUP
-#line 193 "../SqlLexer.lpp"
+#line 194 "../SqlLexer.lpp"
 return TOKEN_CREATE;
 	YY_BREAK
 case 30:
 YY_RULE_SETUP
-#line 194 "../SqlLexer.lpp"
+#line 195 "../SqlLexer.lpp"
 return TOKEN_CURRENT;
 	YY_BREAK
 case 31:
 YY_RULE_SETUP
-#line 195 "../SqlLexer.lpp"
+#line 196 "../SqlLexer.lpp"
 return TOKEN_DATE;
 	YY_BREAK
 case 32:
 YY_RULE_SETUP
-#line 196 "../SqlLexer.lpp"
+#line 197 "../SqlLexer.lpp"
 return TOKEN_DATETIME;
 	YY_BREAK
 case 33:
 YY_RULE_SETUP
-#line 197 "../SqlLexer.lpp"
+#line 198 "../SqlLexer.lpp"
 return TOKEN_DAY;
 	YY_BREAK
 case 34:
 YY_RULE_SETUP
-#line 198 "../SqlLexer.lpp"
+#line 199 "../SqlLexer.lpp"
 return TOKEN_DECIMAL;
 	YY_BREAK
 case 35:
 YY_RULE_SETUP
-#line 199 "../SqlLexer.lpp"
+#line 200 "../SqlLexer.lpp"
 return TOKEN_DEFAULT;
 	YY_BREAK
 case 36:
 YY_RULE_SETUP
-#line 200 "../SqlLexer.lpp"
+#line 201 "../SqlLexer.lpp"
 return TOKEN_DELETE;
 	YY_BREAK
 case 37:
 YY_RULE_SETUP
-#line 201 "../SqlLexer.lpp"
+#line 202 "../SqlLexer.lpp"
 return TOKEN_DELIMITER;
 	YY_BREAK
 case 38:
 YY_RULE_SETUP
-#line 202 "../SqlLexer.lpp"
+#line 203 "../SqlLexer.lpp"
 return TOKEN_DESC;
 	YY_BREAK
 case 39:
 YY_RULE_SETUP
-#line 203 "../SqlLexer.lpp"
+#line 204 "../SqlLexer.lpp"
 return TOKEN_DESC;
 	YY_BREAK
 case 40:
 YY_RULE_SETUP
-#line 204 "../SqlLexer.lpp"
+#line 205 "../SqlLexer.lpp"
 return TOKEN_DISTINCT;
 	YY_BREAK
 case 41:
 YY_RULE_SETUP
-#line 205 "../SqlLexer.lpp"
+#line 206 "../SqlLexer.lpp"
 return TOKEN_DOUBLE;
 	YY_BREAK
 case 42:
 YY_RULE_SETUP
-#line 206 "../SqlLexer.lpp"
+#line 207 "../SqlLexer.lpp"
 return TOKEN_DROP;
 	YY_BREAK
 case 43:
 YY_RULE_SETUP
-#line 207 "../SqlLexer.lpp"
+#line 208 "../SqlLexer.lpp"
 return TOKEN_ELSE;
 	YY_BREAK
 case 44:
 YY_RULE_SETUP
-#line 208 "../SqlLexer.lpp"
+#line 209 "../SqlLexer.lpp"
 return TOKEN_END;
 	YY_BREAK
 case 45:
 YY_RULE_SETUP
-#line 209 "../SqlLexer.lpp"
+#line 210 "../SqlLexer.lpp"
 return TOKEN_ESCAPE_STRINGS;
 	YY_BREAK
 case 46:
 YY_RULE_SETUP
-#line 210 "../SqlLexer.lpp"
+#line 211 "../SqlLexer.lpp"
 return TOKEN_EXISTS;
 	YY_BREAK
 case 47:
 YY_RULE_SETUP
-#line 211 "../SqlLexer.lpp"
+#line 212 "../SqlLexer.lpp"
 return TOKEN_EXTRACT;
 	YY_BREAK
 case 48:
 YY_RULE_SETUP
-#line 212 "../SqlLexer.lpp"
+#line 213 "../SqlLexer.lpp"
 return TOKEN_FALSE;
 	YY_BREAK
 case 49:
 YY_RULE_SETUP
-#line 213 "../SqlLexer.lpp"
+#line 214 "../SqlLexer.lpp"
 return TOKEN_FIRST;
 	YY_BREAK
 case 50:
 YY_RULE_SETUP
-#line 214 "../SqlLexer.lpp"
+#line 215 "../SqlLexer.lpp"
 return TOKEN_FLOAT;
 	YY_BREAK
 case 51:
 YY_RULE_SETUP
-#line 215 "../SqlLexer.lpp"
+#line 216 "../SqlLexer.lpp"
 return TOKEN_FOLLOWING;
 	YY_BREAK
 case 52:
 YY_RULE_SETUP
-#line 216 "../SqlLexer.lpp"
+#line 217 "../SqlLexer.lpp"
 return TOKEN_FOR;
 	YY_BREAK
 case 53:
 YY_RULE_SETUP
-#line 217 "../SqlLexer.lpp"
+#line 218 "../SqlLexer.lpp"
 return TOKEN_FOREIGN;
 	YY_BREAK
 case 54:
 YY_RULE_SETUP
-#line 218 "../SqlLexer.lpp"
+#line 219 "../SqlLexer.lpp"
 return TOKEN_FROM;
 	YY_BREAK
 case 55:
 YY_RULE_SETUP
-#line 219 "../SqlLexer.lpp"
+#line 220 "../SqlLexer.lpp"
 return TOKEN_FULL;
 	YY_BREAK
 case 56:
 YY_RULE_SETUP
-#line 220 "../SqlLexer.lpp"
+#line 221 "../SqlLexer.lpp"
 return TOKEN_GROUP;
 	YY_BREAK
 case 57:
 YY_RULE_SETUP
-#line 221 "../SqlLexer.lpp"
+#line 222 "../SqlLexer.lpp"
 return TOKEN_HASH;
 	YY_BREAK
 case 58:
 YY_RULE_SETUP
-#line 222 "../SqlLexer.lpp"
+#line 223 "../SqlLexer.lpp"
 return TOKEN_HAVING;
 	YY_BREAK
 case 59:
 YY_RULE_SETUP
-#line 223 "../SqlLexer.lpp"
+#line 224 "../SqlLexer.lpp"
 return TOKEN_HOUR;
 	YY_BREAK
 case 60:
 YY_RULE_SETUP
-#line 224 "../SqlLexer.lpp"
+#line 225 "../SqlLexer.lpp"
 return TOKEN_IN;
 	YY_BREAK
 case 61:
 YY_RULE_SETUP
-#line 225 "../SqlLexer.lpp"
+#line 226 "../SqlLexer.lpp"
 return TOKEN_INDEX;
 	YY_BREAK
 case 62:
 YY_RULE_SETUP
-#line 226 "../SqlLexer.lpp"
+#line 227 "../SqlLexer.lpp"
 return TOKEN_INNER;
 	YY_BREAK
 case 63:
 YY_RULE_SETUP
-#line 227 "../SqlLexer.lpp"
+#line 228 "../SqlLexer.lpp"
 return TOKEN_INSERT;
 	YY_BREAK
 case 64:
 YY_RULE_SETUP
-#line 228 "../SqlLexer.lpp"
+#line 229 "../SqlLexer.lpp"
 return TOKEN_INTEGER;
 	YY_BREAK
 case 65:
 YY_RULE_SETUP
-#line 229 "../SqlLexer.lpp"
+#line 230 "../SqlLexer.lpp"
 return TOKEN_INTEGER;
 	YY_BREAK
 case 66:
 YY_RULE_SETUP
-#line 230 "../SqlLexer.lpp"
+#line 231 "../SqlLexer.lpp"
 return TOKEN_INTERVAL;
 	YY_BREAK
 case 67:
 YY_RULE_SETUP
-#line 231 "../SqlLexer.lpp"
+#line 232 "../SqlLexer.lpp"
 return TOKEN_INTO;
 	YY_BREAK
 case 68:
 YY_RULE_SETUP
-#line 232 "../SqlLexer.lpp"
+#line 233 "../SqlLexer.lpp"
 return TOKEN_IS;
 	YY_BREAK
 case 69:
 YY_RULE_SETUP
-#line 233 "../SqlLexer.lpp"
+#line 234 "../SqlLexer.lpp"
 return TOKEN_JOIN;
 	YY_BREAK
 case 70:
 YY_RULE_SETUP
-#line 234 "../SqlLexer.lpp"
+#line 235 "../SqlLexer.lpp"
 return TOKEN_KEY;
 	YY_BREAK
 case 71:
 YY_RULE_SETUP
-#line 235 "../SqlLexer.lpp"
+#line 236 "../SqlLexer.lpp"
 return TOKEN_LAST;
 	YY_BREAK
 case 72:
 YY_RULE_SETUP
-#line 236 "../SqlLexer.lpp"
+#line 237 "../SqlLexer.lpp"
 return TOKEN_LEFT;
 	YY_BREAK
 case 73:
 YY_RULE_SETUP
-#line 237 "../SqlLexer.lpp"
+#line 238 "../SqlLexer.lpp"
 return TOKEN_LIKE;
 	YY_BREAK
 case 74:
 YY_RULE_SETUP
-#line 238 "../SqlLexer.lpp"
+#line 239 "../SqlLexer.lpp"
 return TOKEN_LIMIT;
 	YY_BREAK
 case 75:
 YY_RULE_SETUP
-#line 239 "../SqlLexer.lpp"
+#line 240 "../SqlLexer.lpp"
 return TOKEN_LONG;
 	YY_BREAK
 case 76:
 YY_RULE_SETUP
-#line 240 "../SqlLexer.lpp"
+#line 241 "../SqlLexer.lpp"
 return TOKEN_MINUTE;
 	YY_BREAK
 case 77:
 YY_RULE_SETUP
-#line 241 "../SqlLexer.lpp"
+#line 242 "../SqlLexer.lpp"
 return TOKEN_MONTH;
 	YY_BREAK
 case 78:
 YY_RULE_SETUP
-#line 242 "../SqlLexer.lpp"
+#line 243 "../SqlLexer.lpp"
 return TOKEN_NOT;
 	YY_BREAK
 case 79:
 YY_RULE_SETUP
-#line 243 "../SqlLexer.lpp"
+#line 244 "../SqlLexer.lpp"
 return TOKEN_NULL;
 	YY_BREAK
 case 80:
 YY_RULE_SETUP
-#line 244 "../SqlLexer.lpp"
+#line 245 "../SqlLexer.lpp"
 return TOKEN_NULLS;
 	YY_BREAK
 case 81:
 YY_RULE_SETUP
-#line 245 "../SqlLexer.lpp"
+#line 246 "../SqlLexer.lpp"
 return TOKEN_OFF;
 	YY_BREAK
 case 82:
 YY_RULE_SETUP
-#line 246 "../SqlLexer.lpp"
+#line 247 "../SqlLexer.lpp"
 return TOKEN_ON;
 	YY_BREAK
 case 83:
 YY_RULE_SETUP
-#line 247 "../SqlLexer.lpp"
+#line 248 "../SqlLexer.lpp"
 return TOKEN_OR;
 	YY_BREAK
 case 84:
 YY_RULE_SETUP
-#line 248 "../SqlLexer.lpp"
+#line 249 "../SqlLexer.lpp"
 return TOKEN_ORDER;
 	YY_BREAK
 case 85:
 YY_RULE_SETUP
-#line 249 "../SqlLexer.lpp"
+#line 250 "../SqlLexer.lpp"
 return TOKEN_OUTER;
 	YY_BREAK
 case 86:
 YY_RULE_SETUP
-#line 250 "../SqlLexer.lpp"
+#line 251 "../SqlLexer.lpp"
 return TOKEN_OVER;
 	YY_BREAK
 case 87:
 YY_RULE_SETUP
-#line 251 "../SqlLexer.lpp"
+#line 252 "../SqlLexer.lpp"
 return TOKEN_PARTITION;
 	YY_BREAK
 case 88:
 YY_RULE_SETUP
-#line 252 "../SqlLexer.lpp"
+#line 253 "../SqlLexer.lpp"
 return TOKEN_PARTITIONS;
 	YY_BREAK
 case 89:
 YY_RULE_SETUP
-#line 253 "../SqlLexer.lpp"
+#line 254 "../SqlLexer.lpp"
 return TOKEN_PERCENT;
 	YY_BREAK
 case 90:
 YY_RULE_SETUP
-#line 254 "../SqlLexer.lpp"
+#line 255 "../SqlLexer.lpp"
 return TOKEN_PRECEDING;
 	YY_BREAK
 case 91:
 YY_RULE_SETUP
-#line 255 "../SqlLexer.lpp"
+#line 256 "../SqlLexer.lpp"
 return TOKEN_PRIMARY;
 	YY_BREAK
 case 92:
 YY_RULE_SETUP
-#line 256 "../SqlLexer.lpp"
-return TOKEN_QUIT;
+#line 257 "../SqlLexer.lpp"
+return TOKEN_PRIORITY;
 	YY_BREAK
 case 93:
 YY_RULE_SETUP
-#line 257 "../SqlLexer.lpp"
-return TOKEN_RANGE;
+#line 258 "../SqlLexer.lpp"
+return TOKEN_QUIT;
 	YY_BREAK
 case 94:
 YY_RULE_SETUP
-#line 258 "../SqlLexer.lpp"
-return TOKEN_REAL;
+#line 259 "../SqlLexer.lpp"
+return TOKEN_RANGE;
 	YY_BREAK
 case 95:
 YY_RULE_SETUP
-#line 259 "../SqlLexer.lpp"
-return TOKEN_REFERENCES;
+#line 260 "../SqlLexer.lpp"
+return TOKEN_REAL;
 	YY_BREAK
 case 96:
 YY_RULE_SETUP
-#line 260 "../SqlLexer.lpp"
-return TOKEN_REGEXP;
+#line 261 "../SqlLexer.lpp"
+return TOKEN_REFERENCES;
 	YY_BREAK
 case 97:
 YY_RULE_SETUP
-#line 261 "../SqlLexer.lpp"
-return TOKEN_RIGHT;
+#line 262 "../SqlLexer.lpp"
+return TOKEN_REGEXP;
 	YY_BREAK
 case 98:
 YY_RULE_SETUP
-#line 262 "../SqlLexer.lpp"
-return TOKEN_ROW;
+#line 263 "../SqlLexer.lpp"
+return TOKEN_RIGHT;
 	YY_BREAK
 case 99:
 YY_RULE_SETUP
-#line 263 "../SqlLexer.lpp"
-return TOKEN_ROW_DELIMITER;
+#line 264 "../SqlLexer.lpp"
+return TOKEN_ROW;
 	YY_BREAK
 case 100:
 YY_RULE_SETUP
-#line 264 "../SqlLexer.lpp"
-return TOKEN_ROWS;
+#line 265 "../SqlLexer.lpp"
+return TOKEN_ROW_DELIMITER;
 	YY_BREAK
 case 101:
 YY_RULE_SETUP
-#line 265 "../SqlLexer.lpp"
-return TOKEN_SECOND;
+#line 266 "../SqlLexer.lpp"
+return TOKEN_ROWS;
 	YY_BREAK
 case 102:
 YY_RULE_SETUP
-#line 266 "../SqlLexer.lpp"
-return TOKEN_SELECT;
+#line 267 "../SqlLexer.lpp"
+return TOKEN_SECOND;
 	YY_BREAK
 case 103:
 YY_RULE_SETUP
-#line 267 "../SqlLexer.lpp"
-return TOKEN_SET;
+#line 268 "../SqlLexer.lpp"
+return TOKEN_SELECT;
 	YY_BREAK
 case 104:
 YY_RULE_SETUP
-#line 268 "../SqlLexer.lpp"
-return TOKEN_SMA;
+#line 269 "../SqlLexer.lpp"
+return TOKEN_SET;
 	YY_BREAK
 case 105:
 YY_RULE_SETUP
-#line 269 "../SqlLexer.lpp"
-return TOKEN_SMALLINT;
+#line 270 "../SqlLexer.lpp"
+return TOKEN_SMA;
 	YY_BREAK
 case 106:
 YY_RULE_SETUP
-#line 270 "../SqlLexer.lpp"
-return TOKEN_SUBSTRING;
+#line 271 "../SqlLexer.lpp"
+return TOKEN_SMALLINT;
 	YY_BREAK
 case 107:
 YY_RULE_SETUP
-#line 271 "../SqlLexer.lpp"
-return TOKEN_TABLE;
+#line 272 "../SqlLexer.lpp"
+return TOKEN_SUBSTRING;
 	YY_BREAK
 case 108:
 YY_RULE_SETUP
-#line 272 "../SqlLexer.lpp"
-return TOKEN_THEN;
+#line 273 "../SqlLexer.lpp"
+return TOKEN_TABLE;
 	YY_BREAK
 case 109:
 YY_RULE_SETUP
-#line 273 "../SqlLexer.lpp"
-return TOKEN_TIME;
+#line 274 "../SqlLexer.lpp"
+return TOKEN_THEN;
 	YY_BREAK
 case 110:
 YY_RULE_SETUP
-#line 274 "../SqlLexer.lpp"
-return TOKEN_TIMESTAMP;
+#line 275 "../SqlLexer.lpp"
+return TOKEN_TIME;
 	YY_BREAK
 case 111:
 YY_RULE_SETUP
-#line 275 "../SqlLexer.lpp"
-return TOKEN_TRUE;
+#line 276 "../SqlLexer.lpp"
+return TOKEN_TIMESTAMP;
 	YY_BREAK
 case 112:
 YY_RULE_SETUP
-#line 276 "../SqlLexer.lpp"
-return TOKEN_TUPLESAMPLE;
+#line 277 "../SqlLexer.lpp"
+return TOKEN_TRUE;
 	YY_BREAK
 case 113:
 YY_RULE_SETUP
-#line 277 "../SqlLexer.lpp"
-return TOKEN_UNBOUNDED;
+#line 278 "../SqlLexer.lpp"
+return TOKEN_TUPLESAMPLE;
 	YY_BREAK
 case 114:
 YY_RULE_SETUP
-#line 278 "../SqlLexer.lpp"
-return TOKEN_UNIQUE;
+#line 279 "../SqlLexer.lpp"
+return TOKEN_UNBOUNDED;
 	YY_BREAK
 case 115:
 YY_RULE_SETUP
-#line 279 "../SqlLexer.lpp"
-return TOKEN_UPDATE;
+#line 280 "../SqlLexer.lpp"
+return TOKEN_UNIQUE;
 	YY_BREAK
 case 116:
 YY_RULE_SETUP
-#line 280 "../SqlLexer.lpp"
-return TOKEN_USING;
+#line 281 "../SqlLexer.lpp"
+return TOKEN_UPDATE;
 	YY_BREAK
 case 117:
 YY_RULE_SETUP
-#line 281 "../SqlLexer.lpp"
-return TOKEN_VALUES;
+#line 282 "../SqlLexer.lpp"
+return TOKEN_USING;
 	YY_BREAK
 case 118:
 YY_RULE_SETUP
-#line 282 "../SqlLexer.lpp"
-return TOKEN_VARCHAR;
+#line 283 "../SqlLexer.lpp"
+return TOKEN_VALUES;
 	YY_BREAK
 case 119:
 YY_RULE_SETUP
-#line 283 "../SqlLexer.lpp"
-return TOKEN_WHEN;
+#line 284 "../SqlLexer.lpp"
+return TOKEN_VARCHAR;
 	YY_BREAK
 case 120:
 YY_RULE_SETUP
-#line 284 "../SqlLexer.lpp"
-return TOKEN_WHERE;
+#line 285 "../SqlLexer.lpp"
+return TOKEN_WHEN;
 	YY_BREAK
 case 121:
 YY_RULE_SETUP
-#line 285 "../SqlLexer.lpp"
-return TOKEN_WINDOW;
+#line 286 "../SqlLexer.lpp"
+return TOKEN_WHERE;
 	YY_BREAK
 case 122:
 YY_RULE_SETUP
-#line 286 "../SqlLexer.lpp"
-return TOKEN_WITH;
+#line 287 "../SqlLexer.lpp"
+return TOKEN_WINDOW;
 	YY_BREAK
 case 123:
 YY_RULE_SETUP
-#line 287 "../SqlLexer.lpp"
-return TOKEN_YEAR;
+#line 288 "../SqlLexer.lpp"
+return TOKEN_WITH;
 	YY_BREAK
 case 124:
 YY_RULE_SETUP
-#line 288 "../SqlLexer.lpp"
-return TOKEN_YEARMONTH;
+#line 289 "../SqlLexer.lpp"
+return TOKEN_YEAR;
 	YY_BREAK
 case 125:
 YY_RULE_SETUP
 #line 290 "../SqlLexer.lpp"
-return TOKEN_EQ;
+return TOKEN_YEARMONTH;
 	YY_BREAK
 case 126:
 YY_RULE_SETUP
-#line 291 "../SqlLexer.lpp"
-return TOKEN_NEQ;
+#line 292 "../SqlLexer.lpp"
+return TOKEN_EQ;
 	YY_BREAK
 case 127:
 YY_RULE_SETUP
-#line 292 "../SqlLexer.lpp"
+#line 293 "../SqlLexer.lpp"
 return TOKEN_NEQ;
 	YY_BREAK
 case 128:
 YY_RULE_SETUP
-#line 293 "../SqlLexer.lpp"
-return TOKEN_LT;
+#line 294 "../SqlLexer.lpp"
+return TOKEN_NEQ;
 	YY_BREAK
 case 129:
 YY_RULE_SETUP
-#line 294 "../SqlLexer.lpp"
-return TOKEN_GT;
+#line 295 "../SqlLexer.lpp"
+return TOKEN_LT;
 	YY_BREAK
 case 130:
 YY_RULE_SETUP
-#line 295 "../SqlLexer.lpp"
-return TOKEN_LEQ;
+#line 296 "../SqlLexer.lpp"
+return TOKEN_GT;
 	YY_BREAK
 case 131:
 YY_RULE_SETUP
-#line 296 "../SqlLexer.lpp"
-return TOKEN_GEQ;
+#line 297 "../SqlLexer.lpp"
+return TOKEN_LEQ;
 	YY_BREAK
 case 132:
 YY_RULE_SETUP
 #line 298 "../SqlLexer.lpp"
-return yytext[0];
+return TOKEN_GEQ;
 	YY_BREAK
 case 133:
 YY_RULE_SETUP
-#line 299 "../SqlLexer.lpp"
+#line 300 "../SqlLexer.lpp"
+return yytext[0];
+	YY_BREAK
+case 134:
+YY_RULE_SETUP
+#line 301 "../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 134:
+case 135:
 YY_RULE_SETUP
-#line 305 "../SqlLexer.lpp"
+#line 307 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED_ESCAPED);
   }
 	YY_BREAK
-case 135:
+case 136:
 YY_RULE_SETUP
-#line 310 "../SqlLexer.lpp"
+#line 312 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED);
   }
 	YY_BREAK
-case 136:
+case 137:
 YY_RULE_SETUP
-#line 315 "../SqlLexer.lpp"
+#line 317 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_DOUBLE_QUOTED);
@@ -2161,7 +2178,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 324 "../SqlLexer.lpp"
+#line 326 "../SqlLexer.lpp"
 {
     delete yylval->string_value_;
     BEGIN(INITIAL);
@@ -2172,9 +2189,9 @@ case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
 
 /* Process escape sequences. */
 
-case 137:
+case 138:
 YY_RULE_SETUP
-#line 334 "../SqlLexer.lpp"
+#line 336 "../SqlLexer.lpp"
 {
     /* Octal code */
     unsigned int code;
@@ -2188,9 +2205,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 138:
+case 139:
 YY_RULE_SETUP
-#line 346 "../SqlLexer.lpp"
+#line 348 "../SqlLexer.lpp"
 {
     /* Hexadecimal code */
     unsigned int code;
@@ -2198,9 +2215,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 139:
+case 140:
 YY_RULE_SETUP
-#line 352 "../SqlLexer.lpp"
+#line 354 "../SqlLexer.lpp"
 {
     /* A numeric escape sequence that isn't correctly specified. */
     delete yylval->string_value_;
@@ -2209,58 +2226,58 @@ YY_RULE_SETUP
     return TOKEN_LEX_ERROR;
   }
 	YY_BREAK
-case 140:
+case 141:
 YY_RULE_SETUP
-#line 359 "../SqlLexer.lpp"
+#line 361 "../SqlLexer.lpp"
 {
     /* Backspace */
     yylval->string_value_->push_back('\b');
   }
 	YY_BREAK
-case 141:
+case 142:
 YY_RULE_SETUP
-#line 363 "../SqlLexer.lpp"
+#line 365 "../SqlLexer.lpp"
 {
     /* Form-feed */
     yylval->string_value_->push_back('\f');
   }
 	YY_BREAK
-case 142:
+case 143:
 YY_RULE_SETUP
-#line 367 "../SqlLexer.lpp"
+#line 369 "../SqlLexer.lpp"
 {
     /* Newline */
     yylval->string_value_->push_back('\n');
   }
 	YY_BREAK
-case 143:
+case 144:
 YY_RULE_SETUP
-#line 371 "../SqlLexer.lpp"
+#line 373 "../SqlLexer.lpp"
 {
     /* Carriage-return */
     yylval->string_value_->push_back('\r');
   }
 	YY_BREAK
-case 144:
+case 145:
 YY_RULE_SETUP
-#line 375 "../SqlLexer.lpp"
+#line 377 "../SqlLexer.lpp"
 {
     /* Horizontal Tab */
     yylval->string_value_->push_back('\t');
   }
 	YY_BREAK
-case 145:
-/* rule 145 can match eol */
+case 146:
+/* rule 146 can match eol */
 YY_RULE_SETUP
-#line 379 "../SqlLexer.lpp"
+#line 381 "../SqlLexer.lpp"
 {
     /* Any other character (including actual newline or carriage return) */
     yylval->string_value_->push_back(yytext[1]);
   }
 	YY_BREAK
-case 146:
+case 147:
 YY_RULE_SETUP
-#line 383 "../SqlLexer.lpp"
+#line 385 "../SqlLexer.lpp"
 {
     /* This should only be encountered right before an EOF. */
     delete yylval->string_value_;
@@ -2271,17 +2288,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 147:
+case 148:
 YY_RULE_SETUP
-#line 393 "../SqlLexer.lpp"
+#line 395 "../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 148:
+case 149:
 YY_RULE_SETUP
-#line 397 "../SqlLexer.lpp"
+#line 399 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2290,17 +2307,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 149:
+case 150:
 YY_RULE_SETUP
-#line 405 "../SqlLexer.lpp"
+#line 407 "../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 150:
+case 151:
 YY_RULE_SETUP
-#line 409 "../SqlLexer.lpp"
+#line 411 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2308,94 +2325,94 @@ YY_RULE_SETUP
   }
 	YY_BREAK
 
-case 151:
-/* rule 151 can match eol */
+case 152:
+/* rule 152 can match eol */
 YY_RULE_SETUP
-#line 416 "../SqlLexer.lpp"
+#line 418 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 152:
-/* rule 152 can match eol */
+case 153:
+/* rule 153 can match eol */
 YY_RULE_SETUP
-#line 421 "../SqlLexer.lpp"
+#line 423 "../SqlLexer.lpp"
 {
   /* Scan up to a quote or escape sequence. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 153:
-/* rule 153 can match eol */
+case 154:
+/* rule 154 can match eol */
 YY_RULE_SETUP
-#line 426 "../SqlLexer.lpp"
+#line 428 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
 
-case 154:
+case 155:
 YY_RULE_SETUP
-#line 432 "../SqlLexer.lpp"
+#line 434 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(
         yylloc->first_line, yylloc->first_column, std::string(yytext, yyleng));
     return TOKEN_NAME;
   }
 	YY_BREAK
-case 155:
+case 156:
 YY_RULE_SETUP
-#line 438 "../SqlLexer.lpp"
+#line 440 "../SqlLexer.lpp"
 {
     yylval->numeric_literal_value_ = new quickstep::NumericParseLiteralValue(
         yylloc->first_line, yylloc->first_column, yytext);
     return TOKEN_UNSIGNED_NUMVAL;
   }
 	YY_BREAK
-case 156:
-YY_RULE_SETUP
-#line 444 "../SqlLexer.lpp"
-/* comment */
-	YY_BREAK
 case 157:
-/* rule 157 can match eol */
 YY_RULE_SETUP
 #line 446 "../SqlLexer.lpp"
-{ yycolumn = 0; }
+/* comment */
 	YY_BREAK
 case 158:
+/* rule 158 can match eol */
 YY_RULE_SETUP
 #line 448 "../SqlLexer.lpp"
+{ yycolumn = 0; }
+	YY_BREAK
+case 159:
+YY_RULE_SETUP
+#line 450 "../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 452 "../SqlLexer.lpp"
+#line 454 "../SqlLexer.lpp"
 {
   /* All conditions except for mutli-state string extracting conditions. */
   BEGIN(INITIAL);
   return TOKEN_EOF;
 }
 	YY_BREAK
-case 159:
+case 160:
 YY_RULE_SETUP
-#line 458 "../SqlLexer.lpp"
+#line 460 "../SqlLexer.lpp"
 {
   BEGIN(INITIAL);
   quickstep_yyerror(NULL, yyscanner, NULL, "illegal character");
   return TOKEN_LEX_ERROR;
 }
 	YY_BREAK
-case 160:
+case 161:
 YY_RULE_SETUP
-#line 464 "../SqlLexer.lpp"
+#line 466 "../SqlLexer.lpp"
 YY_FATAL_ERROR( "flex scanner jammed" );
 	YY_BREAK
-#line 2399 "SqlLexer_gen.cpp"
+#line 2416 "SqlLexer_gen.cpp"
 
 	case YY_END_OF_BUFFER:
 		{
@@ -2538,9 +2555,9 @@ YY_FATAL_ERROR( "flex scanner jammed" );
 static int yy_get_next_buffer (yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
-	register char *dest = YY_CURRENT_BUFFER_LVALUE->yy_ch_buf;
-	register char *source = yyg->yytext_ptr;
-	register int number_to_move, i;
+	char *dest = YY_CURRENT_BUFFER_LVALUE->yy_ch_buf;
+	char *source = yyg->yytext_ptr;
+	yy_size_t number_to_move, i;
 	int ret_val;
 
 	if ( yyg->yy_c_buf_p > &YY_CURRENT_BUFFER_LVALUE->yy_ch_buf[yyg->yy_n_chars + 1] )
@@ -2569,7 +2586,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 	/* Try to read more data. */
 
 	/* First move last chars to start of buffer. */
-	number_to_move = (int) (yyg->yy_c_buf_p - yyg->yytext_ptr) - 1;
+	number_to_move = (yy_size_t) (yyg->yy_c_buf_p - yyg->yytext_ptr) - 1;
 
 	for ( i = 0; i < number_to_move; ++i )
 		*(dest++) = *(source++);
@@ -2672,15 +2689,15 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 
     static yy_state_type yy_get_previous_state (yyscan_t yyscanner)
 {
-	register yy_state_type yy_current_state;
-	register char *yy_cp;
+	yy_state_type yy_current_state;
+	char *yy_cp;
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 
 	yy_current_state = yyg->yy_start;
 
 	for ( yy_cp = yyg->yytext_ptr + YY_MORE_ADJ; yy_cp < yyg->yy_c_buf_p; ++yy_cp )
 		{
-		register YY_CHAR yy_c = (*yy_cp ? yy_ec[YY_SC_TO_UI(*yy_cp)] : 1);
+		YY_CHAR yy_c = (*yy_cp ? yy_ec[YY_SC_TO_UI(*yy_cp)] : 1);
 		if ( yy_accept[yy_current_state] )
 			{
 			yyg->yy_last_accepting_state = yy_current_state;
@@ -2689,7 +2706,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 >= 588 )
+			if ( yy_current_state >= 593 )
 				yy_c = yy_meta[(unsigned int) yy_c];
 			}
 		yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
@@ -2705,11 +2722,11 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
  */
     static yy_state_type yy_try_NUL_trans  (yy_state_type yy_current_state , yyscan_t yyscanner)
 {
-	register int yy_is_jam;
+	int yy_is_jam;
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner; /* This var may be unused depending upon options. */
-	register char *yy_cp = yyg->yy_c_buf_p;
+	char *yy_cp = yyg->yy_c_buf_p;
 
-	register YY_CHAR yy_c = 1;
+	YY_CHAR yy_c = 1;
 	if ( yy_accept[yy_current_state] )
 		{
 		yyg->yy_last_accepting_state = yy_current_state;
@@ -2718,16 +2735,20 @@ 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 >= 588 )
+		if ( yy_current_state >= 593 )
 			yy_c = yy_meta[(unsigned int) yy_c];
 		}
 	yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
-	yy_is_jam = (yy_current_state == 587);
+	yy_is_jam = (yy_current_state == 592);
 
 	(void)yyg;
 	return yy_is_jam ? 0 : yy_current_state;
 }
 
+#ifndef YY_NO_UNPUT
+
+#endif
+
 #ifndef YY_NO_INPUT
 #ifdef __cplusplus
     static int yyinput (yyscan_t yyscanner)
@@ -2888,7 +2909,7 @@ static void quickstep_yy_load_buffer_state  (yyscan_t yyscanner)
 	if ( ! b )
 		YY_FATAL_ERROR( "out of dynamic memory in quickstep_yy_create_buffer()" );
 
-	b->yy_buf_size = size;
+	b->yy_buf_size = (yy_size_t)size;
 
 	/* yy_ch_buf has to be 2 characters longer than the size given because
 	 * we need to put in 2 end-of-buffer characters.
@@ -3049,7 +3070,7 @@ static void quickstep_yyensure_buffer_stack (yyscan_t yyscanner)
 		 * scanner will even need a stack. We use 2 instead of 1 to avoid an
 		 * immediate realloc on the next call.
          */
-		num_to_alloc = 1;
+		num_to_alloc = 1; // After all that talk, this was set to 1 anyways...
 		yyg->yy_buffer_stack = (struct yy_buffer_state**)quickstep_yyalloc
 								(num_to_alloc * sizeof(struct yy_buffer_state*)
 								, yyscanner);
@@ -3066,7 +3087,7 @@ static void quickstep_yyensure_buffer_stack (yyscan_t yyscanner)
 	if (yyg->yy_buffer_stack_top >= (yyg->yy_buffer_stack_max) - 1){
 
 		/* Increase the buffer to prepare for a possible push. */
-		int grow_size = 8 /* arbitrary grow size */;
+		yy_size_t grow_size = 8 /* arbitrary grow size */;
 
 		num_to_alloc = yyg->yy_buffer_stack_max + grow_size;
 		yyg->yy_buffer_stack = (struct yy_buffer_state**)quickstep_yyrealloc
@@ -3174,7 +3195,9 @@ YY_BUFFER_STATE quickstep_yy_scan_bytes  (yyconst char * yybytes, yy_size_t  _yy
 
 static void yy_fatal_error (yyconst char* msg , yyscan_t yyscanner)
 {
-    	(void) fprintf( stderr, "%s\n", msg );
+	struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
+	(void)yyg;
+	(void) fprintf( stderr, "%s\n", msg );
 	exit( YY_EXIT_FAILURE );
 }
 
@@ -3280,10 +3303,10 @@ void quickstep_yyset_extra (YY_EXTRA_TYPE  user_defined , yyscan_t yyscanner)
 }
 
 /** Set the current line number.
- * @param line_number
+ * @param _line_number line number
  * @param yyscanner The scanner object.
  */
-void quickstep_yyset_lineno (int  line_number , yyscan_t yyscanner)
+void quickstep_yyset_lineno (int  _line_number , yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 
@@ -3291,14 +3314,14 @@ void quickstep_yyset_lineno (int  line_number , yyscan_t yyscanner)
         if (! YY_CURRENT_BUFFER )
            YY_FATAL_ERROR( "quickstep_yyset_lineno called with no buffer" );
     
-    yylineno = line_number;
+    yylineno = _line_number;
 }
 
 /** Set the current column.
- * @param line_number
+ * @param _column_no column number
  * @param yyscanner The scanner object.
  */
-void quickstep_yyset_column (int  column_no , yyscan_t yyscanner)
+void quickstep_yyset_column (int  _column_no , yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 
@@ -3306,25 +3329,25 @@ void quickstep_yyset_column (int  column_no , yyscan_t yyscanner)
         if (! YY_CURRENT_BUFFER )
            YY_FATAL_ERROR( "quickstep_yyset_column called with no buffer" );
     
-    yycolumn = column_no;
+    yycolumn = _column_no;
 }
 
 /** Set the input stream. This does not discard the current
  * input buffer.
- * @param in_str A readable stream.
+ * @param _in_str A readable stream.
  * @param yyscanner The scanner object.
  * @see quickstep_yy_switch_to_buffer
  */
-void quickstep_yyset_in (FILE *  in_str , yyscan_t yyscanner)
+void quickstep_yyset_in (FILE *  _in_str , yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
-    yyin = in_str ;
+    yyin = _in_str ;
 }
 
-void quickstep_yyset_out (FILE *  out_str , yyscan_t yyscanner)
+void quickstep_yyset_out (FILE *  _out_str , yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
-    yyout = out_str ;
+    yyout = _out_str ;
 }
 
 int quickstep_yyget_debug  (yyscan_t yyscanner)
@@ -3333,10 +3356,10 @@ int quickstep_yyget_debug  (yyscan_t yyscanner)
     return yy_flex_debug;
 }
 
-void quickstep_yyset_debug (int  bdebug , yyscan_t yyscanner)
+void quickstep_yyset_debug (int  _bdebug , yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
-    yy_flex_debug = bdebug ;
+    yy_flex_debug = _bdebug ;
 }
 
 /* Accessor methods for yylval and yylloc */
@@ -3499,7 +3522,10 @@ int quickstep_yylex_destroy  (yyscan_t yyscanner)
 #ifndef yytext_ptr
 static void yy_flex_strncpy (char* s1, yyconst char * s2, int n , yyscan_t yyscanner)
 {
-	register int i;
+	struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
+	(void)yyg;
+
+	int i;
 	for ( i = 0; i < n; ++i )
 		s1[i] = s2[i];
 }
@@ -3508,7 +3534,7 @@ static void yy_flex_strncpy (char* s1, yyconst char * s2, int n , yyscan_t yysca
 #ifdef YY_NEED_STRLEN
 static int yy_flex_strlen (yyconst char * s , yyscan_t yyscanner)
 {
-	register int n;
+	int n;
 	for ( n = 0; s[n]; ++n )
 		;
 
@@ -3518,11 +3544,16 @@ static int yy_flex_strlen (yyconst char * s , yyscan_t yyscanner)
 
 void *quickstep_yyalloc (yy_size_t  size , yyscan_t yyscanner)
 {
+	struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
+	(void)yyg;
 	return (void *) malloc( size );
 }
 
 void *quickstep_yyrealloc  (void * ptr, yy_size_t  size , yyscan_t yyscanner)
 {
+	struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
+	(void)yyg;
+
 	/* The cast to (char *) in the following accommodates both
 	 * implementations that use char* generic pointers, and those
 	 * that use void* generic pointers.  It works with the latter
@@ -3535,12 +3566,14 @@ void *quickstep_yyrealloc  (void * ptr, yy_size_t  size , yyscan_t yyscanner)
 
 void quickstep_yyfree (void * ptr , yyscan_t yyscanner)
 {
+	struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
+	(void)yyg;
 	free( (char *) ptr );	/* see quickstep_yyrealloc() for (char *) cast */
 }
 
 #define YYTABLES_NAME "yytables"
 
-#line 463 "../SqlLexer.lpp"
+#line 466 "../SqlLexer.lpp"
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/preprocessed/SqlLexer_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.hpp b/parser/preprocessed/SqlLexer_gen.hpp
index f6cd8ad..0dd70e4 100644
--- a/parser/preprocessed/SqlLexer_gen.hpp
+++ b/parser/preprocessed/SqlLexer_gen.hpp
@@ -12,8 +12,8 @@
 
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
-#define YY_FLEX_MINOR_VERSION 5
-#define YY_FLEX_SUBMINOR_VERSION 39
+#define YY_FLEX_MINOR_VERSION 6
+#define YY_FLEX_SUBMINOR_VERSION 0
 #if YY_FLEX_SUBMINOR_VERSION > 0
 #define FLEX_BETA
 #endif
@@ -222,7 +222,7 @@ void quickstep_yyfree (void * ,yyscan_t yyscanner );
 
 /* Begin user sect3 */
 
-#define quickstep_yywrap(yyscanner) 1
+#define quickstep_yywrap(yyscanner) (/*CONSTCOND*/1)
 #define YY_SKIP_YYWRAP
 
 #define yytext_ptr yytext_r
@@ -268,11 +268,11 @@ void quickstep_yyset_extra (YY_EXTRA_TYPE user_defined ,yyscan_t yyscanner );
 
 FILE *quickstep_yyget_in (yyscan_t yyscanner );
 
-void quickstep_yyset_in  (FILE * in_str ,yyscan_t yyscanner );
+void quickstep_yyset_in  (FILE * _in_str ,yyscan_t yyscanner );
 
 FILE *quickstep_yyget_out (yyscan_t yyscanner );
 
-void quickstep_yyset_out  (FILE * out_str ,yyscan_t yyscanner );
+void quickstep_yyset_out  (FILE * _out_str ,yyscan_t yyscanner );
 
 yy_size_t quickstep_yyget_leng (yyscan_t yyscanner );
 
@@ -280,11 +280,11 @@ char *quickstep_yyget_text (yyscan_t yyscanner );
 
 int quickstep_yyget_lineno (yyscan_t yyscanner );
 
-void quickstep_yyset_lineno (int line_number ,yyscan_t yyscanner );
+void quickstep_yyset_lineno (int _line_number ,yyscan_t yyscanner );
 
 int quickstep_yyget_column  (yyscan_t yyscanner );
 
-void quickstep_yyset_column (int column_no ,yyscan_t yyscanner );
+void quickstep_yyset_column (int _column_no ,yyscan_t yyscanner );
 
 YYSTYPE * quickstep_yyget_lval (yyscan_t yyscanner );
 
@@ -360,7 +360,7 @@ extern int quickstep_yylex \
 #undef YY_DECL
 #endif
 
-#line 463 "../SqlLexer.lpp"
+#line 466 "../SqlLexer.lpp"
 
 
 #line 367 "SqlLexer_gen.hpp"


[47/50] [abbrv] incubator-quickstep git commit: Fixed the time measurement from milli to microseconds.

Posted by zu...@apache.org.
Fixed the time measurement from milli to microseconds.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 040a511aad35cb958d9d532fabc002313952cb11
Parents: b258821
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Sat Jul 2 15:58:52 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Sat Jul 2 15:58:52 2016 -0500

----------------------------------------------------------------------
 query_execution/Worker.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/040a511a/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index ae889c7..6ba27f1 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -121,7 +121,7 @@ void Worker::executeWorkOrderHelper(const TaggedMessage &tagged_message,
   end = std::chrono::steady_clock::now();
   delete worker_message.getWorkOrder();
   const uint64_t execution_time_microseconds =
-      std::chrono::duration_cast<std::chrono::milliseconds>(end - start)
+      std::chrono::duration_cast<std::chrono::microseconds>(end - start)
           .count();
   // Construct the proto message.
   proto->set_operator_index(worker_message.getRelationalOpIndex());


[25/50] [abbrv] incubator-quickstep git commit: Part I of a sequence of changes to bring the NOTICE and Copyright in line with ASF guidelines

Posted by zu...@apache.org.
Part I of a sequence of changes to bring the NOTICE and Copyright in line with ASF guidelines


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

Branch: refs/heads/tmp-relation-col-store
Commit: dd4495899e22437b9d64bbcb65edcbc7e25ce409
Parents: 29768a7
Author: Jignesh Patel <jm...@hotmail.com>
Authored: Tue Jun 14 12:31:19 2016 -0500
Committer: Jignesh Patel <jm...@hotmail.com>
Committed: Tue Jun 14 12:31:19 2016 -0500

----------------------------------------------------------------------
 NOTICE | 19 ++++++-------------
 1 file changed, 6 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dd449589/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 6395645..9cfd585 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,15 +1,8 @@
-QUICKSTEP
+Apache Quickstep (incubating)
+Copyright 2016 The Apache Software Foundation.
 
-Copyright (c) 2011-2016 Pivotal Software, Inc. All Rights Reserved.
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
 
-Licensed under the Apache License, Version 2.0 (the "License");
-you may not use this file except in compliance with the License.
-You may obtain a copy of the License at
-
-   http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
+Portions Copyright (c) 2011-2015, Quickstep Technologies, LLC.
+Portions Copyright (c) 2015-2016, Pivotal Software, Inc.


[36/50] [abbrv] incubator-quickstep git commit: Added PRIORITY clause in parser.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index fea31d6..0f66d1d 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -1,19 +1,19 @@
-/* A Bison parser, made by GNU Bison 3.0.4.  */
+/* A Bison parser, made by GNU Bison 2.7.  */
 
 /* Bison interface for Yacc-like parsers in C
-
-   Copyright (C) 1984, 1989-1990, 2000-2015 Free Software Foundation, Inc.
-
+   
+      Copyright (C) 1984, 1989-1990, 2000-2012 Free Software Foundation, Inc.
+   
    This program is free software: you can redistribute it and/or modify
    it under the terms of the GNU General Public License as published by
    the Free Software Foundation, either version 3 of the License, or
    (at your option) any later version.
-
+   
    This program is distributed in the hope that it will be useful,
    but WITHOUT ANY WARRANTY; without even the implied warranty of
    MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
    GNU General Public License for more details.
-
+   
    You should have received a copy of the GNU General Public License
    along with this program.  If not, see <http://www.gnu.org/licenses/>.  */
 
@@ -26,13 +26,13 @@
    special exception, which will cause the skeleton and the resulting
    Bison output files to be licensed under the GNU General Public
    License without this special exception.
-
+   
    This special exception was added by the Free Software Foundation in
    version 2.2 of Bison.  */
 
 #ifndef YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED
 # define YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED
-/* Debug traces.  */
+/* Enabling traces.  */
 #ifndef YYDEBUG
 # define YYDEBUG 0
 #endif
@@ -40,150 +40,152 @@
 extern int quickstep_yydebug;
 #endif
 
-/* Token type.  */
+/* Tokens.  */
 #ifndef YYTOKENTYPE
 # define YYTOKENTYPE
-  enum yytokentype
-  {
-    TOKEN_COMMAND = 258,
-    TOKEN_NAME = 259,
-    TOKEN_STRING_SINGLE_QUOTED = 260,
-    TOKEN_STRING_DOUBLE_QUOTED = 261,
-    TOKEN_UNSIGNED_NUMVAL = 262,
-    TOKEN_OR = 263,
-    TOKEN_AND = 264,
-    TOKEN_NOT = 265,
-    TOKEN_EQ = 266,
-    TOKEN_LT = 267,
-    TOKEN_LEQ = 268,
-    TOKEN_GT = 269,
-    TOKEN_GEQ = 270,
-    TOKEN_NEQ = 271,
-    TOKEN_LIKE = 272,
-    TOKEN_REGEXP = 273,
-    TOKEN_BETWEEN = 274,
-    TOKEN_IS = 275,
-    UNARY_PLUS = 276,
-    UNARY_MINUS = 277,
-    TOKEN_ADD = 278,
-    TOKEN_ALL = 279,
-    TOKEN_ALTER = 280,
-    TOKEN_AS = 281,
-    TOKEN_ASC = 282,
-    TOKEN_BIGINT = 283,
-    TOKEN_BIT = 284,
-    TOKEN_BITWEAVING = 285,
-    TOKEN_BLOCKPROPERTIES = 286,
-    TOKEN_BLOCKSAMPLE = 287,
-    TOKEN_BLOOM_FILTER = 288,
-    TOKEN_CSB_TREE = 289,
-    TOKEN_BY = 290,
-    TOKEN_CASE = 291,
-    TOKEN_CHARACTER = 292,
-    TOKEN_CHECK = 293,
-    TOKEN_COLUMN = 294,
-    TOKEN_CONSTRAINT = 295,
-    TOKEN_COPY = 296,
-    TOKEN_CREATE = 297,
-    TOKEN_CURRENT = 298,
-    TOKEN_DATE = 299,
-    TOKEN_DATETIME = 300,
-    TOKEN_DAY = 301,
-    TOKEN_DECIMAL = 302,
-    TOKEN_DEFAULT = 303,
-    TOKEN_DELETE = 304,
-    TOKEN_DELIMITER = 305,
-    TOKEN_DESC = 306,
-    TOKEN_DISTINCT = 307,
-    TOKEN_DOUBLE = 308,
-    TOKEN_DROP = 309,
-    TOKEN_ELSE = 310,
-    TOKEN_END = 311,
-    TOKEN_ESCAPE_STRINGS = 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_QUIT = 354,
-    TOKEN_RANGE = 355,
-    TOKEN_REAL = 356,
-    TOKEN_REFERENCES = 357,
-    TOKEN_RIGHT = 358,
-    TOKEN_ROW = 359,
-    TOKEN_ROW_DELIMITER = 360,
-    TOKEN_ROWS = 361,
-    TOKEN_SECOND = 362,
-    TOKEN_SELECT = 363,
-    TOKEN_SET = 364,
-    TOKEN_SMA = 365,
-    TOKEN_SMALLINT = 366,
-    TOKEN_SUBSTRING = 367,
-    TOKEN_TABLE = 368,
-    TOKEN_THEN = 369,
-    TOKEN_TIME = 370,
-    TOKEN_TIMESTAMP = 371,
-    TOKEN_TRUE = 372,
-    TOKEN_TUPLESAMPLE = 373,
-    TOKEN_UNBOUNDED = 374,
-    TOKEN_UNIQUE = 375,
-    TOKEN_UPDATE = 376,
-    TOKEN_USING = 377,
-    TOKEN_VALUES = 378,
-    TOKEN_VARCHAR = 379,
-    TOKEN_WHEN = 380,
-    TOKEN_WHERE = 381,
-    TOKEN_WINDOW = 382,
-    TOKEN_WITH = 383,
-    TOKEN_YEAR = 384,
-    TOKEN_YEARMONTH = 385,
-    TOKEN_EOF = 386,
-    TOKEN_LEX_ERROR = 387
-  };
+   /* Put the tokens into the symbol table, so that GDB and other debuggers
+      know about them.  */
+   enum yytokentype {
+     TOKEN_COMMAND = 258,
+     TOKEN_NAME = 259,
+     TOKEN_STRING_SINGLE_QUOTED = 260,
+     TOKEN_STRING_DOUBLE_QUOTED = 261,
+     TOKEN_UNSIGNED_NUMVAL = 262,
+     TOKEN_OR = 263,
+     TOKEN_AND = 264,
+     TOKEN_NOT = 265,
+     TOKEN_EQ = 266,
+     TOKEN_NEQ = 267,
+     TOKEN_GEQ = 268,
+     TOKEN_GT = 269,
+     TOKEN_LEQ = 270,
+     TOKEN_LT = 271,
+     TOKEN_REGEXP = 272,
+     TOKEN_LIKE = 273,
+     TOKEN_BETWEEN = 274,
+     TOKEN_IS = 275,
+     UNARY_MINUS = 276,
+     UNARY_PLUS = 277,
+     TOKEN_ADD = 278,
+     TOKEN_ALL = 279,
+     TOKEN_ALTER = 280,
+     TOKEN_AS = 281,
+     TOKEN_ASC = 282,
+     TOKEN_BIGINT = 283,
+     TOKEN_BIT = 284,
+     TOKEN_BITWEAVING = 285,
+     TOKEN_BLOCKPROPERTIES = 286,
+     TOKEN_BLOCKSAMPLE = 287,
+     TOKEN_BLOOM_FILTER = 288,
+     TOKEN_CSB_TREE = 289,
+     TOKEN_BY = 290,
+     TOKEN_CASE = 291,
+     TOKEN_CHARACTER = 292,
+     TOKEN_CHECK = 293,
+     TOKEN_COLUMN = 294,
+     TOKEN_CONSTRAINT = 295,
+     TOKEN_COPY = 296,
+     TOKEN_CREATE = 297,
+     TOKEN_CURRENT = 298,
+     TOKEN_DATE = 299,
+     TOKEN_DATETIME = 300,
+     TOKEN_DAY = 301,
+     TOKEN_DECIMAL = 302,
+     TOKEN_DEFAULT = 303,
+     TOKEN_DELETE = 304,
+     TOKEN_DELIMITER = 305,
+     TOKEN_DESC = 306,
+     TOKEN_DISTINCT = 307,
+     TOKEN_DOUBLE = 308,
+     TOKEN_DROP = 309,
+     TOKEN_ELSE = 310,
+     TOKEN_END = 311,
+     TOKEN_ESCAPE_STRINGS = 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_SUBSTRING = 368,
+     TOKEN_TABLE = 369,
+     TOKEN_THEN = 370,
+     TOKEN_TIME = 371,
+     TOKEN_TIMESTAMP = 372,
+     TOKEN_TRUE = 373,
+     TOKEN_TUPLESAMPLE = 374,
+     TOKEN_UNBOUNDED = 375,
+     TOKEN_UNIQUE = 376,
+     TOKEN_UPDATE = 377,
+     TOKEN_USING = 378,
+     TOKEN_VALUES = 379,
+     TOKEN_VARCHAR = 380,
+     TOKEN_WHEN = 381,
+     TOKEN_WHERE = 382,
+     TOKEN_WINDOW = 383,
+     TOKEN_WITH = 384,
+     TOKEN_YEAR = 385,
+     TOKEN_YEARMONTH = 386,
+     TOKEN_EOF = 387,
+     TOKEN_LEX_ERROR = 388
+   };
 #endif
 
-/* Value type.  */
-#if ! defined YYSTYPE && ! defined YYSTYPE_IS_DECLARED
 
-union YYSTYPE
+#if ! defined YYSTYPE && ! defined YYSTYPE_IS_DECLARED
+typedef union YYSTYPE
 {
-#line 119 "../SqlParser.ypp" /* yacc.c:1909  */
+/* Line 2058 of yacc.c  */
+#line 120 "../SqlParser.ypp"
 
   quickstep::ParseString *string_value_;
 
@@ -281,30 +283,43 @@ union YYSTYPE
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
 
-#line 285 "SqlParser_gen.hpp" /* yacc.c:1909  */
-};
+  quickstep::ParsePriority *opt_priority_clause_;
+
 
-typedef union YYSTYPE YYSTYPE;
+/* Line 2058 of yacc.c  */
+#line 291 "SqlParser_gen.hpp"
+} YYSTYPE;
 # define YYSTYPE_IS_TRIVIAL 1
+# define yystype YYSTYPE /* obsolescent; will be withdrawn */
 # define YYSTYPE_IS_DECLARED 1
 #endif
 
-/* Location type.  */
 #if ! defined YYLTYPE && ! defined YYLTYPE_IS_DECLARED
-typedef struct YYLTYPE YYLTYPE;
-struct YYLTYPE
+typedef struct YYLTYPE
 {
   int first_line;
   int first_column;
   int last_line;
   int last_column;
-};
+} YYLTYPE;
+# define yyltype YYLTYPE /* obsolescent; will be withdrawn */
 # define YYLTYPE_IS_DECLARED 1
 # define YYLTYPE_IS_TRIVIAL 1
 #endif
 
 
-
+#ifdef YYPARSE_PARAM
+#if defined __STDC__ || defined __cplusplus
+int quickstep_yyparse (void *YYPARSE_PARAM);
+#else
+int quickstep_yyparse ();
+#endif
+#else /* ! YYPARSE_PARAM */
+#if defined __STDC__ || defined __cplusplus
 int quickstep_yyparse (yyscan_t yyscanner, quickstep::ParseStatement **parsedStatement);
+#else
+int quickstep_yyparse ();
+#endif
+#endif /* ! YYPARSE_PARAM */
 
 #endif /* !YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED  */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/parser/tests/Select.test
----------------------------------------------------------------------
diff --git a/parser/tests/Select.test b/parser/tests/Select.test
index 79da7ee..464c48f 100644
--- a/parser/tests/Select.test
+++ b/parser/tests/Select.test
@@ -480,6 +480,49 @@ SELECT 1 FROM test LIMIT abc
                          ^
 ==
 
+# Priority
+SELECT 1 FROM test WITH PRIORITY 1
+--
+SelectStatement
++-select_query=Select
+| +-select_clause=SelectList
+| | +-SelectListItem
+| |   +-Literal
+| |     +-NumericLiteral[numeric_string=1,float_like=false]
+| +-from_clause=
+|   +-TableReference[table=test]
++-priority=PRIORITY
+  +-NumericLiteral[numeric_string=1,float_like=false]
+==
+
+SELECT 1 FROM test WITH PRIORITY 1.1
+--
+ERROR: PRIORITY value must be an integer (1 : 34)
+SELECT 1 FROM test WITH PRIORITY 1.1
+                                 ^
+==
+
+SELECT 1 FROM test WITH PRIORITY 0
+--
+ERROR: PRIORITY value must be positive (1 : 34)
+SELECT 1 FROM test WITH PRIORITY 0
+                                 ^
+==
+
+SELECT 1 FROM test WITH PRIORITY -1
+--
+ERROR: syntax error (1 : 34)
+SELECT 1 FROM test WITH PRIORITY -1
+                                 ^
+==
+
+SELECT 1 FROM test WITH PRIORITY abc
+--
+ERROR: syntax error (1 : 34)
+SELECT 1 FROM test WITH PRIORITY abc
+                                 ^
+==
+
 #
 # Subqueries
 #

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/query_optimizer/QueryHandle.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryHandle.hpp b/query_optimizer/QueryHandle.hpp
index a17d3e8..5f3649a 100644
--- a/query_optimizer/QueryHandle.hpp
+++ b/query_optimizer/QueryHandle.hpp
@@ -18,6 +18,7 @@
 #define QUICKSTEP_QUERY_OPTIMIZER_QUERY_HANDLE_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <utility>
 
@@ -44,8 +45,10 @@ class QueryHandle {
    *
    * @param query_id The given query id.
    */
-  explicit QueryHandle(const std::size_t query_id)
+  explicit QueryHandle(const std::size_t query_id,
+                       const std::uint64_t query_priority = 1)
       : query_id_(query_id),
+        query_priority_(query_priority),
         query_plan_(new QueryPlan()),
         query_result_relation_(nullptr) {}
 
@@ -61,6 +64,13 @@ class QueryHandle {
   }
 
   /**
+   * @brief Get the query priority.
+   **/
+  const std::uint64_t query_priority() const {
+    return query_priority_;
+  }
+
+  /**
    * @return The mutable query plan.
    */
   QueryPlan* getQueryPlanMutable() {
@@ -111,6 +121,7 @@ class QueryHandle {
 
  private:
   const std::size_t query_id_;
+  const std::uint64_t query_priority_;
 
   std::unique_ptr<QueryPlan> query_plan_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6428914/query_optimizer/QueryProcessor.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.cpp b/query_optimizer/QueryProcessor.cpp
index b6fa3e0..6381d3a 100644
--- a/query_optimizer/QueryProcessor.cpp
+++ b/query_optimizer/QueryProcessor.cpp
@@ -17,6 +17,7 @@
 
 #include "query_optimizer/QueryProcessor.hpp"
 
+#include <cstdint>
 #include <cstdlib>
 #include <fstream>
 #include <memory>
@@ -37,7 +38,8 @@ using std::ofstream;
 namespace quickstep {
 
 QueryHandle* QueryProcessor::generateQueryHandle(const ParseStatement &statement) {
-  std::unique_ptr<QueryHandle> query_handle(new QueryHandle(query_id_));
+  std::unique_ptr<QueryHandle> query_handle(
+      new QueryHandle(query_id_, statement.getPriority()));
 
   optimizer::Optimizer optimizer(query_id_, getDefaultDatabase(), storage_manager_.get());
   optimizer.generateQueryHandle(statement, query_handle.get());


[20/50] [abbrv] incubator-quickstep git commit: Long lived Foreman thread

Posted by zu...@apache.org.
Long lived Foreman thread

- Foreman thread lives through the lifetime of the Quickstep process.
- Foreman and main thread communicate through TMB messages.
- Foreman admits queries and routes them to PolicyEnforcer.
- Foreman relays messages to policy enforcer which in turn processes it,
  based on the query ID of the message.
- All the tests modified accordingly.

Created PolicyEnforcer class.

- First point of entry for queries in the scheduler.
- Can perform admission control.
- Can talk to the QueryManagers of the active queries to provide them
  messages to process and collect work orders for execution from them.
- Support for admitting multiple queries to the PolicyEnforcer.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 8230b12495297b6837a66485119da44d8fb95a26
Parents: 659967a
Author: Harshad Deshmukh <ha...@cs.wisc.edu>
Authored: Sat Apr 9 15:08:40 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Sun Jun 12 09:18:14 2016 -0500

----------------------------------------------------------------------
 CMakeLists.txt                                  |   2 +
 catalog/CatalogTypedefs.hpp                     |   3 +
 cli/CommandExecutor.cpp                         |  53 +-
 cli/CommandExecutor.hpp                         |  11 +-
 cli/QuickstepCli.cpp                            |  66 +-
 cli/tests/CMakeLists.txt                        |   2 +
 cli/tests/CommandExecutorTestRunner.cpp         |  29 +-
 cli/tests/CommandExecutorTestRunner.hpp         |  37 +-
 query_execution/AdmitRequestMessage.hpp         |  73 ++
 query_execution/CMakeLists.txt                  |  70 +-
 query_execution/Foreman.cpp                     | 578 +++--------
 query_execution/Foreman.hpp                     | 393 +-------
 query_execution/PolicyEnforcer.cpp              | 183 ++++
 query_execution/PolicyEnforcer.hpp              | 167 ++++
 query_execution/QueryContext.cpp                |  16 +-
 query_execution/QueryContext.proto              |   2 +
 query_execution/QueryExecutionMessages.proto    |   4 +
 query_execution/QueryExecutionTypedefs.hpp      |   5 +-
 query_execution/QueryExecutionUtil.hpp          |  52 +
 query_execution/QueryManager.hpp                |   5 +-
 query_execution/WorkOrdersContainer.hpp         |  70 +-
 query_execution/Worker.cpp                      |  11 +-
 query_execution/Worker.hpp                      |   2 +
 query_execution/WorkerMessage.hpp               |  24 +-
 query_execution/tests/Foreman_unittest.cpp      | 952 -------------------
 query_execution/tests/QueryManager_unittest.cpp |   7 +-
 .../tests/WorkOrdersContainer_unittest.cpp      |  26 +
 query_optimizer/ExecutionGenerator.hpp          |   1 +
 query_optimizer/tests/CMakeLists.txt            |   2 +
 .../tests/ExecutionGeneratorTestRunner.cpp      |  21 +-
 .../tests/ExecutionGeneratorTestRunner.hpp      |  33 +-
 query_optimizer/tests/TestDatabaseLoader.cpp    |   1 +
 relational_operators/DeleteOperator.cpp         |   1 +
 relational_operators/DeleteOperator.hpp         |   1 +
 relational_operators/HashJoinOperator.hpp       |   4 +-
 relational_operators/RebuildWorkOrder.hpp       |   1 +
 relational_operators/SortMergeRunOperator.cpp   |   1 +
 relational_operators/UpdateOperator.cpp         |   1 +
 relational_operators/UpdateOperator.hpp         |   1 +
 relational_operators/WorkOrder.hpp              |  20 +-
 .../tests/AggregationOperator_unittest.cpp      |   4 +
 .../tests/HashJoinOperator_unittest.cpp         |   6 +
 .../tests/SortMergeRunOperator_unittest.cpp     |   4 +
 .../SortRunGenerationOperator_unittest.cpp      |   1 +
 .../tests/TextScanOperator_unittest.cpp         |   1 +
 storage/InsertDestination.cpp                   |  50 +-
 storage/InsertDestination.hpp                   |  71 +-
 47 files changed, 1064 insertions(+), 2004 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 207f313..9e445f0 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -721,9 +721,11 @@ target_link_libraries(quickstep_cli_shell
                       quickstep_cli_PrintToScreen
                       quickstep_parser_ParseStatement
                       quickstep_parser_SqlParserWrapper
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_Foreman
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_Worker
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/catalog/CatalogTypedefs.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogTypedefs.hpp b/catalog/CatalogTypedefs.hpp
index 213d91d..44832b6 100644
--- a/catalog/CatalogTypedefs.hpp
+++ b/catalog/CatalogTypedefs.hpp
@@ -46,6 +46,9 @@ const int kCatalogMaxID = INT_MAX;
 // id for the catalog ids.
 constexpr int kInvalidCatalogId = -1;
 
+// Used to indicate no preference for a NUMA Node ID.
+constexpr numa_node_id kAnyNUMANodeID = -1;
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index ddcd38f..dc14741 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -51,6 +51,8 @@
 
 #include "glog/logging.h"
 
+#include "tmb/id_typedefs.h"
+
 using std::fprintf;
 using std::fputc;
 using std::fputs;
@@ -58,6 +60,8 @@ using std::size_t;
 using std::string;
 using std::vector;
 
+namespace tmb { class MessageBus; }
+
 namespace quickstep {
 namespace cli {
 namespace {
@@ -194,11 +198,14 @@ void executeDescribeTable(
 /**
  * @brief A helper function that executes a SQL query to obtain a scalar result.
  */
-inline TypedValue executeQueryForSingleResult(const std::string &query_string,
-                                               StorageManager *storage_manager,
-                                               QueryProcessor *query_processor,
-                                               SqlParserWrapper *parser_wrapper,
-                                               Foreman *foreman) {
+inline TypedValue executeQueryForSingleResult(
+    const tmb::client_id main_thread_client_id,
+    const tmb::client_id foreman_client_id,
+    const std::string &query_string,
+    tmb::MessageBus *bus,
+    StorageManager *storage_manager,
+    QueryProcessor *query_processor,
+    SqlParserWrapper *parser_wrapper) {
   parser_wrapper->feedNextBuffer(new std::string(query_string));
 
   ParseResult result = parser_wrapper->getNextStatement();
@@ -210,11 +217,8 @@ inline TypedValue executeQueryForSingleResult(const std::string &query_string,
   DCHECK(query_handle->getQueryPlanMutable() != nullptr);
 
   // Use foreman to execute the query plan.
-  foreman->setQueryPlan(query_handle->getQueryPlanMutable()->getQueryPlanDAGMutable());
-  foreman->reconstructQueryContextFromProto(query_handle->getQueryContextProto());
-
-  foreman->start();
-  foreman->join();
+  QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
+      main_thread_client_id, foreman_client_id, query_handle.get(), bus);
 
   // Retrieve the scalar result from the result relation.
   const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
@@ -246,8 +250,10 @@ inline TypedValue executeQueryForSingleResult(const std::string &query_string,
   return value;
 }
 
-void executeAnalyze(QueryProcessor *query_processor,
-                    Foreman *foreman,
+void executeAnalyze(const tmb::client_id main_thread_client_id,
+                    const tmb::client_id foreman_client_id,
+                    MessageBus *bus,
+                    QueryProcessor *query_processor,
                     FILE *out) {
   const CatalogDatabase &database = *query_processor->getDefaultDatabase();
   StorageManager *storage_manager = query_processor->getStorageManager();
@@ -273,11 +279,13 @@ void executeAnalyze(QueryProcessor *query_processor,
       query_string.append(";");
 
       TypedValue num_distinct_values =
-          executeQueryForSingleResult(query_string,
+          executeQueryForSingleResult(main_thread_client_id,
+                                      foreman_client_id,
+                                      query_string,
+                                      bus,
                                       storage_manager,
                                       query_processor,
-                                      parser_wrapper.get(),
-                                      foreman);
+                                      parser_wrapper.get());
 
       DCHECK(num_distinct_values.getTypeID() == TypeID::kLong);
       mutable_relation->getStatisticsMutable()->setNumDistinctValues(
@@ -291,11 +299,13 @@ void executeAnalyze(QueryProcessor *query_processor,
     query_string.append(";");
 
     TypedValue num_tuples =
-        executeQueryForSingleResult(query_string,
+        executeQueryForSingleResult(main_thread_client_id,
+                                    foreman_client_id,
+                                    query_string,
+                                    bus,
                                     storage_manager,
                                     query_processor,
-                                    parser_wrapper.get(),
-                                    foreman);
+                                    parser_wrapper.get());
 
     DCHECK(num_tuples.getTypeID() == TypeID::kLong);
     mutable_relation->getStatisticsMutable()->setNumTuples(
@@ -312,9 +322,11 @@ void executeAnalyze(QueryProcessor *query_processor,
 
 void executeCommand(const ParseStatement &statement,
                     const CatalogDatabase &catalog_database,
+                    const tmb::client_id main_thread_client_id,
+                    const tmb::client_id foreman_client_id,
+                    MessageBus *bus,
                     StorageManager *storage_manager,
                     QueryProcessor *query_processor,
-                    Foreman *foreman,
                     FILE *out) {
   const ParseCommand &command = static_cast<const ParseCommand &>(statement);
   const PtrVector<ParseString> *arguments = command.arguments();
@@ -328,7 +340,8 @@ void executeCommand(const ParseStatement &statement,
       executeDescribeTable(arguments, catalog_database, out);
     }
   } else if (command_str == C::kAnalyzeCommand) {
-    executeAnalyze(query_processor, foreman, out);
+    executeAnalyze(
+        main_thread_client_id, foreman_client_id, bus, query_processor, out);
   } else {
     THROW_SQL_ERROR_AT(command.command()) << "Invalid Command";
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/CommandExecutor.hpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.hpp b/cli/CommandExecutor.hpp
index c819981..3435aeb 100644
--- a/cli/CommandExecutor.hpp
+++ b/cli/CommandExecutor.hpp
@@ -21,10 +21,14 @@
 #include <cstdio>
 #include <string>
 
+#include "tmb/id_typedefs.h"
+
 using std::fprintf;
 using std::fputc;
 using std::string;
 
+namespace tmb { class MessageBus; }
+
 namespace quickstep {
 
 class CatalogDatabase;
@@ -53,6 +57,9 @@ constexpr char kAnalyzeCommand[] = "\\analyze";
   *
   * @param statement The parsed statement from the cli.
   * @param catalog_database The catalog information about the current database.
+  * @param main_thread_client_id The TMB client ID of the main thread.
+  * @param foreman_client_id The TMB client ID of the Foreman thread.
+  * @param bus A pointer to the TMB.
   * @param storage_manager The current StorageManager.
   * @param query_processor The query processor to generate plans for SQL queries.
   * @param foreman The foreman to execute query plans.
@@ -60,9 +67,11 @@ constexpr char kAnalyzeCommand[] = "\\analyze";
 */
 void executeCommand(const ParseStatement &statement,
                     const CatalogDatabase &catalog_database,
+                    const tmb::client_id main_thread_client_id,
+                    const tmb::client_id foreman_client_id,
+                    tmb::MessageBus *bus,
                     StorageManager *storage_manager,
                     QueryProcessor *query_processor,
-                    Foreman *foreman,
                     FILE *out);
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 558d6eb..fbe7e3b 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -53,8 +53,10 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include "cli/PrintToScreen.hpp"
 #include "parser/ParseStatement.hpp"
 #include "parser/SqlParserWrapper.hpp"
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/Foreman.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
 #include "query_execution/Worker.hpp"
 #include "query_execution/WorkerDirectory.hpp"
 #include "query_execution/WorkerMessage.hpp"
@@ -95,6 +97,7 @@ using std::string;
 using std::vector;
 
 using quickstep::Address;
+using quickstep::AdmitRequestMessage;
 using quickstep::CatalogRelation;
 using quickstep::DefaultsConfigurator;
 using quickstep::DropRelation;
@@ -107,6 +110,7 @@ using quickstep::ParseResult;
 using quickstep::ParseStatement;
 using quickstep::PrintToScreen;
 using quickstep::PtrVector;
+using quickstep::QueryExecutionUtil;
 using quickstep::QueryHandle;
 using quickstep::QueryPlan;
 using quickstep::QueryProcessor;
@@ -115,9 +119,12 @@ using quickstep::TaggedMessage;
 using quickstep::Worker;
 using quickstep::WorkerDirectory;
 using quickstep::WorkerMessage;
+using quickstep::kAdmitRequestMessage;
 using quickstep::kPoisonMessage;
+using quickstep::kWorkloadCompletionMessage;
 
 using tmb::client_id;
+using tmb::AnnotatedMessage;
 
 namespace quickstep {
 
@@ -197,7 +204,9 @@ int main(int argc, char* argv[]) {
 
   // The TMB client id for the main thread, used to kill workers at the end.
   const client_id main_thread_client_id = bus.Connect();
+  bus.RegisterClientAsSender(main_thread_client_id, kAdmitRequestMessage);
   bus.RegisterClientAsSender(main_thread_client_id, kPoisonMessage);
+  bus.RegisterClientAsReceiver(main_thread_client_id, kWorkloadCompletionMessage);
 
   // Setup the paths used by StorageManager.
   string fixed_storage_path(quickstep::FLAGS_storage_path);
@@ -283,12 +292,6 @@ int main(int argc, char* argv[]) {
            std::chrono::duration<double>(preload_end - preload_start).count());
   }
 
-  Foreman foreman(&bus,
-                  query_processor->getDefaultDatabase(),
-                  query_processor->getStorageManager(),
-                  -1, /* CPU id to bind foreman. -1 is unbound. */
-                  num_numa_nodes_system);
-
   // Get the NUMA affinities for workers.
   vector<int> cpu_numa_nodes = InputParserUtil::GetNUMANodesForCPUs();
   if (cpu_numa_nodes.empty()) {
@@ -323,13 +326,20 @@ int main(int argc, char* argv[]) {
                                    worker_client_ids,
                                    worker_numa_nodes);
 
-  foreman.setWorkerDirectory(&worker_directory);
+  Foreman foreman(main_thread_client_id,
+                  &worker_directory,
+                  &bus,
+                  query_processor->getDefaultDatabase(),
+                  query_processor->getStorageManager(),
+                  num_numa_nodes_system);
 
   // Start the worker threads.
   for (Worker &worker : workers) {
     worker.start();
   }
 
+  foreman.start();
+
   LineReaderImpl line_reader("quickstep> ",
                              "      ...> ");
   std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
@@ -366,9 +376,11 @@ int main(int argc, char* argv[]) {
             quickstep::cli::executeCommand(
                 *result.parsed_statement,
                 *(query_processor->getDefaultDatabase()),
+                main_thread_client_id,
+                foreman.getBusClientID(),
+                &bus,
                 query_processor->getStorageManager(),
                 query_processor.get(),
-                &foreman,
                 stdout);
           } catch (const quickstep::SqlError &sql_error) {
             fprintf(stderr, "%s",
@@ -389,14 +401,18 @@ int main(int argc, char* argv[]) {
         }
 
         DCHECK(query_handle->getQueryPlanMutable() != nullptr);
-        foreman.setQueryPlan(query_handle->getQueryPlanMutable()->getQueryPlanDAGMutable());
-
-        foreman.reconstructQueryContextFromProto(query_handle->getQueryContextProto());
+        start = std::chrono::steady_clock::now();
+        QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
+            main_thread_client_id,
+            foreman.getBusClientID(),
+            query_handle.get(),
+            &bus);
 
         try {
-          start = std::chrono::steady_clock::now();
-          foreman.start();
-          foreman.join();
+          const AnnotatedMessage annotated_msg =
+              bus.Receive(main_thread_client_id, 0, true);
+          const TaggedMessage &tagged_message = annotated_msg.tagged_message;
+          DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
           end = std::chrono::steady_clock::now();
 
           const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
@@ -440,29 +456,13 @@ int main(int argc, char* argv[]) {
     }
   }
 
-  // Terminate all workers before exiting.
-  // The main thread broadcasts poison message to the workers. Each worker dies
-  // after receiving poison message. The order of workers' death is irrelavant.
-  MessageStyle style;
-  style.Broadcast(true);
-  Address address;
-  address.All(true);
-  std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
-  TaggedMessage poison_tagged_message(poison_message.get(),
-                                      sizeof(*poison_message),
-                                      kPoisonMessage);
-
-  const tmb::MessageBus::SendStatus send_status =
-      bus.Send(main_thread_client_id,
-               address,
-               style,
-               std::move(poison_tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
-     "Broadcast message from Foreman to workers failed";
+  // Kill the foreman and workers.
+  QueryExecutionUtil::BroadcastPoisonMessage(main_thread_client_id, &bus);
 
   for (Worker &worker : workers) {
     worker.join();
   }
 
+  foreman.join();
   return 0;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/tests/CMakeLists.txt b/cli/tests/CMakeLists.txt
index ca37e4a..d177d6c 100644
--- a/cli/tests/CMakeLists.txt
+++ b/cli/tests/CMakeLists.txt
@@ -32,9 +32,11 @@ target_link_libraries(quickstep_cli_tests_CommandExecutorTest
                       quickstep_cli_PrintToScreen
                       quickstep_parser_ParseStatement
                       quickstep_parser_SqlParserWrapper
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_Foreman
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_Worker
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/tests/CommandExecutorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTestRunner.cpp b/cli/tests/CommandExecutorTestRunner.cpp
index 9cd493e..794f7e1 100644
--- a/cli/tests/CommandExecutorTestRunner.cpp
+++ b/cli/tests/CommandExecutorTestRunner.cpp
@@ -20,12 +20,15 @@
 #include <cstdio>
 #include <set>
 #include <string>
+#include <utility>
 
 #include "cli/CommandExecutor.hpp"
 #include "cli/DropRelation.hpp"
 #include "cli/PrintToScreen.hpp"
 #include "parser/ParseStatement.hpp"
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/Foreman.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/Worker.hpp"
 #include "query_optimizer/ExecutionGenerator.hpp"
 #include "query_optimizer/LogicalGenerator.hpp"
@@ -41,6 +44,8 @@
 
 #include "glog/logging.h"
 
+#include "tmb/tagged_message.h"
+
 namespace quickstep {
 
 class CatalogRelation;
@@ -87,9 +92,11 @@ void CommandExecutorTestRunner::runTestCase(
           quickstep::cli::executeCommand(
               *result.parsed_statement,
               *(test_database_loader_.catalog_database()),
+              main_thread_client_id_,
+              foreman_->getBusClientID(),
+              &bus_,
               test_database_loader_.storage_manager(),
               nullptr,
-              nullptr,
               output_stream.file());
         } else  {
           QueryHandle query_handle(optimizer_context.query_id());
@@ -100,14 +107,20 @@ void CommandExecutorTestRunner::runTestCase(
               physical_generator.generatePlan(
                   logical_generator.generatePlan(*result.parsed_statement));
           execution_generator.generatePlan(physical_plan);
-          foreman_->setQueryPlan(
-              query_handle.getQueryPlanMutable()->getQueryPlanDAGMutable());
-
-          foreman_->reconstructQueryContextFromProto(query_handle.getQueryContextProto());
-
-          foreman_->start();
-          foreman_->join();
 
+          AdmitRequestMessage request_message(&query_handle);
+          TaggedMessage admit_tagged_message(
+              &request_message, sizeof(request_message), kAdmitRequestMessage);
+          QueryExecutionUtil::SendTMBMessage(&bus_,
+                                             main_thread_client_id_,
+                                             foreman_->getBusClientID(),
+                                             std::move(admit_tagged_message));
+
+          // Receive workload completion message from Foreman.
+          const AnnotatedMessage annotated_msg =
+              bus_.Receive(main_thread_client_id_, 0, true);
+          const TaggedMessage &tagged_message = annotated_msg.tagged_message;
+          DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
           const CatalogRelation *query_result_relation = query_handle.getQueryResultRelation();
           if (query_result_relation) {
             PrintToScreen::PrintRelation(*query_result_relation,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/tests/CommandExecutorTestRunner.hpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTestRunner.hpp b/cli/tests/CommandExecutorTestRunner.hpp
index 94b1d6a..8fb5b65 100644
--- a/cli/tests/CommandExecutorTestRunner.hpp
+++ b/cli/tests/CommandExecutorTestRunner.hpp
@@ -27,6 +27,7 @@
 #include "parser/SqlParserWrapper.hpp"
 #include "query_execution/Foreman.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
 #include "query_execution/Worker.hpp"
 #include "query_execution/WorkerDirectory.hpp"
 #include "query_execution/WorkerMessage.hpp"
@@ -34,6 +35,9 @@
 #include "utility/Macros.hpp"
 #include "utility/textbased_test/TextBasedTestDriver.hpp"
 
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+
 namespace quickstep {
 
 /**
@@ -57,9 +61,11 @@ class CommandExecutorTestRunner : public TextBasedTestRunner {
 
     bus_.Initialize();
 
-    foreman_.reset(new Foreman(&bus_,
-                               test_database_loader_.catalog_database(),
-                               test_database_loader_.storage_manager()));
+    main_thread_client_id_ = bus_.Connect();
+    bus_.RegisterClientAsSender(main_thread_client_id_, kAdmitRequestMessage);
+    bus_.RegisterClientAsSender(main_thread_client_id_, kPoisonMessage);
+    bus_.RegisterClientAsReceiver(main_thread_client_id_, kWorkloadCompletionMessage);
+
     worker_.reset(new Worker(0, &bus_));
 
     std::vector<client_id> worker_client_ids;
@@ -71,27 +77,20 @@ class CommandExecutorTestRunner : public TextBasedTestRunner {
 
     workers_.reset(new WorkerDirectory(1 /* number of workers */,
                                        worker_client_ids, numa_nodes));
-    foreman_->setWorkerDirectory(workers_.get());
+    foreman_.reset(new Foreman(main_thread_client_id_,
+                               workers_.get(),
+                               &bus_,
+                               test_database_loader_.catalog_database(),
+                               test_database_loader_.storage_manager()));
 
+    foreman_->start();
     worker_->start();
   }
 
   ~CommandExecutorTestRunner() {
-    std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
-    TaggedMessage poison_tagged_message(poison_message.get(),
-                                        sizeof(*poison_message),
-                                        quickstep::kPoisonMessage);
-
-    Address worker_address;
-    MessageStyle single_receiver_style;
-
-    worker_address.AddRecipient(worker_->getBusClientID());
-    bus_.Send(foreman_->getBusClientID(),
-              worker_address,
-              single_receiver_style,
-              std::move(poison_tagged_message));
-
+    QueryExecutionUtil::BroadcastPoisonMessage(main_thread_client_id_, &bus_);
     worker_->join();
+    foreman_->join();
   }
 
   void runTestCase(const std::string &input,
@@ -102,6 +101,8 @@ class CommandExecutorTestRunner : public TextBasedTestRunner {
   SqlParserWrapper sql_parser_;
   optimizer::TestDatabaseLoader test_database_loader_;
 
+  tmb::client_id main_thread_client_id_;
+
   MessageBusImpl bus_;
   std::unique_ptr<Foreman> foreman_;
   std::unique_ptr<Worker> worker_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/AdmitRequestMessage.hpp
----------------------------------------------------------------------
diff --git a/query_execution/AdmitRequestMessage.hpp b/query_execution/AdmitRequestMessage.hpp
new file mode 100644
index 0000000..e33b354
--- /dev/null
+++ b/query_execution/AdmitRequestMessage.hpp
@@ -0,0 +1,73 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_
+
+#include <vector>
+
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class QueryHandle;
+
+/** \addtogroup QueryExecution
+ *  @{
+ */
+
+/**
+ * @brief A message requesting a query or queries to be admitted to the system.
+ **/
+class AdmitRequestMessage {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param query_handles The handles of the queries requesting to be admitted
+   *        to the system.
+   **/
+  explicit AdmitRequestMessage(const std::vector<QueryHandle*> &query_handles)
+      : query_handles_(query_handles) {}
+
+  /**
+   * @brief Constructor for requesting single query admission.
+   *
+   * @param query_handle The handle of the query requesting to be admitted.
+   **/
+  explicit AdmitRequestMessage(QueryHandle *query_handle) {
+    query_handles_.push_back(query_handle);
+  }
+
+  /**
+   * @brief Get the query handles from this message.
+   **/
+  const std::vector<QueryHandle*>& getQueryHandles() const {
+    return query_handles_;
+  }
+
+ private:
+  std::vector<QueryHandle*> query_handles_;
+
+  DISALLOW_COPY_AND_ASSIGN(AdmitRequestMessage);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 95bc0d6..323e4a9 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -29,8 +29,10 @@ endif()
 if (ENABLE_DISTRIBUTED)
   add_library(quickstep_queryexecution_BlockLocator BlockLocator.cpp BlockLocator.hpp)
 endif()
+add_library(quickstep_queryexecution_AdmitRequestMessage ../empty_src.cpp AdmitRequestMessage.hpp)
 add_library(quickstep_queryexecution_Foreman Foreman.cpp Foreman.hpp)
 add_library(quickstep_queryexecution_ForemanLite ../empty_src.cpp ForemanLite.hpp)
+add_library(quickstep_queryexecution_PolicyEnforcer PolicyEnforcer.cpp PolicyEnforcer.hpp)
 add_library(quickstep_queryexecution_QueryContext QueryContext.cpp QueryContext.hpp)
 add_library(quickstep_queryexecution_QueryContext_proto
             ${queryexecution_QueryContext_proto_srcs}
@@ -50,6 +52,8 @@ add_library(quickstep_queryexecution_WorkerMessage ../empty_src.cpp WorkerMessag
 add_library(quickstep_queryexecution_WorkerSelectionPolicy ../empty_src.cpp WorkerSelectionPolicy.hpp)
 
 # Link dependencies:
+target_link_libraries(quickstep_queryexecution_AdmitRequestMessage
+                      quickstep_utility_Macros)
 if (ENABLE_DISTRIBUTED)
   target_link_libraries(quickstep_queryexecution_BlockLocator
                         glog
@@ -64,29 +68,17 @@ if (ENABLE_DISTRIBUTED)
                         tmb)
 endif()
 target_link_libraries(quickstep_queryexecution_Foreman
+                      ${GFLAGS_LIB_NAME} 
                       glog
-                      gtest
-                      quickstep_catalog_CatalogDatabase
-                      quickstep_catalog_CatalogRelation
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_catalog_PartitionScheme
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_ForemanLite
-                      quickstep_queryexecution_QueryContext
-                      quickstep_queryexecution_QueryExecutionMessages_proto
-                      quickstep_queryexecution_QueryExecutionState
+                      quickstep_queryexecution_PolicyEnforcer
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
-                      quickstep_queryexecution_WorkOrdersContainer
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage
-                      quickstep_relationaloperators_RebuildWorkOrder
-                      quickstep_relationaloperators_RelationalOperator
-                      quickstep_relationaloperators_WorkOrder
-                      quickstep_storage_InsertDestination
-                      quickstep_storage_StorageBlock
-                      quickstep_storage_StorageBlockInfo
                       quickstep_threading_ThreadUtil
-                      quickstep_utility_DAG
+                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_queryexecution_ForemanLite
@@ -94,6 +86,18 @@ target_link_libraries(quickstep_queryexecution_ForemanLite
                       quickstep_threading_Thread
                       quickstep_utility_Macros
                       tmb)
+target_link_libraries(quickstep_queryexecution_PolicyEnforcer
+                      ${GFLAGS_LIB_NAME}
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_queryexecution_QueryExecutionMessages_proto
+                      quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_QueryManager
+                      quickstep_queryexecution_WorkerMessage
+                      quickstep_queryoptimizer_QueryHandle
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_utility_Macros
+                      tmb)
 target_link_libraries(quickstep_queryexecution_QueryContext
                       glog
                       quickstep_catalog_CatalogDatabaseLite
@@ -135,7 +139,9 @@ target_link_libraries(quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_threading_ThreadIDBasedMap
                       tmb)
 target_link_libraries(quickstep_queryexecution_QueryExecutionUtil
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_WorkerMessage
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_queryexecution_QueryManager
@@ -189,8 +195,10 @@ target_link_libraries(quickstep_queryexecution_WorkerSelectionPolicy
 # Module all-in-one library:
 add_library(quickstep_queryexecution ../empty_src.cpp QueryExecutionModule.hpp)
 target_link_libraries(quickstep_queryexecution
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_Foreman
                       quickstep_queryexecution_ForemanLite
+                      quickstep_queryexecution_PolicyEnforcer
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryContext_proto
                       quickstep_queryexecution_QueryExecutionMessages_proto
@@ -235,36 +243,6 @@ if (ENABLE_DISTRIBUTED)
   add_test(BlockLocator_unittest BlockLocator_unittest)
 endif()
 
-add_executable(Foreman_unittest
-  "${CMAKE_CURRENT_SOURCE_DIR}/tests/Foreman_unittest.cpp")
-target_link_libraries(Foreman_unittest
-                      glog
-                      gtest
-                      gtest_main
-                      quickstep_catalog_CatalogDatabase
-                      quickstep_catalog_CatalogRelation
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_queryexecution_Foreman
-                      quickstep_queryexecution_QueryContext
-                      quickstep_queryexecution_QueryContext_proto
-                      quickstep_queryexecution_QueryExecutionState
-                      quickstep_queryexecution_QueryExecutionTypedefs
-                      quickstep_queryexecution_WorkOrdersContainer
-                      quickstep_queryexecution_WorkerDirectory
-                      quickstep_queryexecution_WorkerMessage
-                      quickstep_queryoptimizer_QueryPlan
-                      quickstep_relationaloperators_RelationalOperator
-                      quickstep_relationaloperators_WorkOrder
-                      quickstep_storage_InsertDestination
-                      quickstep_storage_InsertDestination_proto
-                      quickstep_storage_StorageBlock
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageManager
-                      quickstep_utility_DAG
-                      quickstep_utility_Macros
-                      tmb)
-add_test(Foreman_unittest Foreman_unittest)
-
 add_executable(QueryManager_unittest
   "${CMAKE_CURRENT_SOURCE_DIR}/tests/QueryManager_unittest.cpp")
 target_link_libraries(QueryManager_unittest

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index 7705819..0577e20 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -22,355 +22,189 @@
 #include <utility>
 #include <vector>
 
-#include "catalog/CatalogDatabase.hpp"
-#include "catalog/CatalogRelation.hpp"
-#include "catalog/CatalogTypedefs.hpp"
-#include "catalog/PartitionScheme.hpp"
-#include "query_execution/QueryContext.hpp"
-#include "query_execution/QueryExecutionMessages.pb.h"
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/QueryExecutionUtil.hpp"
 #include "query_execution/WorkerDirectory.hpp"
 #include "query_execution/WorkerMessage.hpp"
-#include "relational_operators/RebuildWorkOrder.hpp"
-#include "relational_operators/RelationalOperator.hpp"
-#include "relational_operators/WorkOrder.hpp"
-#include "storage/InsertDestination.hpp"
-#include "storage/StorageBlock.hpp"
-#include "storage/StorageBlockInfo.hpp"
 #include "threading/ThreadUtil.hpp"
+#include "utility/EqualsAnyConstant.hpp"
 #include "utility/Macros.hpp"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
 
 #include "tmb/message_bus.h"
 #include "tmb/tagged_message.h"
 
 using std::move;
-using std::pair;
 using std::size_t;
+using std::unique_ptr;
 using std::vector;
 
 namespace quickstep {
 
-void Foreman::initialize() {
+DEFINE_uint64(min_load_per_worker, 2, "The minimum load defined as the number "
+              "of pending work orders for the worker. This information is used "
+              "by the Foreman to assign work orders to worker threads");
+
+Foreman::Foreman(const tmb::client_id main_thread_client_id,
+                 WorkerDirectory *worker_directory,
+                 tmb::MessageBus *bus,
+                 CatalogDatabaseLite *catalog_database,
+                 StorageManager *storage_manager,
+                 const int cpu_id,
+                 const size_t num_numa_nodes)
+    : ForemanLite(bus, cpu_id),
+      main_thread_client_id_(main_thread_client_id),
+      worker_directory_(DCHECK_NOTNULL(worker_directory)),
+      catalog_database_(DCHECK_NOTNULL(catalog_database)),
+      storage_manager_(DCHECK_NOTNULL(storage_manager)) {
+  const std::vector<QueryExecutionMessageType> sender_message_types{
+      kPoisonMessage,
+      kRebuildWorkOrderMessage,
+      kWorkOrderMessage,
+      kWorkloadCompletionMessage};
+
+  for (const auto message_type : sender_message_types) {
+    bus_->RegisterClientAsSender(foreman_client_id_, message_type);
+  }
+
+  const std::vector<QueryExecutionMessageType> receiver_message_types{
+      kAdmitRequestMessage,
+      kCatalogRelationNewBlockMessage,
+      kDataPipelineMessage,
+      kPoisonMessage,
+      kRebuildWorkOrderCompleteMessage,
+      kWorkOrderFeedbackMessage,
+      kWorkOrdersAvailableMessage,
+      kWorkOrderCompleteMessage};
+
+  for (const auto message_type : receiver_message_types) {
+    bus_->RegisterClientAsReceiver(foreman_client_id_, message_type);
+  }
+
+  policy_enforcer_.reset(new PolicyEnforcer(
+      foreman_client_id_,
+      num_numa_nodes,
+      catalog_database_,
+      storage_manager_,
+      bus_));
+}
+
+void Foreman::run() {
   if (cpu_id_ >= 0) {
     // We can pin the foreman thread to a CPU if specified.
     ThreadUtil::BindToCPU(cpu_id_);
   }
-  initializeState();
-
-  DEBUG_ASSERT(query_dag_ != nullptr);
-  const dag_node_index dag_size = query_dag_->size();
-
-  // Collect all the workorders from all the relational operators in the DAG.
-  for (dag_node_index index = 0; index < dag_size; ++index) {
-    if (checkAllBlockingDependenciesMet(index)) {
-      query_dag_->getNodePayloadMutable(index)->informAllBlockingDependenciesMet();
-      processOperator(index, false);
-    }
-  }
-
-  // Dispatch the WorkOrders generated so far.
-  dispatchWorkerMessages(0, 0);
-}
-
-void Foreman::processWorkOrderCompleteMessage(const dag_node_index op_index,
-                                              const size_t worker_thread_index) {
-  query_exec_state_->decrementNumQueuedWorkOrders(op_index);
-
-  // As the given worker finished executing a WorkOrder, decrement its number
-  // of queued WorkOrders.
-  workers_->decrementNumQueuedWorkOrders(worker_thread_index);
-
-  // Check if new work orders are available and fetch them if so.
-  fetchNormalWorkOrders(op_index);
-
-  if (checkRebuildRequired(op_index)) {
-    if (checkNormalExecutionOver(op_index)) {
-      if (!checkRebuildInitiated(op_index)) {
-        if (initiateRebuild(op_index)) {
-          // Rebuild initiated and completed right away.
-          markOperatorFinished(op_index);
-        } else {
-          // Rebuild under progress.
-        }
-      } else if (checkRebuildOver(op_index)) {
-        // Rebuild was under progress and now it is over.
-        markOperatorFinished(op_index);
-      }
-    } else {
-      // Normal execution under progress for this operator.
-    }
-  } else if (checkOperatorExecutionOver(op_index)) {
-    // Rebuild not required for this operator and its normal execution is
-    // complete.
-    markOperatorFinished(op_index);
-  }
-
-  for (const pair<dag_node_index, bool> &dependent_link :
-       query_dag_->getDependents(op_index)) {
-    const dag_node_index dependent_op_index = dependent_link.first;
-    if (checkAllBlockingDependenciesMet(dependent_op_index)) {
-      // Process the dependent operator (of the operator whose WorkOrder
-      // was just executed) for which all the dependencies have been met.
-      processOperator(dependent_op_index, true);
-    }
-  }
-
-  // Dispatch the WorkerMessages to the workers. We prefer to start the search
-  // for the schedulable WorkOrders beginning from 'op_index'. The first
-  // candidate worker to receive the next WorkOrder is the one that sent the
-  // response message to Foreman.
-  dispatchWorkerMessages(worker_thread_index, op_index);
-}
-
-void Foreman::processRebuildWorkOrderCompleteMessage(const dag_node_index op_index,
-                                                     const size_t worker_thread_index) {
-  query_exec_state_->decrementNumRebuildWorkOrders(op_index);
-  workers_->decrementNumQueuedWorkOrders(worker_thread_index);
-
-  if (checkRebuildOver(op_index)) {
-    markOperatorFinished(op_index);
-
-    for (const pair<dag_node_index, bool> &dependent_link :
-         query_dag_->getDependents(op_index)) {
-      const dag_node_index dependent_op_index = dependent_link.first;
-      if (checkAllBlockingDependenciesMet(dependent_op_index)) {
-        processOperator(dependent_op_index, true);
-      }
-    }
-  }
-
-  // Dispatch the WorkerMessages to the workers. We prefer to start the search
-  // for the schedulable WorkOrders beginning from 'op_index'. The first
-  // candidate worker to receive the next WorkOrder is the one that sent the
-  // response message to Foreman.
-  dispatchWorkerMessages(worker_thread_index, op_index);
-}
-
-void Foreman::processDataPipelineMessage(const dag_node_index op_index,
-                                         const block_id block,
-                                         const relation_id rel_id) {
-  for (const dag_node_index consumer_index :
-       output_consumers_[op_index]) {
-    // Feed the streamed block to the consumer. Note that 'output_consumers_'
-    // only contain those dependents of operator with index = op_index which are
-    // eligible to receive streamed input.
-    query_dag_->getNodePayloadMutable(consumer_index)->feedInputBlock(block, rel_id);
-    // Because of the streamed input just fed, check if there are any new
-    // WorkOrders available and if so, fetch them.
-    fetchNormalWorkOrders(consumer_index);
-  }
-
-  // Dispatch the WorkerMessages to the workers. We prefer to start the search
-  // for the schedulable WorkOrders beginning from 'op_index'. The first
-  // candidate worker to receive the next WorkOrder is the one that sent the
-  // response message to Foreman.
-  // TODO(zuyu): Improve the data locality for the next WorkOrder.
-  dispatchWorkerMessages(0, op_index);
-}
-
-void Foreman::processFeedbackMessage(const WorkOrder::FeedbackMessage &msg) {
-  RelationalOperator *op =
-      query_dag_->getNodePayloadMutable(msg.header().rel_op_index);
-  op->receiveFeedbackMessage(msg);
-}
-
-void Foreman::run() {
-  // Initialize before for Foreman eventloop.
-  initialize();
 
   // Event loop
-  while (!query_exec_state_->hasQueryExecutionFinished()) {
+  for (;;) {
     // Receive() causes this thread to sleep until next message is received.
-    AnnotatedMessage annotated_msg = bus_->Receive(foreman_client_id_, 0, true);
+    const AnnotatedMessage annotated_msg =
+        bus_->Receive(foreman_client_id_, 0, true);
     const TaggedMessage &tagged_message = annotated_msg.tagged_message;
-    switch (tagged_message.message_type()) {
-      case kWorkOrderCompleteMessage: {
-        serialization::WorkOrderCompletionMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        processWorkOrderCompleteMessage(proto.operator_index(), proto.worker_thread_index());
-        break;
-      }
-      case kRebuildWorkOrderCompleteMessage: {
-        serialization::WorkOrderCompletionMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        processRebuildWorkOrderCompleteMessage(proto.operator_index(), proto.worker_thread_index());
+    const tmb::message_type_id message_type = tagged_message.message_type();
+    switch (message_type) {
+      case kCatalogRelationNewBlockMessage:  // Fall through
+      case kDataPipelineMessage:
+      case kRebuildWorkOrderCompleteMessage:
+      case kWorkOrderCompleteMessage:
+      case kWorkOrderFeedbackMessage:
+      case kWorkOrdersAvailableMessage: {
+        policy_enforcer_->processMessage(tagged_message);
         break;
       }
-      case kCatalogRelationNewBlockMessage: {
-        serialization::CatalogRelationNewBlockMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        const block_id block = proto.block_id();
-
-        CatalogRelation *relation =
-            static_cast<CatalogDatabase*>(catalog_database_)->getRelationByIdMutable(proto.relation_id());
-        relation->addBlock(block);
-
-        if (proto.has_partition_id()) {
-          relation->getPartitionSchemeMutable()->addBlockToPartition(proto.partition_id(), block);
+      case kAdmitRequestMessage: {
+        const AdmitRequestMessage *msg =
+            static_cast<const AdmitRequestMessage *>(tagged_message.message());
+        const vector<QueryHandle *> &query_handles = msg->getQueryHandles();
+
+        DCHECK(!query_handles.empty());
+        bool all_queries_admitted = true;
+        if (query_handles.size() == 1u) {
+          all_queries_admitted =
+              policy_enforcer_->admitQuery(query_handles.front());
+        } else {
+          all_queries_admitted = policy_enforcer_->admitQueries(query_handles);
+        }
+        if (!all_queries_admitted) {
+          LOG(WARNING) << "The scheduler could not admit all the queries";
+          // TODO(harshad) - Inform the main thread about the failure.
         }
         break;
       }
-      case kDataPipelineMessage: {
-        // Possible message senders include InsertDestinations and some
-        // operators which modify existing blocks.
-        serialization::DataPipelineMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        processDataPipelineMessage(proto.operator_index(), proto.block_id(), proto.relation_id());
-        break;
-      }
-      case kWorkOrdersAvailableMessage: {
-        serialization::WorkOrdersAvailableMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        const dag_node_index op_index = proto.operator_index();
-
-        // Check if new work orders are available.
-        fetchNormalWorkOrders(op_index);
-
-        // Dispatch the WorkerMessages to the workers. We prefer to start the search
-        // for the schedulable WorkOrders beginning from 'op_index'. The first
-        // candidate worker to receive the next WorkOrder is the one that sent the
-        // response message to Foreman.
-        // TODO(zuyu): Improve the data locality for the next WorkOrder.
-        dispatchWorkerMessages(0, op_index);
-        break;
-      }
-      case kWorkOrderFeedbackMessage: {
-        WorkOrder::FeedbackMessage msg(const_cast<void *>(tagged_message.message()),
-                                       tagged_message.message_bytes());
-        processFeedbackMessage(msg);
-        break;
+      case kPoisonMessage: {
+        if (policy_enforcer_->hasQueries()) {
+          LOG(WARNING) << "Foreman thread exiting while some queries are "
+                          "under execution or waiting to be admitted";
+        }
+        return;
       }
       default:
         LOG(FATAL) << "Unknown message type to Foreman";
     }
-  }
-
-  // Clean up before exiting.
-  cleanUp();
-}
 
-void Foreman::dispatchWorkerMessages(
-    const size_t start_worker_index,
-    const dag_node_index start_operator_index) {
-  // Loop over all workers. Stopping criteria:
-  // 1. Every worker has been assigned exactly max_msgs_per_worker_ workorders.
-  // OR 2. No schedulable workorders at this time.
-  size_t done_workers_count = 0;
-  for (size_t curr_worker = start_worker_index;
-       done_workers_count < workers_->getNumWorkers();
-       curr_worker = (curr_worker + 1) % workers_->getNumWorkers()) {
-    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
-      std::unique_ptr<WorkerMessage> msg;
-      msg.reset(getNextWorkerMessage(
-          start_operator_index, workers_->getNUMANode(curr_worker)));
-      if (msg.get() != nullptr) {
-        sendWorkerMessage(curr_worker, *msg);
-        workers_->incrementNumQueuedWorkOrders(curr_worker);
-      } else {
-        // No schedulable workorder at this point.
-        ++done_workers_count;
-      }
-    } else {
-      // curr_worker already has been assigned max_msgs_per_worker workorders.
-      ++done_workers_count;
+    if (canCollectNewMessages(message_type)) {
+      vector<unique_ptr<WorkerMessage>> new_messages;
+      policy_enforcer_->getWorkerMessages(&new_messages);
+      dispatchWorkerMessages(new_messages);
+    }
+
+    // We check again, as some queries may produce zero work orders and finish
+    // their execution.
+    if (!policy_enforcer_->hasQueries()) {
+      // Signal the main thread that there are no queries to be executed.
+      // Currently the message doesn't have any real content.
+      const int dummy_payload = 0;
+      TaggedMessage completion_tagged_message(
+          &dummy_payload, sizeof(dummy_payload), kWorkloadCompletionMessage);
+      const tmb::MessageBus::SendStatus send_status =
+          QueryExecutionUtil::SendTMBMessage(
+              bus_,
+              foreman_client_id_,
+              main_thread_client_id_,
+              move(completion_tagged_message));
+      CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+          << "Message could not be sent from Foreman with TMB client ID "
+          << foreman_client_id_ << " to main thread with TMB client ID"
+          << main_thread_client_id_;
     }
   }
 }
 
-void Foreman::initializeState() {
-  const dag_node_index dag_size = query_dag_->size();
-
-  output_consumers_.resize(dag_size);
-  blocking_dependencies_.resize(dag_size);
-
-  query_exec_state_.reset(new QueryExecutionState(dag_size));
-  workorders_container_.reset(new WorkOrdersContainer(dag_size, num_numa_nodes_));
-
-  for (dag_node_index node_index = 0; node_index < dag_size; ++node_index) {
-    const QueryContext::insert_destination_id insert_destination_index =
-        query_dag_->getNodePayload(node_index).getInsertDestinationID();
-    if (insert_destination_index != QueryContext::kInvalidInsertDestinationId) {
-      // Rebuild is necessary whenever InsertDestination is present.
-      query_exec_state_->setRebuildRequired(node_index);
-      query_exec_state_->setRebuildStatus(node_index, 0, false);
-    }
-
-    for (const pair<dag_node_index, bool> &dependent_link :
-         query_dag_->getDependents(node_index)) {
-      const dag_node_index dependent_op_index = dependent_link.first;
-      if (!query_dag_->getLinkMetadata(node_index, dependent_op_index)) {
-        // The link is not a pipeline-breaker. Streaming of blocks is possible
-        // between these two operators.
-        output_consumers_[node_index].push_back(dependent_op_index);
-      } else {
-        // The link is a pipeline-breaker. Streaming of blocks is not possible
-        // between these two operators.
-        blocking_dependencies_[dependent_op_index].push_back(node_index);
-      }
-    }
+bool Foreman::canCollectNewMessages(const tmb::message_type_id message_type) {
+  if (QUICKSTEP_EQUALS_ANY_CONSTANT(message_type,
+                                    kCatalogRelationNewBlockMessage,
+                                    kWorkOrderFeedbackMessage)) {
+    return false;
+  } else if (worker_directory_->getLeastLoadedWorker().second <=
+             FLAGS_min_load_per_worker) {
+    // If the least loaded worker has only one pending work order, we should
+    // collect new messages and dispatch them.
+    return true;
+  } else {
+    return false;
   }
 }
 
-// TODO(harshad) : The default policy may execute remote WorkOrders for an
-// operator with a lower index even when there are local WorkOrders available for
-// an operator with higher index. We should examine if avoiding this behavior
-// has any benefits with respect to execution time and/or memory pressure.
-WorkerMessage* Foreman::getNextWorkerMessage(
-    const dag_node_index start_operator_index, const int numa_node) {
-  // Default policy: Operator with lowest index first.
-  WorkOrder *work_order = nullptr;
-  size_t num_operators_checked = 0;
-  for (dag_node_index index = start_operator_index;
-       num_operators_checked < query_dag_->size();
-       index = (index + 1) % query_dag_->size(), ++num_operators_checked) {
-    if (query_exec_state_->hasExecutionFinished(index)) {
-      continue;
-    }
-    if (numa_node != -1) {
-      // First try to get a normal WorkOrder from the specified NUMA node.
-      work_order = workorders_container_->getNormalWorkOrderForNUMANode(index, numa_node);
-      if (work_order != nullptr) {
-        // A WorkOrder found on the given NUMA node.
-        query_exec_state_->incrementNumQueuedWorkOrders(index);
-        return WorkerMessage::WorkOrderMessage(work_order, index);
-      } else {
-        // Normal workorder not found on this node. Look for a rebuild workorder
-        // on this NUMA node.
-        work_order = workorders_container_->getRebuildWorkOrderForNUMANode(index, numa_node);
-        if (work_order != nullptr) {
-          return WorkerMessage::RebuildWorkOrderMessage(work_order, index);
-        }
-      }
-    }
-    // Either no workorder found on the given NUMA node, or numa_node is -1.
-    // Try to get a normal WorkOrder from other NUMA nodes.
-    work_order = workorders_container_->getNormalWorkOrder(index);
-    if (work_order != nullptr) {
-      query_exec_state_->incrementNumQueuedWorkOrders(index);
-      return WorkerMessage::WorkOrderMessage(work_order, index);
+void Foreman::dispatchWorkerMessages(const vector<unique_ptr<WorkerMessage>> &messages) {
+  for (const auto &message : messages) {
+    DCHECK(message != nullptr);
+    const int recipient_worker_thread_index = message->getRecipientHint();
+    if (recipient_worker_thread_index != WorkerMessage::kInvalidRecipientIndexHint) {
+      sendWorkerMessage(static_cast<size_t>(recipient_worker_thread_index),
+                        *message);
     } else {
-      // Normal WorkOrder not found, look for a RebuildWorkOrder.
-      work_order = workorders_container_->getRebuildWorkOrder(index);
-      if (work_order != nullptr) {
-        return WorkerMessage::RebuildWorkOrderMessage(work_order, index);
-      }
+      sendWorkerMessage(worker_directory_->getLeastLoadedWorker().first,
+                        *message);
     }
   }
-  // No WorkOrders available right now.
-  return nullptr;
 }
 
-void Foreman::sendWorkerMessage(const std::size_t worker_thread_index,
+void Foreman::sendWorkerMessage(const size_t worker_thread_index,
                                 const WorkerMessage &message) {
-  message_type_id type;
+  tmb::message_type_id type;
   if (message.getType() == WorkerMessage::WorkerMessageType::kRebuildWorkOrder) {
     type = kRebuildWorkOrderMessage;
   } else if (message.getType() == WorkerMessage::WorkerMessageType::kWorkOrder) {
@@ -383,152 +217,12 @@ void Foreman::sendWorkerMessage(const std::size_t worker_thread_index,
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          foreman_client_id_,
-                                         workers_->getClientID(worker_thread_index),
+                                         worker_directory_->getClientID(worker_thread_index),
                                          move(worker_tagged_message));
   CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
       "Message could not be sent from Foreman with TMB client ID "
       << foreman_client_id_ << " to Foreman with TMB client ID "
-      << workers_->getClientID(worker_thread_index);
-}
-
-bool Foreman::fetchNormalWorkOrders(const dag_node_index index) {
-  bool generated_new_workorders = false;
-  if (!query_exec_state_->hasDoneGenerationWorkOrders(index)) {
-    // Do not fetch any work units until all blocking dependencies are met.
-    // The releational operator is not aware of blocking dependencies for
-    // uncorrelated scalar queries.
-    if (!checkAllBlockingDependenciesMet(index)) {
-      return false;
-    }
-    const size_t num_pending_workorders_before =
-        workorders_container_->getNumNormalWorkOrders(index);
-    const bool done_generation =
-        query_dag_->getNodePayloadMutable(index)->getAllWorkOrders(workorders_container_.get(),
-                                                                   query_context_.get(),
-                                                                   storage_manager_,
-                                                                   foreman_client_id_,
-                                                                   bus_);
-    if (done_generation) {
-      query_exec_state_->setDoneGenerationWorkOrders(index);
-    }
-
-    // TODO(shoban): It would be a good check to see if operator is making
-    // useful progress, i.e., the operator either generates work orders to
-    // execute or still has pending work orders executing. However, this will not
-    // work if Foreman polls operators without feeding data. This check can be
-    // enabled, if Foreman is refactored to call getAllWorkOrders() only when
-    // pending work orders are completed or new input blocks feed.
-
-    generated_new_workorders =
-        (num_pending_workorders_before <
-         workorders_container_->getNumNormalWorkOrders(index));
-  }
-  return generated_new_workorders;
-}
-
-void Foreman::processOperator(const dag_node_index index,
-                              const bool recursively_check_dependents) {
-  if (fetchNormalWorkOrders(index)) {
-    // Fetched work orders. Return to wait for the generated work orders to
-    // execute, and skip the execution-finished checks.
-    return;
-  }
-
-  if (checkNormalExecutionOver(index)) {
-    if (checkRebuildRequired(index)) {
-      if (!checkRebuildInitiated(index)) {
-        // Rebuild hasn't started, initiate it.
-        if (initiateRebuild(index)) {
-          // Rebuild initiated and completed right away.
-          markOperatorFinished(index);
-        } else {
-          // Rebuild WorkOrders have been generated.
-          return;
-        }
-      } else if (checkRebuildOver(index)) {
-        // Rebuild had been initiated and it is over.
-        markOperatorFinished(index);
-      }
-    } else {
-      // Rebuild is not required and normal execution over, mark finished.
-      markOperatorFinished(index);
-    }
-    // If we reach here, that means the operator has been marked as finished.
-    if (recursively_check_dependents) {
-      for (const pair<dag_node_index, bool> &dependent_link :
-           query_dag_->getDependents(index)) {
-        const dag_node_index dependent_op_index = dependent_link.first;
-        if (checkAllBlockingDependenciesMet(dependent_op_index)) {
-          processOperator(dependent_op_index, true);
-        }
-      }
-    }
-  }
-}
-
-void Foreman::markOperatorFinished(const dag_node_index index) {
-  query_exec_state_->setExecutionFinished(index);
-
-  RelationalOperator *op = query_dag_->getNodePayloadMutable(index);
-  op->updateCatalogOnCompletion();
-
-  const relation_id output_rel = op->getOutputRelationID();
-  for (const pair<dag_node_index, bool> &dependent_link : query_dag_->getDependents(index)) {
-    const dag_node_index dependent_op_index = dependent_link.first;
-    RelationalOperator *dependent_op = query_dag_->getNodePayloadMutable(dependent_op_index);
-    // Signal dependent operator that current operator is done feeding input blocks.
-    if (output_rel >= 0) {
-      dependent_op->doneFeedingInputBlocks(output_rel);
-    }
-    if (checkAllBlockingDependenciesMet(dependent_op_index)) {
-      dependent_op->informAllBlockingDependenciesMet();
-    }
-  }
-}
-
-bool Foreman::initiateRebuild(const dag_node_index index) {
-  DEBUG_ASSERT(!workorders_container_->hasRebuildWorkOrder(index));
-  DEBUG_ASSERT(checkRebuildRequired(index));
-  DEBUG_ASSERT(!checkRebuildInitiated(index));
-
-  getRebuildWorkOrders(index, workorders_container_.get());
-
-  query_exec_state_->setRebuildStatus(
-      index, workorders_container_->getNumRebuildWorkOrders(index), true);
-
-  return (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
-}
-
-void Foreman::getRebuildWorkOrders(const dag_node_index index, WorkOrdersContainer *container) {
-  const RelationalOperator &op = query_dag_->getNodePayload(index);
-  const QueryContext::insert_destination_id insert_destination_index = op.getInsertDestinationID();
-
-  if (insert_destination_index == QueryContext::kInvalidInsertDestinationId) {
-    return;
-  }
-
-  vector<MutableBlockReference> partially_filled_block_refs;
-
-  DCHECK(query_context_ != nullptr);
-  InsertDestination *insert_destination = query_context_->getInsertDestination(insert_destination_index);
-  DCHECK(insert_destination != nullptr);
-
-  insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs);
-
-  for (vector<MutableBlockReference>::size_type i = 0;
-       i < partially_filled_block_refs.size();
-       ++i) {
-    // Note: The query ID used below is dummy for now, it will be replaced with
-    // the true query ID when QueryManager gets used in Foreman.
-    container->addRebuildWorkOrder(
-        new RebuildWorkOrder(0,
-                             move(partially_filled_block_refs[i]),
-                             index,
-                             op.getOutputRelationID(),
-                             foreman_client_id_,
-                             bus_),
-        index);
-  }
+      << worker_directory_->getClientID(worker_thread_index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/Foreman.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.hpp b/query_execution/Foreman.hpp
index 2d6e0d3..94cb9fc 100644
--- a/query_execution/Foreman.hpp
+++ b/query_execution/Foreman.hpp
@@ -22,22 +22,11 @@
 #include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/ForemanLite.hpp"
-#include "query_execution/QueryContext.hpp"
-#include "query_execution/QueryExecutionState.hpp"
-#include "query_execution/QueryExecutionTypedefs.hpp"
-#include "query_execution/WorkOrdersContainer.hpp"
-#include "query_execution/WorkerMessage.hpp"
-#include "relational_operators/RelationalOperator.hpp"
-#include "relational_operators/WorkOrder.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "utility/DAG.hpp"
+#include "query_execution/PolicyEnforcer.hpp"
 #include "utility/Macros.hpp"
 
-#include "glog/logging.h"
-#include "gtest/gtest_prod.h"
-
+#include "tmb/id_typedefs.h"
 #include "tmb/message_bus.h"
 
 namespace quickstep {
@@ -45,23 +34,24 @@ namespace quickstep {
 class CatalogDatabaseLite;
 class StorageManager;
 class WorkerDirectory;
-
-namespace serialization { class QueryContext; }
+class WorkerMessage;
 
 /** \addtogroup QueryExecution
  *  @{
  */
 
 /**
- * @brief The Foreman scans the query DAG, requests each operator to produce
- *        workorders. It also pipelines the intermediate output it receives to
- *        the relational operators which need it.
+ * @brief The Foreman receives queries from the main thread, messages from the
+ *        policy enforcer and dispatches the work to worker threads. It also
+ *        receives work completion messages from workers.
  **/
 class Foreman final : public ForemanLite {
  public:
   /**
    * @brief Constructor.
    *
+   * @param main_thread_client_id The TMB client ID of the main thread.
+   * @param worker_directory The worker directory.
    * @param bus A pointer to the TMB.
    * @param catalog_database The catalog database where this query is executed.
    * @param storage_manager The StorageManager to use.
@@ -71,233 +61,28 @@ class Foreman final : public ForemanLite {
    * @note If cpu_id is not specified, Foreman thread can be possibly moved
    *       around on different CPUs by the OS.
   **/
-  Foreman(tmb::MessageBus *bus,
+  Foreman(const tmb::client_id main_thread_client_id,
+          WorkerDirectory *worker_directory,
+          tmb::MessageBus *bus,
           CatalogDatabaseLite *catalog_database,
           StorageManager *storage_manager,
           const int cpu_id = -1,
-          const int num_numa_nodes = 1)
-      : ForemanLite(bus, cpu_id),
-        catalog_database_(DCHECK_NOTNULL(catalog_database)),
-        storage_manager_(DCHECK_NOTNULL(storage_manager)),
-        max_msgs_per_worker_(1),
-        num_numa_nodes_(num_numa_nodes) {
-    bus_->RegisterClientAsSender(foreman_client_id_, kWorkOrderMessage);
-    bus_->RegisterClientAsSender(foreman_client_id_, kRebuildWorkOrderMessage);
-    // NOTE : Foreman thread sends poison messages in the optimizer's
-    // ExecutionGeneratorTest.
-    bus_->RegisterClientAsSender(foreman_client_id_, kPoisonMessage);
-
-    bus_->RegisterClientAsReceiver(foreman_client_id_,
-                                   kWorkOrderCompleteMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_,
-                                   kRebuildWorkOrderCompleteMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_, kCatalogRelationNewBlockMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_, kDataPipelineMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_,
-                                   kWorkOrdersAvailableMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_,
-                                   kWorkOrderFeedbackMessage);
-  }
+          const std::size_t num_numa_nodes = 1);
 
   ~Foreman() override {}
 
-  /**
-   * @brief Set the Query plan DAG for the query to be executed.
-   *
-   * @param query_plan_dag A pointer to the query plan DAG.
-   **/
-  inline void setQueryPlan(DAG<RelationalOperator, bool> *query_plan_dag) {
-    query_dag_ = query_plan_dag;
-  }
-
-  /**
-   * @brief Reconstruct the QueryContext for the query to be executed.
-   *
-   * @param proto The serialized QueryContext.
-   **/
-  inline void reconstructQueryContextFromProto(const serialization::QueryContext &proto) {
-    query_context_.reset(
-        new QueryContext(proto, *catalog_database_, storage_manager_, foreman_client_id_, bus_));
-  }
-
-  /**
-   * @brief Set the WorkerDirectory pointer.
-   *
-   * @param workers A pointer to the WorkerDirectory.
-   **/
-  void setWorkerDirectory(WorkerDirectory *workers) {
-    workers_ = workers;
-  }
-
-  /**
-   * @brief Set the maximum number of messages that should be allocated to each
-   *        worker during a single round of WorkOrder dispatch.
-   *
-   * @param max_msgs_per_worker Maximum number of messages.
-   **/
-  void setMaxMessagesPerWorker(const std::size_t max_msgs_per_worker) {
-    max_msgs_per_worker_ = max_msgs_per_worker;
-  }
-
  protected:
-  /**
-   * @brief The foreman receives a DAG of relational operators, asks relational
-   *        operators to produce the workorders and based on the response it gets
-   *        pipelines the intermediate output to dependent relational operators.
-   *
-   * @note  The workers who get the messages from the Foreman execute and
-   *        subsequently delete the WorkOrder contained in the message.
-   **/
   void run() override;
 
  private:
-  typedef DAG<RelationalOperator, bool>::size_type_nodes dag_node_index;
-
-  /**
-   * @brief Check if all the dependencies of the node at specified index have
-   *        finished their execution.
-   *
-   * @note This function's true return value is a pre-requisite for calling
-   *       getRebuildWorkOrders()
-   *
-   * @param node_index The index of the specified node in the query DAG.
-   *
-   * @return True if all the dependencies have finished their execution. False
-   *         otherwise.
-   **/
-  inline bool checkAllDependenciesMet(const dag_node_index node_index) const {
-    for (const dag_node_index dependency_index : query_dag_->getDependencies(node_index)) {
-      // If at least one of the dependencies is not met, return false.
-      if (!query_exec_state_->hasExecutionFinished(dependency_index)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * @brief Check if all the blocking dependencies of the node at specified
-   *        index have finished their execution.
-   *
-   * @note A blocking dependency is the one which is pipeline breaker. Output of
-   *       a dependency can't be streamed to its dependent if the link between
-   *       them is pipeline breaker.
-   *
-   * @param node_index The index of the specified node in the query DAG.
-   *
-   * @return True if all the blocking dependencies have finished their
-   *         execution. False otherwise.
-   **/
-  inline bool checkAllBlockingDependenciesMet(const dag_node_index node_index) const {
-    for (const dag_node_index blocking_dependency_index : blocking_dependencies_[node_index]) {
-      if (!query_exec_state_->hasExecutionFinished(blocking_dependency_index)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
   /**
    * @brief Dispatch schedulable WorkOrders, wrapped in WorkerMessages to the
    *        worker threads.
    *
-   * @param start_worker_index The dispatch of WorkOrders preferably begins with
-   *        the worker at this index.
-   * @param start_operator_index The search for a schedulable WorkOrder
-   *        begins with the WorkOrders generated by this operator.
-   **/
-  void dispatchWorkerMessages(const std::size_t start_worker_index,
-                              const dag_node_index start_operator_index);
-
-  /**
-   * @brief Initialize all the local vectors and maps. If the operator has an
-   *        InsertDestination, pass the bus address and Foreman's TMB client ID
-   *        to it.
-   **/
-  void initializeState();
-
-  /**
-   * @brief Initialize the Foreman before starting the event loop. This binds
-   * the Foreman thread to configured CPU, and does initial processing of
-   * operator before waiting for events from Workers.
-   **/
-  void initialize();
-
-  /**
-   * @brief Process the received WorkOrder complete message.
-   *
-   * @param node_index The index of the specified operator node in the query DAG
-   *        for the completed WorkOrder.
-   * @param worker_thread_index The logical index of the worker thread in
-   *        WorkerDirectory for the completed WorkOrder.
-   **/
-  void processWorkOrderCompleteMessage(const dag_node_index op_index,
-                                       const std::size_t worker_thread_index);
-
-  /**
-   * @brief Process the received RebuildWorkOrder complete message.
-   *
-   * @param node_index The index of the specified operator node in the query DAG
-   *        for the completed RebuildWorkOrder.
-   * @param worker_thread_index The logical index of the worker thread in
-   *        WorkerDirectory for the completed RebuildWorkOrder.
-   **/
-  void processRebuildWorkOrderCompleteMessage(const dag_node_index op_index,
-                                              const std::size_t worker_thread_index);
-
-  /**
-   * @brief Process the received data pipeline message.
-   *
-   * @param node_index The index of the specified operator node in the query DAG
-   *        for the pipelining block.
-   * @param block The block id.
-   * @param rel_id The ID of the relation that produced 'block'.
-   **/
-  void processDataPipelineMessage(const dag_node_index op_index,
-                                  const block_id block,
-                                  const relation_id rel_id);
-
-  /**
-   * @brief Process the received work order feedback message and notify relational
-   *        operator.
-   *
-   * @param message Feedback message from work order.
-   **/
-  void processFeedbackMessage(const WorkOrder::FeedbackMessage &message);
-
-  /**
-   * @brief Clear some of the vectors used for a single run of a query.
-   **/
-  void cleanUp() {
-    output_consumers_.clear();
-    blocking_dependencies_.clear();
-  }
-
-  /**
-   * @brief Process a current relational operator: Get its workorders and store
-   *        them in the WorkOrdersContainer for this query. If the operator can
-   *        be marked as done, do so.
-   *
-   * @param index The index of the relational operator to be processed in the
-   *        query plan DAG.
-   * @param recursively_check_dependents If an operator is done, should we
-   *        call processOperator on its dependents recursively.
+   * @param messages The messages to be dispatched.
    **/
-  void processOperator(const dag_node_index index, const bool recursively_check_dependents);
-
- /**
-   * @brief Get the next workorder to be excuted, wrapped in a WorkerMessage.
-   *
-   * @param start_operator_index Begin the search for the schedulable WorkOrder
-   *        with the operator at this index.
-   * @param numa_node The next WorkOrder should preferably have its input(s)
-   *        from this numa_node. This is a hint and not a binding requirement.
-   *
-   * @return A pointer to the WorkerMessage. If there's no WorkOrder to be
-   *         executed, return NULL.
-   **/
-  WorkerMessage* getNextWorkerMessage(
-      const dag_node_index start_operator_index, const int numa_node = -1);
+  void dispatchWorkerMessages(
+      const std::vector<std::unique_ptr<WorkerMessage>> &messages);
 
   /**
    * @brief Send the given message to the specified worker.
@@ -306,156 +91,24 @@ class Foreman final : public ForemanLite {
    *        in WorkerDirectory.
    * @param message The WorkerMessage to be sent.
    **/
-  void sendWorkerMessage(const std::size_t worker_thread_index, const WorkerMessage &message);
-
-  /**
-   * @brief Fetch all work orders currently available in relational operator and
-   *        store them internally.
-   *
-   * @param index The index of the relational operator to be processed in the
-   *        query plan DAG.
-   *
-   * @return Whether any work order was generated by op.
-   **/
-  bool fetchNormalWorkOrders(const dag_node_index index);
-
-  /**
-   * @brief This function does the following things:
-   *        1. Mark the given relational operator as "done".
-   *        2. For all the dependents of this operator, check if all of their
-   *        blocking dependencies are met. If so inform them that the blocking
-   *        dependencies are met.
-   *        3. Check if the given operator is done producing output. If it's
-   *        done, inform the dependents that they won't receive input anymore
-   *        from the given operator.
-   *
-   * @param index The index of the given relational operator in the DAG.
-   **/
-  void markOperatorFinished(const dag_node_index index);
-
-  /**
-   * @brief Check if the execution of the given operator is over.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the execution of the given operator is over, false
-   *         otherwise.
-   **/
-  inline bool checkOperatorExecutionOver(const dag_node_index index) const {
-    if (checkRebuildRequired(index)) {
-      return (checkNormalExecutionOver(index) && checkRebuildOver(index));
-    } else {
-      return checkNormalExecutionOver(index);
-    }
-  }
-
-  /**
-   * @brief Check if the given operator's normal execution is over.
-   *
-   * @note The conditions for a given operator's normal execution to get over:
-   *       1. All of its  normal (i.e. non rebuild) WorkOrders have finished
-   *       execution.
-   *       2. The operator is done generating work orders.
-   *       3. All of the dependencies of the given operator have been met.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the normal execution of the given operator is over, false
-   *         otherwise.
-   **/
-  inline bool checkNormalExecutionOver(const dag_node_index index) const {
-    return (checkAllDependenciesMet(index) &&
-            !workorders_container_->hasNormalWorkOrder(index) &&
-            query_exec_state_->getNumQueuedWorkOrders(index) == 0 &&
-            query_exec_state_->hasDoneGenerationWorkOrders(index));
-  }
+  void sendWorkerMessage(const std::size_t worker_thread_index,
+                         const WorkerMessage &message);
 
   /**
-   * @brief Check if the rebuild operation is required for a given operator.
-   *
-   * @param index The index of the given operator in the DAG.
+   * @brief Check if we can collect new messages from the PolicyEnforcer.
    *
-   * @return True if the rebuild operation is required, false otherwise.
+   * @param message_type The type of the last received message.
    **/
-  inline bool checkRebuildRequired(const dag_node_index index) const {
-    return query_exec_state_->isRebuildRequired(index);
-  }
+  bool canCollectNewMessages(const tmb::message_type_id message_type);
 
-  /**
-   * @brief Check if the rebuild operation for a given operator is over.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the rebuild operation is over, false otherwise.
-   **/
-  inline bool checkRebuildOver(const dag_node_index index) const {
-    return query_exec_state_->hasRebuildInitiated(index) &&
-           !workorders_container_->hasRebuildWorkOrder(index) &&
-           (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
-  }
+  const tmb::client_id main_thread_client_id_;
 
-  /**
-   * @brief Check if the rebuild operation for a given operator has been
-   *        initiated.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the rebuild operation has been initiated, false otherwise.
-   **/
-  inline bool checkRebuildInitiated(const dag_node_index index) const {
-    return query_exec_state_->hasRebuildInitiated(index);
-  }
-
-  /**
-   * @brief Initiate the rebuild process for partially filled blocks generated
-   *        during the execution of the given operator.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the rebuild is over immediately, i.e. the operator didn't
-   *         generate any rebuild WorkOrders, false otherwise.
-   **/
-  bool initiateRebuild(const dag_node_index index);
-
-  /**
-   * @brief Get the rebuild WorkOrders for an operator.
-   *
-   * @note This function should be called only once, when all the normal
-   *       WorkOrders generated by an operator finish their execution.
-   *
-   * @param index The index of the operator in the query plan DAG.
-   * @param container A pointer to a WorkOrdersContainer to be used to store the
-   *        generated WorkOrders.
-   **/
-  void getRebuildWorkOrders(const dag_node_index index, WorkOrdersContainer *container);
+  WorkerDirectory *worker_directory_;
 
   CatalogDatabaseLite *catalog_database_;
   StorageManager *storage_manager_;
 
-  DAG<RelationalOperator, bool> *query_dag_;
-
-  std::unique_ptr<QueryContext> query_context_;
-
-  // During a single round of WorkOrder dispatch, a Worker should be allocated
-  // at most these many WorkOrders.
-  std::size_t max_msgs_per_worker_;
-
-  // For all nodes, store their receiving dependents.
-  std::vector<std::vector<dag_node_index>> output_consumers_;
-
-  // For all nodes, store their pipeline breaking dependencies (if any).
-  std::vector<std::vector<dag_node_index>> blocking_dependencies_;
-
-  std::unique_ptr<QueryExecutionState> query_exec_state_;
-
-  std::unique_ptr<WorkOrdersContainer> workorders_container_;
-
-  const int num_numa_nodes_;
-
-  WorkerDirectory *workers_;
-
-  friend class ForemanTest;
-  FRIEND_TEST(ForemanTest, TwoNodesDAGPartiallyFilledBlocksTest);
+  std::unique_ptr<PolicyEnforcer> policy_enforcer_;
 
   DISALLOW_COPY_AND_ASSIGN(Foreman);
 };


[09/50] [abbrv] incubator-quickstep git commit: Fix Clang problems in Travis.

Posted by zu...@apache.org.
Fix Clang problems in Travis.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 2d39b8ecd8b1ca3fb42ff2505a664a94b67ab9e3
Parents: eab1c9a
Author: Navneet Potti <na...@gmail.com>
Authored: Wed Jun 8 18:15:38 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed Jun 8 20:27:44 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d39b8ec/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 08d6f38..df39fb0 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -12,7 +12,7 @@ cache: ccache
 
 compiler:
   - gcc
-  # clang
+  - clang
 
 env:
   - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
@@ -20,6 +20,19 @@ env:
   - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=none
   - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=none
 
+before_install:
+  - LLVM_VERSION=3.7.1
+  - LLVM_ARCHIVE_PATH=$HOME/clang+llvm.tar.xz
+  - if [[ $CC = "clang" ]]; then
+      wget http://llvm.org/releases/$LLVM_VERSION/clang+llvm-$LLVM_VERSION-x86_64-linux-gnu-ubuntu-14.04.tar.xz -O $LLVM_ARCHIVE_PATH;
+      mkdir -p $HOME/clang-$LLVM_VERSION;
+      tar xf $LLVM_ARCHIVE_PATH -C $HOME/clang-$LLVM_VERSION --strip-components 1;
+      ln -sf $HOME/clang-$LLVM_VERSION/bin/clang++ $HOME/clang-$LLVM_VERSION/bin/clang++-3.7;
+      export PATH=$HOME/clang-$LLVM_VERSION/bin:$PATH;
+      export CPPFLAGS="-I $HOME/clang-$LLVM_VERSION/include/c++/v1";
+      echo "Using clang at " `which $CC-3.7` " and $CXX at " `which $CXX-3.7`;
+    fi
+
 install:
   - if [ "$VECTOR_COPY_ELISION_LEVEL" = "joinwithbinaryexpressions" ] && [ "$CC" = "gcc" ]; then
       export MAKE_JOBS=1;


[19/50] [abbrv] incubator-quickstep git commit: Long lived Foreman thread

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/PolicyEnforcer.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcer.cpp b/query_execution/PolicyEnforcer.cpp
new file mode 100644
index 0000000..2145429
--- /dev/null
+++ b/query_execution/PolicyEnforcer.cpp
@@ -0,0 +1,183 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "query_execution/PolicyEnforcer.hpp"
+
+#include <cstddef>
+#include <memory>
+#include <queue>
+#include <utility>
+#include <unordered_map>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/QueryExecutionMessages.pb.h"
+#include "query_execution/QueryManager.hpp"
+#include "query_optimizer/QueryHandle.hpp"
+#include "relational_operators/WorkOrder.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+namespace quickstep {
+
+DEFINE_uint64(max_msgs_per_dispatch_round, 20, "Maximum number of messages that"
+              " can be allocated in a single round of dispatch of messages to"
+              " the workers.");
+
+bool PolicyEnforcer::admitQuery(QueryHandle *query_handle) {
+  if (admitted_queries_.size() < kMaxConcurrentQueries) {
+    // Ok to admit the query.
+    const std::size_t query_id = query_handle->query_id();
+    if (admitted_queries_.find(query_id) == admitted_queries_.end()) {
+      // Query with the same ID not present, ok to admit.
+      admitted_queries_[query_id].reset(
+          new QueryManager(foreman_client_id_, num_numa_nodes_, query_handle,
+                           catalog_database_, storage_manager_, bus_));
+      return true;
+    } else {
+      LOG(ERROR) << "Query with the same ID " << query_id << " exists";
+      return false;
+    }
+  } else {
+    // This query will have to wait.
+    waiting_queries_.push(query_handle);
+    return false;
+  }
+}
+
+void PolicyEnforcer::processMessage(const TaggedMessage &tagged_message) {
+  // TODO(harshad) : Provide processXMessage() public functions in
+  // QueryManager, so that we need to extract message from the
+  // TaggedMessage only once.
+  std::size_t query_id;
+  switch (tagged_message.message_type()) {
+    case kWorkOrderCompleteMessage:  // Fall through.
+    case kRebuildWorkOrderCompleteMessage: {
+      serialization::WorkOrderCompletionMessage proto;
+      CHECK(proto.ParseFromArray(tagged_message.message(),
+                                 tagged_message.message_bytes()));
+      query_id = proto.query_id();
+      break;
+    }
+    case kCatalogRelationNewBlockMessage: {
+      serialization::CatalogRelationNewBlockMessage proto;
+      CHECK(proto.ParseFromArray(tagged_message.message(),
+                                 tagged_message.message_bytes()));
+      query_id = proto.query_id();
+      break;
+    }
+    case kDataPipelineMessage: {
+      serialization::DataPipelineMessage proto;
+      CHECK(proto.ParseFromArray(tagged_message.message(),
+                                 tagged_message.message_bytes()));
+      query_id = proto.query_id();
+      break;
+    }
+    case kWorkOrdersAvailableMessage: {
+      serialization::WorkOrdersAvailableMessage proto;
+      CHECK(proto.ParseFromArray(tagged_message.message(),
+                                 tagged_message.message_bytes()));
+      query_id = proto.query_id();
+      break;
+    }
+    case kWorkOrderFeedbackMessage: {
+      WorkOrder::FeedbackMessage msg(const_cast<void *>(tagged_message.message()), tagged_message.message_bytes());
+      query_id = msg.header().query_id;
+      break;
+    }
+    default:
+      LOG(FATAL) << "Unknown message type found in PolicyEnforcer";
+  }
+  DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
+  const QueryManager::QueryStatusCode return_code =
+      admitted_queries_[query_id]->processMessage(tagged_message);
+  if (return_code == QueryManager::QueryStatusCode::kQueryExecuted) {
+    removeQuery(query_id);
+    if (!waiting_queries_.empty()) {
+      // Admit the earliest waiting query.
+      QueryHandle *new_query = waiting_queries_.front();
+      waiting_queries_.pop();
+      admitQuery(new_query);
+    }
+  }
+}
+
+void PolicyEnforcer::getWorkerMessages(
+    std::vector<std::unique_ptr<WorkerMessage>> *worker_messages) {
+  // Iterate over admitted queries until either there are no more
+  // messages available, or the maximum number of messages have
+  // been collected.
+  DCHECK(worker_messages->empty());
+  // TODO(harshad) - Make this function generic enough so that it
+  // works well when multiple queries are getting executed.
+  std::size_t per_query_share = 0;
+  if (!admitted_queries_.empty()) {
+    per_query_share = FLAGS_max_msgs_per_dispatch_round / admitted_queries_.size();
+  } else {
+    LOG(WARNING) << "Requesting WorkerMessages when no query is running";
+    return;
+  }
+  DCHECK_GT(per_query_share, 0u);
+  std::vector<std::size_t> finished_queries_ids;
+
+  for (const auto &admitted_query_info : admitted_queries_) {
+    QueryManager *curr_query_manager = admitted_query_info.second.get();
+    DCHECK(curr_query_manager != nullptr);
+    std::size_t messages_collected_curr_query = 0;
+    while (messages_collected_curr_query < per_query_share) {
+      WorkerMessage *next_worker_message =
+          curr_query_manager->getNextWorkerMessage(0, kAnyNUMANodeID);
+      if (next_worker_message != nullptr) {
+        ++messages_collected_curr_query;
+        worker_messages->push_back(std::unique_ptr<WorkerMessage>(next_worker_message));
+      } else {
+        // No more work ordes from the current query at this time.
+        // Check if the query's execution is over.
+        if (curr_query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
+          // If the query has been executed, remove it.
+          finished_queries_ids.push_back(admitted_query_info.first);
+        }
+        break;
+      }
+    }
+  }
+  for (const std::size_t finished_qid : finished_queries_ids) {
+    removeQuery(finished_qid);
+  }
+}
+
+void PolicyEnforcer::removeQuery(const std::size_t query_id) {
+  DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
+  if (!admitted_queries_[query_id]->getQueryExecutionState().hasQueryExecutionFinished()) {
+    LOG(WARNING) << "Removing query with ID " << query_id
+                 << " that hasn't finished its execution";
+  }
+  admitted_queries_.erase(query_id);
+}
+
+bool PolicyEnforcer::admitQueries(
+    const std::vector<QueryHandle*> &query_handles) {
+  for (QueryHandle *curr_query : query_handles) {
+    if (!admitQuery(curr_query)) {
+      return false;
+    }
+  }
+  return true;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/PolicyEnforcer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcer.hpp b/query_execution/PolicyEnforcer.hpp
new file mode 100644
index 0000000..5915b79
--- /dev/null
+++ b/query_execution/PolicyEnforcer.hpp
@@ -0,0 +1,167 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <queue>
+#include <unordered_map>
+#include <vector>
+
+#include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryManager.hpp"
+#include "query_execution/WorkerMessage.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+#include "tmb/tagged_message.h"
+
+namespace quickstep {
+
+class CatalogDatabaseLite;
+class QueryHandle;
+class StorageManager;
+
+/**
+ * @brief A class that ensures that a high level policy is maintained
+ *        in sharing resources among concurrent queries.
+ **/
+class PolicyEnforcer {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param foreman_client_id The TMB client ID of the Foreman.
+   * @param num_numa_nodes Number of NUMA nodes used by the system.
+   * @param catalog_database The CatalogDatabase used.
+   * @param storage_manager The StorageManager used.
+   * @param bus The TMB.
+   **/
+  PolicyEnforcer(const tmb::client_id foreman_client_id,
+                 const std::size_t num_numa_nodes,
+                 CatalogDatabaseLite *catalog_database,
+                 StorageManager *storage_manager,
+                 tmb::MessageBus *bus)
+      : foreman_client_id_(foreman_client_id),
+        num_numa_nodes_(num_numa_nodes),
+        catalog_database_(catalog_database),
+        storage_manager_(storage_manager),
+        bus_(bus) {}
+
+  /**
+   * @brief Destructor.
+   **/
+  ~PolicyEnforcer() {
+    if (hasQueries()) {
+      LOG(WARNING) << "Destructing PolicyEnforcer with some unfinished or "
+                      "waiting queries";
+    }
+  }
+
+  /**
+   * @brief Admit a query to the system.
+   *
+   * @param query_handle The QueryHandle for the new query.
+   *
+   * @return Whether the query was admitted to the system.
+   **/
+  bool admitQuery(QueryHandle *query_handle);
+
+  /**
+   * @brief Admit multiple queries in the system.
+   *
+   * @note In the current simple implementation, we only allow one active
+   *       query in the system. Other queries will have to wait.
+   *
+   * @param query_handles A vector of QueryHandles for the queries to be
+   *        admitted.
+   *
+   * @return True if all the queries were admitted, false if at least one query
+   *         was not admitted.
+   **/
+  bool admitQueries(const std::vector<QueryHandle*> &query_handles);
+
+  /**
+   * @brief Remove a given query that is under execution.
+   *
+   * @note This function is made public so that it is possible for a query
+   *       to be killed. Otherwise, it should only be used privately by the
+   *       class.
+   *
+   * TODO(harshad) - Extend this function to support removal of waiting queries.
+   *
+   * @param query_id The ID of the query to be removed.
+   **/
+  void removeQuery(const std::size_t query_id);
+
+  /**
+   * @brief Get worker messages to be dispatched. These worker messages come
+   *        from the active queries.
+   *
+   * @param worker_messages The worker messages to be dispatched.
+   **/
+  void getWorkerMessages(
+      std::vector<std::unique_ptr<WorkerMessage>> *worker_messages);
+
+  /**
+   * @brief Process a message sent to the Foreman, which gets passed on to the
+   *        policy enforcer.
+   *
+   * @param message The message.
+   **/
+  void processMessage(const TaggedMessage &tagged_message);
+
+  /**
+   * @brief Check if there are any queries to be executed.
+   *
+   * @return True if there is at least one active or waiting query, false if
+   *         the policy enforcer doesn't have any query.
+   **/
+  inline bool hasQueries() const {
+    return !(admitted_queries_.empty() && waiting_queries_.empty());
+  }
+
+ private:
+  static constexpr std::size_t kMaxConcurrentQueries = 1;
+
+  const tmb::client_id foreman_client_id_;
+  const std::size_t num_numa_nodes_;
+
+  CatalogDatabaseLite *catalog_database_;
+  StorageManager *storage_manager_;
+
+  tmb::MessageBus *bus_;
+
+  // Key = query ID, value = QueryManager* for the key query.
+  std::unordered_map<std::size_t, std::unique_ptr<QueryManager>> admitted_queries_;
+
+  // The queries which haven't been admitted yet.
+  std::queue<QueryHandle*> waiting_queries_;
+
+  DISALLOW_COPY_AND_ASSIGN(PolicyEnforcer);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_QUERY_MANAGER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryContext.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 3bfce17..54dd557 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -89,13 +89,13 @@ QueryContext::QueryContext(const serialization::QueryContext &proto,
 
   for (int i = 0; i < proto.insert_destinations_size(); ++i) {
     const serialization::InsertDestination &insert_destination_proto = proto.insert_destinations(i);
-    insert_destinations_.emplace_back(
-        InsertDestination::ReconstructFromProto(insert_destination_proto,
-                                                database.getRelationSchemaById(
-                                                    insert_destination_proto.relation_id()),
-                                                storage_manager,
-                                                scheduler_client_id,
-                                                bus));
+    insert_destinations_.emplace_back(InsertDestination::ReconstructFromProto(
+        proto.query_id(),
+        insert_destination_proto,
+        database.getRelationSchemaById(insert_destination_proto.relation_id()),
+        storage_manager,
+        scheduler_client_id,
+        bus));
   }
 
   for (int i = 0; i < proto.predicates_size(); ++i) {
@@ -231,7 +231,7 @@ bool QueryContext::ProtoIsValid(const serialization::QueryContext &proto,
     }
   }
 
-  return true;
+  return proto.IsInitialized();
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryContext.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.proto b/query_execution/QueryContext.proto
index b37286c..98cd0b6 100644
--- a/query_execution/QueryContext.proto
+++ b/query_execution/QueryContext.proto
@@ -54,4 +54,6 @@ message QueryContext {
 
   // NOTE(zuyu): For UpdateWorkOrder only.
   repeated UpdateGroup update_groups = 10;
+
+  required uint64 query_id = 11;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 15803cf..9d9a9e5 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -27,6 +27,7 @@ message EmptyMessage {
 message WorkOrderCompletionMessage {
   required uint64 operator_index = 1;
   required uint64 worker_thread_index = 2;
+  required uint64 query_id = 3;
 }
 
 message CatalogRelationNewBlockMessage {
@@ -35,16 +36,19 @@ message CatalogRelationNewBlockMessage {
 
   // Used by PartitionAwareInsertDestination.
   optional uint64 partition_id = 3;
+  required uint64 query_id = 4;
 }
 
 message DataPipelineMessage {
   required uint64 operator_index = 1;
   required fixed64 block_id = 2;
   required int32 relation_id = 3;
+  required uint64 query_id = 4;
 }
 
 message WorkOrdersAvailableMessage {
   required uint64 operator_index = 1;
+  required uint64 query_id = 2;
 }
 
 // BlockLocator related messages.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index fc253bc..9d1060f 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -58,6 +58,8 @@ using ClientIDMap = ThreadIDBasedMap<client_id,
 
 // We sort the following message types in the order of a life cycle of a query.
 enum QueryExecutionMessageType : message_type_id {
+  kAdmitRequestMessage,  // Requesting a query (or queries) to be admitted, from
+                         // the main thread to Foreman.
   kWorkOrderMessage,  // From Foreman to Worker.
   kWorkOrderCompleteMessage,  // From Worker to Foreman.
   kCatalogRelationNewBlockMessage,  // From InsertDestination to Foreman.
@@ -67,7 +69,8 @@ enum QueryExecutionMessageType : message_type_id {
                               // their corresponding RelationalOperators.
   kRebuildWorkOrderMessage,  // From Foreman to Worker.
   kRebuildWorkOrderCompleteMessage,  // From Worker to Foreman.
-  kPoisonMessage,  // From the CLI shell to Foreman, then from Foreman to Workers.
+  kWorkloadCompletionMessage,  // From Foreman to main thread.
+  kPoisonMessage,  // From the main thread to Foreman and Workers.
 
 #ifdef QUICKSTEP_DISTRIBUTED
   // BlockLocator related messages, sorted in a life cycle of StorageManager

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryExecutionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionUtil.hpp b/query_execution/QueryExecutionUtil.hpp
index a8b6a38..78fd159 100644
--- a/query_execution/QueryExecutionUtil.hpp
+++ b/query_execution/QueryExecutionUtil.hpp
@@ -17,9 +17,12 @@
 #ifndef QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_UTIL_HPP_
 #define QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_UTIL_HPP_
 
+#include <memory>
 #include <utility>
 
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/WorkerMessage.hpp"
 #include "utility/Macros.hpp"
 
 #include "tmb/address.h"
@@ -60,6 +63,55 @@ class QueryExecutionUtil {
                      std::move(tagged_message));
   }
 
+  /**
+   * @brief Construct and send an AdmitRequestMessage from a given sender to a
+   *        given recipient.
+   *
+   * @param sender_id The TMB client ID of the sender.
+   * @param receiver_id The TMB client ID of the receiver.
+   * @param query_handle The QueryHandle used in the AdmitRequestMessage.
+   * @param bus A pointer to the TMB.
+   * @param tagged_message A moved from reference to the tagged message.
+   *
+   * @return A status code indicating the result of the message delivery.
+   *         The caller should ensure that the status is SendStatus::kOK.
+   **/
+  static tmb::MessageBus::SendStatus ConstructAndSendAdmitRequestMessage(
+      const tmb::client_id sender_id,
+      const tmb::client_id receiver_id,
+      QueryHandle *query_handle,
+      MessageBus *bus) {
+    std::unique_ptr<AdmitRequestMessage> request_message(
+        new AdmitRequestMessage(query_handle));
+    const std::size_t size_of_request_msg = sizeof(*request_message);
+    TaggedMessage admit_tagged_message(
+        request_message.release(), size_of_request_msg, kAdmitRequestMessage);
+
+    return QueryExecutionUtil::SendTMBMessage(
+        bus, sender_id, receiver_id, std::move(admit_tagged_message));
+  }
+
+  static void BroadcastPoisonMessage(const tmb::client_id sender_id, tmb::MessageBus *bus) {
+    // Terminate all threads.
+    // The sender thread broadcasts poison message to the workers and foreman.
+    // Each worker dies after receiving poison message. The order of workers'
+    // death is irrelavant.
+    MessageStyle style;
+    style.Broadcast(true);
+    Address address;
+    address.All(true);
+    std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
+    TaggedMessage poison_tagged_message(poison_message.get(),
+                                        sizeof(*poison_message),
+                                        kPoisonMessage);
+
+    const tmb::MessageBus::SendStatus send_status = bus->Send(
+        sender_id, address, style, std::move(poison_tagged_message));
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
+       "Broadcast poison message from sender with TMB client ID " << sender_id
+       << " failed";
+  }
+
  private:
   /**
    * @brief Constructor. Made private to avoid instantiation.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryManager.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManager.hpp b/query_execution/QueryManager.hpp
index 47f54c5..b52460f 100644
--- a/query_execution/QueryManager.hpp
+++ b/query_execution/QueryManager.hpp
@@ -25,18 +25,21 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryExecutionState.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
 #include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "storage/StorageBlockInfo.hpp"
 #include "utility/DAG.hpp"
 #include "utility/Macros.hpp"
 
+#include "tmb/id_typedefs.h"
 #include "tmb/message_bus.h"
 #include "tmb/tagged_message.h"
 
 namespace quickstep {
 
 class CatalogDatabaseLite;
-class ForemanMessage;
 class QueryHandle;
 class StorageManager;
 class WorkerMessage;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/WorkOrdersContainer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkOrdersContainer.hpp b/query_execution/WorkOrdersContainer.hpp
index eb9aedd..d023daa 100644
--- a/query_execution/WorkOrdersContainer.hpp
+++ b/query_execution/WorkOrdersContainer.hpp
@@ -28,6 +28,8 @@
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 /** \addtogroup QueryExecution
@@ -76,7 +78,7 @@ class WorkOrdersContainer {
    * @return If there are pending WorkOrders.
    **/
   inline bool hasNormalWorkOrder(const std::size_t operator_index) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return normal_workorders_[operator_index].hasWorkOrder();
   }
 
@@ -92,9 +94,9 @@ class WorkOrdersContainer {
    **/
   inline bool hasNormalWorkOrderForNUMANode(
       const std::size_t operator_index, const int numa_node_id) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return normal_workorders_[operator_index].hasWorkOrderForNUMANode(
         numa_node_id);
   }
@@ -108,7 +110,7 @@ class WorkOrdersContainer {
    * @return If there are pending rebuild WorkOrders.
    **/
   inline bool hasRebuildWorkOrder(const std::size_t operator_index) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return rebuild_workorders_[operator_index].hasWorkOrder();
   }
 
@@ -124,9 +126,9 @@ class WorkOrdersContainer {
    **/
   inline bool hasRebuildWorkOrderForNUMANode(
       const std::size_t operator_index, const int numa_node_id) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return rebuild_workorders_[operator_index].hasWorkOrderForNUMANode(
         numa_node_id);
   }
@@ -144,9 +146,9 @@ class WorkOrdersContainer {
    **/
   WorkOrder* getNormalWorkOrderForNUMANode(const std::size_t operator_index,
                                            const int numa_node_id) {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return normal_workorders_[operator_index].getWorkOrderForNUMANode(
         numa_node_id);
   }
@@ -164,7 +166,7 @@ class WorkOrdersContainer {
    **/
   WorkOrder* getNormalWorkOrder(const std::size_t operator_index,
                                 const bool prefer_single_NUMA_node = true) {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return normal_workorders_[operator_index].getWorkOrder(
         prefer_single_NUMA_node);
   }
@@ -182,9 +184,9 @@ class WorkOrdersContainer {
    **/
   WorkOrder* getRebuildWorkOrderForNUMANode(const std::size_t operator_index,
                                             const int numa_node_id) {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return rebuild_workorders_[operator_index].getWorkOrderForNUMANode(
         numa_node_id);
   }
@@ -202,7 +204,7 @@ class WorkOrdersContainer {
    **/
   WorkOrder* getRebuildWorkOrder(const std::size_t operator_index,
                                  const bool prefer_single_NUMA_node = true) {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return rebuild_workorders_[operator_index].getWorkOrder(
         prefer_single_NUMA_node);
   }
@@ -220,8 +222,8 @@ class WorkOrdersContainer {
    * @param operator_index The index of the operator in the query DAG.
    **/
   void addNormalWorkOrder(WorkOrder *workorder, const std::size_t operator_index) {
-    DEBUG_ASSERT(workorder != nullptr);
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK(workorder != nullptr);
+    DCHECK_LT(operator_index, num_operators_);
     normal_workorders_[operator_index].addWorkOrder(workorder);
   }
 
@@ -238,8 +240,8 @@ class WorkOrdersContainer {
    **/
   void addRebuildWorkOrder(WorkOrder *workorder,
                            const std::size_t operator_index) {
-    DEBUG_ASSERT(workorder != nullptr);
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK(workorder != nullptr);
+    DCHECK_LT(operator_index, num_operators_);
     rebuild_workorders_[operator_index].addWorkOrder(workorder);
   }
 
@@ -254,9 +256,9 @@ class WorkOrdersContainer {
    **/
   inline std::size_t getNumNormalWorkOrdersForNUMANode(
       const std::size_t operator_index, const int numa_node_id) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return normal_workorders_[operator_index].getNumWorkOrdersForNUMANode(
         numa_node_id);
   }
@@ -271,7 +273,7 @@ class WorkOrdersContainer {
    **/
   inline std::size_t getNumNormalWorkOrders(
       const std::size_t operator_index) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return normal_workorders_[operator_index].getNumWorkOrders();
   }
 
@@ -286,9 +288,9 @@ class WorkOrdersContainer {
    **/
   inline std::size_t getNumRebuildWorkOrdersForNUMANode(
       const std::size_t operator_index, const int numa_node_id) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return rebuild_workorders_[operator_index].getNumWorkOrdersForNUMANode(
         numa_node_id);
   }
@@ -303,7 +305,7 @@ class WorkOrdersContainer {
    **/
   inline std::size_t getNumRebuildWorkOrders(
       const std::size_t operator_index) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return rebuild_workorders_[operator_index].getNumWorkOrders();
   }
 
@@ -418,8 +420,8 @@ class WorkOrdersContainer {
     void addWorkOrder(WorkOrder *workorder);
 
     bool hasWorkOrderForNUMANode(const int numa_node_id) const {
-      DEBUG_ASSERT(numa_node_id >= 0);
-      DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+      DCHECK_GE(numa_node_id, 0);
+      DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
       return single_numa_node_workorders_[numa_node_id].hasWorkOrder() ||
              multiple_numa_nodes_workorders_.hasWorkOrderForNUMANode(
                  numa_node_id);
@@ -440,8 +442,8 @@ class WorkOrdersContainer {
 
     std::size_t getNumWorkOrdersForNUMANode(
         const int numa_node_id) const {
-      DEBUG_ASSERT(numa_node_id >= 0);
-      DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+      DCHECK_GE(numa_node_id, 0);
+      DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
       return single_numa_node_workorders_[numa_node_id].getNumWorkOrders() +
              multiple_numa_nodes_workorders_.getNumWorkOrdersForNUMANode(
                  numa_node_id);
@@ -463,8 +465,8 @@ class WorkOrdersContainer {
     }
 
     WorkOrder* getWorkOrderForNUMANode(const int numa_node_id) {
-      DEBUG_ASSERT(numa_node_id >= 0);
-      DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+      DCHECK_GE(numa_node_id, 0);
+      DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
       WorkOrder *work_order = single_numa_node_workorders_[numa_node_id].getWorkOrder();
       if (work_order == nullptr) {
         work_order = multiple_numa_nodes_workorders_.getWorkOrderForNUMANode(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index 645fd05..ef596e1 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -58,11 +58,14 @@ void Worker::run() {
         WorkerMessage message(*static_cast<const WorkerMessage*>(tagged_message.message()));
         DCHECK(message.getWorkOrder() != nullptr);
         message.getWorkOrder()->execute();
+        const std::size_t query_id_for_workorder =
+            message.getWorkOrder()->getQueryID();
         delete message.getWorkOrder();
 
-        sendWorkOrderCompleteMessage(annotated_msg.sender,
-                                     message.getRelationalOpIndex(),
-                                     tagged_message.message_type() == kRebuildWorkOrderMessage);
+        sendWorkOrderCompleteMessage(
+            annotated_msg.sender, message.getRelationalOpIndex(),
+            query_id_for_workorder,
+            tagged_message.message_type() == kRebuildWorkOrderMessage);
         break;
       }
       case kPoisonMessage: {
@@ -76,10 +79,12 @@ void Worker::run() {
 
 void Worker::sendWorkOrderCompleteMessage(const tmb::client_id receiver,
                                           const size_t op_index,
+                                          const size_t query_id,
                                           const bool is_rebuild_work_order) {
   serialization::WorkOrderCompletionMessage proto;
   proto.set_operator_index(op_index);
   proto.set_worker_thread_index(worker_thread_index_);
+  proto.set_query_id(query_id);
 
   // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
   const size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/Worker.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.hpp b/query_execution/Worker.hpp
index b94e937..c0bafdc 100644
--- a/query_execution/Worker.hpp
+++ b/query_execution/Worker.hpp
@@ -97,11 +97,13 @@ class Worker : public Thread {
    *
    * @param receiver The id of the TMB client which should receive the response.
    * @param op_index The index of the operator to which the WorkOrder belongs.
+   * @param query_id The ID of the query which the WorkOrder belongs to.
    * @param is_rebuild_work_order True if it is a RebuildWorkOrder. Otherwise
    *        false.
    **/
   void sendWorkOrderCompleteMessage(const tmb::client_id receiver,
                                     const std::size_t op_index,
+                                    const std::size_t query_id,
                                     const bool is_rebuild_work_order);
 
   const std::size_t worker_thread_index_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/WorkerMessage.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkerMessage.hpp b/query_execution/WorkerMessage.hpp
index ec63af9..560c1ba 100644
--- a/query_execution/WorkerMessage.hpp
+++ b/query_execution/WorkerMessage.hpp
@@ -30,6 +30,8 @@ class WorkOrder;
  **/
 class WorkerMessage {
  public:
+  static constexpr int kInvalidRecipientIndexHint = -1;
+
   enum class WorkerMessageType {
     kRebuildWorkOrder = 0,
     kWorkOrder,
@@ -105,6 +107,23 @@ class WorkerMessage {
     return type_;
   }
 
+  /**
+   * @brief Set a hint for the recipient worker thread.
+   *
+   * @param recipient_index_hint The hint i.e. the worker thread index.
+   **/
+  inline void setRecipientHint(const int recipient_index_hint) {
+    recipient_index_hint_ = recipient_index_hint;
+  }
+
+  /**
+   * @brief Get the hint for the recipient worker thread. The hint is invalid if
+   *        it is kInvalidRecipientIndexHint.
+   **/
+  inline int getRecipientHint() const {
+    return recipient_index_hint_;
+  }
+
  private:
   /**
    * @brief Constructor.
@@ -120,12 +139,13 @@ class WorkerMessage {
                 const WorkerMessageType type)
       : work_unit_(work_unit),
         relational_op_index_(relational_op_index),
-        type_(type) {
-  }
+        type_(type),
+        recipient_index_hint_(kInvalidRecipientIndexHint) {}
 
   WorkOrder *work_unit_;
   const std::size_t relational_op_index_;
   const WorkerMessageType type_;
+  int recipient_index_hint_;
 };
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/tests/Foreman_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/Foreman_unittest.cpp b/query_execution/tests/Foreman_unittest.cpp
deleted file mode 100644
index cbe5088..0000000
--- a/query_execution/tests/Foreman_unittest.cpp
+++ /dev/null
@@ -1,952 +0,0 @@
-/**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *
- *   Licensed under the Apache License, Version 2.0 (the "License");
- *   you may not use this file except in compliance with the License.
- *   You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- **/
-
-#include <climits>
-#include <memory>
-#include <utility>
-#include <vector>
-
-#include "catalog/CatalogDatabase.hpp"
-#include "catalog/CatalogRelation.hpp"
-#include "catalog/CatalogTypedefs.hpp"
-#include "query_execution/Foreman.hpp"
-#include "query_execution/QueryContext.hpp"
-#include "query_execution/QueryContext.pb.h"
-#include "query_execution/QueryExecutionState.hpp"
-#include "query_execution/QueryExecutionTypedefs.hpp"
-#include "query_execution/WorkOrdersContainer.hpp"
-#include "query_execution/WorkerDirectory.hpp"
-#include "query_execution/WorkerMessage.hpp"
-#include "query_optimizer/QueryPlan.hpp"
-#include "relational_operators/RelationalOperator.hpp"
-#include "relational_operators/WorkOrder.hpp"
-#include "storage/InsertDestination.hpp"
-#include "storage/InsertDestination.pb.h"
-#include "storage/StorageBlock.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageManager.hpp"
-#include "utility/DAG.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-#include "gtest/gtest.h"
-
-#include "tmb/id_typedefs.h"
-#include "tmb/message_bus.h"
-#include "tmb/tagged_message.h"
-
-using std::move;
-using std::unique_ptr;
-using std::vector;
-
-using tmb::client_id;
-
-namespace quickstep {
-
-class WorkOrderProtosContainer;
-
-class MockWorkOrder : public WorkOrder {
- public:
-  explicit MockWorkOrder(const int op_index)
-      : WorkOrder(0), op_index_(op_index) {}
-
-  void execute() override {
-    VLOG(3) << "WorkOrder[" << op_index_ << "] executing.";
-  }
-
-  inline QueryPlan::DAGNodeIndex getOpIndex() const {
-    return op_index_;
-  }
-
- private:
-  const QueryPlan::DAGNodeIndex op_index_;
-
-  DISALLOW_COPY_AND_ASSIGN(MockWorkOrder);
-};
-
-class MockOperator: public RelationalOperator {
- public:
-  enum function_name {
-    kFeedInputBlock = 0,
-    kFeedInputBlocks,
-    kDoneFeedingInputBlocks,
-    kGetAllWorkOrders
-  };
-
-  MockOperator(const bool produce_workorders,
-               const bool has_streaming_input,
-               const int max_getworkorder_iters = 1,
-               const int max_workorders = INT_MAX)
-      : RelationalOperator(0 /* Query Id */),
-        produce_workorders_(produce_workorders),
-        has_streaming_input_(has_streaming_input),
-        max_workorders_(max_workorders),
-        max_getworkorder_iters_(max_getworkorder_iters),
-        num_calls_get_workorders_(0),
-        num_workorders_generated_(0),
-        num_calls_feedblock_(0),
-        num_calls_feedblocks_(0),
-        num_calls_donefeedingblocks_(0) {
-  }
-
-#define MOCK_OP_LOG(x) VLOG(x) << "Op[" << op_index_ << "]: " << __func__ << ": "
-
-  // The methods below are used to check whether Foreman calls the Relational
-  // operator, how many times it calls a particular method etc.
-  inline int getNumWorkOrders() const {
-    return num_workorders_generated_;
-  }
-
-  inline int getNumCalls(const function_name fname) const {
-    switch (fname) {
-      case kFeedInputBlock:
-        return num_calls_feedblock_;
-      case kFeedInputBlocks:
-        return num_calls_feedblocks_;
-      case kDoneFeedingInputBlocks:
-        return num_calls_donefeedingblocks_;
-      case kGetAllWorkOrders:
-        return num_calls_get_workorders_;
-      default:
-        return -1;
-    }
-  }
-
-  inline bool getBlockingDependenciesMet() const {
-    MOCK_OP_LOG(3) << "met.";
-    return blocking_dependencies_met_;
-  }
-
-  void setInsertDestinationID(const QueryContext::insert_destination_id insert_destination_index) {
-    insert_destination_index_ = insert_destination_index;
-  }
-
-  // Mock to trigger doneFeedingInputBlocks for the dependent operators
-  // in Foreman::markOperatorFinished.
-  void setOutputRelationID(const relation_id rel_id) {
-    output_relation_id_ = rel_id;
-  }
-
-  // Override methods from the base class.
-  bool getAllWorkOrders(
-      WorkOrdersContainer *container,
-      QueryContext *query_context,
-      StorageManager *storage_manager,
-      const tmb::client_id foreman_client_id,
-      tmb::MessageBus *bus) override {
-    ++num_calls_get_workorders_;
-    if (produce_workorders_) {
-      if (has_streaming_input_) {
-        if ((num_calls_feedblock_ > 0 || num_calls_feedblocks_ > 0) && (num_workorders_generated_ < max_workorders_)) {
-          MOCK_OP_LOG(3) << "[stream] generate WorkOrder";
-          container->addNormalWorkOrder(new MockWorkOrder(op_index_), op_index_);
-          ++num_workorders_generated_;
-        }
-      } else {
-        if (blocking_dependencies_met_ && (num_workorders_generated_ < max_workorders_)) {
-          MOCK_OP_LOG(3) << "[static] generate WorkOrder";
-          container->addNormalWorkOrder(new MockWorkOrder(op_index_), op_index_);
-          ++num_workorders_generated_;
-        }
-      }
-    }
-    MOCK_OP_LOG(3) << "count(" << num_calls_get_workorders_ << ") "
-                   << "return(" << (num_calls_get_workorders_ == max_getworkorder_iters_) << ")";
-    return num_calls_get_workorders_ == max_getworkorder_iters_;
-  }
-
-  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
-    return true;
-  }
-
-  void feedInputBlock(const block_id input_block_id,
-                      const relation_id input_relation_id) override {
-    ++num_calls_feedblock_;
-    MOCK_OP_LOG(3) << "count(" << num_calls_feedblock_ << ")";
-  }
-
-  void feedInputBlocks(const relation_id rel_id,
-                       std::vector<block_id> *partially_filled_blocks) override {
-    ++num_calls_feedblocks_;
-    MOCK_OP_LOG(3) << "count(" << num_calls_feedblocks_ << ")";
-  }
-
-  void doneFeedingInputBlocks(const relation_id rel_id) override {
-    ++num_calls_donefeedingblocks_;
-    MOCK_OP_LOG(3) << "count(" << num_calls_donefeedingblocks_ << ")";
-  }
-
-  QueryContext::insert_destination_id getInsertDestinationID() const override {
-    return insert_destination_index_;
-  }
-
-  const relation_id getOutputRelationID() const override {
-    return output_relation_id_;
-  }
-
- private:
-  const bool produce_workorders_;
-  const bool has_streaming_input_;
-  const int max_workorders_;
-  const int max_getworkorder_iters_;
-
-  int num_calls_get_workorders_;
-  int num_workorders_generated_;
-  int num_calls_feedblock_;
-  int num_calls_feedblocks_;
-  int num_calls_donefeedingblocks_;
-
-  QueryContext::insert_destination_id insert_destination_index_ = QueryContext::kInvalidInsertDestinationId;
-
-  relation_id output_relation_id_ = -1;
-
-#undef MOCK_OP_LOG
-
-  DISALLOW_COPY_AND_ASSIGN(MockOperator);
-};
-
-
-class ForemanTest : public ::testing::Test {
- protected:
-  // Class ForemanTest is the friend of class Foreman. Each TEST_F behaves
-  // as a separate class, so we can't access Foreman's private members in
-  // TEST_F.
-  virtual void SetUp() {
-    db_.reset(new CatalogDatabase(nullptr /* catalog */, "database"));
-    storage_manager_.reset(new StorageManager("./"));
-
-    query_plan_.reset(new QueryPlan());
-
-    bus_.Initialize();
-
-    foreman_.reset(new Foreman(&bus_, db_.get(), storage_manager_.get()));
-
-    // This thread acts both as Foreman as well as Worker. Foreman connects to
-    // the bus in its constructor.
-    worker_client_id_ = bus_.Connect();
-
-    // Register as sender and receiver for relevant types of messages.
-    bus_.RegisterClientAsSender(worker_client_id_, kWorkOrderCompleteMessage);
-    bus_.RegisterClientAsSender(worker_client_id_, kRebuildWorkOrderCompleteMessage);
-    bus_.RegisterClientAsSender(worker_client_id_, kDataPipelineMessage);
-    bus_.RegisterClientAsReceiver(worker_client_id_, kWorkOrderMessage);
-    bus_.RegisterClientAsReceiver(worker_client_id_, kRebuildWorkOrderMessage);
-    bus_.RegisterClientAsReceiver(worker_client_id_, kPoisonMessage);
-
-    std::vector<client_id> worker_client_ids;
-    worker_client_ids.push_back(worker_client_id_);
-
-    std::vector<numa_node_id> numa_nodes;
-    numa_nodes.push_back(static_cast<numa_node_id>(-1));
-
-    workers_.reset(new WorkerDirectory(1, worker_client_ids, numa_nodes));
-    foreman_->setWorkerDirectory(workers_.get());
-  }
-
-  inline const int getNumWorkOrdersInExecution(const QueryPlan::DAGNodeIndex index) const {
-    return foreman_->query_exec_state_->getNumQueuedWorkOrders(index);
-  }
-
-  inline const int getNumOperatorsFinished() const {
-    return foreman_->query_exec_state_->getNumOperatorsFinished();
-  }
-
-  inline bool getOperatorFinishedStatus(const QueryPlan::DAGNodeIndex index) const {
-    return foreman_->query_exec_state_->hasExecutionFinished(index);
-  }
-
-  inline bool popWorkOrderIfAvailable(MockWorkOrder **workorder) {
-    AnnotatedMessage msg;
-    if (bus_.ReceiveIfAvailable(worker_client_id_, &msg)) {
-      WorkerMessage message(*static_cast<const WorkerMessage*>(msg.tagged_message.message()));
-      *workorder = static_cast<MockWorkOrder*>(message.getWorkOrder());
-      return true;
-    }
-    return false;
-  }
-
-  inline bool popRebuildWorkOrderIfAvailable(MockWorkOrder **workorder) {
-    return popWorkOrderIfAvailable(workorder);
-  }
-
-  inline bool placeDataPipelineMessage(const QueryPlan::DAGNodeIndex source_operator_index) {
-    VLOG(3) << "Place DataPipeline message for Op[" << source_operator_index << "]";
-    foreman_->processDataPipelineMessage(source_operator_index, 0 /* block_id */, 0 /* relation_id */);
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline bool placeWorkOrderCompleteMessage(const QueryPlan::DAGNodeIndex index) {
-    VLOG(3) << "Place WorkOrderComplete message for Op[" << index << "]";
-    foreman_->processWorkOrderCompleteMessage(index, 0 /* worker id */);
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline bool placeRebuildWorkOrderCompleteMessage(const QueryPlan::DAGNodeIndex index) {
-    VLOG(3) << "Place RebuildWorkOrderComplete message for Op[" << index << "]";
-    foreman_->processRebuildWorkOrderCompleteMessage(index, 0 /* worker id */);
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline bool placeOutputBlockMessage(const QueryPlan::DAGNodeIndex index) {
-    VLOG(3) << "Place OutputBlock message for Op[" << index << "]";
-    foreman_->processDataPipelineMessage(index,
-                                         BlockIdUtil::GetBlockId(1 /* domain */, 1),
-                                         0 /* relation_id */);
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline bool startForeman() {
-    foreman_->initialize();
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline int getWorkerInputQueueSize() {
-    return bus_.CountQueuedMessagesForClient(worker_client_id_);
-  }
-
-  unique_ptr<CatalogDatabase> db_;
-  unique_ptr<StorageManager> storage_manager_;
-
-  unique_ptr<QueryPlan> query_plan_;
-
-  unique_ptr<Foreman> foreman_;
-  MessageBusImpl bus_;
-
-  client_id worker_client_id_;
-
-  unique_ptr<WorkerDirectory> workers_;
-};
-
-TEST_F(ForemanTest, SingleNodeDAGNoWorkOrdersTest) {
-  // This test creates a DAG of a single node. No workorders are generated.
-  query_plan_->addRelationalOperator(new MockOperator(false, false));
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  const MockOperator &op = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(0));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // Foreman exits after initialize, since no workorders are generated.
-  EXPECT_TRUE(startForeman());
-
-  // op doesn't have any dependencies.
-  EXPECT_TRUE(op.getBlockingDependenciesMet());
-
-  // No workorder is generated. No response is received.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // We expect one call for op's getAllWorkOrders().
-  EXPECT_EQ(1, op.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlocks));
-}
-
-TEST_F(ForemanTest, SingleNodeDAGStaticWorkOrdersTest) {
-  // This test creates a DAG of a single node. Static workorders are generated.
-  const QueryPlan::DAGNodeIndex id = query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  const MockOperator &op = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-  EXPECT_FALSE(startForeman());
-
-  // op doesn't have any dependencies.
-  EXPECT_TRUE(op.getBlockingDependenciesMet());
-
-  // We expect one call for op's getAllWorkOrders().
-  EXPECT_EQ(1, op.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlocks));
-
-  // One workorder is generated.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-  EXPECT_EQ(1, op.getNumWorkOrders());
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  EXPECT_EQ(id, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id));
-  EXPECT_EQ(0, getNumOperatorsFinished());
-
-  // Send a message to Foreman upon workorder completion.
-  // Last event processed by Foreman.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id));
-  EXPECT_EQ(1, getNumOperatorsFinished());
-  EXPECT_TRUE(getOperatorFinishedStatus(id));
-}
-
-TEST_F(ForemanTest, SingleNodeDAGDynamicWorkOrdersTest) {
-  // This test creates a DAG of a single node. WorkOrders are generated
-  // dynamically as pending work orders complete execution, i.e.,
-  // getAllWorkOrders() is called multiple times.  getAllWorkOrders() will be
-  // called 5 times and 3 work orders will be returned, i.e., 1st 3 calls to
-  // getAllWorkOrders() insert 1 WorkOrder and return false, and the next will insert no
-  // WorkOrder and return true.
-
-  // TODO(shoban): This test can not be more robust than this because of fixed
-  // scaffolding of mocking. If we use gMock, we can do much better.
-  const QueryPlan::DAGNodeIndex id = query_plan_->addRelationalOperator(new MockOperator(true, false, 4, 3));
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  const MockOperator &op = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-  EXPECT_FALSE(startForeman());
-
-  // op doesn't have any dependencies.
-  EXPECT_TRUE(op.getBlockingDependenciesMet());
-
-  for (int i = 0; i < 3; i++) {
-    // We expect one call for op's getAllWorkOrders().
-    EXPECT_EQ(i + 1, op.getNumCalls(MockOperator::kGetAllWorkOrders));
-
-    // One workorder is generated.
-    EXPECT_EQ(1, getWorkerInputQueueSize());
-    EXPECT_EQ(i + 1, op.getNumWorkOrders());
-
-    // Worker receives a WorkOrder.
-    MockWorkOrder *work_order;
-    ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-    EXPECT_EQ(id, work_order->getOpIndex());
-
-    work_order->execute();
-    delete work_order;
-
-    EXPECT_EQ(1, getNumWorkOrdersInExecution(id));
-    EXPECT_EQ(0, getNumOperatorsFinished());
-
-    if (i < 2) {
-      // Send a message to Foreman upon workorder completion.
-      EXPECT_FALSE(placeWorkOrderCompleteMessage(id));
-    } else {
-      // Send a message to Foreman upon workorder completion.
-      // Last event.
-      EXPECT_TRUE(placeWorkOrderCompleteMessage(id));
-    }
-  }
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id));
-
-  EXPECT_EQ(1, getNumOperatorsFinished());
-  EXPECT_TRUE(getOperatorFinishedStatus(id));
-
-  // We place this check in the end, since it's true throughout the test.
-  EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlocks));
-}
-
-TEST_F(ForemanTest, TwoNodesDAGBlockingLinkTest) {
-  // We use two nodes in the DAG with a blocking link between them.
-  // There is no streaming of data involved in this test.
-  const QueryPlan::DAGNodeIndex id1 = query_plan_->addRelationalOperator(new MockOperator(true, false));
-  const QueryPlan::DAGNodeIndex id2 = query_plan_->addRelationalOperator(new MockOperator(true, false));
-
-  // Create a blocking link.
-  query_plan_->addDirectDependency(id2, id1, true);
-
-  static_cast<MockOperator*>(query_plan_->getQueryPlanDAGMutable()->getNodePayloadMutable(id1))
-      ->setOutputRelationID(0xdead);
-
-  const MockOperator &op1 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id1));
-  const MockOperator &op2 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id2));
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  EXPECT_FALSE(startForeman());
-
-  // op1 doesn't have any dependencies
-  EXPECT_TRUE(op1.getBlockingDependenciesMet());
-
-  // Only op1 should receive a call to getAllWorkOrders initially.
-  EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlocks));
-
-  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlocks));
-
-  // Only op1 should produce a workorder.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-  EXPECT_EQ(1, op1.getNumWorkOrders());
-  EXPECT_EQ(0, op2.getNumWorkOrders());
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // This workorder's source should be op1.
-  EXPECT_EQ(id1, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-  // Foreman hasn't yet got workorder completion response for the workorder.
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id1));
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id2));
-  EXPECT_EQ(0, getNumOperatorsFinished());
-
-  // Send a message to Foreman upon workorder (generated by op1) completion.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id1));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id1));
-  // op1 is over now, op2 still to go.
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-  EXPECT_EQ(1, getNumOperatorsFinished());
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id2));
-
-  // op1 is op2's blocking dependency.
-  EXPECT_TRUE(op2.getBlockingDependenciesMet());
-
-  EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
-  // op2 should get first call of getAllWorkOrders() when op1 is over.
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-
-  EXPECT_EQ(1, op2.getNumWorkOrders());
-
-  // Send a message to Foreman upon workorder (generated by op2) completion.
-
-  // Note that the worker hasn't yet popped the workorder. Usually this won't
-  // happen as workers pop workorders first, execute and then send the response.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id2));
-
-  // WorkOrder yet to be popped by the worker.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id1));
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id2));
-
-  EXPECT_EQ(2, getNumOperatorsFinished());
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_TRUE(getOperatorFinishedStatus(id2));
-
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have come from op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // Expect no additional calls to getAllWorkOrders.
-  EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-}
-
-TEST_F(ForemanTest, TwoNodesDAGPipeLinkTest) {
-  // We use two nodes in the DAG with a non-blocking link between them.
-  // We stream output of op1 to op2. Sequeuce of events is as follows:
-  // 1. op1 creates a workorder.
-  // 2. We send a "block full" (from op1) to Foreman.
-  // 3. op2 creates a workorder because of step 2.
-  const QueryPlan::DAGNodeIndex id1 = query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
-  const QueryPlan::DAGNodeIndex id2 = query_plan_->addRelationalOperator(new MockOperator(true, true, 3));
-
-  // Create a non-blocking link.
-  query_plan_->addDirectDependency(id2, id1, false);
-
-  static_cast<MockOperator*>(query_plan_->getQueryPlanDAGMutable()->getNodePayloadMutable(id1))
-      ->setOutputRelationID(0xdead);
-
-  const MockOperator &op1 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id1));
-  const MockOperator &op2 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id2));
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  startForeman();
-
-  // As none of the operators have a blocking link, blocking dependencies should
-  // be met.
-  EXPECT_TRUE(op1.getBlockingDependenciesMet());
-  EXPECT_TRUE(op2.getBlockingDependenciesMet());
-
-  EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(1, op1.getNumWorkOrders());
-  EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlocks));
-
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-  // op2 will generate workorder only after receiving a streaming input.
-  EXPECT_EQ(0, op2.getNumWorkOrders());
-  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlocks));
-
-  // There should be one workorder sent to the worker so far.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // This workorder's source be op1.
-  EXPECT_EQ(id1, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // Send a message to Foreman upon block getting full (output of op1).
-  EXPECT_FALSE(placeOutputBlockMessage(id1));
-
-  // op1 is not finished yet because the response of workorder completion hasn't
-  // been received yet by the Foreman.
-  EXPECT_FALSE(getOperatorFinishedStatus(id1));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-
-  // No additional call to op1's getAllWorkOrders.
-  EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlocks));
-
-  // Output from op1 should be fed to op2.
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlocks));
-
-  // A call to op2's getAllWorkOrders because of the streamed input.
-  EXPECT_EQ(2, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(1, op2.getNumWorkOrders());
-
-  // Place a message of a workorder completion of op1 on Foreman's input queue.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id1));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id1));
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-
-  // An additional call to op2's getAllWorkOrders because of completion of op1.
-  EXPECT_EQ(3, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(2, op2.getNumWorkOrders());
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Pop a workorder from Foreman's output queue.
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-  work_order->execute();
-  delete work_order;
-
-  // Place a message of a workorder completion of op2 on Foreman's input queue.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id2));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id2));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Pop a workorder from Foreman's output queue.
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // Send a message to Foreman upon workorder (generated by op2) completion.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id2));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id2));
-  EXPECT_TRUE(getOperatorFinishedStatus(id2));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-}
-
-TEST_F(ForemanTest, TwoNodesDAGPartiallyFilledBlocksTest) {
-  // In this test, we create a 2-node DAG with a non-blocking link between them.
-  // There is no streaming of data from op1 to op2 during the execution of op1.
-  // op1 produces a partially filled block at the end of its execution which is
-  // rebuilt and then fed to op2.
-  const QueryPlan::DAGNodeIndex id1 = query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
-  const QueryPlan::DAGNodeIndex id2 = query_plan_->addRelationalOperator(new MockOperator(true, true, 3, 1));
-
-  // Create a non-blocking link.
-  query_plan_->addDirectDependency(id2, id1, false);
-
-  // Create a relation, owned by db_.
-  CatalogRelation *relation = new CatalogRelation(nullptr /* catalog_database */, "test_relation");
-  const relation_id output_relation_id = db_->addRelation(relation);
-
-  // Setup the InsertDestination proto in the query context proto.
-  serialization::QueryContext query_context_proto;
-
-  const QueryContext::insert_destination_id insert_destination_index =
-      query_context_proto.insert_destinations_size();
-  serialization::InsertDestination *insert_destination_proto = query_context_proto.add_insert_destinations();
-
-  insert_destination_proto->set_insert_destination_type(serialization::InsertDestinationType::BLOCK_POOL);
-  insert_destination_proto->set_relation_id(output_relation_id);
-  insert_destination_proto->set_relational_op_index(id1);
-
-  MockOperator *op1_mutable =
-      static_cast<MockOperator*>(query_plan_->getQueryPlanDAGMutable()->getNodePayloadMutable(id1));
-  op1_mutable->setInsertDestinationID(insert_destination_index);
-  op1_mutable->setOutputRelationID(output_relation_id);
-
-  const MockOperator &op1 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id1));
-  const MockOperator &op2 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id2));
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-  foreman_->reconstructQueryContextFromProto(query_context_proto);
-
-  // NOTE(zuyu): An operator generally has no ideas about partially filled
-  // blocks, but InsertDestination in QueryContext does.
-  // Mock to add partially filled blocks in the InsertDestination.
-  InsertDestination *insert_destination =
-      foreman_->query_context_->getInsertDestination(insert_destination_index);
-  DCHECK(insert_destination != nullptr);
-  MutableBlockReference block_ref;
-  static_cast<BlockPoolInsertDestination*>(insert_destination)->available_block_refs_.push_back(move(block_ref));
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  startForeman();
-
-  // There's no blocking dependency in the DAG.
-  EXPECT_TRUE(op1.getBlockingDependenciesMet());
-  EXPECT_TRUE(op2.getBlockingDependenciesMet());
-
-  EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(1, op1.getNumWorkOrders());
-
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(0, op2.getNumWorkOrders());
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by op1.
-  EXPECT_EQ(id1, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // Send a message to Foreman upon workorder (generated by op1) completion.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id1));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id1));
-
-  // op1 generates a rebuild workorder. The block is rebuilt and streamed
-  // to Foreman.
-  EXPECT_FALSE(placeDataPipelineMessage(id1));
-
-  // Based on the streamed input, op2's getAllWorkOrders should produce a
-  // workorder.
-  EXPECT_EQ(3, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(1, op2.getNumWorkOrders());
-
-  // Worker receives a rebuild WorkOrder.
-  MockWorkOrder *rebuild_op1;
-  ASSERT_TRUE(popRebuildWorkOrderIfAvailable(&rebuild_op1));
-  // We skip the check for relation ID of the rebuild WorkOrder, as the partially
-  // filled block reference is a mock reference with garbage contents.
-  delete rebuild_op1;
-
-  EXPECT_FALSE(placeRebuildWorkOrderCompleteMessage(id1));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id2));
-
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Worker receives a WorkOrder.
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // Send a message to Foreman upon workorder (generated by op2) completion.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id2));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id2));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id2));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-}
-
-TEST_F(ForemanTest, MultipleNodesNoOutputTest) {
-  // When an operator produces workorders but no output, the Foreman should
-  // check the dependents of this operator to make progress.
-  const QueryPlan::DAGNodeIndex kNumNodes = 5;
-  std::vector<QueryPlan::DAGNodeIndex> ids;
-  ids.reserve(kNumNodes);
-
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    if (i == 0) {
-      ids[i] = query_plan_->addRelationalOperator(new MockOperator(true, false));
-    } else {
-      ids[i] = query_plan_->addRelationalOperator(new MockOperator(true, true));
-    }
-    VLOG(3) << ids[i];
-  }
-
-  /**
-   * The DAG looks like this:
-   *
-   * op1 -> op2 -> op3 -> op4 -> op5
-   *
-   **/
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes - 1; ++i) {
-    query_plan_->addDirectDependency(ids[i + 1], ids[i], false);
-    static_cast<MockOperator*>(query_plan_->getQueryPlanDAGMutable()->getNodePayloadMutable(ids[i]))
-        ->setOutputRelationID(0xdead);
-  }
-
-  std::vector<const MockOperator*> operators;
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    operators.push_back(static_cast<const MockOperator*>(&query_plan_->getQueryPlanDAG().getNodePayload(ids[i])));
-  }
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  startForeman();
-
-  // operators[0] should have produced a workorder by now.
-  EXPECT_EQ(1, operators[0]->getNumWorkOrders());
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(ids[0]));
-  EXPECT_FALSE(getOperatorFinishedStatus(ids[0]));
-
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    EXPECT_EQ(1, operators[ids[i]]->getNumCalls(MockOperator::kGetAllWorkOrders));
-  }
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by operators[0].
-  EXPECT_EQ(ids[0], work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // Send a message to Foreman upon workorder (generated by operators[0])
-  // completion.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(ids[0]));
-
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    EXPECT_EQ(0, getNumWorkOrdersInExecution(ids[i]));
-    EXPECT_TRUE(getOperatorFinishedStatus(ids[i]));
-    if (i < kNumNodes - 1) {
-      EXPECT_EQ(1, operators[i + 1]->getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-    }
-  }
-}
-
-TEST_F(ForemanTest, OutOfOrderWorkOrderCompletionTest) {
-  // Consider two operators, both generate one workorder each. The dependent's
-  // workorder finishes before dependency's workorder.
-  const QueryPlan::DAGNodeIndex id1 = query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
-  const QueryPlan::DAGNodeIndex id2 = query_plan_->addRelationalOperator(new MockOperator(true, true, 2, 1));
-
-  // Create a non-blocking link.
-  query_plan_->addDirectDependency(id2, id1, false);
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-  // There should be two workorders on Worker's private queue, for this test.
-  foreman_->setMaxMessagesPerWorker(2);
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  startForeman();
-
-  // Expect one workorder produced by op1.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Pop a workorder from Foreman's output queue.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // This workorder's source be op1.
-  EXPECT_EQ(id1, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // Send a message to Foreman upon a block (output of op1) getting full.
-  EXPECT_FALSE(placeOutputBlockMessage(id1));
-
-  // op1 is not finished yet because the response of workorder completion hasn't
-  // been received yet.
-  EXPECT_FALSE(getOperatorFinishedStatus(id1));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-
-  // Expect one workorder produced by op2.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Worker receives a WorkOrder.
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // This workorder's source should be op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // As mentioned earlier, op2 finishes before op1.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id2));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // op1's workorder execution is over.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id1));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_TRUE(getOperatorFinishedStatus(id2));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 9ba5978..4f98748 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -228,8 +228,9 @@ class QueryManagerTest : public ::testing::Test {
     db_.reset(new CatalogDatabase(nullptr /* catalog */, "database"));
     storage_manager_.reset(new StorageManager("./"));
     bus_.Initialize();
-    query_handle_.reset(new QueryHandle(0));
+    query_handle_.reset(new QueryHandle(0));  // dummy query ID.
     query_plan_ = query_handle_->getQueryPlanMutable();
+    query_handle_->getQueryContextProtoMutable()->set_query_id(query_handle_->query_id());
   }
 
   inline void constructQueryManager() {
@@ -256,6 +257,7 @@ class QueryManagerTest : public ::testing::Test {
 
     proto.set_block_id(0);  // dummy block ID
     proto.set_relation_id(0);  // dummy relation ID.
+    proto.set_query_id(0);  // dummy query ID.
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const std::size_t proto_length = proto.ByteSize();
@@ -276,6 +278,7 @@ class QueryManagerTest : public ::testing::Test {
     serialization::WorkOrderCompletionMessage proto;
     proto.set_operator_index(index);
     proto.set_worker_thread_index(1);  // dummy worker ID.
+    proto.set_query_id(0);  // dummy query ID.
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const size_t proto_length = proto.ByteSize();
@@ -297,6 +300,7 @@ class QueryManagerTest : public ::testing::Test {
     serialization::WorkOrderCompletionMessage proto;
     proto.set_operator_index(index);
     proto.set_worker_thread_index(1);  // dummy worker thread ID.
+    proto.set_query_id(0);  // dummy query ID.
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const size_t proto_length = proto.ByteSize();
@@ -320,6 +324,7 @@ class QueryManagerTest : public ::testing::Test {
 
     proto.set_block_id(0);  // dummy block ID
     proto.set_relation_id(0);  // dummy relation ID.
+    proto.set_query_id(0);  // dummy query ID.
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const std::size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/tests/WorkOrdersContainer_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/WorkOrdersContainer_unittest.cpp b/query_execution/tests/WorkOrdersContainer_unittest.cpp
index cf133c4..cb583ab 100644
--- a/query_execution/tests/WorkOrdersContainer_unittest.cpp
+++ b/query_execution/tests/WorkOrdersContainer_unittest.cpp
@@ -72,6 +72,7 @@ TEST(WorkOrdersContainerTest, ZeroNUMANodesAddWorkOrderTest) {
   // they get inserted and retrieved correctly.
   std::vector<int> numa_node_ids;
   // A container for one operator and no NUMA nodes.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, 0);
 
   EXPECT_EQ(0u, w.getNumNormalWorkOrders(0));
@@ -104,11 +105,15 @@ TEST(WorkOrdersContainerTest, ZeroNUMANodesAddWorkOrderTest) {
   ASSERT_TRUE(returned_work_order != nullptr);
   EXPECT_EQ(work_order.getID(), static_cast<MockNUMAWorkOrder*>(returned_work_order)->getID());
 
+  EXPECT_EQ(query_id, returned_work_order->getQueryID());
+
   WorkOrder *returned_rebuild_work_order = w.getRebuildWorkOrder(0);
   ASSERT_TRUE(returned_rebuild_work_order != nullptr);
   EXPECT_EQ(work_order1.getID(),
             static_cast<MockNUMAWorkOrder *>(returned_rebuild_work_order)->getID());
 
+  EXPECT_EQ(query_id, returned_rebuild_work_order->getQueryID());
+
   // Container should be empty now.
   EXPECT_EQ(0u, w.getNumNormalWorkOrders(0));
   EXPECT_EQ(0u, w.getNumRebuildWorkOrders(0));
@@ -123,6 +128,7 @@ TEST(WorkOrdersContainerTest, ZeroNUMANodesMultipleWorkOrdersTest) {
   // if they get inserted and retrieved correctly and the order of retrieval.
   // A container for one operator and no NUMA nodes.
   std::vector<int> numa_node_ids;
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, 0);
 
   EXPECT_EQ(0u, w.getNumNormalWorkOrders(0));
@@ -164,6 +170,8 @@ TEST(WorkOrdersContainerTest, ZeroNUMANodesMultipleWorkOrdersTest) {
     ASSERT_TRUE(returned_work_order != nullptr);
     EXPECT_EQ(static_cast<int>(kNumWorkOrders + i),
               static_cast<MockNUMAWorkOrder *>(returned_rebuild_work_order)->getID());
+    EXPECT_EQ(query_id, returned_work_order->getQueryID());
+    EXPECT_EQ(query_id, returned_rebuild_work_order->getQueryID());
   }
 
   // Container should be empty now.
@@ -190,6 +198,7 @@ TEST(WorkOrdersContainerTest, MultipleNUMANodesTest) {
   const std::size_t kNUMANodesUsed = numa_node_ids.size();
 
   // A container for one operator and kNUMANodes.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, kNUMANodes);
 
   for (std::size_t i = 0; i < kNUMANodesUsed; ++i) {
@@ -246,6 +255,9 @@ TEST(WorkOrdersContainerTest, MultipleNUMANodesTest) {
     ASSERT_TRUE(returned_rebuild_work_order != nullptr);
     EXPECT_EQ(rebuild_workorders[i].getID(),
               static_cast<MockNUMAWorkOrder *>(returned_rebuild_work_order)->getID());
+
+    EXPECT_EQ(query_id, returned_work_order->getQueryID());
+    EXPECT_EQ(query_id, returned_rebuild_work_order->getQueryID());
   }
 
   // No workorder should be left for this operator on any NUMA node.
@@ -291,6 +303,7 @@ TEST(WorkOrdersContainerTest, AllTypesWorkOrdersTest) {
   const std::size_t kNUMANodesUsed = numa_nodes.size();
 
   // Create the container.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, kNUMANodes);
 
   w.addNormalWorkOrder(&multiple_numa_work_order, 0);
@@ -331,6 +344,7 @@ TEST(WorkOrdersContainerTest, AllTypesWorkOrdersTest) {
       w.getNormalWorkOrderForNUMANode(0, numa_nodes[0]));
   ASSERT_TRUE(observed_work_order != nullptr);
 
+  EXPECT_EQ(query_id, observed_work_order->getQueryID());
   EXPECT_EQ(one_numa_work_order.getPreferredNUMANodes().front(),
             observed_work_order->getPreferredNUMANodes().front());
   EXPECT_EQ(one_numa_work_order.getID(), observed_work_order->getID());
@@ -348,6 +362,7 @@ TEST(WorkOrdersContainerTest, AllTypesWorkOrdersTest) {
   EXPECT_EQ(no_numa_work_order.getID(),
             static_cast<MockNUMAWorkOrder *>(observed_non_numa_work_order)->getID());
 
+  EXPECT_EQ(query_id, observed_non_numa_work_order->getQueryID());
   EXPECT_EQ(1u, w.getNumNormalWorkOrdersForNUMANode(0, numa_nodes[0]));
   EXPECT_EQ(1u, w.getNumNormalWorkOrdersForNUMANode(0, numa_nodes[1]));
 
@@ -361,6 +376,7 @@ TEST(WorkOrdersContainerTest, AllTypesWorkOrdersTest) {
   ASSERT_TRUE(observed_work_order_multiple_numa_nodes != nullptr);
   EXPECT_EQ(multiple_numa_work_order.getID(), observed_work_order_multiple_numa_nodes->getID());
 
+  EXPECT_EQ(query_id, observed_work_order_multiple_numa_nodes->getQueryID());
   std::vector<int> observed_numa_nodes(
       observed_work_order_multiple_numa_nodes->getPreferredNUMANodes());
   // Look up the expected numa nodes in the observed_numa_nodes vector.
@@ -427,6 +443,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsNormalWorkOrderTest) {
   const std::size_t kNUMANodes = numa_node_ids.size();
 
   // Create the container.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(kNumOperators, kNUMANodes);
 
   std::vector<std::size_t> operator_ids;
@@ -538,6 +555,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsNormalWorkOrderTest) {
             curr_operator_id, single_numa_node_id));
     ASSERT_TRUE(observed_work_order_single_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_single_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_single_numa =
         normal_workorders_one_numa_ids[curr_operator_id];
@@ -550,6 +568,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsNormalWorkOrderTest) {
             curr_operator_id, multiple_numa_node_id));
     ASSERT_TRUE(observed_work_order_multiple_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_multiple_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_multiple_numa =
         normal_workorders_multiple_numa_ids[curr_operator_id];
@@ -562,6 +581,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsNormalWorkOrderTest) {
         static_cast<MockNUMAWorkOrder *>(w.getNormalWorkOrder(curr_operator_id));
     ASSERT_TRUE(observed_work_order_no_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_no_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_no_numa =
         normal_workorders_no_numa_ids[curr_operator_id];
@@ -620,6 +640,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsRebuildWorkOrderTest) {
   const std::size_t kNUMANodes = numa_node_ids.size();
 
   // Create the container.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(kNumOperators, kNUMANodes);
 
   std::vector<std::size_t> operator_ids;
@@ -732,6 +753,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsRebuildWorkOrderTest) {
             curr_operator_id, single_numa_node_id));
     ASSERT_TRUE(observed_work_order_single_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_single_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_single_numa =
         rebuild_workorders_one_numa_ids[curr_operator_id];
@@ -744,6 +766,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsRebuildWorkOrderTest) {
             curr_operator_id, multiple_numa_node_id));
     ASSERT_TRUE(observed_work_order_multiple_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_multiple_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_multiple_numa =
         rebuild_workorders_multiple_numa_ids[curr_operator_id];
@@ -755,6 +778,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsRebuildWorkOrderTest) {
     MockNUMAWorkOrder *observed_work_order_no_numa =
         static_cast<MockNUMAWorkOrder *>(w.getRebuildWorkOrder(curr_operator_id));
 
+    EXPECT_EQ(query_id, observed_work_order_no_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_no_numa =
         rebuild_workorders_no_numa_ids[curr_operator_id];
@@ -772,6 +796,7 @@ TEST(WorkOrdersContainerTest, RetrievalOrderTest) {
   numa_node_ids.push_back(0);
   const std::size_t kNumWorkOrdersPerType = 100;
 
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, 2);
 
   std::vector<int> single_numa_node_workorder_ids;
@@ -820,6 +845,7 @@ TEST(WorkOrdersContainerTest, RetrievalOrderTest) {
     MockNUMAWorkOrder *observed_work_order = static_cast<MockNUMAWorkOrder *>(
         w.getNormalWorkOrder(0, prefer_single_NUMA_node));
     ASSERT_TRUE(observed_work_order != nullptr);
+    EXPECT_EQ(query_id, observed_work_order->getQueryID());
     if (prefer_single_NUMA_node) {
       EXPECT_EQ(*single_numa_it, observed_work_order->getID());
       EXPECT_EQ(1u, observed_work_order->getPreferredNUMANodes().size());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index 0630bca..c7fd018 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -105,6 +105,7 @@ class ExecutionGenerator {
         execution_plan_(DCHECK_NOTNULL(query_handle->getQueryPlanMutable())),
         query_context_proto_(DCHECK_NOTNULL(query_handle->getQueryContextProtoMutable())),
         execution_heuristics_(new ExecutionHeuristics()) {
+    query_context_proto_->set_query_id(query_handle_->query_id());
 #ifdef QUICKSTEP_DISTRIBUTED
     catalog_database_cache_proto_ = DCHECK_NOTNULL(query_handle->getCatalogDatabaseCacheProtoMutable());
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/CMakeLists.txt b/query_optimizer/tests/CMakeLists.txt
index 6ef2a03..5b58f75 100644
--- a/query_optimizer/tests/CMakeLists.txt
+++ b/query_optimizer/tests/CMakeLists.txt
@@ -116,9 +116,11 @@ target_link_libraries(quickstep_queryoptimizer_tests_ExecutionGeneratorTest
                       quickstep_cli_PrintToScreen
                       quickstep_parser_ParseStatement
                       quickstep_parser_SqlParserWrapper
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_Foreman
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_Worker
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
index 56b53ba..ea871d0 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
@@ -24,7 +24,9 @@
 #include "cli/DropRelation.hpp"
 #include "cli/PrintToScreen.hpp"
 #include "parser/ParseStatement.hpp"
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/Foreman.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
 #include "query_execution/Worker.hpp"
 #include "query_optimizer/ExecutionGenerator.hpp"
 #include "query_optimizer/LogicalGenerator.hpp"
@@ -40,6 +42,8 @@
 
 #include "glog/logging.h"
 
+#include "tmb/tagged_message.h"
+
 namespace quickstep {
 
 class CatalogRelation;
@@ -90,13 +94,18 @@ void ExecutionGeneratorTestRunner::runTestCase(
             physical_generator.generatePlan(
                 logical_generator.generatePlan(*result.parsed_statement));
         execution_generator.generatePlan(physical_plan);
-        foreman_->setQueryPlan(
-            query_handle.getQueryPlanMutable()->getQueryPlanDAGMutable());
-
-        foreman_->reconstructQueryContextFromProto(query_handle.getQueryContextProto());
 
-        foreman_->start();
-        foreman_->join();
+        QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
+            main_thread_client_id_,
+            foreman_->getBusClientID(),
+            &query_handle,
+            &bus_);
+
+        // Receive workload completion message from Foreman.
+        const AnnotatedMessage annotated_msg =
+            bus_.Receive(main_thread_client_id_, 0, true);
+        const TaggedMessage &tagged_message = annotated_msg.tagged_message;
+        DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
 
         const CatalogRelation *query_result_relation = query_handle.getQueryResultRelation();
         if (query_result_relation) {


[33/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-20: Added parser support for SQL window aggregation function

Posted by zu...@apache.org.
QUICKSTEP-20: Added parser support for SQL window aggregation function


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

Branch: refs/heads/tmp-relation-col-store
Commit: 00ca1e4b3a9c9838dcb9509058b8a40b0f573617
Parents: 8e825f1
Author: shixuan <sh...@wisc.edu>
Authored: Tue Jun 14 23:07:32 2016 +0000
Committer: shixuan <sh...@wisc.edu>
Committed: Fri Jun 17 20:34:24 2016 +0000

----------------------------------------------------------------------
 parser/CMakeLists.txt                   |   11 +
 parser/ParseBasicExpressions.cpp        |   10 +
 parser/ParseBasicExpressions.hpp        |   45 +
 parser/ParseSelect.hpp                  |   27 +-
 parser/ParseWindow.hpp                  |  201 ++
 parser/SqlLexer.lpp                     |   10 +
 parser/SqlParser.ypp                    |  128 +-
 parser/preprocessed/SqlLexer_gen.cpp    | 1671 ++++++-------
 parser/preprocessed/SqlLexer_gen.hpp    |   16 +-
 parser/preprocessed/SqlParser_gen.cpp   | 3367 ++++++++++++++------------
 parser/preprocessed/SqlParser_gen.hpp   |  182 +-
 parser/tests/Select.test                |  120 +
 query_optimizer/resolver/CMakeLists.txt |    1 +
 query_optimizer/resolver/Resolver.cpp   |    7 +
 14 files changed, 3368 insertions(+), 2428 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index 2488d30..55c4a67 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -113,6 +113,7 @@ add_library(quickstep_parser_ParseSubqueryExpression ParseSubqueryExpression.cpp
 add_library(quickstep_parser_ParseSubqueryTableReference ParseSubqueryTableReference.cpp ParseSubqueryTableReference.hpp)
 add_library(quickstep_parser_ParseTableReference ParseTableReference.cpp ParseTableReference.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)
 add_library(quickstep_parser_SqlParserWrapper SqlParserWrapper.cpp SqlParserWrapper.hpp)
 add_library(quickstep_parser_SqlParser ${BISON_SqlParser_OUTPUTS})
@@ -135,6 +136,7 @@ target_link_libraries(quickstep_parser_ParseBasicExpressions
                       quickstep_parser_ParseLiteralValue
                       quickstep_parser_ParseString
                       quickstep_parser_ParseTreeNode
+                      quickstep_parser_ParseWindow
                       quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_utility_Macros
@@ -241,6 +243,7 @@ target_link_libraries(quickstep_parser_ParseSelect
                       quickstep_parser_ParseSelectionClause
                       quickstep_parser_ParseTableReference
                       quickstep_parser_ParseTreeNode
+                      quickstep_parser_ParseWindow
                       quickstep_utility_Macros
                       quickstep_utility_PtrList)
 target_link_libraries(quickstep_parser_ParseSelectionClause
@@ -300,6 +303,12 @@ target_link_libraries(quickstep_parser_ParseTreeNode
                       quickstep_utility_TreeStringSerializable)
 target_link_libraries(quickstep_parser_ParserUtil
                       quickstep_utility_SqlError)
+target_link_libraries(quickstep_parser_ParseWindow
+                      quickstep_parser_ParseExpression
+                      quickstep_parser_ParseOrderBy
+                      quickstep_parser_ParseString
+                      quickstep_parser_ParseTreeNode
+                      quickstep_utility_PtrList)
 target_link_libraries(quickstep_parser_SqlLexer
                       quickstep_parser_ParseJoinedTableReference
                       quickstep_parser_ParseLiteralValue
@@ -337,6 +346,7 @@ target_link_libraries(quickstep_parser_SqlParser
                       quickstep_parser_ParseSubqueryExpression
                       quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseTableReference
+                      quickstep_parser_ParseWindow
                       quickstep_parser_ParserUtil
                       quickstep_storage_StorageBlockInfo
                       quickstep_types_Type
@@ -419,6 +429,7 @@ target_link_libraries(quickstep_parser
                       quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseTableReference
                       quickstep_parser_ParseTreeNode
+                      quickstep_parser_ParseWindow
                       quickstep_parser_SqlLexer
                       quickstep_parser_SqlParser
                       quickstep_parser_SqlParserWrapper)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/ParseBasicExpressions.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.cpp b/parser/ParseBasicExpressions.cpp
index a9d84ea..bbb6801 100644
--- a/parser/ParseBasicExpressions.cpp
+++ b/parser/ParseBasicExpressions.cpp
@@ -162,6 +162,16 @@ void ParseFunctionCall::getFieldStringItems(
       non_container_child_field_names->push_back("");
       non_container_child_fields->push_back(&argument);
     }
+
+    if (window_name_ != nullptr) {
+      inline_field_names->push_back("window_name");
+      inline_field_values->push_back(window_name_->value());
+    }
+
+    if (window_ != nullptr) {
+      non_container_child_field_names->push_back("window");
+      non_container_child_fields->push_back(window_.get());
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/ParseBasicExpressions.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.hpp b/parser/ParseBasicExpressions.hpp
index dea25d7..64ac119 100644
--- a/parser/ParseBasicExpressions.hpp
+++ b/parser/ParseBasicExpressions.hpp
@@ -30,6 +30,7 @@
 #include "parser/ParseLiteralValue.hpp"
 #include "parser/ParseString.hpp"
 #include "parser/ParseTreeNode.hpp"
+#include "parser/ParseWindow.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrList.hpp"
 
@@ -429,6 +430,46 @@ class ParseFunctionCall : public ParseExpression {
     return star_.get();
   }
 
+  /**
+   * @return The window name.
+   **/
+  const ParseString* window_name() const {
+    return window_name_.get();
+  }
+
+  /**
+   * @return The window.
+   **/
+  const ParseWindow* window() const {
+    return window_.get();
+  }
+
+  /**
+   * @brief Check if this function is a window aggregation function
+   *
+   * @return True if this function is a window aggregation function; false
+   *         otherwise.
+   **/
+  bool isWindow() const {
+    return window_name_ != nullptr || window_ != nullptr;
+  }
+
+  /**
+   * @brief Set the window name.
+   * @param window_name The window name.
+   **/
+  void setWindowName(ParseString *window_name) {
+    window_name_.reset(window_name);
+  }
+
+  /**
+   * @brief Set the window.
+   * @param window The window.
+   **/
+  void setWindow(ParseWindow *window) {
+    window_.reset(window);
+  }
+
   std::string generateName() const override;
 
  protected:
@@ -446,6 +487,10 @@ class ParseFunctionCall : public ParseExpression {
   // Either <arguments_> or <star_> is NULL.
   std::unique_ptr<PtrList<ParseExpression>> arguments_;
   std::unique_ptr<ParseStar> star_;
+  // A window aggregation function should have either <window_name_> or <window_> but not both.
+  // <window_name_> and <window_> will both be NULL if it is not a window function.
+  std::unique_ptr<ParseString> window_name_;
+  std::unique_ptr<ParseWindow> window_;
 
   DISALLOW_COPY_AND_ASSIGN(ParseFunctionCall);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/ParseSelect.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSelect.hpp b/parser/ParseSelect.hpp
index 81e9ab8..930a215 100644
--- a/parser/ParseSelect.hpp
+++ b/parser/ParseSelect.hpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -30,6 +32,7 @@
 #include "parser/ParseSelectionClause.hpp"
 #include "parser/ParseTableReference.hpp"
 #include "parser/ParseTreeNode.hpp"
+#include "parser/ParseWindow.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrList.hpp"
 
@@ -68,7 +71,8 @@ class ParseSelect : public ParseTreeNode {
               ParseGroupBy *group_by,
               ParseHaving *having,
               ParseOrderBy *order_by,
-              ParseLimit *limit)
+              ParseLimit *limit,
+              PtrList<ParseWindow> *window_list)
       : ParseTreeNode(line_number, column_number),
         selection_(selection),
         from_list_(from_list),
@@ -76,7 +80,8 @@ class ParseSelect : public ParseTreeNode {
         group_by_(group_by),
         having_(having),
         order_by_(order_by),
-        limit_(limit) {
+        limit_(limit),
+        window_list_(window_list) {
   }
 
   ~ParseSelect() override {
@@ -152,6 +157,13 @@ class ParseSelect : public ParseTreeNode {
    */
   const ParseLimit* limit() const { return limit_.get(); }
 
+  /**
+   * @brief Gets the parsed WINDOW.
+   *
+   * @return The parsed WINDOW.
+   */
+  const PtrList<ParseWindow>* window_list() const { return window_list_.get(); }
+
  protected:
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
@@ -171,7 +183,7 @@ class ParseSelect : public ParseTreeNode {
     if (from_list_ != nullptr) {
       container_child_field_names->push_back("from_clause");
       container_child_fields->emplace_back();
-      for (const ParseTableReference& from_item : *from_list_) {
+      for (const ParseTableReference &from_item : *from_list_) {
         container_child_fields->back().push_back(&from_item);
       }
     }
@@ -195,6 +207,14 @@ class ParseSelect : public ParseTreeNode {
       non_container_child_field_names->push_back("limit");
       non_container_child_fields->push_back(limit_.get());
     }
+
+    if (window_list_ != nullptr) {
+      container_child_field_names->push_back("window_list");
+      container_child_fields->emplace_back();
+      for (const ParseWindow &window : *window_list_) {
+        container_child_fields->back().push_back(&window);
+      }
+    }
   }
 
  private:
@@ -205,6 +225,7 @@ class ParseSelect : public ParseTreeNode {
   std::unique_ptr<ParseHaving> having_;
   std::unique_ptr<ParseOrderBy> order_by_;
   std::unique_ptr<ParseLimit> limit_;
+  std::unique_ptr<PtrList<ParseWindow>> window_list_;
 
   DISALLOW_COPY_AND_ASSIGN(ParseSelect);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/ParseWindow.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseWindow.hpp b/parser/ParseWindow.hpp
new file mode 100644
index 0000000..7b41683
--- /dev/null
+++ b/parser/ParseWindow.hpp
@@ -0,0 +1,201 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_PARSER_PARSE_WINDOW_HPP_
+#define QUICKSTEP_PARSER_PARSE_WINDOW_HPP_
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parser/ParseExpression.hpp"
+#include "parser/ParseOrderBy.hpp"
+#include "parser/ParseString.hpp"
+#include "parser/ParseTreeNode.hpp"
+#include "utility/PtrList.hpp"
+
+namespace quickstep {
+
+/**
+ * @brief The information of the how the framing in the window is defined
+ **/
+struct ParseFrameInfo : ParseTreeNode {
+  /**
+   * @brief Constructor.
+   * @param row True if the frame mode is ROW, false if it is RANGE.
+   * @param num_pre The number of rows/value of range that is preceding
+   *                the current row in the frame.
+   * @param num_follow The number of rows/value of range that is following
+   *                   the current row in the frame.
+   **/  
+  ParseFrameInfo(const int line_number,
+                 const int column_number,
+                 const bool is_row_in,
+                 const std::int64_t num_preceding_in,
+                 const std::int64_t num_following_in)
+      : ParseTreeNode(line_number, column_number),
+        is_row(is_row_in),
+        num_preceding(num_preceding_in),
+        num_following(num_following_in) {
+  }
+
+  std::string getName() const override { return "FrameInfo"; }
+
+  const bool is_row;
+  const std::int64_t num_preceding;
+  const std::int64_t num_following;
+
+ 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 {
+    inline_field_names->push_back("frame_mode");
+    inline_field_values->push_back(is_row ? "row" : "range");
+
+    inline_field_names->push_back("num_preceding");
+    inline_field_values->push_back(std::to_string(num_preceding));
+
+    inline_field_names->push_back("num_following");
+    inline_field_values->push_back(std::to_string(num_following));
+  }
+};
+
+/**
+ * @brief The parsed representation of a WINDOW definition.
+ **/
+class ParseWindow : public ParseTreeNode {
+ public:
+  /**
+   * @brief Constructor.
+   * @param line_number The line number of the first token of this WINDOW clause
+   *                    in the SQL statement.
+   * @param column_number The column number of the first token of this WINDOW
+   *                      clause in the SQL statement.
+   * @param partition_by_expressions Optional grouping expressions that might be
+   *                                 specified in the SQL statement. Similar to
+   *                                 GROUP BY with regular aggregates.
+   * @param order_by_expressions Optional ordering expressions that might be
+   *                             specified in the SQL statement.
+   * @param frame_info The information about framing.
+   **/
+  ParseWindow(const int line_number,
+              const int column_number,
+              PtrList<ParseExpression> *partition_by_expressions,
+              PtrList<ParseOrderByItem> *order_by_expressions,
+              ParseFrameInfo *frame_info)
+      : ParseTreeNode(line_number, column_number),
+        partition_by_expressions_(partition_by_expressions),
+        order_by_expressions_(order_by_expressions),
+        frame_info_(frame_info) {
+  }
+
+  /**
+   * @brief Destructor.
+   **/
+  ~ParseWindow() override {}
+
+  std::string getName() const override {
+    return "window";
+  }
+
+  /**
+   * @brief Grouping expressions.
+   **/
+  const PtrList<ParseExpression>* partition_by_expressions() const {
+    return partition_by_expressions_.get();
+  }
+
+  /**
+   * @brief Ordering expressions.
+   **/
+  const PtrList<ParseOrderByItem>* order_by_expressions() const {
+    return order_by_expressions_.get();
+  }
+
+  /**
+   * @brief Frame information.
+   **/
+  const ParseFrameInfo* frame_info() const {
+    return frame_info_.get();
+  }
+
+  /**
+   * @return The window name.
+   */
+  const ParseString* name() const {
+    return name_.get();
+  }
+
+  /**
+   * @brief Set the name of the window.
+   * @param name The name of the window.
+   **/
+  void setName(ParseString *name) {
+    name_.reset(name);
+  }
+
+ 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 {
+    if (name_ != nullptr) {
+      inline_field_names->push_back("window_name");
+      inline_field_values->push_back(name_->value());
+    }
+
+    container_child_field_names->push_back("partition_by");
+    container_child_fields->emplace_back();
+    if (partition_by_expressions_ != nullptr) {
+      for (const auto &e : *partition_by_expressions_) {
+        container_child_fields->back().emplace_back(&e);
+      }
+    }
+
+    container_child_field_names->push_back("order_by");
+    container_child_fields->emplace_back();
+    if (order_by_expressions_ != nullptr) {
+      for (const auto &e : *order_by_expressions_) {
+        container_child_fields->back().emplace_back(&e);
+      }
+    }
+
+    if (frame_info_ != nullptr) {
+      non_container_child_field_names->push_back("frame_info");
+      non_container_child_fields->push_back(frame_info_.get());
+    }
+  }
+
+ private:
+  std::unique_ptr<PtrList<ParseExpression>> partition_by_expressions_;
+  std::unique_ptr<PtrList<ParseOrderByItem>> order_by_expressions_;
+  std::unique_ptr<ParseFrameInfo> frame_info_;
+  std::unique_ptr<ParseString> name_;
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_PARSER_PARSE_WINDOW_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index ac1c708..ee34400 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -48,6 +48,7 @@ class ParseCommand;
 struct ParseCopyFromParams;
 class ParseDataType;
 class ParseExpression;
+struct ParseFrameInfo;
 class ParseFunctionCall;
 class ParseGroupBy;
 class ParseHaving;
@@ -86,6 +87,7 @@ class ParseSubqueryExpression;
 class ParseSubqueryTableReference;
 class ParseTableReference;
 class ParseTableReferenceSignature;
+class ParseWindow;
 class Type;
 class UnaryOperation;
 
@@ -189,6 +191,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "constraint"       return TOKEN_CONSTRAINT;
   "copy"             return TOKEN_COPY;
   "create"           return TOKEN_CREATE;
+  "current"          return TOKEN_CURRENT;
   "date"             return TOKEN_DATE;
   "datetime"         return TOKEN_DATETIME;
   "day"              return TOKEN_DAY;
@@ -209,6 +212,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "false"            return TOKEN_FALSE;
   "first"            return TOKEN_FIRST;
   "float"            return TOKEN_FLOAT;
+  "following"        return TOKEN_FOLLOWING;
   "for"              return TOKEN_FOR;
   "foreign"          return TOKEN_FOREIGN;
   "from"             return TOKEN_FROM;
@@ -243,9 +247,11 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "or"               return TOKEN_OR;
   "order"            return TOKEN_ORDER;
   "outer"            return TOKEN_OUTER;
+  "over"             return TOKEN_OVER;
   "partition"        return TOKEN_PARTITION;
   "partitions"       return TOKEN_PARTITIONS;
   "percent"          return TOKEN_PERCENT;
+  "preceding"        return TOKEN_PRECEDING;
   "primary"          return TOKEN_PRIMARY;
   "quit"             return TOKEN_QUIT;
   "range"            return TOKEN_RANGE;
@@ -253,7 +259,9 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "references"       return TOKEN_REFERENCES;
   "regexp"           return TOKEN_REGEXP;
   "right"            return TOKEN_RIGHT;
+  "row"              return TOKEN_ROW;
   "row_delimiter"    return TOKEN_ROW_DELIMITER;
+  "rows"             return TOKEN_ROWS;
   "second"           return TOKEN_SECOND;
   "select"           return TOKEN_SELECT;
   "set"              return TOKEN_SET;
@@ -266,6 +274,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "timestamp"        return TOKEN_TIMESTAMP;
   "true"             return TOKEN_TRUE;
   "tuplesample"      return TOKEN_TUPLESAMPLE;
+  "unbounded"        return TOKEN_UNBOUNDED;
   "unique"           return TOKEN_UNIQUE;
   "update"           return TOKEN_UPDATE;
   "using"            return TOKEN_USING;
@@ -273,6 +282,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "varchar"          return TOKEN_VARCHAR;
   "when"             return TOKEN_WHEN;
   "where"            return TOKEN_WHERE;
+  "window"           return TOKEN_WINDOW;
   "with"             return TOKEN_WITH;
   "year"             return TOKEN_YEAR;
   "yearmonth"        return TOKEN_YEARMONTH;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index b07c48e..81fa3ae 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -95,6 +95,7 @@ typedef struct YYLTYPE {
 #include "parser/ParseSubqueryExpression.hpp"
 #include "parser/ParseSubqueryTableReference.hpp"
 #include "parser/ParseTableReference.hpp"
+#include "parser/ParseWindow.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
@@ -200,6 +201,12 @@ typedef void* yyscan_t;
 
   quickstep::ParseSample *opt_sample_clause_;
 
+  quickstep::PtrList<quickstep::ParseWindow> *opt_window_clause_;
+  quickstep::ParseWindow *window_definition_;
+  quickstep::PtrList<quickstep::ParseExpression> *window_partition_by_list_;
+  quickstep::PtrList<quickstep::ParseOrderByItem> *window_order_by_list_;
+  quickstep::ParseFrameInfo *window_frame_info_;
+
   quickstep::PtrList<quickstep::ParseOrderByItem> *order_commalist_;
   quickstep::ParseOrderByItem *order_item_;
 
@@ -254,6 +261,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_CONSTRAINT;
 %token TOKEN_COPY;
 %token TOKEN_CREATE;
+%token TOKEN_CURRENT;
 %token TOKEN_DATE;
 %token TOKEN_DATETIME;
 %token TOKEN_DAY;
@@ -273,6 +281,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_FALSE;
 %token TOKEN_FIRST;
 %token TOKEN_FLOAT;
+%token TOKEN_FOLLOWING;
 %token TOKEN_FOR;
 %token TOKEN_FOREIGN;
 %token TOKEN_FROM;
@@ -304,9 +313,11 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_OR;
 %token TOKEN_ORDER;
 %token TOKEN_OUTER;
+%token TOKEN_OVER;
 %token TOKEN_PARTITION;
 %token TOKEN_PARTITIONS;
 %token TOKEN_PERCENT;
+%token TOKEN_PRECEDING;
 %token TOKEN_PRIMARY;
 %token TOKEN_QUIT;
 %token TOKEN_RANGE;
@@ -314,7 +325,9 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_REFERENCES;
 %token TOKEN_REGEXP;
 %token TOKEN_RIGHT;
+%token TOKEN_ROW;
 %token TOKEN_ROW_DELIMITER;
+%token TOKEN_ROWS;
 %token TOKEN_SECOND;
 %token TOKEN_SELECT;
 %token TOKEN_SET;
@@ -327,6 +340,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_TIMESTAMP;
 %token TOKEN_TRUE;
 %token TOKEN_TUPLESAMPLE;
+%token TOKEN_UNBOUNDED;
 %token TOKEN_UNIQUE;
 %token TOKEN_UPDATE;
 %token TOKEN_USING;
@@ -334,6 +348,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_VARCHAR;
 %token TOKEN_WHEN;
 %token TOKEN_WHERE;
+%token TOKEN_WINDOW;
 %token TOKEN_WITH;
 %token TOKEN_YEAR;
 %token TOKEN_YEARMONTH;
@@ -348,10 +363,15 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 
 %type <boolean_value_>
   boolean_value
+  frame_mode
 
 %type <literal_value_>
   literal_value
 
+%type <numeric_literal_value_>
+  frame_preceding
+  frame_following
+
 %type <literal_value_list_>
   literal_value_commalist
 
@@ -546,6 +566,23 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %type <opt_sample_clause_>
   opt_sample_clause
 
+%type <opt_window_clause_>
+  opt_window_clause
+  window_declaration_list
+
+%type <window_definition_>
+  window_declaration
+  window_definition
+
+%type <window_partition_by_list_>
+  opt_window_partition
+
+%type <window_order_by_list_>
+  opt_window_order
+
+%type <window_frame_info_>
+  opt_window_frame
+
 %type <with_list_>
   with_clause
   with_list
@@ -1136,8 +1173,8 @@ with_list_element:
 
 select_query:
   TOKEN_SELECT opt_all_distinct selection from_clause opt_where_clause opt_group_by_clause opt_having_clause
-      opt_order_by_clause opt_limit_clause {
-    $$ = new quickstep::ParseSelect(@1.first_line, @1.first_column, $3, $4, $5, $6, $7, $8, $9);
+      opt_order_by_clause opt_limit_clause opt_window_clause {
+    $$ = new quickstep::ParseSelect(@1.first_line, @1.first_column, $3, $4, $5, $6, $7, $8, $9, $10);
   };
 
 opt_all_distinct:
@@ -1332,6 +1369,85 @@ opt_limit_clause:
     }
   }
 
+opt_window_clause:
+  {
+    $$ = nullptr;
+  }
+  | window_declaration_list {
+    $$ = $1;
+  }
+
+window_declaration_list:
+  window_declaration {
+    $$ = new quickstep::PtrList<quickstep::ParseWindow>();
+    $$->push_back($1);
+  }
+  | window_declaration_list window_declaration {
+    $$ = $1;
+    $$->push_back($2);
+  }
+
+window_declaration:
+  TOKEN_WINDOW any_name TOKEN_AS '(' window_definition ')' {
+    $$ = $5;
+    $$->setName($2);
+  } 
+
+window_definition:
+  opt_window_partition opt_window_order opt_window_frame {
+    $$ = new quickstep::ParseWindow(@1.first_line, @1.first_column, $1, $2, $3);
+  };
+
+opt_window_partition:
+  {
+    $$ = nullptr;
+  }
+  | TOKEN_PARTITION TOKEN_BY expression_list {
+    $$ = $3;
+  };
+
+opt_window_order:
+  {
+    $$ = nullptr;
+  }
+  | TOKEN_ORDER TOKEN_BY order_commalist  {
+    $$ = $3;
+  };
+
+opt_window_frame:
+  {
+    $$ = nullptr;
+  }
+  | frame_mode TOKEN_BETWEEN frame_preceding TOKEN_AND frame_following {
+    $$ = new quickstep::ParseFrameInfo(@1.first_line, @1.first_column, $1, $3->long_value(), $5->long_value());
+  };
+
+frame_mode:
+  TOKEN_ROWS {
+    $$ = true;
+  }
+  | TOKEN_RANGE {
+    $$ = false;
+  };
+
+frame_preceding:
+  TOKEN_UNSIGNED_NUMVAL TOKEN_PRECEDING
+  | TOKEN_UNBOUNDED TOKEN_PRECEDING { 
+    $$ = new quickstep::NumericParseLiteralValue(@1.first_line, @1.first_column, "-1");
+  }
+  | TOKEN_CURRENT TOKEN_ROW {
+    $$ = new quickstep::NumericParseLiteralValue(@1.first_line, @1.first_column, "0");
+  };
+
+frame_following:
+  TOKEN_UNSIGNED_NUMVAL TOKEN_FOLLOWING
+  | TOKEN_UNBOUNDED TOKEN_FOLLOWING {
+    $$ = new quickstep::NumericParseLiteralValue(@1.first_line, @1.first_column, "-1");
+  }
+  | TOKEN_CURRENT TOKEN_ROW {
+    $$ = new quickstep::NumericParseLiteralValue(@1.first_line, @1.first_column, "0");
+  };
+
 order_commalist:
   order_item {
     $$ = new quickstep::PtrList<quickstep::ParseOrderByItem>();
@@ -1505,6 +1621,14 @@ expression_base:
   | function_call {
     $$ = $1;
   }
+  | function_call TOKEN_OVER any_name {
+    $1->setWindowName($3);
+    $$ = $1;
+  }
+  | function_call TOKEN_OVER '(' window_definition ')' {
+    $1->setWindow($4);
+    $$ = $1;
+  }
   | extract_function {
     $$ = $1;
   }


[31/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-20: Added parser support for SQL window aggregation function

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/preprocessed/SqlLexer_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.hpp b/parser/preprocessed/SqlLexer_gen.hpp
index c14559b..f6cd8ad 100644
--- a/parser/preprocessed/SqlLexer_gen.hpp
+++ b/parser/preprocessed/SqlLexer_gen.hpp
@@ -12,8 +12,8 @@
 
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
-#define YY_FLEX_MINOR_VERSION 6
-#define YY_FLEX_SUBMINOR_VERSION 0
+#define YY_FLEX_MINOR_VERSION 5
+#define YY_FLEX_SUBMINOR_VERSION 39
 #if YY_FLEX_SUBMINOR_VERSION > 0
 #define FLEX_BETA
 #endif
@@ -222,7 +222,7 @@ void quickstep_yyfree (void * ,yyscan_t yyscanner );
 
 /* Begin user sect3 */
 
-#define quickstep_yywrap(yyscanner) (/*CONSTCOND*/1)
+#define quickstep_yywrap(yyscanner) 1
 #define YY_SKIP_YYWRAP
 
 #define yytext_ptr yytext_r
@@ -268,11 +268,11 @@ void quickstep_yyset_extra (YY_EXTRA_TYPE user_defined ,yyscan_t yyscanner );
 
 FILE *quickstep_yyget_in (yyscan_t yyscanner );
 
-void quickstep_yyset_in  (FILE * _in_str ,yyscan_t yyscanner );
+void quickstep_yyset_in  (FILE * in_str ,yyscan_t yyscanner );
 
 FILE *quickstep_yyget_out (yyscan_t yyscanner );
 
-void quickstep_yyset_out  (FILE * _out_str ,yyscan_t yyscanner );
+void quickstep_yyset_out  (FILE * out_str ,yyscan_t yyscanner );
 
 yy_size_t quickstep_yyget_leng (yyscan_t yyscanner );
 
@@ -280,11 +280,11 @@ char *quickstep_yyget_text (yyscan_t yyscanner );
 
 int quickstep_yyget_lineno (yyscan_t yyscanner );
 
-void quickstep_yyset_lineno (int _line_number ,yyscan_t yyscanner );
+void quickstep_yyset_lineno (int line_number ,yyscan_t yyscanner );
 
 int quickstep_yyget_column  (yyscan_t yyscanner );
 
-void quickstep_yyset_column (int _column_no ,yyscan_t yyscanner );
+void quickstep_yyset_column (int column_no ,yyscan_t yyscanner );
 
 YYSTYPE * quickstep_yyget_lval (yyscan_t yyscanner );
 
@@ -360,7 +360,7 @@ extern int quickstep_yylex \
 #undef YY_DECL
 #endif
 
-#line 454 "../SqlLexer.lpp"
+#line 463 "../SqlLexer.lpp"
 
 
 #line 367 "SqlLexer_gen.hpp"


[15/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-10: Serialized WorkOrders as proto.

Posted by zu...@apache.org.
QUICKSTEP-10: Serialized WorkOrders as proto.


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

Branch: refs/heads/tmp-relation-col-store
Commit: c9214ecb1d481b3d1b02db0ffdf53852b11b540f
Parents: ccd11c0
Author: Zuyu Zhang <zz...@pivotal.io>
Authored: Tue Apr 12 16:55:48 2016 -0700
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Jun 9 17:06:02 2016 -0700

----------------------------------------------------------------------
 query_execution/CMakeLists.txt                  |   6 +
 query_execution/WorkOrderProtosContainer.hpp    | 146 +++++++++++++++++++
 query_execution/tests/Foreman_unittest.cpp      |   6 +
 query_execution/tests/QueryManager_unittest.cpp |   6 +
 relational_operators/AggregationOperator.cpp    |  34 +++++
 relational_operators/AggregationOperator.hpp    |  12 ++
 relational_operators/BuildHashOperator.cpp      |  40 +++++
 relational_operators/BuildHashOperator.hpp      |  12 ++
 relational_operators/CMakeLists.txt             |  36 ++++-
 relational_operators/CreateIndexOperator.hpp    |   9 ++
 relational_operators/CreateTableOperator.hpp    |   8 +
 relational_operators/DeleteOperator.cpp         |  38 ++++-
 relational_operators/DeleteOperator.hpp         |  14 +-
 relational_operators/DestroyHashOperator.cpp    |  17 +++
 relational_operators/DestroyHashOperator.hpp    |   3 +
 relational_operators/DropTableOperator.cpp      |  23 +++
 relational_operators/DropTableOperator.hpp      |   3 +
 .../FinalizeAggregationOperator.cpp             |  20 +++
 .../FinalizeAggregationOperator.hpp             |   3 +
 relational_operators/HashJoinOperator.cpp       | 124 ++++++++++++++++
 relational_operators/HashJoinOperator.hpp       |  21 +++
 relational_operators/InsertOperator.cpp         |  19 +++
 relational_operators/InsertOperator.hpp         |   3 +
 .../NestedLoopsJoinOperator.cpp                 | 142 ++++++++++++++++++
 .../NestedLoopsJoinOperator.hpp                 |  51 +++++++
 relational_operators/RelationalOperator.hpp     |  22 +++
 relational_operators/SampleOperator.cpp         | 101 ++++++++++---
 relational_operators/SampleOperator.hpp         |  12 ++
 relational_operators/SaveBlocksOperator.cpp     |  18 +++
 relational_operators/SaveBlocksOperator.hpp     |   3 +
 relational_operators/SelectOperator.cpp         |  43 ++++++
 relational_operators/SelectOperator.hpp         |  12 ++
 relational_operators/SortMergeRunOperator.cpp   |  68 +++++++++
 relational_operators/SortMergeRunOperator.hpp   |  12 ++
 .../SortRunGenerationOperator.cpp               |  39 +++++
 .../SortRunGenerationOperator.hpp               |  12 ++
 relational_operators/TableGeneratorOperator.cpp |  20 ++-
 relational_operators/TableGeneratorOperator.hpp |   5 +-
 relational_operators/TextScanOperator.cpp       | 125 ++++++++++++----
 relational_operators/TextScanOperator.hpp       |  24 ++-
 relational_operators/TextScanOperator.proto     |  22 ---
 relational_operators/UpdateOperator.cpp         |  23 +++
 relational_operators/UpdateOperator.hpp         |   3 +
 relational_operators/WorkOrder.proto            |  21 +--
 relational_operators/WorkOrderFactory.cpp       |   7 +-
 45 files changed, 1291 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 8306f78..95bc0d6 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -42,6 +42,7 @@ add_library(quickstep_queryexecution_QueryExecutionState ../empty_src.cpp QueryE
 add_library(quickstep_queryexecution_QueryExecutionTypedefs ../empty_src.cpp QueryExecutionTypedefs.hpp)
 add_library(quickstep_queryexecution_QueryExecutionUtil ../empty_src.cpp QueryExecutionUtil.hpp)
 add_library(quickstep_queryexecution_QueryManager QueryManager.cpp QueryManager.hpp)
+add_library(quickstep_queryexecution_WorkOrderProtosContainer ../empty_src.cpp WorkOrderProtosContainer.hpp)
 add_library(quickstep_queryexecution_WorkOrdersContainer WorkOrdersContainer.cpp WorkOrdersContainer.hpp)
 add_library(quickstep_queryexecution_Worker Worker.cpp Worker.hpp)
 add_library(quickstep_queryexecution_WorkerDirectory ../empty_src.cpp WorkerDirectory.hpp)
@@ -157,6 +158,10 @@ target_link_libraries(quickstep_queryexecution_QueryManager
                       quickstep_utility_DAG
                       quickstep_utility_Macros
                       tmb)
+target_link_libraries(quickstep_queryexecution_WorkOrderProtosContainer
+                      glog
+                      quickstep_relationaloperators_WorkOrder_proto
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryexecution_WorkOrdersContainer
                       glog
                       quickstep_relationaloperators_WorkOrder
@@ -193,6 +198,7 @@ target_link_libraries(quickstep_queryexecution
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_QueryManager
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_queryexecution_Worker
                       quickstep_queryexecution_WorkerDirectory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/query_execution/WorkOrderProtosContainer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkOrderProtosContainer.hpp b/query_execution/WorkOrderProtosContainer.hpp
new file mode 100644
index 0000000..5043755
--- /dev/null
+++ b/query_execution/WorkOrderProtosContainer.hpp
@@ -0,0 +1,146 @@
+/**
+ *   Copyright 2015-2016 Pivotal Software, Inc.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_WORKORDER_PROTOS_CONTAINER_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_WORKORDER_PROTOS_CONTAINER_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <queue>
+#include <vector>
+
+#include "relational_operators/WorkOrder.pb.h"  // IWYU pragma: keep
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup QueryExecution
+ *  @{
+ */
+
+ /**
+  * @brief A container used in the distributed version to hold the normal
+  *        (non-rebuild) WorkOrder protos for a given query.
+  *
+  * @note This container stays alive during the lifetime of the query.
+  **/
+class WorkOrderProtosContainer {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param num_operators Number of operators in the query DAG.
+   **/
+  explicit WorkOrderProtosContainer(const std::size_t num_operators)
+      : num_operators_(num_operators),
+        operator_containers_(num_operators_) {
+    DCHECK_NE(num_operators_, 0u);
+  }
+
+  /**
+   * @brief Destructor.
+   *
+   * @note If the query is executed normally, we should never encounter a
+   *       situation where at the time of deletion the WorkOrderProtosContainer has
+   *       pending WorkOrders.
+   **/
+  ~WorkOrderProtosContainer() {
+    for (std::size_t op = 0; op < num_operators_; ++op) {
+      if (hasWorkOrderProto(op)) {
+        LOG(WARNING) << "Destroying a WorkOrderProtosContainer that still has pending WorkOrder protos.";
+        break;
+      }
+    }
+  }
+
+  /**
+   * @brief Check if there are some pending WorkOrders for the given operator.
+   *
+   * @param operator_index Index of the operator.
+   *
+   * @return If there are pending WorkOrders.
+   **/
+  bool hasWorkOrderProto(const std::size_t operator_index) const {
+    DCHECK_LT(operator_index, num_operators_);
+    return !operator_containers_[operator_index].empty();
+  }
+
+  /**
+   * @brief Get a WorkOrder for a given operator.
+   *
+   * @param operator_index The index of the operator.
+   *
+   * @return Release a WorkOrder proto. If no WorkOrder proto is available,
+   *         return nullptr.
+   **/
+  serialization::WorkOrder* getWorkOrderProto(const std::size_t operator_index) {
+    DCHECK_LT(operator_index, num_operators_);
+
+    if (operator_containers_[operator_index].empty()) {
+      return nullptr;
+    }
+
+    serialization::WorkOrder *proto =
+        operator_containers_[operator_index].front().release();
+    operator_containers_[operator_index].pop();
+
+    return proto;
+  }
+
+  /**
+   * @brief Add a WorkOrder generated from a given
+   *        operator.
+   *
+   * @param workorder A pointer to the WorkOrder to be added.
+   * @param operator_index The index of the operator in the query DAG.
+   **/
+  void addWorkOrderProto(serialization::WorkOrder *proto,
+                         const std::size_t operator_index) {
+    DCHECK(proto != nullptr);
+    DCHECK_LT(operator_index, num_operators_);
+
+    operator_containers_[operator_index].emplace(
+        std::unique_ptr<serialization::WorkOrder>(proto));
+  }
+
+  /**
+   * @brief Get the number of all pending WorkOrders
+   *        for a given operator.
+   *
+   * @param operator_index The index of the operator.
+   *
+   * @return The number of pending WorkOrders.
+   **/
+  std::size_t getNumWorkOrderProtos(const std::size_t operator_index) const {
+    DCHECK_LT(operator_index, num_operators_);
+    return operator_containers_[operator_index].size();
+  }
+
+ private:
+  const std::size_t num_operators_;
+
+  std::vector<std::queue<std::unique_ptr<serialization::WorkOrder>>> operator_containers_;
+
+  DISALLOW_COPY_AND_ASSIGN(WorkOrderProtosContainer);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_WORKORDER_PROTOS_CONTAINER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/query_execution/tests/Foreman_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/Foreman_unittest.cpp b/query_execution/tests/Foreman_unittest.cpp
index 79f8f4a..cbe5088 100644
--- a/query_execution/tests/Foreman_unittest.cpp
+++ b/query_execution/tests/Foreman_unittest.cpp
@@ -58,6 +58,8 @@ using tmb::client_id;
 
 namespace quickstep {
 
+class WorkOrderProtosContainer;
+
 class MockWorkOrder : public WorkOrder {
  public:
   explicit MockWorkOrder(const int op_index)
@@ -168,6 +170,10 @@ class MockOperator: public RelationalOperator {
     return num_calls_get_workorders_ == max_getworkorder_iters_;
   }
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
+    return true;
+  }
+
   void feedInputBlock(const block_id input_block_id,
                       const relation_id input_relation_id) override {
     ++num_calls_feedblock_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 308d5ca..9ba5978 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -59,6 +59,8 @@ using tmb::client_id;
 
 namespace quickstep {
 
+class WorkOrderProtosContainer;
+
 class MockWorkOrder : public WorkOrder {
  public:
   explicit MockWorkOrder(const int op_index)
@@ -169,6 +171,10 @@ class MockOperator: public RelationalOperator {
     return num_calls_get_workorders_ == max_getworkorder_iters_;
   }
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
+    return true;
+  }
+
   void feedInputBlock(const block_id input_block_id,
                       const relation_id input_relation_id) override {
     ++num_calls_feedblock_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/AggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.cpp b/relational_operators/AggregationOperator.cpp
index 7252541..6d16930 100644
--- a/relational_operators/AggregationOperator.cpp
+++ b/relational_operators/AggregationOperator.cpp
@@ -20,7 +20,9 @@
 #include <vector>
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/AggregationOperationState.hpp"
 #include "storage/StorageBlockInfo.hpp"
 
@@ -61,6 +63,38 @@ bool AggregationOperator::getAllWorkOrders(
   }
 }
 
+bool AggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id input_block_id : input_relation_block_ids_) {
+        container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* AggregationOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::AGGREGATION);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::AggregationWorkOrder::block_id, block);
+  proto->SetExtension(serialization::AggregationWorkOrder::aggr_state_index, aggr_state_index_);
+
+  return proto;
+}
+
+
 void AggregationWorkOrder::execute() {
   state_->aggregateBlock(input_block_id_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index f340d4e..4bcbcf6 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -38,8 +38,11 @@ namespace quickstep {
 
 class AggregationOperationState;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -80,6 +83,8 @@ class AggregationOperator : public RelationalOperator {
                         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) override {
     input_relation_block_ids_.push_back(input_block_id);
   }
@@ -91,6 +96,13 @@ class AggregationOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const bool input_relation_is_stored_;
   std::vector<block_id> input_relation_block_ids_;
   const QueryContext::aggregation_state_id aggr_state_index_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index 9dc4afe..1c2ff05 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -22,7 +22,9 @@
 
 #include "catalog/CatalogRelation.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/HashTable.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -99,6 +101,44 @@ bool BuildHashOperator::getAllWorkOrders(
   }
 }
 
+bool BuildHashOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id input_block_id : input_relation_block_ids_) {
+        container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* BuildHashOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::BUILD_HASH);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::BuildHashWorkOrder::relation_id, input_relation_.getID());
+  for (const attribute_id attr_id : join_key_attributes_) {
+    proto->AddExtension(serialization::BuildHashWorkOrder::join_key_attributes, attr_id);
+  }
+  proto->SetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable,
+                      any_join_key_attributes_nullable_);
+  proto->SetExtension(serialization::BuildHashWorkOrder::join_hash_table_index, hash_table_index_);
+  proto->SetExtension(serialization::BuildHashWorkOrder::block_id, block);
+
+  return proto;
+}
+
+
 void BuildHashWorkOrder::execute() {
   BlockReference block(
       storage_manager_->getBlock(build_block_id_, input_relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 50dd7d6..464bbf8 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -39,6 +39,7 @@ namespace quickstep {
 
 class CatalogRelationSchema;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 struct TupleReference;
@@ -46,6 +47,8 @@ struct TupleReference;
 template <typename, bool, bool, bool, bool> class HashTable;
 typedef HashTable<TupleReference, true, false, false, true> JoinHashTable;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -96,6 +99,8 @@ class BuildHashOperator : public RelationalOperator {
                         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) override {
     input_relation_block_ids_.push_back(input_block_id);
@@ -109,6 +114,13 @@ class BuildHashOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &input_relation_;
   const bool input_relation_is_stored_;
   const std::vector<attribute_id> join_key_attributes_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index eb73c07..91d1097 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -73,9 +73,11 @@ target_link_libraries(quickstep_relationaloperators_AggregationOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_AggregationOperationState
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_Macros
@@ -85,9 +87,11 @@ target_link_libraries(quickstep_relationaloperators_BuildHashOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_HashTable
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -120,9 +124,11 @@ target_link_libraries(quickstep_relationaloperators_DeleteOperator
                       quickstep_queryexecution_QueryExecutionMessages_proto
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
+                      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
@@ -132,9 +138,11 @@ target_link_libraries(quickstep_relationaloperators_DeleteOperator
 target_link_libraries(quickstep_relationaloperators_DestroyHashOperator
                       glog
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_relationaloperators_DropTableOperator
@@ -143,9 +151,11 @@ target_link_libraries(quickstep_relationaloperators_DropTableOperator
                       quickstep_catalog_CatalogDatabaseLite
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
                       quickstep_utility_Macros
@@ -155,9 +165,11 @@ target_link_libraries(quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_AggregationOperationState
                       quickstep_utility_Macros
                       tmb)
@@ -170,9 +182,11 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_HashTable
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
@@ -194,9 +208,11 @@ target_link_libraries(quickstep_relationaloperators_InsertOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_types_containers_Tuple
                       quickstep_utility_Macros
@@ -209,9 +225,11 @@ target_link_libraries(quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -245,9 +263,11 @@ target_link_libraries(quickstep_relationaloperators_SampleOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
-                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
+                      quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -257,9 +277,11 @@ target_link_libraries(quickstep_relationaloperators_SampleOperator
 target_link_libraries(quickstep_relationaloperators_SaveBlocksOperator
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
                       quickstep_utility_Macros
@@ -270,9 +292,11 @@ target_link_libraries(quickstep_relationaloperators_SelectOperator
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_PartitionSchemeHeader
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -289,11 +313,13 @@ target_link_libraries(quickstep_relationaloperators_SortMergeRunOperator
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_SortMergeRunOperatorHelpers
                       quickstep_relationaloperators_SortMergeRunOperator_proto
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_threading_ThreadIDBasedMap
                       quickstep_utility_Macros
@@ -326,9 +352,11 @@ target_link_libraries(quickstep_relationaloperators_SortRunGenerationOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -343,9 +371,11 @@ target_link_libraries(quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_tablegenerator_GeneratorFunctionHandle
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlockInfo
                       quickstep_types_containers_ColumnVectorsValueAccessor
@@ -358,9 +388,11 @@ target_link_libraries(quickstep_relationaloperators_TextScanOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_types_Type
                       quickstep_types_TypedValue
@@ -379,9 +411,11 @@ target_link_libraries(quickstep_relationaloperators_UpdateOperator
                       quickstep_queryexecution_QueryExecutionMessages_proto
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index 11a01ae..18ca656 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -1,6 +1,7 @@
 /**
  *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
  *     University of Wisconsin\u2014Madison.
+ *   Copyright 2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -37,6 +38,7 @@ namespace quickstep {
 class CatalogRelation;
 class QueryContext;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -76,6 +78,13 @@ class CreateIndexOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  /**
+   * @note no WorkOrder proto generated for this operator.
+   **/
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
+    return true;
+  }
+
   void updateCatalogOnCompletion() override;
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 60bcef4..6d91142 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -36,6 +36,7 @@ namespace quickstep {
 class CatalogDatabase;
 class QueryContext;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -74,6 +75,13 @@ class CreateTableOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  /**
+   * @note no WorkOrder proto generated for this operator.
+   **/
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
+    return true;
+  }
+
   void updateCatalogOnCompletion() override;
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 15dc9e3..47e36e9 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -26,13 +26,14 @@
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryExecutionUtil.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 "threading/ThreadIDBasedMap.hpp"
 
-
 #include "glog/logging.h"
 
 #include "tmb/id_typedefs.h"
@@ -85,6 +86,41 @@ bool DeleteOperator::getAllWorkOrders(
   }
 }
 
+bool DeleteOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (relation_is_stored_) {
+    // If relation_ is stored, iterate over the list of blocks in relation_.
+    if (!started_) {
+      for (const block_id input_block_id : relation_block_ids_) {
+        container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* DeleteOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::DELETE);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::DeleteWorkOrder::operator_index, op_index_);
+  proto->SetExtension(serialization::DeleteWorkOrder::relation_id, relation_.getID());
+  proto->SetExtension(serialization::DeleteWorkOrder::predicate_index, predicate_index_);
+  proto->SetExtension(serialization::DeleteWorkOrder::block_id, block);
+
+  return proto;
+}
+
+
 void DeleteWorkOrder::execute() {
   MutableBlockReference block(
       storage_manager_->getBlockMutable(input_block_id_, input_relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index c55f585..aa8a688 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -41,8 +41,11 @@ namespace quickstep {
 class CatalogRelationSchema;
 class Predicate;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -67,7 +70,7 @@ class DeleteOperator : public RelationalOperator {
                  const CatalogRelation &relation,
                  const QueryContext::predicate_id predicate_index,
                  const bool relation_is_stored)
-     :  RelationalOperator(query_id),
+      : RelationalOperator(query_id),
         relation_(relation),
         predicate_index_(predicate_index),
         relation_is_stored_(relation_is_stored),
@@ -84,6 +87,8 @@ class DeleteOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   const relation_id getOutputRelationID() const override {
     return relation_.getID();
   }
@@ -101,6 +106,13 @@ class DeleteOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &relation_;
   const QueryContext::predicate_id predicate_index_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DestroyHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.cpp b/relational_operators/DestroyHashOperator.cpp
index 8aa40b4..e748470 100644
--- a/relational_operators/DestroyHashOperator.cpp
+++ b/relational_operators/DestroyHashOperator.cpp
@@ -18,7 +18,9 @@
 #include "relational_operators/DestroyHashOperator.hpp"
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 
 #include "tmb/id_typedefs.h"
 
@@ -39,6 +41,21 @@ bool DestroyHashOperator::getAllWorkOrders(
   return work_generated_;
 }
 
+bool DestroyHashOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !work_generated_) {
+    work_generated_ = true;
+
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::DESTROY_HASH);
+    proto->set_query_id(query_id_);
+    proto->SetExtension(serialization::DestroyHashWorkOrder::join_hash_table_index, hash_table_index_);
+
+    container->addWorkOrderProto(proto, op_index_);
+  }
+  return work_generated_;
+}
+
+
 void DestroyHashWorkOrder::execute() {
   query_context_->destroyJoinHashTable(hash_table_index_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 7d8acb7..181386f 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -32,6 +32,7 @@ namespace tmb { class MessageBus; }
 namespace quickstep {
 
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -63,6 +64,8 @@ class DestroyHashOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
  private:
   const QueryContext::join_hash_table_id hash_table_index_;
   bool work_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DropTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.cpp b/relational_operators/DropTableOperator.cpp
index 256f6a1..727aa46 100644
--- a/relational_operators/DropTableOperator.cpp
+++ b/relational_operators/DropTableOperator.cpp
@@ -24,7 +24,9 @@
 #include "catalog/CatalogDatabaseLite.hpp"
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
 
@@ -55,6 +57,27 @@ bool DropTableOperator::getAllWorkOrders(
   return work_generated_;
 }
 
+bool DropTableOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !work_generated_) {
+    work_generated_ = true;
+
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::DROP_TABLE);
+    proto->set_query_id(query_id_);
+
+    std::vector<block_id> relation_blocks(relation_.getBlocksSnapshot());
+    for (const block_id relation_block : relation_blocks) {
+      proto->AddExtension(serialization::DropTableWorkOrder::block_ids, relation_block);
+    }
+
+    container->addWorkOrderProto(proto, op_index_);
+
+    database_->setStatus(CatalogDatabase::Status::kPendingBlockDeletions);
+  }
+
+  return work_generated_;
+}
+
 void DropTableOperator::updateCatalogOnCompletion() {
   const relation_id rel_id = relation_.getID();
   if (only_drop_blocks_) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index a0a8d6e..6c7fca3 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -41,6 +41,7 @@ class CatalogDatabaseLite;
 class CatalogRelation;
 class QueryContext;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -79,6 +80,8 @@ class DropTableOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void updateCatalogOnCompletion() override;
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 1dc4188..20d0ee5 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -18,7 +18,9 @@
 #include "relational_operators/FinalizeAggregationOperator.hpp"
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/AggregationOperationState.hpp"
 
 #include "glog/logging.h"
@@ -47,6 +49,24 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
   return started_;
 }
 
+bool FinalizeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !started_) {
+    started_ = true;
+
+    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::insert_destination_index,
+                        output_destination_index_);
+
+    container->addWorkOrderProto(proto, op_index_);
+  }
+  return started_;
+}
+
+
 void FinalizeAggregationWorkOrder::execute() {
   state_->finalizeAggregate(output_destination_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index e8a403f..158a637 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -39,6 +39,7 @@ namespace quickstep {
 
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -79,6 +80,8 @@ class FinalizeAggregationOperator : public RelationalOperator {
                         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_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index b89cfb3..5a47b50 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -31,7 +31,9 @@
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/HashTable.hpp"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
@@ -391,6 +393,128 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
   return false;
 }
 
+bool HashJoinOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  switch (join_type_) {
+    case JoinType::kInnerJoin:
+      return getAllNonOuterJoinWorkOrderProtos(container, serialization::HashJoinWorkOrder::HASH_INNER_JOIN);
+    case JoinType::kLeftSemiJoin:
+      return getAllNonOuterJoinWorkOrderProtos(container, serialization::HashJoinWorkOrder::HASH_SEMI_JOIN);
+    case JoinType::kLeftAntiJoin:
+      return getAllNonOuterJoinWorkOrderProtos(container, serialization::HashJoinWorkOrder::HASH_ANTI_JOIN);
+    case JoinType::kLeftOuterJoin:
+      return getAllOuterJoinWorkOrderProtos(container);
+    default:
+      LOG(FATAL) << "Unknown join type in HashJoinOperator::getAllWorkOrderProtos()";
+  }
+}
+
+bool HashJoinOperator::getAllNonOuterJoinWorkOrderProtos(
+    WorkOrderProtosContainer *container,
+    const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type) {
+  // We wait until the building of global hash table is complete.
+  if (!blocking_dependencies_met_) {
+    return false;
+  }
+
+  if (probe_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id probe_block_id : probe_relation_block_ids_) {
+        container->addWorkOrderProto(
+            createNonOuterJoinWorkOrderProto(hash_join_type, probe_block_id),
+            op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createNonOuterJoinWorkOrderProto(hash_join_type,
+                                           probe_relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* HashJoinOperator::createNonOuterJoinWorkOrderProto(
+    const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type,
+    const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::HASH_JOIN);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type, hash_join_type);
+  proto->SetExtension(serialization::HashJoinWorkOrder::build_relation_id, build_relation_.getID());
+  proto->SetExtension(serialization::HashJoinWorkOrder::probe_relation_id, probe_relation_.getID());
+  for (const attribute_id attr_id : join_key_attributes_) {
+    proto->AddExtension(serialization::HashJoinWorkOrder::join_key_attributes, attr_id);
+  }
+  proto->SetExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable,
+                      any_join_key_attributes_nullable_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::insert_destination_index, output_destination_index_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::join_hash_table_index, hash_table_index_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::selection_index, selection_index_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::block_id, block);
+  proto->SetExtension(serialization::HashJoinWorkOrder::residual_predicate_index, residual_predicate_index_);
+
+  return proto;
+}
+
+bool HashJoinOperator::getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *container) {
+  // We wait until the building of global hash table is complete.
+  if (!blocking_dependencies_met_) {
+    return false;
+  }
+
+  if (probe_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id probe_block_id : probe_relation_block_ids_) {
+        container->addWorkOrderProto(createOuterJoinWorkOrderProto(probe_block_id), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createOuterJoinWorkOrderProto(probe_relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::HASH_JOIN);
+
+  proto->SetExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type,
+                      serialization::HashJoinWorkOrder::HASH_OUTER_JOIN);
+  proto->SetExtension(serialization::HashJoinWorkOrder::build_relation_id, build_relation_.getID());
+  proto->SetExtension(serialization::HashJoinWorkOrder::probe_relation_id, probe_relation_.getID());
+  for (const attribute_id attr_id : join_key_attributes_) {
+    proto->AddExtension(serialization::HashJoinWorkOrder::join_key_attributes, attr_id);
+  }
+  proto->SetExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable,
+                      any_join_key_attributes_nullable_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::insert_destination_index, output_destination_index_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::join_hash_table_index, hash_table_index_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::selection_index, selection_index_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::block_id, block);
+
+  for (const bool is_attribute_on_build : is_selection_on_build_) {
+    proto->AddExtension(serialization::HashJoinWorkOrder::is_selection_on_build, is_attribute_on_build);
+  }
+
+  return proto;
+}
+
+
 void HashInnerJoinWorkOrder::execute() {
   if (FLAGS_vector_based_joined_tuple_collector) {
     executeWithCollectorType<VectorBasedJoinedTupleCollector>();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 1d5d4e3..6f4271d 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -30,6 +30,7 @@
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/HashTable.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "utility/Macros.hpp"
@@ -47,6 +48,7 @@ class InsertDestination;
 class Predicate;
 class Scalar;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -161,6 +163,8 @@ class HashJoinOperator : public RelationalOperator {
                         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) override {
     DCHECK(input_relation_id == probe_relation_.getID());
@@ -202,6 +206,23 @@ class HashJoinOperator : public RelationalOperator {
                                  QueryContext *query_context,
                                  StorageManager *storage_manager);
 
+  bool getAllNonOuterJoinWorkOrderProtos(
+      WorkOrderProtosContainer *container,
+      const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type);
+
+  serialization::WorkOrder* createNonOuterJoinWorkOrderProto(
+      const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type,
+      const block_id block);
+
+  bool getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *container);
+
+  /**
+   * @brief Create HashOuterJoinWorkOrder proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createOuterJoinWorkOrderProto(const block_id block);
+
   const CatalogRelation &build_relation_;
   const CatalogRelation &probe_relation_;
   const bool probe_relation_is_stored_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/InsertOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.cpp b/relational_operators/InsertOperator.cpp
index 3ec9933..963cdcd 100644
--- a/relational_operators/InsertOperator.cpp
+++ b/relational_operators/InsertOperator.cpp
@@ -20,7 +20,9 @@
 #include <memory>
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 
 #include "glog/logging.h"
@@ -49,6 +51,23 @@ bool InsertOperator::getAllWorkOrders(
   return work_generated_;
 }
 
+bool InsertOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !work_generated_) {
+    work_generated_ = true;
+
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::INSERT);
+    proto->set_query_id(query_id_);
+    proto->SetExtension(serialization::InsertWorkOrder::insert_destination_index, output_destination_index_);
+    proto->SetExtension(serialization::InsertWorkOrder::tuple_index, tuple_index_);
+
+    container->addWorkOrderProto(proto, op_index_);
+  }
+
+  return work_generated_;
+}
+
+
 void InsertWorkOrder::execute() {
   output_destination_->insertTuple(*tuple_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index 51c606d..78f5199 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -39,6 +39,7 @@ namespace quickstep {
 
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -78,6 +79,8 @@ class InsertOperator : public RelationalOperator {
                         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_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/NestedLoopsJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.cpp b/relational_operators/NestedLoopsJoinOperator.cpp
index 5a47fca..43588ee 100644
--- a/relational_operators/NestedLoopsJoinOperator.cpp
+++ b/relational_operators/NestedLoopsJoinOperator.cpp
@@ -26,7 +26,9 @@
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -150,6 +152,72 @@ bool NestedLoopsJoinOperator::getAllWorkOrders(
   }
 }
 
+bool NestedLoopsJoinOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (left_relation_is_stored_ && right_relation_is_stored_) {
+    // Make sure we generate workorders only once.
+    if (!all_workorders_generated_) {
+      for (const block_id left_block_id : left_relation_block_ids_) {
+        for (const block_id right_block_id : right_relation_block_ids_) {
+          container->addWorkOrderProto(createWorkOrderProto(left_block_id, right_block_id),
+                                       op_index_);
+        }
+      }
+      all_workorders_generated_ = true;
+    }
+    return true;
+  } else if (!(left_relation_is_stored_ || right_relation_is_stored_)) {
+    // Both relations are not stored.
+    const std::vector<block_id>::size_type new_left_blocks
+        = left_relation_block_ids_.size() - num_left_workorders_generated_;
+    const std::vector<block_id>::size_type new_right_blocks
+        = right_relation_block_ids_.size() - num_right_workorders_generated_;
+
+    std::size_t new_workorders = 0;
+    if (new_left_blocks > 0 && new_right_blocks > 0) {
+      // Blocks added to both left and right relations.
+      // First generate (left + new_left_blocks) * (new_right_blocks).
+      new_workorders =
+          getAllWorkOrderProtosHelperBothNotStored(container,
+                                                   0,
+                                                   left_relation_block_ids_.size(),
+                                                   num_right_workorders_generated_,
+                                                   right_relation_block_ids_.size());
+
+      // Now generate new_left_blocks * (right).
+      new_workorders +=
+          getAllWorkOrderProtosHelperBothNotStored(container,
+                                                   num_left_workorders_generated_,
+                                                   left_relation_block_ids_.size(),
+                                                   0,
+                                                   num_right_workorders_generated_);
+    } else if (new_left_blocks == 0 && new_right_blocks > 0) {
+      // Only new right blocks are added. Generate left * new_right_blocks.
+      new_workorders =
+          getAllWorkOrderProtosHelperBothNotStored(container,
+                                                   0,
+                                                   left_relation_block_ids_.size(),
+                                                   num_right_workorders_generated_,
+                                                   right_relation_block_ids_.size());
+    } else if (new_left_blocks > 0 && new_right_blocks == 0) {
+      // Generate new_left_blocks * right
+      new_workorders =
+          getAllWorkOrderProtosHelperBothNotStored(container,
+                                                   num_left_workorders_generated_,
+                                                   left_relation_block_ids_.size(),
+                                                   0,
+                                                   right_relation_block_ids_.size());
+    }
+    if (new_workorders > 0) {
+      num_left_workorders_generated_ = left_relation_block_ids_.size();
+      num_right_workorders_generated_ = right_relation_block_ids_.size();
+    }
+    return done_feeding_left_relation_ && done_feeding_right_relation_;
+  } else {
+    // Only one relation is a stored relation.
+    return getAllWorkOrderProtosHelperOneStored(container);
+  }
+}
+
 std::size_t NestedLoopsJoinOperator::getAllWorkOrdersHelperBothNotStored(WorkOrdersContainer *container,
                                                                          QueryContext *query_context,
                                                                          StorageManager *storage_manager,
@@ -241,6 +309,80 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
   }
 }
 
+std::size_t NestedLoopsJoinOperator::getAllWorkOrderProtosHelperBothNotStored(
+    WorkOrderProtosContainer *container,
+    const std::vector<block_id>::size_type left_min,
+    const std::vector<block_id>::size_type left_max,
+    const std::vector<block_id>::size_type right_min,
+    const std::vector<block_id>::size_type right_max) {
+  DCHECK(!(left_relation_is_stored_ || right_relation_is_stored_));
+  DCHECK_LE(left_min, left_max);
+  DCHECK_LE(right_min, right_max);
+
+  for (std::vector<block_id>::size_type left_index = left_min;
+       left_index < left_max;
+       ++left_index) {
+    for (std::vector<block_id>::size_type right_index = right_min;
+         right_index < right_max;
+         ++right_index) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(left_relation_block_ids_[left_index], right_relation_block_ids_[right_index]),
+          op_index_);
+    }
+  }
+  // Return the number of workorders produced.
+  return (left_max - left_min) * (right_max - right_min);
+}
+
+bool NestedLoopsJoinOperator::getAllWorkOrderProtosHelperOneStored(WorkOrderProtosContainer *container) {
+  DCHECK(left_relation_is_stored_ ^ right_relation_is_stored_);
+
+  if (left_relation_is_stored_) {
+    for (std::vector<block_id>::size_type right_index = num_right_workorders_generated_;
+         right_index < right_relation_block_ids_.size();
+         ++right_index) {
+      for (const block_id left_block_id : left_relation_block_ids_) {
+        container->addWorkOrderProto(
+            createWorkOrderProto(left_block_id, right_relation_block_ids_[right_index]),
+            op_index_);
+      }
+    }
+    num_right_workorders_generated_ = right_relation_block_ids_.size();
+    return done_feeding_right_relation_;
+  } else {
+    for (std::vector<block_id>::size_type left_index = num_left_workorders_generated_;
+         left_index < left_relation_block_ids_.size();
+         ++left_index) {
+      for (const block_id right_block_id : right_relation_block_ids_) {
+        container->addWorkOrderProto(
+            createWorkOrderProto(left_relation_block_ids_[left_index], right_block_id),
+            op_index_);
+      }
+    }
+    num_left_workorders_generated_ = left_relation_block_ids_.size();
+    return done_feeding_left_relation_;
+  }
+}
+
+serialization::WorkOrder* NestedLoopsJoinOperator::createWorkOrderProto(const block_id left_block,
+                                                                        const block_id right_block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::NESTED_LOOP_JOIN);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::left_relation_id, left_input_relation_.getID());
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::right_relation_id, right_input_relation_.getID());
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::left_block_id, left_block);
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::right_block_id, right_block);
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::insert_destination_index,
+                      output_destination_index_);
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::join_predicate_index, join_predicate_index_);
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::selection_index, selection_index_);
+
+  return proto;
+}
+
+
 template <bool LEFT_PACKED, bool RIGHT_PACKED>
 void NestedLoopsJoinWorkOrder::executeHelper(const TupleStorageSubBlock &left_store,
                                              const TupleStorageSubBlock &right_store) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index 0b13842..992e76d 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -44,8 +44,11 @@ class Predicate;
 class Scalar;
 class StorageManager;
 class TupleStorageSubBlock;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -119,6 +122,8 @@ class NestedLoopsJoinOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void doneFeedingInputBlocks(const relation_id rel_id) override {
     if (rel_id == left_input_relation_.getID()) {
       done_feeding_left_relation_ = true;
@@ -187,6 +192,52 @@ class NestedLoopsJoinOperator : public RelationalOperator {
                                        QueryContext *query_context,
                                        StorageManager *storage_manager);
 
+  /**
+   * @brief Pairs block IDs from left and right relation block IDs and generates
+   *        NestedLoopsJoinWorkOrder protos and pushes them to the
+   *        WorkOrderProtosContainer when both relations are not stored
+   *        relations.
+   *
+   * @param container A pointer to the WorkOrderProtosContainer to store the
+   *                  resulting WorkOrder protos.
+   * @param left_min The starting index in left_relation_block_ids_ from where
+   *                 we begin generating NestedLoopsJoinWorkOrders.
+   * @param left_max The index in left_relation_block_ids_ until which we
+   *                 generate NestedLoopsJoinWorkOrders (excluding left_max).
+   * @param right_min The starting index in right_relation_block_ids_ from where
+   *                  we begin generating NestedLoopsJoinWorkOrders.
+   * @param right_max The index in right_relation_block_ids_ until which we
+   *                  generate NestedLoopsJoinWorkOrders. (excluding right_max).
+   *
+   * @return The number of workorder protos generated during the execution of this
+   *         function.
+   **/
+  std::size_t getAllWorkOrderProtosHelperBothNotStored(WorkOrderProtosContainer *container,
+                                                       const std::vector<block_id>::size_type left_min,
+                                                       const std::vector<block_id>::size_type left_max,
+                                                       const std::vector<block_id>::size_type right_min,
+                                                       const std::vector<block_id>::size_type right_max);
+
+  /**
+   * @brief Pairs block IDs from left and right relation block IDs and generates
+   *        NestedLoopsJoinWorkOrder protos and pushes them to the
+   *        WorkOrderProtosContainer when only one relation is a stored relation.
+   *
+   * @param container A pointer to the WorkOrderProtosContainer to store the
+   *                  resulting WorkOrder protos.
+   *
+   * @return Whether all work orders have been generated.
+   **/
+  bool getAllWorkOrderProtosHelperOneStored(WorkOrderProtosContainer *container);
+
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id left_block,
+                                                 const block_id right_block);
+
   const CatalogRelation &left_input_relation_;
   const CatalogRelation &right_input_relation_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index c173a0a..116727b 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -36,6 +36,7 @@ namespace tmb { class MessageBus; }
 namespace quickstep {
 
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -82,6 +83,27 @@ class RelationalOperator {
                                 tmb::MessageBus *bus) = 0;
 
   /**
+    * @brief For the distributed version, generate all the next WorkOrder protos
+    *        for this RelationalOperator
+    *
+    * @note If a RelationalOperator has blocking dependencies, it should not
+    *       generate workorders unless all of the blocking dependencies have been
+    *       met.
+    *
+    * @note If a RelationalOperator is not parallelizeable on a block-level, then
+    *       only one WorkOrder consisting of all the work for this
+    *       RelationalOperator should be generated.
+    *
+    * @param container A pointer to a WorkOrderProtosContainer to be used to
+    *        store the generated WorkOrder protos.
+    *
+    * @return Whether the operator has finished generating work order protos. If
+    *         \c false, the execution engine will invoke this method after at
+    *         least one pending work order has finished executing.
+    **/
+  virtual bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) = 0;
+
+  /**
    * @brief Update Catalog upon the completion of this RelationalOperator, if
    *        necessary.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SampleOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.cpp b/relational_operators/SampleOperator.cpp
index 8d5fade..5e5a417 100644
--- a/relational_operators/SampleOperator.cpp
+++ b/relational_operators/SampleOperator.cpp
@@ -1,6 +1,7 @@
 /**
  *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
  *     University of Wisconsin\u2014Madison.
+ *   Copyright 2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -18,10 +19,13 @@
 #include "relational_operators/SampleOperator.hpp"
 
 #include <memory>
+#include <random>
 #include <vector>
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -82,24 +86,9 @@ bool SampleOperator::getAllWorkOrders(
     }
     return started_;
   } else {
-      if (is_block_sample_) {
-          while (num_workorders_generated_ < input_relation_block_ids_.size()) {
-            if (distribution(generator) <= probability) {
-              container->addNormalWorkOrder(
-                  new SampleWorkOrder(
-                      query_id_,
-                      input_relation_,
-                      input_relation_block_ids_[num_workorders_generated_],
-                      is_block_sample_,
-                      percentage_,
-                      output_destination,
-                      storage_manager),
-                  op_index_);
-            ++num_workorders_generated_;
-          }
-        }
-      } else  {
-        while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+    if (is_block_sample_) {
+      while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+        if (distribution(generator) <= probability) {
           container->addNormalWorkOrder(
               new SampleWorkOrder(
                   query_id_,
@@ -113,10 +102,86 @@ bool SampleOperator::getAllWorkOrders(
           ++num_workorders_generated_;
         }
       }
+    } else {
+      while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+        container->addNormalWorkOrder(
+            new SampleWorkOrder(
+                query_id_,
+                input_relation_,
+                input_relation_block_ids_[num_workorders_generated_],
+                is_block_sample_,
+                percentage_,
+                output_destination,
+                storage_manager),
+            op_index_);
+        ++num_workorders_generated_;
+      }
+    }
     return done_feeding_input_relation_;
   }
 }
 
+bool SampleOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  std::random_device random_device;
+  std::mt19937 generator(random_device());
+  std::uniform_real_distribution<> distribution(0, 1);
+  const double probability = static_cast<double>(percentage_) / 100;
+
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      // If the sampling is by block choose blocks randomly
+      if (is_block_sample_) {
+        for (const block_id input_block_id : input_relation_block_ids_) {
+          if (distribution(generator) <= probability) {
+            container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+          }
+        }
+      } else {
+        // Add all the blocks for tuple sampling which would handle
+        // the sampling from each block
+        for (const block_id input_block_id : input_relation_block_ids_) {
+          container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+        }
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    if (is_block_sample_) {
+      while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+        if (distribution(generator) <= probability) {
+          container->addWorkOrderProto(
+              createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+              op_index_);
+          ++num_workorders_generated_;
+        }
+      }
+    } else {
+      while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+        container->addWorkOrderProto(
+            createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+            op_index_);
+        ++num_workorders_generated_;
+      }
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* SampleOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::SAMPLE);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::SampleWorkOrder::relation_id, input_relation_.getID());
+  proto->SetExtension(serialization::SampleWorkOrder::block_id, block);
+  proto->SetExtension(serialization::SampleWorkOrder::is_block_sample, is_block_sample_);
+  proto->SetExtension(serialization::SampleWorkOrder::percentage, percentage_);
+  proto->SetExtension(serialization::SampleWorkOrder::insert_destination_index, output_destination_index_);
+
+  return proto;
+}
+
 void SampleWorkOrder::execute() {
   BlockReference block(
       storage_manager_->getBlock(input_block_id_, input_relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index 505daa2..f8fe5f6 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -42,8 +42,11 @@ class CatalogDatabase;
 class CatalogRelationSchema;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -96,6 +99,8 @@ class SampleOperator : public RelationalOperator {
                         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) override {
     input_relation_block_ids_.push_back(input_block_id);
   }
@@ -115,6 +120,13 @@ class SampleOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &input_relation_;
   const CatalogRelationSchema &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index 8127d88..5e0f33d 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -19,7 +19,9 @@
 
 #include <vector>
 
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
 
@@ -46,6 +48,22 @@ bool SaveBlocksOperator::getAllWorkOrders(
   return done_feeding_input_relation_;
 }
 
+bool SaveBlocksOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  while (num_workorders_generated_ < destination_block_ids_.size()) {
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::SAVE_BLOCKS);
+    proto->set_query_id(query_id_);
+    proto->SetExtension(serialization::SaveBlocksWorkOrder::block_id,
+                        destination_block_ids_[num_workorders_generated_]);
+    proto->SetExtension(serialization::SaveBlocksWorkOrder::force, force_);
+
+    container->addWorkOrderProto(proto, op_index_);
+
+    ++num_workorders_generated_;
+  }
+  return done_feeding_input_relation_;
+}
+
 void SaveBlocksOperator::feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) {
   destination_block_ids_.push_back(input_block_id);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 6e2c72b..50032b6 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -37,6 +37,7 @@ namespace quickstep {
 
 class QueryContext;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -69,6 +70,8 @@ class SaveBlocksOperator : public RelationalOperator {
                         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) override;
 
   void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index e9a96f3..eb6277e 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -21,7 +21,9 @@
 #include <vector>
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -170,6 +172,47 @@ bool SelectOperator::getAllWorkOrders(
   }
 }
 
+bool SelectOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id input_block_id : input_relation_block_ids_) {
+        container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* SelectOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::SELECT);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::SelectWorkOrder::relation_id, input_relation_.getID());
+  proto->SetExtension(serialization::SelectWorkOrder::insert_destination_index, output_destination_index_);
+  proto->SetExtension(serialization::SelectWorkOrder::predicate_index, predicate_index_);
+  proto->SetExtension(serialization::SelectWorkOrder::block_id, block);
+  proto->SetExtension(serialization::SelectWorkOrder::simple_projection, simple_projection_);
+  if (simple_projection_) {
+    for (const attribute_id attr_id : simple_selection_) {
+      proto->AddExtension(serialization::SelectWorkOrder::simple_selection, attr_id);
+    }
+  }
+  proto->SetExtension(serialization::SelectWorkOrder::selection_index, selection_index_);
+
+  return proto;
+}
+
+
 void SelectWorkOrder::execute() {
   BlockReference block(
       storage_manager_->getBlock(input_block_id_, input_relation_, getPreferredNUMANodes()[0]));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index ac7b038..0c10686 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -49,8 +49,11 @@ class InsertDestination;
 class Predicate;
 class Scalar;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -192,6 +195,8 @@ class SelectOperator : public RelationalOperator {
                         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) override {
     if (input_relation_.hasPartitionScheme()) {
       const partition_id part_id =
@@ -243,6 +248,13 @@ class SelectOperator : public RelationalOperator {
                                    InsertDestination *output_destination);
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &input_relation_;
   const CatalogRelation &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;



[48/50] [abbrv] incubator-quickstep git commit: Disallow negative number of worker threads.

Posted by zu...@apache.org.
Disallow negative number of worker threads.

- Fixed a bug thereby Quickstep command line now disallows negative
  number of worker threads.
- If the user provides zero or fewer worker threads, we switch to the
  default number of worker threasd, instead of terminating the process.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 31f1bbb1c71d9a18af27ee540c83f513125b656f
Parents: 040a511
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Thu Jun 30 11:04:29 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Sun Jul 3 23:20:00 2016 -0500

----------------------------------------------------------------------
 cli/QuickstepCli.cpp | 23 +++++++++++------------
 1 file changed, 11 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/31f1bbb1/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 3f99130..02a55a0 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -200,19 +200,18 @@ int main(int argc, char* argv[]) {
   // that we computed above, provided it did return a valid value.
   // TODO(jmp): May need to change this at some point to keep one thread
   //            available for the OS if the hardware concurrency level is high.
-  const unsigned int real_num_workers = quickstep::FLAGS_num_workers != 0
-                                      ? quickstep::FLAGS_num_workers
-                                      : (num_hw_threads != 0 ?
-                                         num_hw_threads
-                                         : 1);
-
-  if (real_num_workers > 0) {
-    printf("Starting Quickstep with %d worker thread(s) and a %.2f GB buffer pool\n",
-           real_num_workers,
-           (static_cast<double>(quickstep::FLAGS_buffer_pool_slots) * quickstep::kSlotSizeBytes)/quickstep::kAGigaByte);
-  } else {
-    LOG(FATAL) << "Quickstep needs at least one worker thread to run";
+  if (quickstep::FLAGS_num_workers <= 0) {
+    LOG(INFO) << "Quickstep expects at least one worker thread, switching to "
+                 "the default number of worker threads";
   }
+  const int real_num_workers = quickstep::FLAGS_num_workers > 0
+                                   ? quickstep::FLAGS_num_workers
+                                   : (num_hw_threads != 0 ? num_hw_threads : 1);
+
+  DCHECK_GT(real_num_workers, 0);
+  printf("Starting Quickstep with %d worker thread(s) and a %.2f GB buffer pool\n",
+         real_num_workers,
+         (static_cast<double>(quickstep::FLAGS_buffer_pool_slots) * quickstep::kSlotSizeBytes)/quickstep::kAGigaByte);
 
 #ifdef QUICKSTEP_HAVE_FILE_MANAGER_HDFS
   if (quickstep::FLAGS_use_hdfs) {


[45/50] [abbrv] incubator-quickstep git commit: Remove unused vector_based HashJoin collector type

Posted by zu...@apache.org.
Remove unused vector_based HashJoin collector type


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

Branch: refs/heads/tmp-relation-col-store
Commit: 33470032a946a5a216df931b56be0eb8c6bfa0c4
Parents: 5c4e8db
Author: Navneet Potti <na...@gmail.com>
Authored: Mon Jun 27 11:00:04 2016 -0500
Committer: Navneet Potti <na...@cs.wisc.edu>
Committed: Wed Jun 29 12:54:09 2016 -0500

----------------------------------------------------------------------
 relational_operators/HashJoinOperator.cpp | 116 +------------------------
 relational_operators/HashJoinOperator.hpp |   3 -
 2 files changed, 2 insertions(+), 117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/33470032/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 5a47b50..667df1e 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -61,25 +61,9 @@ namespace quickstep {
 
 namespace {
 
-DEFINE_bool(vector_based_joined_tuple_collector, false,
-            "If true, use simple vector-based joined tuple collector in "
-            "hash join, with a final sort pass to group joined tuple pairs "
-            "by inner block. If false, use unordered_map based collector that "
-            "keeps joined pairs grouped by inner block as they are found "
-            "(this latter option has exhibited performance/scaling problems, "
-            "particularly in NUMA contexts).");
-
 // Functor passed to HashTable::getAllFromValueAccessor() to collect matching
-// tuples from the inner relation. This version stores matching tuple ID pairs
+// tuples from the inner relation. It stores matching tuple ID pairs
 // in an unordered_map keyed by inner block ID.
-//
-// NOTE(chasseur): Performance testing has shown that this particular
-// implementation has problems scaling in a multisocket NUMA machine.
-// Additional benchmarking revealed problems using the STL unordered_map class
-// in a NUMA system (at least for the implementation in GNU libstdc++), even
-// though instances of this class and the internal unordered_map are private to
-// a single thread. Because of this, VectorBasedJoinedTupleCollector is used by
-// default instead.
 class MapBasedJoinedTupleCollector {
  public:
   MapBasedJoinedTupleCollector() {
@@ -91,13 +75,6 @@ class MapBasedJoinedTupleCollector {
     joined_tuples_[tref.block].emplace_back(tref.tuple, accessor.getCurrentPosition());
   }
 
-  // Consolidation is a no-op for this version, but we provide this trivial
-  // call so that MapBasedJoinedTupleCollector and
-  // VectorBasedJoinedTupleCollector have the same interface and can both be
-  // used in the templated HashInnerJoinWorkOrder::executeWithCollectorType() method.
-  inline void consolidate() const {
-  }
-
   // Get a mutable pointer to the collected map of joined tuple ID pairs. The
   // key is inner block_id, values are vectors of joined tuple ID pairs with
   // tuple ID from the inner block on the left and the outer block on the
@@ -116,82 +93,6 @@ class MapBasedJoinedTupleCollector {
   std::unordered_map<block_id, std::vector<std::pair<tuple_id, tuple_id>>> joined_tuples_;
 };
 
-// Compare std::pair instances based on their first element only.
-template <typename PairT>
-inline bool CompareFirst(const PairT &left, const PairT &right) {
-  return left.first < right.first;
-}
-
-// Functor passed to HashTable::getAllFromValueAccessor() to collect matching
-// tuples from the inner relation. This version stores inner block ID and pairs
-// of joined tuple IDs in an unsorted vector, which should then be sorted with
-// a call to consolidate() before materializing join output.
-//
-// NOTE(chasseur): Because of NUMA scaling issues for
-// MapBasedJoinedTupleCollector noted above, this implementation is the
-// default.
-class VectorBasedJoinedTupleCollector {
- public:
-  VectorBasedJoinedTupleCollector() {
-  }
-
-  template <typename ValueAccessorT>
-  inline void operator()(const ValueAccessorT &accessor,
-                         const TupleReference &tref) {
-    joined_tuples_.emplace_back(tref.block,
-                                std::make_pair(tref.tuple, accessor.getCurrentPosition()));
-  }
-
-  // Sorts joined tuple pairs by inner block ID. Must be called before
-  // getJoinedTuples().
-  void consolidate() {
-    if (joined_tuples_.empty()) {
-      return;
-    }
-
-    // Sort joined tuple_id pairs by inner block_id.
-    std::sort(joined_tuples_.begin(),
-              joined_tuples_.end(),
-              CompareFirst<std::pair<block_id, std::pair<tuple_id, tuple_id>>>);
-
-    // Make a single vector of joined block_id pairs for each inner block for
-    // compatibility with other join-related APIs.
-    consolidated_joined_tuples_.emplace_back(joined_tuples_.front().first,
-                                             std::vector<std::pair<tuple_id, tuple_id>>());
-
-    for (const std::pair<block_id, std::pair<tuple_id, tuple_id>> &match_entry
-         : joined_tuples_) {
-      if (match_entry.first == consolidated_joined_tuples_.back().first) {
-        consolidated_joined_tuples_.back().second.emplace_back(match_entry.second);
-      } else {
-        consolidated_joined_tuples_.emplace_back(
-            match_entry.first,
-            std::vector<std::pair<tuple_id, tuple_id>>(1, match_entry.second));
-      }
-    }
-  }
-
-  // Get a mutable pointer to the collected joined tuple ID pairs. The returned
-  // vector has a single entry for each inner block where there are matching
-  // joined tuples (the inner block's ID is the first element of the pair). The
-  // second element of each pair is another vector consisting of pairs of
-  // joined tuple IDs (tuple ID from inner block on the left, from outer block
-  // on the right).
-  inline std::vector<std::pair<const block_id, std::vector<std::pair<tuple_id, tuple_id>>>>*
-      getJoinedTuples() {
-    return &consolidated_joined_tuples_;
-  }
-
- private:
-  // Unsorted vector of join matches that is appended to by call operator().
-  std::vector<std::pair<block_id, std::pair<tuple_id, tuple_id>>> joined_tuples_;
-
-  // Joined tuples sorted by inner block_id. consolidate() populates this from
-  // 'joined_tuples_'.
-  std::vector<std::pair<const block_id, std::vector<std::pair<tuple_id, tuple_id>>>>
-      consolidated_joined_tuples_;
-};
-
 class SemiAntiJoinTupleCollector {
  public:
   explicit SemiAntiJoinTupleCollector(const TupleStorageSubBlock &tuple_store) {
@@ -516,21 +417,12 @@ serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const
 
 
 void HashInnerJoinWorkOrder::execute() {
-  if (FLAGS_vector_based_joined_tuple_collector) {
-    executeWithCollectorType<VectorBasedJoinedTupleCollector>();
-  } else {
-    executeWithCollectorType<MapBasedJoinedTupleCollector>();
-  }
-}
-
-template <typename CollectorT>
-void HashInnerJoinWorkOrder::executeWithCollectorType() {
   BlockReference probe_block(
       storage_manager_->getBlock(block_id_, probe_relation_));
   const TupleStorageSubBlock &probe_store = probe_block->getTupleStorageSubBlock();
 
   std::unique_ptr<ValueAccessor> probe_accessor(probe_store.createValueAccessor());
-  CollectorT collector;
+  MapBasedJoinedTupleCollector collector;
   if (join_key_attributes_.size() == 1) {
     hash_table_.getAllFromValueAccessor(
         probe_accessor.get(),
@@ -544,7 +436,6 @@ void HashInnerJoinWorkOrder::executeWithCollectorType() {
         any_join_key_attributes_nullable_,
         &collector);
   }
-  collector.consolidate();
 
   const relation_id build_relation_id = build_relation_.getID();
   const relation_id probe_relation_id = probe_relation_.getID();
@@ -637,8 +528,6 @@ void HashSemiJoinWorkOrder::executeWithResidualPredicate() {
 
   std::unique_ptr<ValueAccessor> probe_accessor(probe_store.createValueAccessor());
 
-  // TODO(harshad) - Make this function work with both types of collectors.
-
   // We collect all the matching probe relation tuples, as there's a residual
   // preidcate that needs to be applied after collecting these matches.
   MapBasedJoinedTupleCollector collector;
@@ -810,7 +699,6 @@ void HashAntiJoinWorkOrder::executeWithResidualPredicate() {
   const TupleStorageSubBlock &probe_store = probe_block->getTupleStorageSubBlock();
 
   std::unique_ptr<ValueAccessor> probe_accessor(probe_store.createValueAccessor());
-  // TODO(harshad) - Make the following code work with both types of collectors.
   MapBasedJoinedTupleCollector collector;
   // We probe the hash table and get all the matches. Unlike
   // executeWithoutResidualPredicate(), we have to collect all the matching

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/33470032/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 9762f04..5d3d7da 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -356,9 +356,6 @@ class HashInnerJoinWorkOrder : public WorkOrder {
   void execute() override;
 
  private:
-  template <typename CollectorT>
-  void executeWithCollectorType();
-
   const CatalogRelationSchema &build_relation_;
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;


[24/50] [abbrv] incubator-quickstep git commit: Fix conditional per-target flags for lexer

Posted by zu...@apache.org.
Fix conditional per-target flags for lexer


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

Branch: refs/heads/tmp-relation-col-store
Commit: 29768a7297746606089c6b2ebca1a9bf44180614
Parents: 31e9e4c
Author: Craig Chasseur <sp...@gmail.com>
Authored: Tue Jun 14 00:51:52 2016 -0700
Committer: Craig Chasseur <sp...@gmail.com>
Committed: Tue Jun 14 00:51:52 2016 -0700

----------------------------------------------------------------------
 parser/CMakeLists.txt | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/29768a72/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index 9738c2c..2488d30 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -374,12 +374,15 @@ set_property(SOURCE ${CMAKE_CURRENT_BINARY_DIR}/SqlParser_gen.cpp APPEND PROPERT
 include(CheckCXXCompilerFlag)
 CHECK_CXX_COMPILER_FLAG("-Wno-deprecated-register" COMPILER_HAS_WNO_DEPRECATED_REGISTER)
 if (COMPILER_HAS_WNO_DEPRECATED_REGISTER)
-  set_target_properties(quickstep_parser_SqlLexer PROPERTIES COMPILE_FLAGS "-Wno-deprecated-register")
+  set_property(TARGET quickstep_parser_SqlLexer APPEND_STRING PROPERTY COMPILE_FLAGS " -Wno-deprecated-register")
 endif()
 
 # GCC will make a warning for unsigned-signed comparisons which are inherent
 # in the lexer. For this, we turn off the sign compare.
-set_target_properties(quickstep_parser_SqlLexer PROPERTIES COMPILE_FLAGS "-Wno-sign-compare")
+CHECK_CXX_COMPILER_FLAG("-Wno-sign-compare" COMPILER_HAS_WNO_SIGN_COMPARE)
+if (COMPILER_HAS_WNO_SIGN_COMPARE)
+  set_property(TARGET quickstep_parser_SqlLexer APPEND_STRING PROPERTY COMPILE_FLAGS " -Wno-sign-compare")
+endif()
 
 add_subdirectory(tests)
 


[30/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-20: Added parser support for SQL window aggregation function

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 038db14..c02baf4 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -131,6 +131,7 @@ typedef struct YYLTYPE {
 #include "parser/ParseSubqueryExpression.hpp"
 #include "parser/ParseSubqueryTableReference.hpp"
 #include "parser/ParseTableReference.hpp"
+#include "parser/ParseWindow.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
@@ -150,7 +151,7 @@ typedef struct YYLTYPE {
 // Needed for Bison 2.6 and higher, which do not automatically provide this typedef.
 typedef void* yyscan_t;
 
-#line 154 "SqlParser_gen.cpp" /* yacc.c:339  */
+#line 155 "SqlParser_gen.cpp" /* yacc.c:339  */
 
 # ifndef YY_NULLPTR
 #  if defined __cplusplus && 201103L <= __cplusplus
@@ -225,88 +226,96 @@ extern int quickstep_yydebug;
     TOKEN_CONSTRAINT = 295,
     TOKEN_COPY = 296,
     TOKEN_CREATE = 297,
-    TOKEN_DATE = 298,
-    TOKEN_DATETIME = 299,
-    TOKEN_DAY = 300,
-    TOKEN_DECIMAL = 301,
-    TOKEN_DEFAULT = 302,
-    TOKEN_DELETE = 303,
-    TOKEN_DELIMITER = 304,
-    TOKEN_DESC = 305,
-    TOKEN_DISTINCT = 306,
-    TOKEN_DOUBLE = 307,
-    TOKEN_DROP = 308,
-    TOKEN_ELSE = 309,
-    TOKEN_END = 310,
-    TOKEN_ESCAPE_STRINGS = 311,
-    TOKEN_EXISTS = 312,
-    TOKEN_EXTRACT = 313,
-    TOKEN_FALSE = 314,
-    TOKEN_FIRST = 315,
-    TOKEN_FLOAT = 316,
-    TOKEN_FOR = 317,
-    TOKEN_FOREIGN = 318,
-    TOKEN_FROM = 319,
-    TOKEN_FULL = 320,
-    TOKEN_GROUP = 321,
-    TOKEN_HASH = 322,
-    TOKEN_HAVING = 323,
-    TOKEN_HOUR = 324,
-    TOKEN_IN = 325,
-    TOKEN_INDEX = 326,
-    TOKEN_INNER = 327,
-    TOKEN_INSERT = 328,
-    TOKEN_INTEGER = 329,
-    TOKEN_INTERVAL = 330,
-    TOKEN_INTO = 331,
-    TOKEN_JOIN = 332,
-    TOKEN_KEY = 333,
-    TOKEN_LAST = 334,
-    TOKEN_LEFT = 335,
-    TOKEN_LIMIT = 336,
-    TOKEN_LONG = 337,
-    TOKEN_MINUTE = 338,
-    TOKEN_MONTH = 339,
-    TOKEN_NULL = 340,
-    TOKEN_NULLS = 341,
-    TOKEN_OFF = 342,
-    TOKEN_ON = 343,
-    TOKEN_ORDER = 344,
-    TOKEN_OUTER = 345,
-    TOKEN_PARTITION = 346,
-    TOKEN_PARTITIONS = 347,
-    TOKEN_PERCENT = 348,
-    TOKEN_PRIMARY = 349,
-    TOKEN_QUIT = 350,
-    TOKEN_RANGE = 351,
-    TOKEN_REAL = 352,
-    TOKEN_REFERENCES = 353,
-    TOKEN_RIGHT = 354,
-    TOKEN_ROW_DELIMITER = 355,
-    TOKEN_SECOND = 356,
-    TOKEN_SELECT = 357,
-    TOKEN_SET = 358,
-    TOKEN_SMA = 359,
-    TOKEN_SMALLINT = 360,
-    TOKEN_SUBSTRING = 361,
-    TOKEN_TABLE = 362,
-    TOKEN_THEN = 363,
-    TOKEN_TIME = 364,
-    TOKEN_TIMESTAMP = 365,
-    TOKEN_TRUE = 366,
-    TOKEN_TUPLESAMPLE = 367,
-    TOKEN_UNIQUE = 368,
-    TOKEN_UPDATE = 369,
-    TOKEN_USING = 370,
-    TOKEN_VALUES = 371,
-    TOKEN_VARCHAR = 372,
-    TOKEN_WHEN = 373,
-    TOKEN_WHERE = 374,
-    TOKEN_WITH = 375,
-    TOKEN_YEAR = 376,
-    TOKEN_YEARMONTH = 377,
-    TOKEN_EOF = 378,
-    TOKEN_LEX_ERROR = 379
+    TOKEN_CURRENT = 298,
+    TOKEN_DATE = 299,
+    TOKEN_DATETIME = 300,
+    TOKEN_DAY = 301,
+    TOKEN_DECIMAL = 302,
+    TOKEN_DEFAULT = 303,
+    TOKEN_DELETE = 304,
+    TOKEN_DELIMITER = 305,
+    TOKEN_DESC = 306,
+    TOKEN_DISTINCT = 307,
+    TOKEN_DOUBLE = 308,
+    TOKEN_DROP = 309,
+    TOKEN_ELSE = 310,
+    TOKEN_END = 311,
+    TOKEN_ESCAPE_STRINGS = 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_QUIT = 354,
+    TOKEN_RANGE = 355,
+    TOKEN_REAL = 356,
+    TOKEN_REFERENCES = 357,
+    TOKEN_RIGHT = 358,
+    TOKEN_ROW = 359,
+    TOKEN_ROW_DELIMITER = 360,
+    TOKEN_ROWS = 361,
+    TOKEN_SECOND = 362,
+    TOKEN_SELECT = 363,
+    TOKEN_SET = 364,
+    TOKEN_SMA = 365,
+    TOKEN_SMALLINT = 366,
+    TOKEN_SUBSTRING = 367,
+    TOKEN_TABLE = 368,
+    TOKEN_THEN = 369,
+    TOKEN_TIME = 370,
+    TOKEN_TIMESTAMP = 371,
+    TOKEN_TRUE = 372,
+    TOKEN_TUPLESAMPLE = 373,
+    TOKEN_UNBOUNDED = 374,
+    TOKEN_UNIQUE = 375,
+    TOKEN_UPDATE = 376,
+    TOKEN_USING = 377,
+    TOKEN_VALUES = 378,
+    TOKEN_VARCHAR = 379,
+    TOKEN_WHEN = 380,
+    TOKEN_WHERE = 381,
+    TOKEN_WINDOW = 382,
+    TOKEN_WITH = 383,
+    TOKEN_YEAR = 384,
+    TOKEN_YEARMONTH = 385,
+    TOKEN_EOF = 386,
+    TOKEN_LEX_ERROR = 387
   };
 #endif
 
@@ -315,7 +324,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 118 "../SqlParser.ypp" /* yacc.c:355  */
+#line 119 "../SqlParser.ypp" /* yacc.c:355  */
 
   quickstep::ParseString *string_value_;
 
@@ -401,13 +410,19 @@ union YYSTYPE
 
   quickstep::ParseSample *opt_sample_clause_;
 
+  quickstep::PtrList<quickstep::ParseWindow> *opt_window_clause_;
+  quickstep::ParseWindow *window_definition_;
+  quickstep::PtrList<quickstep::ParseExpression> *window_partition_by_list_;
+  quickstep::PtrList<quickstep::ParseOrderByItem> *window_order_by_list_;
+  quickstep::ParseFrameInfo *window_frame_info_;
+
   quickstep::PtrList<quickstep::ParseOrderByItem> *order_commalist_;
   quickstep::ParseOrderByItem *order_item_;
 
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
 
-#line 411 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 426 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -436,13 +451,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 210 "../SqlParser.ypp" /* yacc.c:358  */
+#line 217 "../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 446 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 461 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -686,21 +701,21 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  47
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1215
+#define YYLAST   1347
 
 /* YYNTOKENS -- Number of terminals.  */
-#define YYNTOKENS  136
+#define YYNTOKENS  144
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  96
+#define YYNNTS  106
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  265
+#define YYNRULES  287
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  488
+#define YYNSTATES  531
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
 #define YYUNDEFTOK  2
-#define YYMAXUTOK   379
+#define YYMAXUTOK   387
 
 #define YYTRANSLATE(YYX)                                                \
   ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK)
@@ -710,11 +725,11 @@ union yyalloc
 static const yytype_uint8 yytranslate[] =
 {
        0,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-     131,     2,     2,     2,     2,     2,     2,     2,     2,     2,
+     139,     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,   135,     2,     2,
-     132,   133,    23,    21,   134,    22,    27,    24,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,     2,     2,   130,
+       2,     2,     2,     2,     2,     2,     2,   143,     2,     2,
+     140,   141,    23,    21,   142,    22,    27,    24,     2,     2,
+       2,     2,     2,     2,     2,     2,     2,     2,     2,   138,
        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,
@@ -746,40 +761,43 @@ static const yytype_uint8 yytranslate[] =
       90,    91,    92,    93,    94,    95,    96,    97,    98,    99,
      100,   101,   102,   103,   104,   105,   106,   107,   108,   109,
      110,   111,   112,   113,   114,   115,   116,   117,   118,   119,
-     120,   121,   122,   123,   124,   125,   126,   127,   128,   129
+     120,   121,   122,   123,   124,   125,   126,   127,   128,   129,
+     130,   131,   132,   133,   134,   135,   136,   137
 };
 
 #if YYDEBUG
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   585,   585,   589,   593,   597,   601,   604,   611,   614,
-     617,   620,   623,   626,   629,   632,   635,   638,   644,   650,
-     657,   663,   670,   679,   684,   693,   698,   703,   707,   713,
-     718,   721,   724,   729,   732,   735,   738,   741,   744,   747,
-     750,   753,   756,   768,   771,   774,   792,   812,   815,   818,
-     823,   828,   834,   840,   849,   853,   859,   862,   867,   872,
-     877,   884,   891,   895,   901,   904,   909,   912,   917,   920,
-     925,   928,   947,   951,   957,   961,   967,   970,   973,   978,
-     981,   988,   993,  1004,  1009,  1013,  1017,  1023,  1026,  1032,
-    1040,  1043,  1046,  1052,  1057,  1060,  1065,  1069,  1073,  1077,
-    1083,  1088,  1093,  1097,  1103,  1109,  1112,  1117,  1122,  1126,
-    1132,  1138,  1144,  1147,  1151,  1157,  1160,  1165,  1169,  1175,
-    1178,  1181,  1186,  1191,  1196,  1199,  1202,  1207,  1210,  1213,
-    1216,  1219,  1222,  1225,  1228,  1233,  1236,  1241,  1245,  1249,
-    1252,  1256,  1259,  1264,  1267,  1272,  1275,  1280,  1284,  1290,
-    1293,  1298,  1301,  1306,  1309,  1314,  1317,  1336,  1340,  1346,
-    1353,  1356,  1359,  1364,  1367,  1370,  1376,  1379,  1384,  1389,
-    1398,  1403,  1412,  1417,  1420,  1425,  1428,  1433,  1439,  1445,
-    1448,  1451,  1454,  1457,  1460,  1466,  1475,  1478,  1483,  1486,
-    1491,  1494,  1499,  1502,  1505,  1508,  1511,  1514,  1517,  1520,
-    1525,  1529,  1533,  1536,  1541,  1546,  1550,  1556,  1559,  1564,
-    1568,  1574,  1579,  1583,  1589,  1594,  1597,  1602,  1606,  1612,
-    1615,  1618,  1621,  1633,  1637,  1656,  1669,  1684,  1687,  1690,
-    1693,  1696,  1699,  1704,  1708,  1714,  1717,  1722,  1726,  1733,
-    1736,  1739,  1742,  1745,  1748,  1751,  1754,  1757,  1760,  1765,
-    1776,  1779,  1784,  1787,  1790,  1796,  1800,  1806,  1809,  1817,
-    1820,  1823,  1826,  1832,  1837,  1842
+       0,   622,   622,   626,   630,   634,   638,   641,   648,   651,
+     654,   657,   660,   663,   666,   669,   672,   675,   681,   687,
+     694,   700,   707,   716,   721,   730,   735,   740,   744,   750,
+     755,   758,   761,   766,   769,   772,   775,   778,   781,   784,
+     787,   790,   793,   805,   808,   811,   829,   849,   852,   855,
+     860,   865,   871,   877,   886,   890,   896,   899,   904,   909,
+     914,   921,   928,   932,   938,   941,   946,   949,   954,   957,
+     962,   965,   984,   988,   994,   998,  1004,  1007,  1010,  1015,
+    1018,  1025,  1030,  1041,  1046,  1050,  1054,  1060,  1063,  1069,
+    1077,  1080,  1083,  1089,  1094,  1097,  1102,  1106,  1110,  1114,
+    1120,  1125,  1130,  1134,  1140,  1146,  1149,  1154,  1159,  1163,
+    1169,  1175,  1181,  1184,  1188,  1194,  1197,  1202,  1206,  1212,
+    1215,  1218,  1223,  1228,  1233,  1236,  1239,  1244,  1247,  1250,
+    1253,  1256,  1259,  1262,  1265,  1270,  1273,  1278,  1282,  1286,
+    1289,  1293,  1296,  1301,  1304,  1309,  1312,  1317,  1321,  1327,
+    1330,  1335,  1338,  1343,  1346,  1351,  1354,  1373,  1376,  1379,
+    1383,  1389,  1395,  1400,  1403,  1408,  1411,  1416,  1419,  1424,
+    1427,  1432,  1433,  1436,  1441,  1442,  1445,  1450,  1454,  1460,
+    1467,  1470,  1473,  1478,  1481,  1484,  1490,  1493,  1498,  1503,
+    1512,  1517,  1526,  1531,  1534,  1539,  1542,  1547,  1553,  1559,
+    1562,  1565,  1568,  1571,  1574,  1580,  1589,  1592,  1597,  1600,
+    1605,  1608,  1613,  1616,  1619,  1622,  1626,  1630,  1633,  1636,
+    1639,  1642,  1647,  1651,  1655,  1658,  1663,  1668,  1672,  1678,
+    1681,  1686,  1690,  1696,  1701,  1705,  1711,  1716,  1719,  1724,
+    1728,  1734,  1737,  1740,  1743,  1755,  1759,  1778,  1791,  1806,
+    1809,  1812,  1815,  1818,  1821,  1826,  1830,  1836,  1839,  1844,
+    1848,  1855,  1858,  1861,  1864,  1867,  1870,  1873,  1876,  1879,
+    1882,  1887,  1898,  1901,  1906,  1909,  1912,  1918,  1922,  1928,
+    1931,  1939,  1942,  1945,  1948,  1954,  1959,  1964
 };
 #endif
 
@@ -798,47 +816,51 @@ static const char *const yytname[] =
   "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_DATE",
-  "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL", "TOKEN_DEFAULT",
-  "TOKEN_DELETE", "TOKEN_DELIMITER", "TOKEN_DESC", "TOKEN_DISTINCT",
-  "TOKEN_DOUBLE", "TOKEN_DROP", "TOKEN_ELSE", "TOKEN_END",
-  "TOKEN_ESCAPE_STRINGS", "TOKEN_EXISTS", "TOKEN_EXTRACT", "TOKEN_FALSE",
-  "TOKEN_FIRST", "TOKEN_FLOAT", "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_PARTITION",
-  "TOKEN_PARTITIONS", "TOKEN_PERCENT", "TOKEN_PRIMARY", "TOKEN_QUIT",
-  "TOKEN_RANGE", "TOKEN_REAL", "TOKEN_REFERENCES", "TOKEN_RIGHT",
-  "TOKEN_ROW_DELIMITER", "TOKEN_SECOND", "TOKEN_SELECT", "TOKEN_SET",
-  "TOKEN_SMA", "TOKEN_SMALLINT", "TOKEN_SUBSTRING", "TOKEN_TABLE",
-  "TOKEN_THEN", "TOKEN_TIME", "TOKEN_TIMESTAMP", "TOKEN_TRUE",
-  "TOKEN_TUPLESAMPLE", "TOKEN_UNIQUE", "TOKEN_UPDATE", "TOKEN_USING",
-  "TOKEN_VALUES", "TOKEN_VARCHAR", "TOKEN_WHEN", "TOKEN_WHERE",
-  "TOKEN_WITH", "TOKEN_YEAR", "TOKEN_YEARMONTH", "TOKEN_EOF",
-  "TOKEN_LEX_ERROR", "';'", "'\\n'", "'('", "')'", "','", "'%'", "$accept",
-  "start", "sql_statement", "quit_statement", "alter_table_statement",
-  "create_table_statement", "create_index_statement",
-  "drop_table_statement", "column_def", "column_def_commalist",
-  "data_type", "column_constraint_def", "column_constraint_def_list",
-  "opt_column_constraint_def_list", "table_constraint_def",
-  "table_constraint_def_commalist", "opt_table_constraint_def_commalist",
-  "opt_column_list", "opt_block_properties", "opt_partition_clause",
-  "partition_type", "key_value_list", "key_value", "key_string_value",
-  "key_string_list", "key_integer_value", "index_type",
-  "opt_index_properties", "insert_statement", "copy_from_statement",
-  "opt_copy_from_params", "copy_from_params", "update_statement",
-  "delete_statement", "assignment_list", "assignment_item",
-  "select_statement", "with_clause", "with_list", "with_list_element",
-  "select_query", "opt_all_distinct", "selection",
-  "selection_item_commalist", "selection_item", "from_clause",
+  "TOKEN_CONSTRAINT", "TOKEN_COPY", "TOKEN_CREATE", "TOKEN_CURRENT",
+  "TOKEN_DATE", "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL",
+  "TOKEN_DEFAULT", "TOKEN_DELETE", "TOKEN_DELIMITER", "TOKEN_DESC",
+  "TOKEN_DISTINCT", "TOKEN_DOUBLE", "TOKEN_DROP", "TOKEN_ELSE",
+  "TOKEN_END", "TOKEN_ESCAPE_STRINGS", "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_QUIT", "TOKEN_RANGE",
+  "TOKEN_REAL", "TOKEN_REFERENCES", "TOKEN_RIGHT", "TOKEN_ROW",
+  "TOKEN_ROW_DELIMITER", "TOKEN_ROWS", "TOKEN_SECOND", "TOKEN_SELECT",
+  "TOKEN_SET", "TOKEN_SMA", "TOKEN_SMALLINT", "TOKEN_SUBSTRING",
+  "TOKEN_TABLE", "TOKEN_THEN", "TOKEN_TIME", "TOKEN_TIMESTAMP",
+  "TOKEN_TRUE", "TOKEN_TUPLESAMPLE", "TOKEN_UNBOUNDED", "TOKEN_UNIQUE",
+  "TOKEN_UPDATE", "TOKEN_USING", "TOKEN_VALUES", "TOKEN_VARCHAR",
+  "TOKEN_WHEN", "TOKEN_WHERE", "TOKEN_WINDOW", "TOKEN_WITH", "TOKEN_YEAR",
+  "TOKEN_YEARMONTH", "TOKEN_EOF", "TOKEN_LEX_ERROR", "';'", "'\\n'", "'('",
+  "')'", "','", "'%'", "$accept", "start", "sql_statement",
+  "quit_statement", "alter_table_statement", "create_table_statement",
+  "create_index_statement", "drop_table_statement", "column_def",
+  "column_def_commalist", "data_type", "column_constraint_def",
+  "column_constraint_def_list", "opt_column_constraint_def_list",
+  "table_constraint_def", "table_constraint_def_commalist",
+  "opt_table_constraint_def_commalist", "opt_column_list",
+  "opt_block_properties", "opt_partition_clause", "partition_type",
+  "key_value_list", "key_value", "key_string_value", "key_string_list",
+  "key_integer_value", "index_type", "opt_index_properties",
+  "insert_statement", "copy_from_statement", "opt_copy_from_params",
+  "copy_from_params", "update_statement", "delete_statement",
+  "assignment_list", "assignment_item", "select_statement", "with_clause",
+  "with_list", "with_list_element", "select_query", "opt_all_distinct",
+  "selection", "selection_item_commalist", "selection_item", "from_clause",
   "subquery_expression", "opt_sample_clause", "join_type",
   "joined_table_reference", "table_reference", "table_reference_signature",
   "table_reference_signature_primary", "joined_table_reference_commalist",
   "opt_group_by_clause", "opt_having_clause", "opt_order_by_clause",
-  "opt_limit_clause", "order_commalist", "order_item",
+  "opt_limit_clause", "opt_window_clause", "window_declaration_list",
+  "window_declaration", "window_definition", "opt_window_partition",
+  "opt_window_order", "opt_window_frame", "frame_mode", "frame_preceding",
+  "frame_following", "order_commalist", "order_item",
   "opt_order_direction", "opt_nulls_first", "opt_where_clause",
   "where_clause", "or_expression", "and_expression", "not_expression",
   "predicate_expression_base", "add_expression", "multiply_expression",
@@ -872,14 +894,15 @@ static const yytype_uint16 yytoknum[] =
      350,   351,   352,   353,   354,   355,   356,   357,   358,   359,
      360,   361,   362,   363,   364,   365,   366,   367,   368,   369,
      370,   371,   372,   373,   374,   375,   376,   377,   378,   379,
-      59,    10,    40,    41,    44,    37
+     380,   381,   382,   383,   384,   385,   386,   387,    59,    10,
+      40,    41,    44,    37
 };
 # endif
 
-#define YYPACT_NINF -230
+#define YYPACT_NINF -239
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-230)))
+  (!!((Yystate) == (-239)))
 
 #define YYTABLE_NINF -128
 
@@ -890,55 +913,60 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-     162,  -230,  -230,   -75,   181,     7,    17,    23,    19,  -230,
-      41,   181,   181,  -230,   151,   129,  -230,  -230,  -230,  -230,
-    -230,  -230,  -230,  -230,  -230,  -230,   -38,  -230,   -36,   166,
-     181,  -230,  -230,   138,   181,   181,   181,   181,   181,  -230,
-    -230,   644,   111,    93,  -230,   220,   128,  -230,  -230,  -230,
-     202,  -230,  -230,  -230,  -230,    65,   273,   192,   163,   172,
-    -230,    11,  -230,  -230,   295,   300,  -230,  -230,  -230,   699,
-     207,  -230,   233,  -230,  -230,   218,  -230,  -230,   310,  -230,
-    -230,  -230,  -230,   219,  -230,  -230,   229,   245,   786,   322,
-     261,   224,  -230,  -230,   182,    35,  -230,  -230,  -230,  -230,
-    -230,  -230,  -230,  -230,   928,   -12,   181,   181,   235,   181,
-     181,   194,   226,   238,   181,   181,   542,  -230,  -230,   237,
-     181,  -230,  -230,  -230,   542,    53,   -34,  -230,   359,  -230,
-     105,   105,   983,   363,  -230,   241,    40,  -230,    18,   172,
-     983,  -230,  -230,   181,   983,  -230,  -230,  -230,  -230,   983,
-     300,  -230,   181,   331,    82,  -230,   360,  -230,   268,  -230,
-      -7,  -230,   268,   181,   146,   181,   181,   244,  -230,   246,
-    -230,   143,  1088,   841,   235,   455,   369,   372,  -230,  -230,
-     624,   364,  1069,   155,    15,   983,     8,  -230,   983,  -230,
-     323,   252,  -230,  -230,  -230,  -230,  -230,  -230,   317,  -230,
-      60,   256,  -230,  -230,    10,    90,   197,  -230,   257,    90,
-     -10,   319,  -230,  -230,    35,  -230,  -230,   259,   983,  -230,
-     270,   160,   181,  -230,   983,  -230,   181,  -230,  -230,   263,
-     313,   315,   267,  -230,  -230,  -230,   174,   181,   281,   146,
-     181,  -230,    81,  -230,  -230,    20,    34,   542,   542,    61,
-    -230,  -230,  -230,  -230,  -230,  -230,  -230,  -230,   983,   271,
-     983,    47,  -230,   165,   283,   983,    55,  -230,   342,   270,
-    -230,  -230,   983,   398,  -230,   125,   181,  -230,  -230,   311,
-    -230,   312,   318,   326,    18,  -230,   405,   407,    90,   375,
-     344,  -230,   171,  -230,   983,  -230,   270,  -230,   542,   287,
-     288,   181,   417,   157,   175,  -230,   177,   396,    49,  -230,
-     290,   301,  -230,   335,   296,  1069,  -230,   346,   181,  -230,
-    -230,    81,  -230,  -230,   372,  -230,  -230,  -230,   983,   298,
-     203,   786,  -230,   270,   337,  -230,  -230,  1069,   299,   270,
-     983,  -230,    42,   -25,  -230,  -230,  -230,  -230,  -230,    18,
-     197,   334,   336,  -230,   983,   542,   341,  -230,   270,    21,
-     181,   181,   183,  -230,  -230,  -230,  -230,  -230,  -230,  -230,
-     193,  -230,   181,  -230,  -230,  -230,  -230,   307,   146,   404,
-     347,  -230,   542,  -230,  -230,   316,  -230,   234,   786,  -230,
-     983,   185,  -230,  -230,  1069,   270,  -230,   437,  -230,   354,
-    -230,  -230,   320,   369,   409,   365,  -230,   188,   190,  -230,
-     451,   157,  -230,   181,  -230,  -230,   325,   426,  -230,    31,
-     181,   983,   195,   270,  -230,   198,   338,   542,   983,   460,
-    -230,   366,  -230,  -230,  -230,   200,  -230,  -230,  -230,  -230,
-      14,   181,    13,  -230,   339,   270,  -230,  -230,  -230,   369,
-     345,  -230,   231,  -230,   181,  -230,   181,  -230,  -230,   181,
-    -230,   208,  -230,  -230,   343,  -230,   983,  -230,  -230,   377,
-     348,  -230,   210,  -230,   181,  -230,   113,  -230,   181,  -230,
-     213,  -230,  -230,   215,   376,  -230,   471,  -230
+      71,  -239,  -239,   -52,   270,   -19,    22,   -43,    31,  -239,
+      16,   270,   270,  -239,    97,   162,  -239,  -239,  -239,  -239,
+    -239,  -239,  -239,  -239,  -239,  -239,   -24,  -239,    98,   117,
+     270,  -239,  -239,    58,   270,   270,   270,   270,   270,  -239,
+    -239,   619,    69,    29,  -239,   155,    52,  -239,  -239,  -239,
+     126,  -239,  -239,  -239,  -239,    36,   233,   151,    60,   127,
+    -239,    68,  -239,  -239,   255,   260,  -239,  -239,  -239,   700,
+     131,  -239,   217,  -239,  -239,   137,  -239,  -239,   288,  -239,
+    -239,  -239,  -239,   172,  -239,  -239,   176,   241,   792,   320,
+     297,   189,  -239,  -239,   286,    -1,  -239,  -239,   271,  -239,
+    -239,  -239,  -239,  -239,   954,     0,   270,   270,   234,   270,
+     270,   184,   210,   240,   270,   270,   538,  -239,  -239,   239,
+     270,  -239,  -239,  -239,   538,    47,    -9,  -239,   373,  -239,
+     154,   154,  1035,   374,  -239,   242,    26,  -239,    33,   127,
+    1035,  -239,  -239,   270,  1035,  -239,  -239,  -239,  -239,  1035,
+      38,   260,  -239,   270,   354,    48,  -239,   371,  -239,   272,
+    -239,   140,  -239,   272,   270,   -18,   270,   270,   244,  -239,
+     246,  -239,   163,  1144,   873,   234,   457,   382,   383,  -239,
+    -239,  1270,   377,  1116,   169,    46,  1035,    -2,  -239,  1035,
+    -239,   332,   253,  -239,  -239,  -239,  -239,  -239,  -239,   327,
+    -239,    65,   259,  -239,  -239,    34,   220,   177,  -239,   263,
+     220,    -8,   328,  -239,  -239,    -1,  -239,   303,  -239,  -239,
+     266,  1035,  -239,   293,   180,   270,  -239,  1035,  -239,   270,
+    -239,  -239,   268,   325,   329,   273,  -239,  -239,  -239,   125,
+     270,   290,   -18,   270,  -239,   188,  -239,  -239,     2,   201,
+     538,   538,   105,  -239,  -239,  -239,  -239,  -239,  -239,  -239,
+    -239,  1035,   279,  1035,    43,  -239,   185,   287,  1035,    57,
+    -239,   359,   293,  -239,  -239,  1035,   415,  -239,   178,   270,
+    -239,  -239,   326,  -239,   330,   331,   340,    33,  -239,   419,
+     422,   220,   390,   356,   393,   296,   338,  -239,   187,  -239,
+    1035,  -239,   293,  -239,   538,   298,   299,   270,   435,    -3,
+     192,  -239,   195,   414,    42,  -239,   300,   311,  -239,   353,
+     307,  1116,  -239,   364,   270,  -239,  -239,   188,  -239,  -239,
+     383,  -239,  -239,  -239,  1035,   310,   275,   792,  -239,   293,
+     360,  -239,  -239,  1116,   313,   293,  1035,  -239,    28,   -33,
+    -239,  -239,  -239,  -239,  -239,    33,   177,   350,   357,  -239,
+    1035,   538,   358,  1035,  -239,   425,   -17,  -239,   293,     4,
+     270,   270,   197,  -239,  -239,  -239,  -239,  -239,  -239,  -239,
+     148,  -239,   270,  -239,  -239,  -239,  -239,   315,   -18,   423,
+     367,  -239,   538,  -239,  -239,   333,  -239,   280,   792,  -239,
+    1035,   199,  -239,  -239,  1116,   293,  -239,   462,  -239,   381,
+    -239,  -239,   335,   382,   432,   396,   335,  1035,  -239,  -239,
+    -239,   467,  -239,   202,   204,  -239,   476,    -3,  -239,   270,
+    -239,  -239,   347,   448,  -239,    11,   270,  1035,   206,   293,
+    -239,   208,   351,   538,  1035,   486,   365,   361,  -239,   238,
+       7,   389,  -239,  -239,  -239,   211,  -239,  -239,  -239,  -239,
+      14,   270,    -7,  -239,   363,   293,  -239,  -239,  -239,   382,
+     361,  -239,   270,  -239,   365,  -239,  1035,  -239,  -239,   407,
+     399,   401,   400,   499,   270,  -239,   270,  -239,  -239,   270,
+    -239,   213,  -239,  -239,   372,  -239,   480,  -239,  -239,   102,
+    -239,  -239,  -239,  -239,    17,   376,  -239,   215,  -239,   270,
+     378,  -239,  -239,   446,   408,   451,  -239,   270,  -239,   221,
+     303,  -239,  -239,  -239,   223,   426,   384,  -239,   515,  -239,
+    -239
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -946,85 +974,92 @@ static const yytype_int16 yypact[] =
      means the default is an error.  */
 static const yytype_uint16 yydefact[] =
 {
-       0,     6,   265,     0,     0,     0,     0,     0,     0,    18,
+       0,     6,   287,     0,     0,     0,     0,     0,     0,    18,
      112,     0,     0,     7,     0,     0,    15,     8,    10,    11,
-      13,    14,     9,    17,    12,    16,     0,   105,     0,   263,
-       0,   257,   258,     0,     0,     0,     0,     0,     0,   113,
+      13,    14,     9,    17,    12,    16,     0,   105,     0,   285,
+       0,   279,   280,     0,     0,     0,     0,     0,     0,   113,
      114,     0,     0,   107,   108,     0,   145,     1,     3,     2,
-       0,   106,     5,     4,   264,     0,     0,     0,     0,   166,
-      25,     0,   223,   220,     0,   249,   115,    40,    29,     0,
+       0,   106,     5,     4,   286,     0,     0,     0,     0,   186,
+      25,     0,   245,   242,     0,   271,   115,    40,    29,     0,
        0,    30,    31,    34,    36,     0,    37,    39,     0,    41,
-     219,    35,    38,     0,    32,    33,     0,     0,     0,     0,
-       0,   116,   117,   199,   121,   187,   189,   191,   194,   195,
-     196,   197,   193,   192,     0,   235,     0,     0,     0,     0,
-       0,     0,     0,    94,     0,     0,     0,   101,   167,     0,
-       0,    91,   221,   222,     0,     0,   215,   212,     0,    43,
-       0,   224,     0,     0,    44,     0,     0,   226,     0,   166,
-       0,   250,   251,     0,     0,   120,   253,   254,   252,     0,
-       0,   190,     0,     0,   166,   103,     0,   109,     0,   110,
-       0,   255,     0,     0,     0,     0,     0,     0,    93,    66,
-      27,     0,     0,     0,     0,     0,   168,   170,   172,   174,
-       0,   192,     0,     0,     0,     0,   215,   209,     0,   213,
-       0,     0,   229,   230,   231,   228,   232,   227,     0,   225,
-       0,     0,   123,   198,     0,     0,   147,   136,   122,   141,
-     124,   149,   118,   119,   186,   188,   236,     0,     0,   200,
-     217,     0,     0,   100,     0,   146,     0,    92,    19,     0,
-       0,     0,     0,    20,    21,    22,     0,     0,     0,    64,
-       0,    42,    56,   173,   181,     0,     0,     0,     0,     0,
-     239,   241,   242,   243,   244,   240,   245,   247,     0,     0,
-       0,     0,   233,     0,     0,     0,     0,   210,     0,   216,
-     208,    45,     0,     0,    46,   127,     0,   137,   143,   133,
-     128,   129,   131,     0,     0,   140,     0,     0,   139,     0,
-     151,   201,     0,   202,     0,   102,   104,   256,     0,     0,
-       0,     0,     0,     0,     0,   237,     0,   235,     0,    63,
-      65,    68,    28,     0,     0,     0,    47,     0,     0,    49,
-      55,    57,    26,   180,   169,   171,   246,   248,     0,     0,
-       0,     0,   182,   179,     0,   178,    90,     0,     0,   214,
-       0,   207,     0,     0,   142,   144,   134,   130,   132,     0,
-     148,     0,     0,   138,     0,     0,   153,   203,   218,     0,
-       0,     0,     0,    96,   261,   262,   260,   259,    97,    95,
+     241,    35,    38,     0,    32,    33,     0,     0,     0,     0,
+       0,   116,   117,   221,   121,   207,   209,   211,   214,   217,
+     218,   219,   213,   212,     0,   257,     0,     0,     0,     0,
+       0,     0,     0,    94,     0,     0,     0,   101,   187,     0,
+       0,    91,   243,   244,     0,     0,   237,   234,     0,    43,
+       0,   246,     0,     0,    44,     0,     0,   248,     0,   186,
+       0,   272,   273,     0,     0,   120,   275,   276,   274,     0,
+       0,     0,   210,     0,     0,   186,   103,     0,   109,     0,
+     110,     0,   277,     0,     0,     0,     0,     0,     0,    93,
+      66,    27,     0,     0,     0,     0,     0,   188,   190,   192,
+     194,     0,   212,     0,     0,     0,     0,   237,   231,     0,
+     235,     0,     0,   251,   252,   253,   250,   254,   249,     0,
+     247,     0,     0,   123,   220,     0,     0,   147,   136,   122,
+     141,   124,   149,   118,   119,   206,   208,   163,   215,   258,
+       0,     0,   222,   239,     0,     0,   100,     0,   146,     0,
+      92,    19,     0,     0,     0,     0,    20,    21,    22,     0,
+       0,     0,    64,     0,    42,    56,   193,   201,     0,     0,
+       0,     0,     0,   261,   263,   264,   265,   266,   262,   267,
+     269,     0,     0,     0,     0,   255,     0,     0,     0,     0,
+     232,     0,   238,   230,    45,     0,     0,    46,   127,     0,
+     137,   143,   133,   128,   129,   131,     0,     0,   140,     0,
+       0,   139,     0,   151,     0,     0,   165,   223,     0,   224,
+       0,   102,   104,   278,     0,     0,     0,     0,     0,     0,
+       0,   259,     0,   257,     0,    63,    65,    68,    28,     0,
+       0,     0,    47,     0,     0,    49,    55,    57,    26,   200,
+     189,   191,   268,   270,     0,     0,     0,     0,   202,   199,
+       0,   198,    90,     0,     0,   236,     0,   229,     0,     0,
+     142,   144,   134,   130,   132,     0,   148,     0,     0,   138,
+       0,     0,   153,     0,   216,     0,   167,   225,   240,     0,
+       0,     0,     0,    96,   283,   284,   282,   281,    97,    95,
        0,    67,     0,    83,    84,    85,    86,    87,     0,     0,
-      70,    48,     0,    51,    50,     0,    54,     0,     0,   184,
-       0,     0,   177,   234,     0,   211,   204,     0,   205,     0,
-     125,   126,   150,   152,     0,   155,    61,     0,     0,    58,
-       0,     0,   238,     0,    24,    62,     0,     0,    23,     0,
-       0,     0,     0,   175,   183,     0,     0,     0,     0,     0,
-     111,     0,    59,    98,    99,     0,    74,    76,    77,    78,
-       0,     0,     0,    52,     0,   176,   185,    89,   206,   135,
-     154,   157,   160,   156,     0,    88,     0,    82,    80,     0,
-      79,     0,    72,    73,     0,    53,     0,   161,   162,   163,
-       0,    75,     0,    69,     0,   158,     0,   159,     0,    81,
-       0,   164,   165,     0,     0,    60,     0,    71
+      70,    48,     0,    51,    50,     0,    54,     0,     0,   204,
+       0,     0,   197,   256,     0,   233,   226,     0,   227,     0,
+     125,   126,   150,   152,     0,   155,   164,     0,   170,   169,
+     162,     0,    61,     0,     0,    58,     0,     0,   260,     0,
+      24,    62,     0,     0,    23,     0,     0,     0,     0,   195,
+     203,     0,     0,     0,     0,     0,   157,   166,   177,   180,
+       0,     0,    59,    98,    99,     0,    74,    76,    77,    78,
+       0,     0,     0,    52,     0,   196,   205,    89,   228,   135,
+     154,   156,     0,   111,   158,   159,     0,   181,   182,   183,
+       0,     0,     0,     0,     0,    88,     0,    82,    80,     0,
+      79,     0,    72,    73,     0,    53,     0,   160,   178,     0,
+     179,   171,   173,   172,     0,     0,    75,     0,    69,     0,
+       0,   184,   185,     0,     0,     0,   168,     0,    81,     0,
+     163,   174,   176,   175,     0,     0,     0,    60,     0,   161,
+      71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -230,  -230,  -230,  -230,  -230,  -230,  -230,  -230,  -127,  -230,
-     309,   161,  -230,  -230,  -229,  -230,  -230,  -230,  -230,  -230,
-    -230,    43,    27,  -230,  -230,  -230,  -230,  -230,  -230,  -230,
-    -230,  -230,  -230,  -230,  -230,   264,  -230,  -230,  -230,   378,
-      12,  -230,  -230,  -230,   350,  -230,  -103,  -230,  -230,  -139,
-     142,  -190,    -8,  -230,  -230,  -230,  -230,  -230,  -230,    26,
-    -230,  -230,    44,  -230,  -121,   240,   247,   321,   -28,   349,
-     351,   394,  -132,  -230,  -230,  -230,  -230,   324,  -230,   373,
-     327,  -216,  -171,   370,   108,  -107,  -230,  -230,  -230,  -230,
-    -230,  -119,    -4,    94,  -230,  -230
+    -239,  -239,  -239,  -239,  -239,  -239,  -239,  -239,   -88,  -239,
+     355,   196,  -239,  -239,  -238,  -239,  -239,  -239,  -239,  -239,
+    -239,    66,    44,  -239,  -239,  -239,  -239,  -239,  -239,  -239,
+    -239,  -239,  -239,  -239,  -239,   304,  -239,  -239,  -239,   424,
+       9,  -239,  -239,  -239,   392,  -239,   -92,  -239,  -239,  -134,
+     179,  -182,   -11,  -239,  -239,  -239,  -239,  -239,  -239,  -239,
+      59,    15,  -239,  -239,  -239,  -239,  -239,  -239,    92,    61,
+    -239,  -239,   -55,  -239,  -119,   291,   289,   379,   -28,   403,
+     402,   450,  -123,  -239,  -239,  -239,  -239,   368,  -239,   430,
+     370,  -204,  -177,   421,   146,  -113,  -239,  -239,  -239,  -239,
+    -239,  -118,    -4,   134,  -239,  -239
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
-      -1,    14,    15,    16,    17,    18,    19,    20,   170,   171,
-      89,   320,   321,   322,   233,   310,   311,   238,   380,   418,
-     464,   435,   436,   437,   438,   439,   377,   414,    21,    22,
-     168,   304,    23,    24,   154,   155,    25,    26,    43,    44,
-     135,    41,    90,    91,    92,   139,    93,   288,   283,   206,
-     207,   277,   278,   208,   290,   356,   405,   430,   450,   451,
-     469,   477,   117,   118,   176,   177,   178,   179,   180,    95,
-      96,    97,    98,    99,   100,   101,   186,   187,   126,   127,
-     190,   221,   102,   198,   263,   103,   306,   260,   104,   144,
-     149,   160,   105,   368,    28,    29
+      -1,    14,    15,    16,    17,    18,    19,    20,   171,   172,
+      89,   326,   327,   328,   236,   316,   317,   241,   390,   434,
+     494,   455,   456,   457,   458,   459,   387,   430,    21,    22,
+     169,   310,    23,    24,   155,   156,    25,    26,    43,    44,
+     135,    41,    90,    91,    92,   139,    93,   291,   286,   207,
+     208,   280,   281,   209,   293,   362,   415,   446,   473,   474,
+     475,   295,   296,   366,   420,   421,   483,   516,   447,   448,
+     479,   500,   117,   118,   177,   178,   179,   180,   181,    95,
+      96,    97,    98,    99,   100,   101,   187,   188,   126,   127,
+     191,   224,   102,   199,   266,   103,   312,   263,   104,   144,
+     149,   161,   105,   378,    28,    29
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
@@ -1032,341 +1067,374 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      33,   183,   292,   184,    45,   159,   209,    42,    46,   181,
-     309,   262,    27,    94,    31,   152,    32,   181,    31,   285,
-      32,   457,    31,   247,    32,   188,    55,   286,   247,   247,
-      57,    58,    59,    60,    61,   205,   228,    30,    51,   247,
-      50,   125,   397,   458,   249,   250,   251,   252,   253,   254,
-     255,   256,   257,   258,   245,   141,   142,   334,   146,   147,
-     136,   141,   142,   141,   142,   275,   181,   188,   181,    10,
-      39,   244,   209,   121,   141,   142,   141,   142,   326,   327,
-     328,   141,   142,    34,   373,   462,    36,   374,   375,   124,
-     145,   313,    52,   111,    31,    53,    32,    40,   353,    45,
-      38,   205,   156,    46,   200,   161,   162,   287,   398,   259,
-     169,   172,    94,   312,   463,   391,   161,    10,    10,    35,
-     153,   276,   153,   112,   314,   220,   225,   226,   265,   273,
-     305,   185,   119,   315,   210,    37,   329,   335,   402,   213,
-     181,   181,   204,   120,   383,   350,   459,   246,   216,   415,
-     204,    47,   209,   323,   406,   192,   332,   266,   376,   172,
-     269,   234,   235,     1,   443,     2,   393,   203,   340,    54,
-     148,   316,   422,   203,   227,   396,   185,   359,   481,   193,
-     317,   205,   362,   211,   318,    31,    31,    32,    32,   229,
-     220,   181,     3,   194,   195,   279,   296,   482,   223,   319,
-     210,    46,   280,   141,   142,    46,   116,    56,     4,     5,
-     281,   196,   390,   143,   230,     6,   222,   209,   156,   106,
-       7,   364,   297,   262,   141,   142,   389,   107,   302,   282,
-     330,   197,   333,   307,   403,   303,   172,   339,   163,   164,
-       8,   407,   408,   421,   342,   231,   205,   410,   181,   365,
-     366,   108,   141,   142,   411,   141,   142,    48,   344,    49,
-     109,   419,     9,   467,   232,   412,   358,   279,   345,    10,
-     165,   166,    46,   367,   280,   181,   239,   240,   113,  -127,
-     210,    11,   281,   110,    46,   114,   468,    12,   264,   226,
-      13,   141,   142,   293,   294,   115,   116,   161,   336,   337,
-     387,   282,   122,   220,   357,   294,   449,   123,   369,   370,
-     371,   372,   395,   129,   385,   131,   409,   226,   424,   294,
-     181,   431,   226,   432,   226,   134,   220,   137,   446,   294,
-     138,   447,   337,   455,   456,    31,    62,    32,    63,   128,
-     472,   473,   456,   479,   226,   210,   484,   226,   485,   226,
-     130,   132,    64,    65,   217,   480,   161,   161,   140,   483,
-     220,   133,   423,   167,    67,    68,   191,   158,   307,   182,
-     201,   224,    69,    70,   202,    10,   236,   247,   237,    71,
-      72,   248,    73,   270,   261,   271,   272,   218,    74,   274,
-     289,   284,   291,   445,    75,   298,   299,    76,   300,   301,
-     452,   308,   341,   331,   338,   343,   346,   347,   349,   440,
-      77,    78,   351,   348,   352,   354,   444,   355,    79,   360,
-     361,    80,   363,   152,   378,   381,   379,   392,   382,   384,
-     388,   394,   400,    81,   401,   404,   460,   440,   452,   413,
-     416,    82,    83,   417,   426,    84,    85,   427,   420,   428,
-     470,   429,   440,    86,   294,   161,   433,   441,    87,    31,
-      62,    32,    63,    88,   219,   173,   442,   453,   476,   454,
-     161,   448,   465,   486,   161,   474,    64,    65,   487,   466,
-     478,   242,   386,   471,   461,   157,   295,   324,    67,    68,
-     212,   399,   475,   214,   243,   325,    69,    70,   151,   189,
-     215,   199,   425,    71,    72,   434,    73,     0,     0,     0,
-     267,     0,    74,   268,     0,     0,     0,   174,    75,     0,
-       0,    76,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    77,    78,     0,     0,     0,     0,
-       0,     0,    79,     0,     0,    80,    31,    62,    32,    63,
-       0,     0,   173,     0,     0,     0,     0,    81,     0,     0,
-       0,     0,    10,    64,    65,    82,    83,     0,     0,    84,
-      85,     0,     0,     0,     0,    67,    68,    86,     0,     0,
-       0,     0,    87,    69,    70,     0,     0,   175,     0,     0,
-      71,    72,     0,    73,     0,     0,     0,     0,     0,    74,
-       0,     0,     0,     0,   174,    75,     0,     0,    76,     0,
+      33,    45,   184,   182,   315,   185,   265,    42,    46,    27,
+     250,   182,   250,    94,   480,   210,   160,   298,    31,   250,
+      32,   487,   146,   147,   513,   232,    55,   153,   288,   289,
+      57,    58,    59,    60,    61,    51,   407,    31,    31,    32,
+      32,   125,    31,   488,    32,    39,   206,   141,   142,   141,
+     142,   189,   233,   340,   250,   481,    50,   248,   189,    34,
+     136,   182,   374,   182,   111,   514,    30,   492,   141,   142,
+     121,   278,     1,    40,     2,    37,   231,   383,   141,   142,
+     384,   385,   210,   247,   212,   234,   141,   142,   418,    10,
+     145,   375,   376,    36,   419,   112,    45,    47,   493,    35,
+     226,     3,   157,    46,   201,   162,   163,   235,   408,   359,
+     170,   173,    94,   206,    38,   290,   162,     4,     5,   377,
+      54,   124,   332,   333,   334,     6,   223,   311,   186,    56,
+       7,   482,   154,   401,   211,   341,   276,   182,   182,   214,
+     154,   515,   148,   329,   393,   422,   218,    10,   249,   219,
+     431,     8,   463,   356,   489,   318,   412,   386,   269,   416,
+     173,   272,   237,   238,   210,   268,   403,   204,   511,   406,
+     338,   107,   230,   205,   205,     9,   346,   186,   217,   116,
+     308,    10,   335,   106,    10,   369,   108,   309,   512,   372,
+     225,   182,   109,   223,   438,   206,   119,    11,   319,   302,
+     115,   211,    46,   426,    12,   193,    46,    13,   120,   110,
+     427,   252,   253,   254,   255,   256,   257,   258,   259,   260,
+     261,   157,   141,   142,    31,   303,    32,   265,   164,   165,
+     194,   320,   210,   336,    52,   339,   313,    53,   113,   173,
+     345,   321,   413,   399,   195,   196,   114,   348,   182,   282,
+     282,   279,   423,   424,   166,   167,   283,   283,   116,   141,
+     142,  -127,   122,   206,   284,   284,   197,   123,   351,   428,
+     477,   128,   368,   435,    31,    46,    32,   130,   262,   182,
+     322,   228,   229,   211,   400,   285,   285,    46,   198,   437,
+      31,   323,    32,   131,   478,   324,   141,   142,    48,   129,
+      49,   141,   142,   162,   242,   243,   397,   141,   142,   223,
+     267,   229,   132,   325,   141,   142,   133,   143,   405,   350,
+     395,   299,   300,   134,   469,   137,   342,   343,   367,   300,
+     182,   140,   223,   379,   380,   223,   381,   382,   425,   229,
+     440,   300,   204,   451,   229,   452,   229,   466,   300,   467,
+     343,   211,   485,   486,   508,   486,   518,   229,    31,    62,
+      32,    63,   525,   229,   527,   229,   162,   162,   138,   150,
+     223,   507,   439,   168,   159,    64,    65,   220,   313,   183,
+     192,   202,   227,   203,   239,    10,   240,    67,    68,   449,
+     250,   519,   251,   273,   274,    69,    70,   264,   275,   524,
+     277,   292,   294,    71,    72,   287,    73,   297,   304,   465,
+     305,   221,    74,   307,   306,   344,   449,   314,    75,   337,
+     347,    76,   349,   352,   355,   460,   357,   353,   354,   358,
+     360,   361,   464,   363,   365,    77,    78,   364,   370,   371,
+     373,   153,   388,    79,   389,   391,    80,   392,   449,   394,
+     398,   410,   402,   404,   414,   429,   490,   460,   411,   432,
+      81,    31,    62,    32,    63,   417,   433,   174,   496,   442,
+      82,    83,   444,   436,    84,    85,   443,   300,    64,    65,
+     505,   453,   460,    86,   445,   162,   450,   461,   462,    87,
+      67,    68,   468,   471,    88,   222,   484,   472,    69,    70,
+     499,   501,   503,   476,   495,   162,    71,    72,   504,    73,
+     502,   510,   509,   162,   521,    74,   517,   522,   520,   523,
+     175,    75,   530,   396,    76,   529,   528,   491,   245,   301,
+     506,   158,   213,   497,   409,   526,   470,   498,    77,    78,
+     331,   330,    31,    62,    32,    63,    79,   215,   174,    80,
+     441,   216,   200,   246,   152,   270,   190,   271,     0,    64,
+      65,   454,     0,    81,     0,     0,     0,     0,     0,     0,
+      10,    67,    68,    82,    83,     0,     0,    84,    85,    69,
+      70,     0,     0,     0,     0,     0,    86,    71,    72,     0,
+      73,     0,    87,     0,     0,     0,    74,   176,     0,     0,
+       0,   175,    75,     0,     0,    76,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,    77,
+      78,     0,     0,    31,    62,    32,    63,    79,     0,     0,
+      80,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+      64,    65,    66,     0,    81,     0,     0,     0,     0,     0,
+       0,     0,    67,    68,    82,    83,     0,     0,    84,    85,
+      69,    70,     0,     0,     0,     0,     0,    86,    71,    72,
+       0,    73,     0,    87,     0,     0,     0,    74,   176,     0,
+       0,     0,     0,    75,     0,     0,    76,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+      77,    78,     0,     0,    31,    62,    32,    63,    79,     0,
+       0,    80,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,    64,    65,     0,     0,    81,     0,     0,     0,     0,
+       0,     0,     0,    67,    68,    82,    83,     0,     0,    84,
+      85,    69,    70,     0,     0,     0,     0,     0,    86,    71,
+      72,     0,    73,     0,    87,     0,     0,     0,    74,    88,
+       0,     0,     0,     0,    75,     0,     0,    76,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,    77,    78,     0,     0,     0,     0,     0,     0,    79,
-       0,     0,    80,     0,   249,   250,   251,   252,   253,   254,
-     255,   256,   257,   258,    81,   141,   142,     0,    31,    62,
-      32,    63,    82,    83,     0,     0,    84,    85,     0,     0,
-       0,     0,     0,     0,    86,    64,    65,    66,     0,    87,
-       0,     0,     0,     0,   175,     0,     0,    67,    68,     0,
-       0,     0,     0,     0,     0,    69,    70,     0,     0,     0,
-       0,     0,    71,    72,     0,    73,     0,     0,     0,   259,
-       0,    74,     0,    31,    62,    32,    63,    75,     0,     0,
+       0,     0,    80,     0,     0,     0,    31,    62,    32,    63,
+       0,     0,     0,     0,     0,     0,    81,     0,     0,     0,
+       0,     0,     0,    64,    65,     0,    82,    83,     0,     0,
+      84,    85,     0,     0,     0,    67,    68,     0,     0,    86,
+     124,     0,     0,    69,    70,    87,     0,     0,     0,     0,
+      88,    71,    72,     0,    73,     0,     0,     0,     0,     0,
+      74,     0,     0,     0,     0,     0,    75,     0,     0,    76,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    77,    78,     0,     0,    31,    62,    32,
+      63,    79,     0,     0,    80,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,    64,    65,     0,     0,    81,     0,
+       0,     0,     0,     0,     0,    10,    67,    68,    82,    83,
+       0,     0,    84,    85,    69,    70,     0,     0,     0,     0,
+       0,    86,    71,    72,     0,    73,     0,    87,     0,     0,
+       0,    74,    88,     0,     0,     0,   175,    75,     0,     0,
       76,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-      64,    65,     0,    77,    78,     0,     0,     0,     0,     0,
-       0,    79,    67,    68,    80,     0,     0,     0,     0,     0,
-      69,    70,     0,     0,     0,     0,    81,    71,    72,     0,
-      73,     0,     0,     0,    82,    83,    74,     0,    84,    85,
-       0,     0,    75,     0,     0,    76,    86,     0,     0,     0,
-       0,    87,     0,     0,     0,     0,    88,     0,    77,    78,
-       0,     0,     0,     0,     0,     0,    79,     0,     0,    80,
-      31,    62,    32,    63,     0,     0,     0,     0,     0,     0,
-       0,    81,     0,     0,     0,     0,     0,    64,    65,    82,
-      83,     0,     0,    84,    85,     0,     0,     0,     0,    67,
-      68,    86,   124,     0,     0,     0,    87,    69,    70,     0,
-       0,    88,     0,     0,    71,    72,     0,    73,     0,     0,
-       0,     0,     0,    74,     0,    31,    62,    32,    63,    75,
+       0,     0,     0,     0,    77,    78,     0,     0,    31,    62,
+      32,    63,    79,     0,     0,    80,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,    64,   151,     0,     0,    81,
+       0,     0,     0,     0,     0,     0,     0,    67,    68,    82,
+      83,     0,     0,    84,    85,    69,    70,     0,     0,     0,
+       0,     0,    86,    71,    72,     0,    73,     0,    87,     0,
+       0,     0,    74,   176,     0,     0,     0,     0,    75,     0,
+       0,    76,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,    77,    78,     0,     0,    31,
+      62,    32,    63,    79,     0,     0,    80,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,    64,    65,     0,     0,
+      81,     0,     0,     0,     0,     0,     0,     0,    67,    68,
+      82,    83,     0,     0,    84,    85,    69,    70,     0,     0,
+       0,     0,     0,    86,    71,    72,     0,    73,     0,    87,
+       0,     0,     0,    74,    88,     0,     0,     0,     0,    75,
        0,     0,    76,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,    64,    65,     0,    77,    78,     0,     0,     0,
-       0,     0,     0,    79,    67,    68,    80,     0,     0,     0,
-       0,     0,    69,    70,     0,     0,     0,     0,    81,    71,
-      72,     0,    73,    10,     0,     0,    82,    83,    74,     0,
-      84,    85,     0,   174,    75,     0,     0,    76,    86,     0,
-       0,     0,     0,    87,     0,     0,     0,     0,    88,     0,
-      77,    78,     0,     0,     0,     0,     0,     0,    79,     0,
-       0,    80,    31,    62,    32,    63,     0,     0,     0,     0,
-       0,     0,     0,    81,     0,     0,     0,     0,     0,    64,
-     150,    82,    83,     0,     0,    84,    85,     0,     0,     0,
-       0,    67,    68,    86,     0,     0,     0,     0,    87,    69,
-      70,     0,     0,   175,     0,     0,    71,    72,     0,    73,
-       0,     0,     0,     0,     0,    74,     0,    31,    62,    32,
-      63,    75,     0,     0,    76,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    64,    65,     0,    77,    78,     0,
-       0,     0,     0,     0,     0,    79,    67,    68,    80,     0,
-       0,     0,     0,     0,    69,    70,     0,     0,     0,     0,
-      81,    71,    72,     0,    73,     0,     0,     0,    82,    83,
-      74,     0,    84,    85,     0,     0,    75,     0,     0,    76,
-      86,     0,     0,     0,     0,    87,     0,     0,     0,     0,
-      88,     0,    77,    78,     0,     0,     0,     0,     0,     0,
-      79,     0,     0,    80,    62,     0,    63,     0,     0,     0,
-       0,     0,     0,     0,     0,    81,     0,     0,     0,     0,
-      64,   150,     0,    82,    83,     0,     0,    84,    85,     0,
-       0,     0,    67,    68,     0,    86,     0,     0,     0,     0,
-      87,    70,     0,     0,     0,    88,     0,    71,    72,     0,
-      73,    67,    68,     0,     0,     0,    74,     0,     0,     0,
-      70,     0,     0,     0,     0,    76,    71,    72,     0,    73,
-       0,     0,     0,     0,     0,    74,     0,     0,    77,    78,
-       0,     0,     0,     0,    76,     0,    79,     0,     0,    80,
-       0,     0,     0,     0,     0,     0,     0,    77,   241,     0,
-       0,    81,     0,     0,     0,    79,     0,     0,     0,    82,
-       0,     0,     0,    84,    85,     0,     0,     0,     0,     0,
-      81,    86,     0,     0,     0,     0,    87,     0,    82,     0,
-       0,     0,    84,    85,     0,     0,     0,     0,     0,     0,
-      86,     0,     0,     0,     0,    87
+       0,     0,     0,     0,     0,     0,    77,    78,     0,     0,
+       0,    62,     0,    63,    79,     0,     0,    80,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    64,   151,     0,
+       0,    81,     0,     0,     0,     0,     0,     0,     0,    67,
+      68,    82,    83,     0,     0,    84,    85,     0,    70,     0,
+       0,     0,     0,     0,    86,    71,    72,     0,    73,     0,
+      87,     0,     0,     0,    74,    88,     0,    67,    68,     0,
+       0,     0,     0,    76,     0,     0,    70,     0,     0,     0,
+       0,     0,     0,    71,    72,     0,    73,    77,    78,     0,
+       0,     0,    74,     0,     0,    79,     0,     0,    80,     0,
+       0,    76,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,    81,     0,     0,    77,   244,     0,     0,     0,
+       0,     0,    82,    79,     0,     0,    84,    85,     0,     0,
+       0,     0,     0,     0,     0,    86,     0,     0,     0,     0,
+      81,    87,     0,     0,     0,     0,     0,     0,     0,     0,
+      82,     0,     0,     0,    84,    85,     0,     0,     0,     0,
+       0,     0,     0,    86,     0,     0,     0,     0,     0,    87,
+     252,   253,   254,   255,   256,   257,   258,   259,   260,   261,
+       0,   141,   142,     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,   262
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   120,   218,   124,    12,   108,   138,    11,    12,   116,
-     239,   182,     0,    41,     4,    27,     6,   124,     4,   209,
-       6,     7,     4,     8,     6,    59,    30,    37,     8,     8,
-      34,    35,    36,    37,    38,   138,   163,   112,    26,     8,
-      78,    69,    67,    29,    10,    11,    12,    13,    14,    15,
-      16,    17,    18,    19,   175,    21,    22,    10,    23,    24,
-      88,    21,    22,    21,    22,   204,   173,    59,   175,   107,
-      29,   174,   204,    61,    21,    22,    21,    22,    17,    18,
-      19,    21,    22,    76,    35,    72,    69,    38,    39,   123,
-      94,    10,   128,    28,     4,   131,     6,    56,   288,   107,
-      81,   204,   106,   107,   132,   109,   110,   117,   133,    75,
-     114,   115,   140,   240,   101,   331,   120,   107,   107,   112,
-     132,    31,   132,    58,    43,   153,   133,   134,   113,    69,
-     237,   123,   121,    52,   138,   112,    75,    90,   354,   143,
-     247,   248,   132,   132,   315,   284,   132,   175,   152,   378,
-     132,     0,   284,   133,   133,    50,   259,   185,   109,   163,
-     188,   165,   166,     1,   133,     3,   337,   133,   113,     3,
-     135,    90,   388,   133,   162,   133,   123,   298,    65,    74,
-      99,   284,   301,   139,   103,     4,     4,     6,     6,    43,
-     218,   298,    30,    88,    89,    70,   224,    84,   154,   118,
-     204,   205,    77,    21,    22,   209,   124,    69,    46,    47,
-      85,   106,     9,    31,    68,    53,   134,   349,   222,   108,
-      58,    64,   226,   394,    21,    22,   329,   134,    54,   104,
-     258,   126,   260,   237,   355,    61,   240,   265,    44,    45,
-      78,   360,   361,     9,   272,    99,   349,    54,   355,    92,
-      93,    31,    21,    22,    61,    21,    22,   128,   133,   130,
-     132,   382,   100,    32,   118,   372,   294,    70,   276,   107,
-      44,    45,   276,   116,    77,   382,   133,   134,     5,    82,
-     284,   119,    85,    81,   288,    93,    55,   125,   133,   134,
-     128,    21,    22,   133,   134,   132,   124,   301,   133,   134,
-     328,   104,     7,   331,   133,   134,   427,     7,   133,   134,
-     133,   134,   340,    80,   318,     5,   133,   134,   133,   134,
-     427,   133,   134,   133,   134,    80,   354,     5,   133,   134,
-      69,   133,   134,   133,   134,     4,     5,     6,     7,   132,
-     459,   133,   134,   133,   134,   349,   133,   134,   133,   134,
-     132,   132,    21,    22,    23,   474,   360,   361,   134,   478,
-     388,   132,   390,   125,    33,    34,     7,   132,   372,   132,
-       7,    11,    41,    42,   133,   107,   132,     8,   132,    48,
-      49,     9,    51,    60,    20,   133,    69,    56,    57,   133,
-      71,   134,   133,   421,    63,   132,    83,    66,    83,   132,
-     428,   120,    60,   132,   121,     7,    95,    95,    82,   413,
-      79,    80,     7,    95,     7,    40,   420,    73,    87,   132,
-     132,    90,     5,    27,   134,    90,   125,    90,   132,    83,
-     132,   132,    98,   102,    98,    94,   440,   441,   466,   132,
-      36,   110,   111,    96,     7,   114,   115,    93,   132,    40,
-     454,    86,   456,   122,   134,   459,     5,   132,   127,     4,
-       5,     6,     7,   132,   133,    10,    40,     7,    91,   103,
-     474,   133,   133,    97,   478,   132,    21,    22,     7,   134,
-     132,   172,   321,   456,   441,   107,   222,   247,    33,    34,
-     140,   349,   466,   144,   173,   248,    41,    42,   104,   126,
-     149,   131,   394,    48,    49,   411,    51,    -1,    -1,    -1,
-     186,    -1,    57,   186,    -1,    -1,    -1,    62,    63,    -1,
-      -1,    66,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    79,    80,    -1,    -1,    -1,    -1,
-      -1,    -1,    87,    -1,    -1,    90,     4,     5,     6,     7,
-      -1,    -1,    10,    -1,    -1,    -1,    -1,   102,    -1,    -1,
-      -1,    -1,   107,    21,    22,   110,   111,    -1,    -1,   114,
-     115,    -1,    -1,    -1,    -1,    33,    34,   122,    -1,    -1,
-      -1,    -1,   127,    41,    42,    -1,    -1,   132,    -1,    -1,
-      48,    49,    -1,    51,    -1,    -1,    -1,    -1,    -1,    57,
-      -1,    -1,    -1,    -1,    62,    63,    -1,    -1,    66,    -1,
+       4,    12,   120,   116,   242,   124,   183,    11,    12,     0,
+       8,   124,     8,    41,     7,   138,   108,   221,     4,     8,
+       6,     7,    23,    24,     7,    43,    30,    27,   210,    37,
+      34,    35,    36,    37,    38,    26,    69,     4,     4,     6,
+       6,    69,     4,    29,     6,    29,   138,    21,    22,    21,
+      22,    60,    70,    10,     8,    48,    80,   176,    60,    78,
+      88,   174,    65,   176,    28,    48,   118,    74,    21,    22,
+      61,   205,     1,    57,     3,   118,   164,    35,    21,    22,
+      38,    39,   205,   175,   139,   103,    21,    22,   105,   113,
+      94,    94,    95,    71,   111,    59,   107,     0,   105,   118,
+     155,    30,   106,   107,   132,   109,   110,   125,   141,   291,
+     114,   115,   140,   205,    83,   123,   120,    46,    47,   122,
+       3,   130,    17,    18,    19,    54,   154,   240,   130,    71,
+      59,   124,   140,   337,   138,    92,    71,   250,   251,   143,
+     140,   124,   143,   141,   321,   141,   150,   113,   176,   153,
+     388,    80,   141,   287,   140,   243,   360,   115,   186,   363,
+     164,   189,   166,   167,   287,   119,   343,   141,    66,   141,
+     262,   142,   163,   140,   140,   104,   119,   130,   140,   131,
+      55,   113,    77,   114,   113,   304,    31,    62,    86,   307,
+     142,   304,   140,   221,   398,   287,   128,   126,    10,   227,
+     140,   205,   206,    55,   133,    51,   210,   136,   140,    83,
+      62,    10,    11,    12,    13,    14,    15,    16,    17,    18,
+      19,   225,    21,    22,     4,   229,     6,   404,    44,    45,
+      76,    43,   355,   261,   136,   263,   240,   139,     5,   243,
+     268,    53,   361,   335,    90,    91,    95,   275,   361,    72,
+      72,    31,   370,   371,    44,    45,    79,    79,   131,    21,
+      22,    84,     7,   355,    87,    87,   112,     7,   279,   382,
+      32,   140,   300,   392,     4,   279,     6,   140,    77,   392,
+      92,   141,   142,   287,     9,   108,   108,   291,   134,     9,
+       4,   103,     6,     5,    56,   107,    21,    22,   136,    82,
+     138,    21,    22,   307,   141,   142,   334,    21,    22,   337,
+     141,   142,   140,   125,    21,    22,   140,    31,   346,   141,
+     324,   141,   142,    82,   443,     5,   141,   142,   141,   142,
+     443,   142,   360,   141,   142,   363,   141,   142,   141,   142,
+     141,   142,   141,   141,   142,   141,   142,   141,   142,   141,
+     142,   355,   141,   142,   141,   142,   141,   142,     4,     5,
+       6,     7,   141,   142,   141,   142,   370,   371,    71,    98,
+     398,   489,   400,   133,   140,    21,    22,    23,   382,   140,
+       7,     7,    11,   141,   140,   113,   140,    33,    34,   417,
+       8,   509,     9,    61,   141,    41,    42,    20,    71,   517,
+     141,    73,    99,    49,    50,   142,    52,   141,   140,   437,
+      85,    57,    58,   140,    85,   128,   444,   127,    64,   140,
+      61,    67,     7,    97,    84,   429,     7,    97,    97,     7,
+      40,    75,   436,    40,    96,    81,    82,   141,   140,   140,
+       5,    27,   142,    89,   133,    92,    92,   140,   476,    85,
+     140,   101,    92,   140,    96,   140,   460,   461,   101,    36,
+     106,     4,     5,     6,     7,    40,    99,    10,   472,     7,
+     116,   117,    40,   140,   120,   121,    95,   142,    21,    22,
+     484,     5,   486,   129,    88,   489,    19,   140,    40,   135,
+      33,    34,   141,     7,   140,   141,   107,   132,    41,    42,
+      93,   102,   102,   142,   141,   509,    49,    50,     9,    52,
+     109,    31,   140,   517,    68,    58,   140,   109,   140,    68,
+      63,    64,     7,   327,    67,   141,   100,   461,   173,   225,
+     486,   107,   140,   474,   355,   520,   444,   476,    81,    82,
+     251,   250,     4,     5,     6,     7,    89,   144,    10,    92,
+     404,   149,   131,   174,   104,   187,   126,   187,    -1,    21,
+      22,   427,    -1,   106,    -1,    -1,    -1,    -1,    -1,    -1,
+     113,    33,    34,   116,   117,    -1,    -1,   120,   121,    41,
+      42,    -1,    -1,    -1,    -1,    -1,   129,    49,    50,    -1,
+      52,    -1,   135,    -1,    -1,    -1,    58,   140,    -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,     4,     5,     6,     7,    89,    -1,    -1,
+      92,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      21,    22,    23,    -1,   106,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    33,    34,   116,   117,    -1,    -1,   120,   121,
+      41,    42,    -1,    -1,    -1,    -1,    -1,   129,    49,    50,
+      -1,    52,    -1,   135,    -1,    -1,    -1,    58,   140,    -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,     4,     5,     6,     7,    89,    -1,
+      -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    21,    22,    -1,    -1,   106,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    33,    34,   116,   117,    -1,    -1,   120,
+     121,    41,    42,    -1,    -1,    -1,    -1,    -1,   129,    49,
+      50,    -1,    52,    -1,   135,    -1,    -1,    -1,    58,   140,
+      -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,     4,     5,     6,     7,
+      -1,    -1,    -1,    -1,    -1,    -1,   106,    -1,    -1,    -1,
+      -1,    -1,    -1,    21,    22,    -1,   116,   117,    -1,    -1,
+     120,   121,    -1,    -1,    -1,    33,    34,    -1,    -1,   129,
+     130,    -1,    -1,    41,    42,   135,    -1,    -1,    -1,    -1,
+     140,    49,    50,    -1,    52,    -1,    -1,    -1,    -1,    -1,
+      58,    -1,    -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,     4,     5,     6,
+       7,    89,    -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    21,    22,    -1,    -1,   106,    -1,
+      -1,    -1,    -1,    -1,    -1,   113,    33,    34,   116,   117,
+      -1,    -1,   120,   121,    41,    42,    -1,    -1,    -1,    -1,
+      -1,   129,    49,    50,    -1,    52,    -1,   135,    -1,    -1,
+      -1,    58,   140,    -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,     4,     5,
+       6,     7,    89,    -1,    -1,    92,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    21,    22,    -1,    -1,   106,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    33,    34,   116,
+     117,    -1,    -1,   120,   121,    41,    42,    -1,    -1,    -1,
+      -1,    -1,   129,    49,    50,    -1,    52,    -1,   135,    -1,
+      -1,    -1,    58,   140,    -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,     4,
+       5,     6,     7,    89,    -1,    -1,    92,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    21,    22,    -1,    -1,
+     106,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    33,    34,
+     116,   117,    -1,    -1,   120,   121,    41,    42,    -1,    -1,
+      -1,    -1,    -1,   129,    49,    50,    -1,    52,    -1,   135,
+      -1,    -1,    -1,    58,   140,    -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,     5,    -1,     7,    89,    -1,    -1,    92,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    21,    22,    -1,
+      -1,   106,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    33,
+      34,   116,   117,    -1,    -1,   120,   121,    -1,    42,    -1,
+      -1,    -1,    -1,    -1,   129,    49,    50,    -1,    52,    -1,
+     135,    -1,    -1,    -1,    58,   140,    -1,    33,    34,    -1,
+      -1,    -1,    -1,    67,    -1,    -1,    42,    -1,    -1,    -1,
+      -1,    -1,    -1,    49,    50,    -1,    52,    81,    82,    -1,
+      -1,    -1,    58,    -1,    -1,    89,    -1,    -1,    92,    -1,
+      -1,    67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,   106,    -1,    -1,    81,    82,    -1,    -1,    -1,
+      -1,    -1,   116,    89,    -1,    -1,   120,   121,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,   129,    -1,    -1,    -1,    -1,
+     106,   135,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+     116,    -1,    -1,    -1,   120,   121,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,   129,    -1,    -1,    -1,    -1,    -1,   135,
+      10,    11,    12,    13,    14,    15,    16,    17,    18,    19,
+      -1,    21,    22,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    79,    80,    -1,    -1,    -1,    -1,    -1,    -1,    87,
-      -1,    -1,    90,    -1,    10,    11,    12,    13,    14,    15,
-      16,    17,    18,    19,   102,    21,    22,    -1,     4,     5,
-       6,     7,   110,   111,    -1,    -1,   114,   115,    -1,    -1,
-      -1,    -1,    -1,    -1,   122,    21,    22,    23,    -1,   127,
-      -1,    -1,    -1,    -1,   132,    -1,    -1,    33,    34,    -1,
-      -1,    -1,    -1,    -1,    -1,    41,    42,    -1,    -1,    -1,
-      -1,    -1,    48,    49,    -1,    51,    -1,    -1,    -1,    75,
-      -1,    57,    -1,     4,     5,     6,     7,    63,    -1,    -1,
-      66,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      21,    22,    -1,    79,    80,    -1,    -1,    -1,    -1,    -1,
-      -1,    87,    33,    34,    90,    -1,    -1,    -1,    -1,    -1,
-      41,    42,    -1,    -1,    -1,    -1,   102,    48,    49,    -1,
-      51,    -1,    -1,    -1,   110,   111,    57,    -1,   114,   115,
-      -1,    -1,    63,    -1,    -1,    66,   122,    -1,    -1,    -1,
-      -1,   127,    -1,    -1,    -1,    -1,   132,    -1,    79,    80,
-      -1,    -1,    -1,    -1,    -1,    -1,    87,    -1,    -1,    90,
-       4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,   102,    -1,    -1,    -1,    -1,    -1,    21,    22,   110,
-     111,    -1,    -1,   114,   115,    -1,    -1,    -1,    -1,    33,
-      34,   122,   123,    -1,    -1,    -1,   127,    41,    42,    -1,
-      -1,   132,    -1,    -1,    48,    49,    -1,    51,    -1,    -1,
-      -1,    -1,    -1,    57,    -1,     4,     5,     6,     7,    63,
-      -1,    -1,    66,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    21,    22,    -1,    79,    80,    -1,    -1,    -1,
-      -1,    -1,    -1,    87,    33,    34,    90,    -1,    -1,    -1,
-      -1,    -1,    41,    42,    -1,    -1,    -1,    -1,   102,    48,
-      49,    -1,    51,   107,    -1,    -1,   110,   111,    57,    -1,
-     114,   115,    -1,    62,    63,    -1,    -1,    66,   122,    -1,
-      -1,    -1,    -1,   127,    -1,    -1,    -1,    -1,   132,    -1,
-      79,    80,    -1,    -1,    -1,    -1,    -1,    -1,    87,    -1,
-      -1,    90,     4,     5,     6,     7,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,   102,    -1,    -1,    -1,    -1,    -1,    21,
-      22,   110,   111,    -1,    -1,   114,   115,    -1,    -1,    -1,
-      -1,    33,    34,   122,    -1,    -1,    -1,    -1,   127,    41,
-      42,    -1,    -1,   132,    -1,    -1,    48,    49,    -1,    51,
-      -1,    -1,    -1,    -1,    -1,    57,    -1,     4,     5,     6,
-       7,    63,    -1,    -1,    66,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    21,    22,    -1,    79,    80,    -1,
-      -1,    -1,    -1,    -1,    -1,    87,    33,    34,    90,    -1,
-      -1,    -1,    -1,    -1,    41,    42,    -1,    -1,    -1,    -1,
-     102,    48,    49,    -1,    51,    -1,    -1,    -1,   110,   111,
-      57,    -1,   114,   115,    -1,    -1,    63,    -1,    -1,    66,
-     122,    -1,    -1,    -1,    -1,   127,    -1,    -1,    -1,    -1,
-     132,    -1,    79,    80,    -1,    -1,    -1,    -1,    -1,    -1,
-      87,    -1,    -1,    90,     5,    -1,     7,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,   102,    -1,    -1,    -1,    -1,
-      21,    22,    -1,   110,   111,    -1,    -1,   114,   115,    -1,
-      -1,    -1,    33,    34,    -1,   122,    -1,    -1,    -1,    -1,
-     127,    42,    -1,    -1,    -1,   132,    -1,    48,    49,    -1,
-      51,    33,    34,    -1,    -1,    -1,    57,    -1,    -1,    -1,
-      42,    -1,    -1,    -1,    -1,    66,    48,    49,    -1,    51,
-      -1,    -1,    -1,    -1,    -1,    57,    -1,    -1,    79,    80,
-      -1,    -1,    -1,    -1,    66,    -1,    87,    -1,    -1,    90,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    79,    80,    -1,
-      -1,   102,    -1,    -1,    -1,    87,    -1,    -1,    -1,   110,
-      -1,    -1,    -1,   114,   115,    -1,    -1,    -1,    -1,    -1,
-     102,   122,    -1,    -1,    -1,    -1,   127,    -1,   110,    -1,
-      -1,    -1,   114,   115,    -1,    -1,    -1,    -1,    -1,    -1,
-     122,    -1,    -1,    -1,    -1,   127
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    77
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
      symbol of state STATE-NUM.  */
 static const yytype_uint8 yystos[] =
 {
-       0,     1,     3,    30,    46,    47,    53,    58,    78,   100,
-     107,   119,   125,   128,   137,   138,   139,   140,   141,   142,
-     143,   164,   165,   168,   169,   172,   173,   176,   230,   231,
-     112,     4,     6,   228,    76,   112,    69,   112,    81,    29,
-      56,   177,   228,   174,   175,   188,   228,     0,   128,   130,
-      78,   176,   128,   131,     3,   228,    69,   228,   228,   228,
-     228,   228,     5,     7,    21,    22,    23,    33,    34,    41,
-      42,    48,    49,    51,    57,    63,    66,    79,    80,    87,
-      90,   102,   110,   111,   114,   115,   122,   127,   132,   146,
-     178,   179,   180,   182,   204,   205,   206,   207,   208,   209,
-     210,   211,   218,   221,   224,   228,   108,   134,    31,   132,
-      81,    28,    58,     5,    93,   132,   124,   198,   199,   121,
-     132,   176,     7,     7,   123,   204,   214,   215,   132,    80,
-     132,     5,   132,   132,    80,   176,   204,     5,    69,   181,
-     134,    21,    22,    31,   225,   228,    23,    24,   135,   226,
-      22,   207,    27,   132,   170,   171,   228,   175,   132,   182,
-     227,   228,   228,    44,    45,    44,    45,   125,   166,   228,
-     144,   145,   228,    10,    62,   132,   200,   201,   202,   203,
-     204,   221,   132,   227,   200,   123,   212,   213,    59,   215,
-     216,     7,    50,    74,    88,    89,   106,   126,   219,   219,
-     204,     7,   133,   133,   132,   182,   185,   186,   189,   208,
-     228,   198,   180,   228,   205,   206,   228,    23,    56,   133,
-     204,   217,   134,   198,    11,   133,   134,   176,   144,    43,
-      68,    99,   118,   150,   228,   228,   132,   132,   153,   133,
-     134,    80,   146,   203,   182,   200,   204,     8,     9,    10,
-      11,    12,    13,    14,    15,    16,    17,    18,    19,    75,
-     223,    20,   218,   220,   133,   113,   204,   213,   216,   204,
-      60,   133,    69,    69,   133,   185,    31,   187,   188,    70,
-      77,    85,   104,   184,   134,   187,    37,   117,   183,    71,
-     190,   133,   217,   133,   134,   171,   204,   228,   132,    83,
-      83,   132,    54,    61,   167,   221,   222,   228,   120,   150,
-     151,   152,   144,    10,    43,    52,    90,    99,   103,   118,
-     147,   148,   149,   133,   201,   202,    17,    18,    19,    75,
-     204,   132,   182,   204,    10,    90,   133,   134,   121,   204,
-     113,    60,   204,     7,   133,   188,    95,    95,    95,    82,
-     185,     7,     7,   187,    40,    73,   191,   133,   204,   200,
-     132,   132,   227,     5,    64,    92,    93,   116,   229,   133,
-     134,   133,   134,    35,    38,    39,   109,   162,   134,   125,
-     154,    90,   132,   218,    83,   228,   147,   204,   132,   182,
-       9,   217,    90,   218,   132,   204,   133,    67,   133,   186,
-      98,    98,   217,   200,    94,   192,   133,   227,   227,   133,
-      54,    61,   221,   132,   163,   150,    36,    96,   155,   200,
-     132,     9,   217,   204,   133,   220,     7,    93,    40,    86,
-     193,   133,   133,     5,   229,   157,   158,   159,   160,   161,
-     228,   132,    40,   133,   228,   204,   133,   133,   133,   200,
-     194,   195,   204,     7,   103,   133,   134,     7,    29,   132,
-     228,   157,    72,   101,   156,   133,   134,    32,    55,   196,
-     228,   158,   227,   133,   132,   195,    91,   197,   132,   133,
-     227,    65,    84,   227,   133,   133,    97,     7
+       0,     1,     3,    30,    46,    47,    54,    59,    80,   104,
+     113,   126,   133,   136,   145,   146,   147,   148,   149,   150,
+     151,   172,   173,   176,   177,   180,   181,   184,   248,   249,
+     118,     4,     6,   246,    78,   118,    71,   118,    83,    29,
+      57,   185,   246,   182,   183,   196,   246,     0,   136,   138,
+      80,   184,   136,   139,     3,   246,    71,   246,   246,   246,
+     246,   246,     5,     7,    21,    22,    23,    33,    34,    41,
+      42,    49,    50,    52,    58,    64,    67,    81,    82,    89,
+      92,   106,   116,   117,   120,   121,   129,   135,   140,   154,
+     186,   187,   188,   190,   222,   223,   224,   225,   226,   227,
+     228,   229,   236,   239,   242,   246,   114,   142,    31,   140,
+      83,    28,    59,     5,    95,   140,   131,   216,   217,   128,
+     140,   184,     7,     7,   130,   222,   232,   233,   140,    82,
+     140,     5,   140,   140,    82,   184,   222,     5,    71,   189,
+     142,    21,    22,    31,   243,   246,    23,    24,   143,   244,
+      98,    22,   225,    27,   140,   178,   179,   246,   183,   140,
+     190,   245,   246,   246,    44,    45,    44,    45,   133,   174,
+     246,   152,   153,   246,    10,    63,   140,   218,   219,   220,
+     221,   222,   239,   140,   245,   218,   130,   230,   231,    60,
+     233,   234,     7,    51,    76,    90,    91,   112,   134,   237,
+     237,   222,     7,   141,   141,   140,   190,   193,   194,   197,
+     226,   246,   216,   188,   246,   223,   224,   140,   246,   246,
+      23,    57,   141,   222,   235,   142,   216,    11,   141,   142,
+     184,   152,    43,    70,   103,   125,   158,   246,   246,   140,
+     140,   161,   141,   142,    82,   154,   221,   190,   218,   222,
+       8,     9,    10,    11,    12,    13,    14,    15,    16,    17,
+      18,    19,    77,   241,    20,   236,   238,   141,   119,   222,
+     231,   234,   222,    61,   141,    71,    71,   141,   193,    31,
+     195,   196,    72,    79,    87,   108,   192,   142,   195,    37,
+     123,   191,    73,   198,    99,   205,   206,   141,   235,   141,
+     142,   179,   222,   246,   140,    85,    85,   140,    55,    62,
+     175,   239,   240,   246,   127,   158,   159,   160,   152,    10,
+      43,    53,    92,   103,   107,   125,   155,   156,   157,   141,
+     219,   220,    17,    18,    19,    77,   222,   140,   190,   222,
+      10,    92,   141,   142,   128,   222,   119,    61,   222,     7,
+     141,   196,    97,    97,    97,    84,   193,     7,     7,   195,
+      40,    75,   199,    40,   141,    96,   207,   141,   222,   218,
+     140,   140,   245,     5,    65,    94,    95,   122,   247,   141,
+     142,   141,   142,    35,    38,    39,   115,   170,   142,   133,
+     162,    92,   140,   236,    85,   246,   155,   222,   140,   190,
+       9,   235,    92,   236,   140,   222,   141,    69,   141,   194,
+     101,   101,   235,   218,    96,   200,   235,    40,   105,   111,
+     208,   209,   141,   245,   245,   141,    55,    62,   239,   140,
+     171,   158,    36,    99,   163,   218,   140,     9,   235,   222,
+     141,   238,     7,    95,    40,    88,   201,   212,   213,   222,
+      19,   141,   141,     5,   247,   165,   166,   167,   168,   169,
+     246,   140,    40,   141,   246,   222,   141,   141,   141,   218,
+     212,     7,   132,   202,   203,   204,   142,    32,    56,   214,
+       7,    48,   124,   210,   107,   141,   142,     7,    29,   140,
+     246,   165,    74,   105,   164,   141,   246,   204,   213,    93,
+     215,   102,   109,   102,     9,   246,   166,   245,   141,   140,
+      31,    66,    86,     7,    48,   124,   211,   140,   141,   245,
+     140,    68,   109,    68,   245,   141,   205,   141,   100,   141,
+       7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
 static const yytype_uint8 yyr1[] =
 {
-       0,   136,   137,   137,   137,   137,   137,   137,   138,   138,
-     138,   138,   138,   138,   138,   138,   138,   138,   139,   140,
-     140,   140,   140,   141,   142,   143,   144,   145,   145,   146,
-     146,   146,   146,   146,   146,   146,   146,   146,   146,   146,
-     146,   146,   146,   146,   146,   146,   146,   147,   147,   147,
-     147,   147,   147,   147,   148,   148,   149,   149,   150,   150,
-     150,   150,   151,   151,   152,   152,   153,   153,   154,   154,
-     155,   155,   156,   156,   157,   157,   158,   158,   158,   159,
-     159,   160,   161,   162,   162,   162,   162,   163,   163,   164,
-     164,   164,   164,   165,   166,   166,   167,   167,   167,   167,
-     168,   169,   170,   170,   171,   172,   172,   173,   174,   174,
-     175,   176,   177,   177,   177,   178,   178,   179,   179,   180,
-     180,   180,   181,   182,   183,   183,   183,   184,   184,   184,
-     184,   184,   184,   184,   184,   185,   185,   186,   186,   186,
-     186,   186,   186,   187,   187,   188,   188,   189,   189,   190,
-     190,   191,   191,   192,   192,   193,   193,   194,   194,   195,
-     196,   196,   196,   197,   197,   197,   198,   198,   199,   200,
-     200,   201,   201,   202,   202,   203,   203,   203,   203,   203,
-     203,   203,   203,   203,   203,   203,   204,   204,   205,   205,
-     206,   206,   207,   207,   207,   207,   207,   207,   207,   207,
-     208,   208,   208,   208,   209,   210,   210,   211,   211,   212,
-     212,   213,   214,   214,   215,   216,   216,   217,   217,   218,
-     218,   218,   218,   218,   218,   218,   218,   219,   219,   219,
-     219,   219,   219,   220,   220,   221,   221,   222,   222,   223,
-     223,   223,   223,   223,   223,   223,   223,   223,   223,   224,
-     225,   225,   226,   226,   226,   227,   227,   228,   228,   229,
-     229,   229,   229,   230,   231,   231
+       0,   144,   145,   145,   145,   145,   145,   145,   146,   146,
+     146,   146,   146,   146,   146,   146,   146,   146,   147,   148,
+     148,   148,   148,   149,   150,   151,   152,   153,   153,   154,
+     154,   154,   154,   154,   154,   154,   154,   154,   154,   154,
+     154,   154,   154,   154,   154,   154,   154,   155,   155,   155,
+     155,   155,   155,   155,   156,   156,   157,   157,   158,   158,
+     158,   158,   159,   159,   160,   160,   161,   161,   162,   162,
+     163,   163,   164,   164,   165,   165,   166,   166,   166,   167,
+     167,   168,   169,   170,   170,   170,   170,   171,   171,   172,
+     172,   172,   172,   173,   174,   174,   175,   175,   175,   175,
+     176,   177,   178,   178,   179,   180,   180,   181,   182,   182,
+     183,   184,   185,   185,   185,   186,   186,   187,   187,   188,
+     188,   188,   189,   190,   191,   191,   191,   192,   192,   192,
+     192,   192,   192,   192,   192,   193,   193,   194,   194,   194,
+     194,   194,   194,   195,   195,   196,   196,   197,   197,   198,
+     198,   199,   199,   200,   200,   201,   201,   202,   202,   203,
+     203,   204,   205,   206,   206,   207,   207,   208,   208,   209,
+     209,   210,   210,   210,   211,   211,   211,   212,   212,   213,
+     214,   214,   214,   215,   215,   215,   216,   216,   217,   218,
+     218,   219,   219,   220,   220,   221,   221,   221,   221,   221,
+     221,   221,   221,   221,   221,   221,   222,   222,   223,   223,
+     224,   224,   225,   225,   225,   225,   225,   225,   225,   225,
+     225,   225,   226,   226,   226,   226,   227,   228,   228,   229,
+     229,   230,   230,   231,   232,   232,   233,   234,   234,   235,
+     235,   236,   236,   236,   236,   236,   236,   236,   236,   237,
+     237,   237,   237,   237,   237,   238,   238,   239,   239,   240,
+     240,   241,   241,   241,   241,   241,   241,   241,   241,   241,
+     241,   242,   243,   243,   244,   244,   244,   245,   245,   246,
+     246,   247,   247,   247,   247,   248,   249,   249
 };
 
   /* YYR2[YYN] -- Number of symbols on the right hand side of rule YYN.  */
@@ -1383,22 +1451,24 @@ static const yytype_uint8 yyr2[] =
        2,     4,     2,     1,     1,     1,     1,     0,     3,    10,
        7,     4,     5,     5,     0,     4,     2,     2,     4,     4,
        5,     4,     3,     1,     3,     1,     2,     2,     1,     3,
-       3,     9,     0,     1,     1,     1,     1,     1,     3,     3,
+       3,    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,     1,     3,     3,
+       3,     0,     2,     0,     3,     0,     2,     0,     1,     1,
+       2,     6,     3,    

<TRUNCATED>


[11/50] [abbrv] incubator-quickstep git commit: Fixed a potential segfault with CompressedBlockBuilder.

Posted by zu...@apache.org.
Fixed a potential segfault with CompressedBlockBuilder.


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

Branch: refs/heads/tmp-relation-col-store
Commit: eebb4644f195fd82b28e77aafcf60344c33d6197
Parents: 096abe2
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Jun 9 00:43:16 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Jun 9 10:52:12 2016 -0700

----------------------------------------------------------------------
 storage/CompressedBlockBuilder.cpp | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/eebb4644/storage/CompressedBlockBuilder.cpp
----------------------------------------------------------------------
diff --git a/storage/CompressedBlockBuilder.cpp b/storage/CompressedBlockBuilder.cpp
index 4a181eb..1ca0c07 100644
--- a/storage/CompressedBlockBuilder.cpp
+++ b/storage/CompressedBlockBuilder.cpp
@@ -321,6 +321,9 @@ void CompressedBlockBuilder::buildCompressedColumnStoreTupleStorageSubBlock(void
 bool CompressedBlockBuilder::addTupleInternal(Tuple *candidate_tuple) {
   DEBUG_ASSERT(candidate_tuple->size() == relation_.size());
 
+  // Ensure that the tuple is the owner of its values.
+  candidate_tuple->ensureLiteral();
+
   // Modify dictionaries and maximum integers to reflect the new tuple's
   // values. Keep track of what has changed in case a rollback is needed.
   vector<CompressionDictionaryBuilder*> modified_dictionaries;


[14/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-10: Serialized WorkOrders as proto.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.cpp b/relational_operators/SortMergeRunOperator.cpp
index 6bf5719..e398d62 100644
--- a/relational_operators/SortMergeRunOperator.cpp
+++ b/relational_operators/SortMergeRunOperator.cpp
@@ -23,9 +23,11 @@
 #include <vector>
 
 #include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
 #include "relational_operators/SortMergeRunOperator.pb.h"
 #include "relational_operators/SortMergeRunOperatorHelpers.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "threading/ThreadIDBasedMap.hpp"
 
 #include "glog/logging.h"
@@ -69,6 +71,72 @@ bool SortMergeRunOperator::getAllWorkOrders(
   return generateWorkOrders(container, query_context, storage_manager, scheduler_client_id, bus);
 }
 
+bool SortMergeRunOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (input_relation_is_stored_) {
+    // Input blocks (or runs) are from base relation. Only possible when base
+    // relation is stored sorted.
+    if (!started_) {
+      // Initialize merge tree completely, since all input runs are known.
+      merge_tree_.initializeTree(input_relation_block_ids_.size());
+      started_ = true;
+      initializeInputRuns();
+    }
+  } else {
+    // Input blocks (or runs) are pipelined from the sorted run generation
+    // operator.
+    if (!started_ && !input_stream_done_) {
+      // Initialize merge tree for first pipeline mode.
+      merge_tree_.initializeForPipeline();
+      started_ = true;
+      initializeInputRuns();
+    }
+  }
+
+  // Get merge jobs from merge tree.
+  std::vector<MergeTree::MergeJob> jobs;
+  const bool done_generating = merge_tree_.getMergeJobs(&jobs);
+
+  for (std::vector<MergeTree::MergeJob>::size_type job_id = 0;
+       job_id < jobs.size();
+       ++job_id) {
+    // Add work order for each merge job.
+    container->addWorkOrderProto(createWorkOrderProto(&jobs[job_id]), op_index_);
+  }
+
+  return done_generating;
+}
+
+serialization::WorkOrder* SortMergeRunOperator::createWorkOrderProto(
+    merge_run_operator::MergeTree::MergeJob *job) {
+  DCHECK(job != nullptr);
+  DCHECK(!job->runs.empty());
+
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::SORT_MERGE_RUN);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::operator_index, op_index_);
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::sort_config_index, sort_config_index_);
+
+  for (const merge_run_operator::Run &run : job->runs) {
+    serialization::Run *run_proto = proto->AddExtension(serialization::SortMergeRunWorkOrder::runs);
+    for (const block_id block : run) {
+      run_proto->add_blocks(block);
+    }
+  }
+
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::top_k, top_k_);
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::merge_level, job->level);
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::relation_id,
+                      job->level > 0 ? run_relation_.getID()
+                                     : input_relation_.getID());
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::insert_destination_index,
+                      job->is_final_level ? output_destination_index_
+                                          : run_block_destination_index_);
+
+  return proto;
+}
+
 WorkOrder *SortMergeRunOperator::createWorkOrder(
     merge_run_operator::MergeTree::MergeJob *job,
     QueryContext *query_context,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index cfff8b9..177836f 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -44,8 +44,11 @@ namespace quickstep {
 class CatalogRelationSchema;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /**
  * @defgroup SortMergeRun Merging Sorted Runs
  * @ingroup Sort
@@ -132,6 +135,8 @@ class SortMergeRunOperator : public RelationalOperator {
                         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) override {
     input_relation_block_ids_.push_back(input_block_id);
@@ -182,6 +187,13 @@ class SortMergeRunOperator : public RelationalOperator {
                              const tmb::client_id scheduler_client_id,
                              tmb::MessageBus *bus);
 
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param job The merge job.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(merge_run_operator::MergeTree::MergeJob *job);
+
   const CatalogRelation &input_relation_;
 
   const CatalogRelation &output_relation_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SortRunGenerationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.cpp b/relational_operators/SortRunGenerationOperator.cpp
index 37b8fb8..d7362db 100644
--- a/relational_operators/SortRunGenerationOperator.cpp
+++ b/relational_operators/SortRunGenerationOperator.cpp
@@ -21,7 +21,9 @@
 
 #include "catalog/CatalogRelation.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageManager.hpp"
@@ -80,6 +82,43 @@ bool SortRunGenerationOperator::getAllWorkOrders(
   }
 }
 
+bool SortRunGenerationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (input_relation_is_stored_) {
+    // Input blocks are from a base relation.
+    if (!started_) {
+      for (const block_id input_block_id : input_relation_block_ids_) {
+        container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    // Input blocks are pipelined.
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* SortRunGenerationOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::SORT_RUN_GENERATION);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::SortRunGenerationWorkOrder::sort_config_index, sort_config_index_);
+  proto->SetExtension(serialization::SortRunGenerationWorkOrder::relation_id, input_relation_.getID());
+  proto->SetExtension(serialization::SortRunGenerationWorkOrder::insert_destination_index,
+                      output_destination_index_);
+  proto->SetExtension(serialization::SortRunGenerationWorkOrder::block_id, block);
+
+  return proto;
+}
+
+
 void SortRunGenerationWorkOrder::execute() {
   BlockReference block(
       storage_manager_->getBlock(input_block_id_, input_relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index f96e6a6..96a3ce1 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -40,8 +40,11 @@ namespace quickstep {
 class CatalogRelationSchema;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /**
  * \defgroup Sort Sorting
  * \ingroup RelationalOperators
@@ -112,6 +115,8 @@ class SortRunGenerationOperator : public RelationalOperator {
                         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) override {
     DCHECK(input_relation_id == input_relation_.getID());
     input_relation_block_ids_.push_back(input_block_id);
@@ -133,6 +138,13 @@ class SortRunGenerationOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &input_relation_;
 
   const CatalogRelation &output_relation_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TableGeneratorOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.cpp b/relational_operators/TableGeneratorOperator.cpp
index a3f9340..d5a08ec 100644
--- a/relational_operators/TableGeneratorOperator.cpp
+++ b/relational_operators/TableGeneratorOperator.cpp
@@ -1,6 +1,6 @@
 /**
  *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ *     University of Wisconsin\u2014Madison.
  *   Copyright 2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
@@ -20,7 +20,9 @@
 
 #include "expressions/table_generator/GeneratorFunctionHandle.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 
@@ -53,6 +55,22 @@ bool TableGeneratorOperator::getAllWorkOrders(
   return started_;
 }
 
+bool TableGeneratorOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (!started_) {
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::TABLE_GENERATOR);
+    proto->set_query_id(query_id_);
+
+    proto->SetExtension(serialization::TableGeneratorWorkOrder::generator_function_index, generator_function_index_);
+    proto->SetExtension(serialization::TableGeneratorWorkOrder::insert_destination_index, output_destination_index_);
+
+    container->addWorkOrderProto(proto, op_index_);
+    started_ = true;
+  }
+  return true;
+}
+
+
 void TableGeneratorWorkOrder::execute() {
   ColumnVectorsValueAccessor temp_result;
   function_handle_.populateColumns(&temp_result);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index 6a6af4b..1b791a6 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -1,6 +1,6 @@
 /**
  *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ *     University of Wisconsin\u2014Madison.
  *   Copyright 2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
@@ -40,6 +40,7 @@ namespace quickstep {
 class GeneratorFunctionHandle;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -81,6 +82,8 @@ class TableGeneratorOperator : public RelationalOperator {
                         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) override {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index d2fd0cd..49c9150 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -22,6 +22,7 @@
 #include <algorithm>
 #include <cctype>
 #include <cstddef>
+#include <cstdint>
 #include <cstdio>
 #include <cstdlib>
 #include <memory>
@@ -31,21 +32,46 @@
 
 #include "catalog/CatalogAttribute.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/Tuple.hpp"
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "types/containers/Tuple.hpp"
 #include "utility/Glob.hpp"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
 
 #include "tmb/id_typedefs.h"
 
+using std::size_t;
+using std::string;
+
 namespace quickstep {
 
+// Text segment size set to 256KB.
+DEFINE_uint64(textscan_text_segment_size, 0x40000,
+              "Size of text segment in bytes the input text files "
+              "are split into in the TextScanOperator.");
+
+// Check if the segment size is positive.
+static bool ValidateTextScanTextSegmentSize(const char *flagname,
+                                            std::uint64_t text_segment_size) {
+  if (text_segment_size == 0) {
+    LOG(ERROR) << "--" << flagname << " must be greater than 0";
+    return false;
+  }
+
+  return true;
+}
+
+static const volatile bool text_scan_text_segment_size_dummy = gflags::RegisterFlagValidator(
+    &FLAGS_textscan_text_segment_size, &ValidateTextScanTextSegmentSize);
+
 bool TextScanOperator::getAllWorkOrders(
     WorkOrdersContainer *container,
     QueryContext *query_context,
@@ -56,16 +82,12 @@ bool TextScanOperator::getAllWorkOrders(
 
   const std::vector<std::string> files = utility::file::GlobExpand(file_pattern_);
 
-  if (files.size() == 0) {
-    LOG(FATAL) << "No files matched '" << file_pattern_ << "'. Exiting.";
-  }
+  CHECK_NE(files.size(), 0u)
+      << "No files matched '" << file_pattern_ << "'. Exiting.";
 
   InsertDestination *output_destination =
       query_context->getInsertDestination(output_destination_index_);
 
-  // Text segment size set to 256KB.
-  constexpr std::size_t kTextSegmentSize = 0x40000u;
-
   if (blocking_dependencies_met_ && !work_generated_) {
     for (const std::string &file : files) {
       // Use standard C libary to retrieve the file size.
@@ -75,18 +97,32 @@ bool TextScanOperator::getAllWorkOrders(
       std::fclose(fp);
 
       std::size_t text_offset = 0;
-      while (text_offset < file_size) {
+      for (size_t num_full_segments = file_size / FLAGS_textscan_text_segment_size;
+           num_full_segments > 0;
+           --num_full_segments, text_offset += FLAGS_textscan_text_segment_size) {
+        container->addNormalWorkOrder(
+            new TextScanWorkOrder(query_id_,
+                                  file,
+                                  text_offset,
+                                  FLAGS_textscan_text_segment_size,
+                                  field_terminator_,
+                                  process_escape_sequences_,
+                                  output_destination),
+            op_index_);
+      }
+
+      // Deal with the residual partial segment whose size is less than
+      // 'FLAGS_textscan_text_segment_size'.
+      if (text_offset < file_size) {
         container->addNormalWorkOrder(
             new TextScanWorkOrder(query_id_,
                                   file,
                                   text_offset,
-                                  std::min(kTextSegmentSize, file_size - text_offset),
+                                  file_size - text_offset,
                                   field_terminator_,
                                   process_escape_sequences_,
-                                  output_destination,
-                                  storage_manager),
+                                  output_destination),
             op_index_);
-        text_offset += kTextSegmentSize;
       }
     }
     work_generated_ = true;
@@ -94,24 +130,53 @@ bool TextScanOperator::getAllWorkOrders(
   return work_generated_;
 }
 
-TextScanWorkOrder::TextScanWorkOrder(const std::size_t query_id,
-                                     const std::string &filename,
-                                     const std::size_t text_offset,
-                                     const std::size_t text_segment_size,
-                                     const char field_terminator,
-                                     const bool process_escape_sequences,
-                                     InsertDestination *output_destination,
-                                     StorageManager *storage_manager)
-    : WorkOrder(query_id),
-      filename_(filename),
-      text_offset_(text_offset),
-      text_segment_size_(text_segment_size),
-      field_terminator_(field_terminator),
-      process_escape_sequences_(process_escape_sequences),
-      output_destination_(output_destination),
-      storage_manager_(storage_manager) {
-  DCHECK(output_destination_ != nullptr);
-  DCHECK(storage_manager_ != nullptr);
+bool TextScanOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  const std::vector<std::string> files = utility::file::GlobExpand(file_pattern_);
+  if (blocking_dependencies_met_ && !work_generated_) {
+    for (const string &file : files) {
+      // Use standard C libary to retrieve the file size.
+      FILE *fp = std::fopen(file.c_str(), "rb");
+      std::fseek(fp, 0, SEEK_END);
+      const std::size_t file_size = std::ftell(fp);
+      std::fclose(fp);
+
+      size_t text_offset = 0;
+      for (size_t num_full_segments = file_size / FLAGS_textscan_text_segment_size;
+           num_full_segments > 0;
+           --num_full_segments, text_offset += FLAGS_textscan_text_segment_size) {
+        container->addWorkOrderProto(createWorkOrderProto(file, text_offset, FLAGS_textscan_text_segment_size),
+                                     op_index_);
+      }
+
+      // Deal with the residual partial segment whose size is less than
+      // 'FLAGS_textscan_text_segment_size'.
+      if (text_offset < file_size) {
+        container->addWorkOrderProto(createWorkOrderProto(file, text_offset, file_size - text_offset),
+                                     op_index_);
+      }
+    }
+    work_generated_ = true;
+  }
+  return work_generated_;
+}
+
+serialization::WorkOrder* TextScanOperator::createWorkOrderProto(const string &filename,
+                                                                 const size_t text_offset,
+                                                                 const size_t text_segment_size) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::TEXT_SCAN);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::TextScanWorkOrder::filename, filename);
+  proto->SetExtension(serialization::TextScanWorkOrder::text_offset, text_offset);
+  proto->SetExtension(serialization::TextScanWorkOrder::text_segment_size, text_segment_size);
+  proto->SetExtension(serialization::TextScanWorkOrder::field_terminator, field_terminator_);
+  proto->SetExtension(serialization::TextScanWorkOrder::process_escape_sequences,
+                      process_escape_sequences_);
+  proto->SetExtension(serialization::TextScanWorkOrder::insert_destination_index,
+                      output_destination_index_);
+
+  return proto;
 }
 
 void TextScanWorkOrder::execute() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index d73e7dd..1a62ded 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -33,6 +33,8 @@
 #include "types/containers/Tuple.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 #include "tmb/id_typedefs.h"
 
 namespace tmb { class MessageBus; }
@@ -42,8 +44,11 @@ namespace quickstep {
 class CatalogRelationSchema;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -135,6 +140,8 @@ class TextScanOperator : public RelationalOperator {
                         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_;
   }
@@ -144,6 +151,10 @@ class TextScanOperator : public RelationalOperator {
   }
 
  private:
+  serialization::WorkOrder* createWorkOrderProto(const std::string &filename,
+                                                 const std::size_t text_offset,
+                                                 const std::size_t text_segment_size);
+
   const std::string file_pattern_;
   const char field_terminator_;
   const bool process_escape_sequences_;
@@ -173,7 +184,6 @@ class TextScanWorkOrder : public WorkOrder {
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
    * @param output_destination The InsertDestination to insert tuples.
-   * @param storage_manager The StorageManager to use.
    **/
   TextScanWorkOrder(
       const std::size_t query_id,
@@ -182,8 +192,14 @@ class TextScanWorkOrder : public WorkOrder {
       const std::size_t text_segment_size,
       const char field_terminator,
       const bool process_escape_sequences,
-      InsertDestination *output_destination,
-      StorageManager *storage_manager);
+      InsertDestination *output_destination)
+      : WorkOrder(query_id),
+        filename_(filename),
+        text_offset_(text_offset),
+        text_segment_size_(text_segment_size),
+        field_terminator_(field_terminator),
+        process_escape_sequences_(process_escape_sequences),
+        output_destination_(DCHECK_NOTNULL(output_destination)) {}
 
   ~TextScanWorkOrder() override {}
 
@@ -233,7 +249,6 @@ class TextScanWorkOrder : public WorkOrder {
   Tuple parseRow(const char **row_ptr,
                  const CatalogRelationSchema &relation) const;
 
-
   /**
    * @brief Parse up to three octal digits (0-7) starting at \p *literal_ptr as
    *        a char literal. \p *literal_ptr will be modified to the last position
@@ -297,7 +312,6 @@ class TextScanWorkOrder : public WorkOrder {
   const bool process_escape_sequences_;
 
   InsertDestination *output_destination_;
-  StorageManager *storage_manager_;
 
   DISALLOW_COPY_AND_ASSIGN(TextScanWorkOrder);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TextScanOperator.proto
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.proto b/relational_operators/TextScanOperator.proto
deleted file mode 100644
index 8ead3f3..0000000
--- a/relational_operators/TextScanOperator.proto
+++ /dev/null
@@ -1,22 +0,0 @@
-//   Copyright 2015 Pivotal Software, Inc.
-//
-//   Licensed under the Apache License, Version 2.0 (the "License");
-//   you may not use this file except in compliance with the License.
-//   You may obtain a copy of the License at
-//
-//       http://www.apache.org/licenses/LICENSE-2.0
-//
-//   Unless required by applicable law or agreed to in writing, software
-//   distributed under the License is distributed on an "AS IS" BASIS,
-//   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-//   See the License for the specific language governing permissions and
-//   limitations under the License.
-
-syntax = "proto2";
-
-package quickstep.serialization;
-
-message TextBlob {
-  required fixed64 blob_id = 1;
-  required uint64 size = 2;
-}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index 2130563..1b2979e 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -26,7 +26,9 @@
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryExecutionUtil.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -72,6 +74,27 @@ bool UpdateOperator::getAllWorkOrders(
   return started_;
 }
 
+bool UpdateOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !started_) {
+    for (const block_id input_block_id : input_blocks_) {
+      serialization::WorkOrder *proto = new serialization::WorkOrder;
+      proto->set_work_order_type(serialization::UPDATE);
+      proto->set_query_id(query_id_);
+
+      proto->SetExtension(serialization::UpdateWorkOrder::operator_index, op_index_);
+      proto->SetExtension(serialization::UpdateWorkOrder::relation_id, relation_.getID());
+      proto->SetExtension(serialization::UpdateWorkOrder::insert_destination_index, relocation_destination_index_);
+      proto->SetExtension(serialization::UpdateWorkOrder::predicate_index, predicate_index_);
+      proto->SetExtension(serialization::UpdateWorkOrder::update_group_index, update_group_index_);
+      proto->SetExtension(serialization::UpdateWorkOrder::block_id, input_block_id);
+
+      container->addWorkOrderProto(proto, op_index_);
+    }
+    started_ = true;
+  }
+  return started_;
+}
+
 void UpdateWorkOrder::execute() {
   MutableBlockReference block(
       storage_manager_->getBlockMutable(input_block_id_, relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index cebb9b5..f6c5053 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -45,6 +45,7 @@ class InsertDestination;
 class Predicate;
 class Scalar;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -99,6 +100,8 @@ class UpdateOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return relocation_destination_index_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 60d4c8f..3ed065a 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -232,23 +232,14 @@ message TextScanWorkOrder {
   }
 }
 
-message TextSplitWorkOrder {
-  extend WorkOrder {
-    // All required.
-    optional uint64 operator_index = 320;
-    optional string filename = 321;
-    optional bool process_escape_sequences = 322;
-  }
-}
-
 message UpdateWorkOrder {
   extend WorkOrder {
     // All required.
-    optional uint64 operator_index = 336;
-    optional int32 relation_id = 337;
-    optional int32 insert_destination_index = 338;
-    optional int32 predicate_index = 339;
-    optional uint32 update_group_index = 340;
-    optional fixed64 block_id = 341;
+    optional uint64 operator_index = 320;
+    optional int32 relation_id = 321;
+    optional int32 insert_destination_index = 322;
+    optional int32 predicate_index = 323;
+    optional uint32 update_group_index = 324;
+    optional fixed64 block_id = 325;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index da42b4d..e078b84 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -396,8 +396,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
           proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
           query_context->getInsertDestination(
-              proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
-          storage_manager);
+              proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)));
     }
     case serialization::UPDATE: {
       LOG(INFO) << "Creating UpdateWorkOrder";
@@ -425,6 +424,10 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
 bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
                                     const CatalogDatabaseLite &catalog_database,
                                     const QueryContext &query_context) {
+  if (!proto.IsInitialized()) {
+    return false;
+  }
+
   switch (proto.work_order_type()) {
     case serialization::AGGREGATION: {
       return proto.HasExtension(serialization::AggregationWorkOrder::block_id) &&


[06/50] [abbrv] incubator-quickstep git commit: Reordered Query ID in operators and work orders.

Posted by zu...@apache.org.
Reordered Query ID in operators and work orders.


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

Branch: refs/heads/tmp-relation-col-store
Commit: 1be47dcbd79e6e2ee01f237d0dbaa8a97e562519
Parents: 4054268
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Jun 3 15:40:34 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed Jun 8 14:00:13 2016 -0700

----------------------------------------------------------------------
 query_execution/Foreman.cpp                     |  12 +-
 query_execution/QueryManager.cpp                |  12 +-
 query_execution/tests/Foreman_unittest.cpp      |   3 +-
 query_execution/tests/QueryManager_unittest.cpp |   3 +-
 query_optimizer/ExecutionGenerator.cpp          | 194 ++++++++--------
 .../tests/ExecutionHeuristics_unittest.cpp      |  34 +--
 relational_operators/AggregationOperator.cpp    |   4 +-
 relational_operators/AggregationOperator.hpp    |  14 +-
 relational_operators/BuildHashOperator.cpp      |   6 +-
 relational_operators/BuildHashOperator.hpp      |  20 +-
 relational_operators/CreateIndexOperator.hpp    |   6 +-
 relational_operators/DeleteOperator.cpp         |   8 +-
 relational_operators/DeleteOperator.hpp         |  14 +-
 relational_operators/DestroyHashOperator.cpp    |   2 +-
 relational_operators/DestroyHashOperator.hpp    |  12 +-
 relational_operators/DropTableOperator.hpp      |   6 +-
 .../FinalizeAggregationOperator.hpp             |  11 +-
 relational_operators/HashJoinOperator.cpp       |  31 +--
 relational_operators/HashJoinOperator.hpp       | 229 ++++++++++---------
 relational_operators/InsertOperator.hpp         |  11 +-
 .../NestedLoopsJoinOperator.cpp                 |  45 ++--
 .../NestedLoopsJoinOperator.hpp                 |  52 +++--
 relational_operators/RebuildWorkOrder.hpp       |  15 +-
 relational_operators/RelationalOperator.hpp     |  11 +-
 relational_operators/SampleOperator.cpp         |  52 +++--
 relational_operators/SampleOperator.hpp         |  39 ++--
 relational_operators/SaveBlocksOperator.cpp     |   2 +-
 relational_operators/SaveBlocksOperator.hpp     |   9 +-
 relational_operators/SelectOperator.cpp         |  27 ++-
 relational_operators/SelectOperator.hpp         |  56 ++---
 relational_operators/SortMergeRunOperator.cpp   |   2 +-
 relational_operators/SortMergeRunOperator.hpp   |  27 +--
 .../SortRunGenerationOperator.cpp               |   6 +-
 .../SortRunGenerationOperator.hpp               |  26 ++-
 relational_operators/TableGeneratorOperator.cpp |   2 +-
 relational_operators/TableGeneratorOperator.hpp |  20 +-
 relational_operators/TextScanOperator.cpp       |  21 +-
 relational_operators/TextScanOperator.hpp       |  32 +--
 relational_operators/UpdateOperator.cpp         |  22 +-
 relational_operators/UpdateOperator.hpp         |  37 +--
 relational_operators/WorkOrderFactory.cpp       |  38 +--
 .../tests/AggregationOperator_unittest.cpp      |  23 +-
 .../tests/HashJoinOperator_unittest.cpp         | 181 ++++++++-------
 .../tests/SortMergeRunOperator_unittest.cpp     |  14 +-
 .../SortRunGenerationOperator_unittest.cpp      |  16 +-
 .../tests/TextScanOperator_unittest.cpp         |   7 +-
 46 files changed, 745 insertions(+), 669 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index b358f70..7705819 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -521,12 +521,12 @@ void Foreman::getRebuildWorkOrders(const dag_node_index index, WorkOrdersContain
     // Note: The query ID used below is dummy for now, it will be replaced with
     // the true query ID when QueryManager gets used in Foreman.
     container->addRebuildWorkOrder(
-        new RebuildWorkOrder(move(partially_filled_block_refs[i]),
-                            index,
-                            op.getOutputRelationID(),
-                            foreman_client_id_,
-                            0,
-                            bus_),
+        new RebuildWorkOrder(0,
+                             move(partially_filled_block_refs[i]),
+                             index,
+                             op.getOutputRelationID(),
+                             foreman_client_id_,
+                             bus_),
         index);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/query_execution/QueryManager.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManager.cpp b/query_execution/QueryManager.cpp
index 21f5820..e4e4c9d 100644
--- a/query_execution/QueryManager.cpp
+++ b/query_execution/QueryManager.cpp
@@ -457,12 +457,12 @@ void QueryManager::getRebuildWorkOrders(const dag_node_index index,
        i < partially_filled_block_refs.size();
        ++i) {
     container->addRebuildWorkOrder(
-        new RebuildWorkOrder(std::move(partially_filled_block_refs[i]),
-                            index,
-                            op.getOutputRelationID(),
-                            foreman_client_id_,
-                            query_id_,
-                            bus_),
+        new RebuildWorkOrder(query_id_,
+                             std::move(partially_filled_block_refs[i]),
+                             index,
+                             op.getOutputRelationID(),
+                             foreman_client_id_,
+                             bus_),
         index);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/query_execution/tests/Foreman_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/Foreman_unittest.cpp b/query_execution/tests/Foreman_unittest.cpp
index d2f43a4..79f8f4a 100644
--- a/query_execution/tests/Foreman_unittest.cpp
+++ b/query_execution/tests/Foreman_unittest.cpp
@@ -90,7 +90,8 @@ class MockOperator: public RelationalOperator {
                const bool has_streaming_input,
                const int max_getworkorder_iters = 1,
                const int max_workorders = INT_MAX)
-      : produce_workorders_(produce_workorders),
+      : RelationalOperator(0 /* Query Id */),
+        produce_workorders_(produce_workorders),
         has_streaming_input_(has_streaming_input),
         max_workorders_(max_workorders),
         max_getworkorder_iters_(max_getworkorder_iters),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 80876f2..308d5ca 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -91,7 +91,8 @@ class MockOperator: public RelationalOperator {
                const bool has_streaming_input,
                const int max_getworkorder_iters = 1,
                const int max_workorders = INT_MAX)
-      : produce_workorders_(produce_workorders),
+      : RelationalOperator(0 /* Query Id */),
+        produce_workorders_(produce_workorders),
         has_streaming_input_(has_streaming_input),
         max_workorders_(max_workorders),
         max_getworkorder_iters_(max_getworkorder_iters),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 30dfa8e..99c2a21 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -193,8 +193,8 @@ void ExecutionGenerator::generatePlan(const P::PhysicalPtr &physical_plan) {
     }
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
-            new DropTableOperator(*temporary_relation,
-                                  query_handle_->query_id(),
+            new DropTableOperator(query_handle_->query_id(),
+                                  *temporary_relation,
                                   optimizer_context_->catalog_database(),
                                   false /* only_drop_blocks */));
     DCHECK(!temporary_relation_info.isStoredRelation());
@@ -411,13 +411,14 @@ void ExecutionGenerator::convertSample(const P::SamplePtr &physical_sample) {
       findRelationInfoOutputByPhysical(physical_sample->input());
   DCHECK(input_relation_info != nullptr);
 
-  SampleOperator *sample_op = new SampleOperator(*input_relation_info->relation,
-                                                 *output_relation,
-                                                 insert_destination_index,
-                                                 input_relation_info->isStoredRelation(),
-                                                 physical_sample->is_block_sample(),
-                                                 physical_sample->percentage(),
-                                                 query_handle_->query_id());
+  SampleOperator *sample_op =
+      new SampleOperator(query_handle_->query_id(),
+                         *input_relation_info->relation,
+                         *output_relation,
+                         insert_destination_index,
+                         input_relation_info->isStoredRelation(),
+                         physical_sample->is_block_sample(),
+                         physical_sample->percentage());
   const QueryPlan::DAGNodeIndex sample_index =
       execution_plan_->addRelationalOperator(sample_op);
   insert_destination_proto->set_relational_op_index(sample_index);
@@ -526,22 +527,22 @@ void ExecutionGenerator::convertSelection(
   // doesn't require any expression evaluation or intermediate copies) if
   // possible.
   std::vector<attribute_id> attributes;
-  SelectOperator *op
-      = convertSimpleProjection(project_expressions_group_index, &attributes)
-        ? new SelectOperator(*input_relation_info->relation,
-                             *output_relation,
-                             insert_destination_index,
-                             execution_predicate_index,
-                             move(attributes),
-                             input_relation_info->isStoredRelation(),
-                             query_handle_->query_id())
-        : new SelectOperator(*input_relation_info->relation,
-                             *output_relation,
-                             insert_destination_index,
-                             execution_predicate_index,
-                             project_expressions_group_index,
-                             input_relation_info->isStoredRelation(),
-                             query_handle_->query_id());
+  SelectOperator *op =
+      convertSimpleProjection(project_expressions_group_index, &attributes)
+          ? new SelectOperator(query_handle_->query_id(),
+                               *input_relation_info->relation,
+                               *output_relation,
+                               insert_destination_index,
+                               execution_predicate_index,
+                               move(attributes),
+                               input_relation_info->isStoredRelation())
+          : new SelectOperator(query_handle_->query_id(),
+                               *input_relation_info->relation,
+                               *output_relation,
+                               insert_destination_index,
+                               execution_predicate_index,
+                               project_expressions_group_index,
+                               input_relation_info->isStoredRelation());
 
   const QueryPlan::DAGNodeIndex select_index =
       execution_plan_->addRelationalOperator(op);
@@ -741,12 +742,12 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const QueryPlan::DAGNodeIndex build_operator_index =
       execution_plan_->addRelationalOperator(
           new BuildHashOperator(
+              query_handle_->query_id(),
               *build_relation_info->relation,
               build_relation_info->isStoredRelation(),
               build_attribute_ids,
               any_build_attributes_nullable,
-              join_hash_table_index,
-              query_handle_->query_id()));
+              join_hash_table_index));
 
   // Create InsertDestination proto.
   const CatalogRelation *output_relation = nullptr;
@@ -782,6 +783,7 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const QueryPlan::DAGNodeIndex join_operator_index =
       execution_plan_->addRelationalOperator(
           new HashJoinOperator(
+              query_handle_->query_id(),
               *build_relation_info->relation,
               *probe_operator_info->relation,
               probe_operator_info->isStoredRelation(),
@@ -792,14 +794,13 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
               join_hash_table_index,
               residual_predicate_index,
               project_expressions_group_index,
-              query_handle_->query_id(),
               is_selection_on_build.get(),
               join_type));
   insert_destination_proto->set_relational_op_index(join_operator_index);
 
   const QueryPlan::DAGNodeIndex destroy_operator_index =
       execution_plan_->addRelationalOperator(new DestroyHashOperator(
-          join_hash_table_index, query_handle_->query_id()));
+          query_handle_->query_id(), join_hash_table_index));
 
   if (!build_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(build_operator_index,
@@ -885,16 +886,16 @@ void ExecutionGenerator::convertNestedLoopsJoin(
 
   // Create and add a NestedLoopsJoin operator.
   const QueryPlan::DAGNodeIndex join_operator_index =
-      execution_plan_->addRelationalOperator(new NestedLoopsJoinOperator(
-          *left_relation_info->relation,
-          *right_relation_info->relation,
-          *output_relation,
-          insert_destination_index,
-          execution_join_predicate_index,
-          project_expressions_group_index,
-          left_relation_info->isStoredRelation(),
-          right_relation_info->isStoredRelation(),
-          query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(
+          new NestedLoopsJoinOperator(query_handle_->query_id(),
+                                      *left_relation_info->relation,
+                                      *right_relation_info->relation,
+                                      *output_relation,
+                                      insert_destination_index,
+                                      execution_join_predicate_index,
+                                      project_expressions_group_index,
+                                      left_relation_info->isStoredRelation(),
+                                      right_relation_info->isStoredRelation()));
   insert_destination_proto->set_relational_op_index(join_operator_index);
 
   if (!left_relation_info->isStoredRelation()) {
@@ -940,13 +941,13 @@ void ExecutionGenerator::convertCopyFrom(
   const QueryPlan::DAGNodeIndex scan_operator_index =
       execution_plan_->addRelationalOperator(
           new TextScanOperator(
+              query_handle_->query_id(),
               physical_plan->file_name(),
               physical_plan->column_delimiter(),
               physical_plan->escape_strings(),
               FLAGS_parallelize_load,
               *output_relation,
-              insert_destination_index,
-              query_handle_->query_id()));
+              insert_destination_index));
   insert_destination_proto->set_relational_op_index(scan_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_operator_index =
@@ -997,10 +998,11 @@ void ExecutionGenerator::convertCreateIndex(
     // Check if the given index description is valid.
     THROW_SQL_ERROR() << "The index with given properties cannot be created.";
   }
-  execution_plan_->addRelationalOperator(new CreateIndexOperator(input_relation,
-                                                                 physical_plan->index_name(),
-                                                                 query_handle_->query_id(),
-                                                                 std::move(index_description)));
+  execution_plan_->addRelationalOperator(
+      new CreateIndexOperator(query_handle_->query_id(),
+                              input_relation,
+                              physical_plan->index_name(),
+                              std::move(index_description)));
 }
 
 void ExecutionGenerator::convertCreateTable(
@@ -1067,8 +1069,8 @@ void ExecutionGenerator::convertDeleteTuples(
        execution_predicate->getStaticResult())) {
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
-            new DropTableOperator(*input_relation_info->relation,
-                                  query_handle_->query_id(),
+            new DropTableOperator(query_handle_->query_id(),
+                                  *input_relation_info->relation,
                                   optimizer_context_->catalog_database(),
                                   true /* only_drop_blocks */));
     if (!input_relation_info->isStoredRelation()) {
@@ -1081,11 +1083,12 @@ void ExecutionGenerator::convertDeleteTuples(
     query_context_proto_->add_predicates()->CopyFrom(execution_predicate->getProto());
 
     const QueryPlan::DAGNodeIndex delete_tuples_index =
-        execution_plan_->addRelationalOperator(new DeleteOperator(
-            *input_relation_info->relation,
-            execution_predicate_index,
-            input_relation_info->isStoredRelation(),
-            query_handle_->query_id()));
+        execution_plan_->addRelationalOperator(
+            new DeleteOperator(query_handle_->query_id(),
+                               *input_relation_info->relation,
+                               execution_predicate_index,
+                               input_relation_info->isStoredRelation()));
+
     if (!input_relation_info->isStoredRelation()) {
       execution_plan_->addDirectDependency(delete_tuples_index,
                                            input_relation_info->producer_operator_index,
@@ -1111,8 +1114,8 @@ void ExecutionGenerator::convertDropTable(
 #endif
 
   execution_plan_->addRelationalOperator(
-      new DropTableOperator(catalog_relation,
-                            query_handle_->query_id(),
+      new DropTableOperator(query_handle_->query_id(),
+                            catalog_relation,
                             optimizer_context_->catalog_database()));
 }
 
@@ -1164,10 +1167,10 @@ void ExecutionGenerator::convertInsertTuple(
 
   const QueryPlan::DAGNodeIndex insert_operator_index =
       execution_plan_->addRelationalOperator(
-          new InsertOperator(input_relation,
+          new InsertOperator(query_handle_->query_id(),
+                             input_relation,
                              insert_destination_index,
-                             tuple_index,
-                             query_handle_->query_id()));
+                             tuple_index));
   insert_destination_proto->set_relational_op_index(insert_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
@@ -1240,13 +1243,13 @@ void ExecutionGenerator::convertInsertSelection(
   // optimization is to enable specifying a specific output relation for each
   // physical plan by modifying class Physical.
   SelectOperator *insert_selection_op =
-      new SelectOperator(*selection_relation_info->relation,
+      new SelectOperator(query_handle_->query_id(),
+                         *selection_relation_info->relation,
                          destination_relation,
                          insert_destination_index,
                          QueryContext::kInvalidPredicateId,
                          move(attributes),
-                         selection_relation_info->isStoredRelation(),
-                         query_handle_->query_id());
+                         selection_relation_info->isStoredRelation());
 
   const QueryPlan::DAGNodeIndex insert_selection_index =
       execution_plan_->addRelationalOperator(insert_selection_op);
@@ -1321,13 +1324,13 @@ void ExecutionGenerator::convertUpdateTable(
   }
 
   const QueryPlan::DAGNodeIndex update_operator_index =
-      execution_plan_->addRelationalOperator(
-          new UpdateOperator(
-              *optimizer_context_->catalog_database()->getRelationById(input_rel_id),
-              relocation_destination_index,
-              execution_predicate_index,
-              update_group_index,
-              query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(new UpdateOperator(
+          query_handle_->query_id(),
+          *optimizer_context_->catalog_database()->getRelationById(
+              input_rel_id),
+          relocation_destination_index,
+          execution_predicate_index,
+          update_group_index));
   relocation_destination_proto->set_relational_op_index(update_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
@@ -1429,10 +1432,11 @@ void ExecutionGenerator::convertAggregate(
   const QueryPlan::DAGNodeIndex aggregation_operator_index =
       execution_plan_->addRelationalOperator(
           new AggregationOperator(
+              query_handle_->query_id(),
               *input_relation_info->relation,
               input_relation_info->isStoredRelation(),
-              aggr_state_index,
-              query_handle_->query_id()));
+              aggr_state_index));
+
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(aggregation_operator_index,
                                          input_relation_info->producer_operator_index,
@@ -1450,10 +1454,11 @@ void ExecutionGenerator::convertAggregate(
 
   const QueryPlan::DAGNodeIndex finalize_aggregation_operator_index =
       execution_plan_->addRelationalOperator(
-          new FinalizeAggregationOperator(aggr_state_index,
+          new FinalizeAggregationOperator(query_handle_->query_id(),
+                                          aggr_state_index,
                                           *output_relation,
-                                          insert_destination_index,
-                                          query_handle_->query_id()));
+                                          insert_destination_index));
+
   insert_destination_proto->set_relational_op_index(finalize_aggregation_operator_index);
 
   execution_plan_->addDirectDependency(finalize_aggregation_operator_index,
@@ -1499,13 +1504,13 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
   const CatalogRelationInfo *input_relation_info =
       findRelationInfoOutputByPhysical(physical_sort->input());
   const QueryPlan::DAGNodeIndex run_generator_index =
-      execution_plan_->addRelationalOperator(
-          new SortRunGenerationOperator(*input_relation_info->relation,
-                                        *initial_runs_relation,
-                                        initial_runs_destination_id,
-                                        sort_run_gen_config_id,
-                                        input_relation_info->isStoredRelation(),
-                                        query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(new SortRunGenerationOperator(
+          query_handle_->query_id(),
+          *input_relation_info->relation,
+          *initial_runs_relation,
+          initial_runs_destination_id,
+          sort_run_gen_config_id,
+          input_relation_info->isStoredRelation()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(run_generator_index,
                                          input_relation_info->producer_operator_index,
@@ -1553,17 +1558,18 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
 
   // TODO(qzeng): Make the merge factor configurable.
   const QueryPlan::DAGNodeIndex merge_run_operator_index =
-      execution_plan_->addRelationalOperator(
-          new SortMergeRunOperator(*initial_runs_relation,
-                                   *sorted_relation,
-                                   sorted_output_destination_id,
-                                   *merged_runs_relation,
-                                   merged_runs_destination_id,
-                                   sort_merge_run_config_id,
-                                   64 /* merge_factor */,
-                                   physical_sort->limit(),
-                                   false /* input_relation_is_stored */,
-                                   query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(new SortMergeRunOperator(
+          query_handle_->query_id(),
+          *initial_runs_relation,
+          *sorted_relation,
+          sorted_output_destination_id,
+          *merged_runs_relation,
+          merged_runs_destination_id,
+          sort_merge_run_config_id,
+          64 /* merge_factor */,
+          physical_sort->limit(),
+          false /* input_relation_is_stored */));
+
   execution_plan_->addDirectDependency(merge_run_operator_index,
                                        run_generator_index,
                                        false /* is_pipeline_breaker */);
@@ -1576,8 +1582,8 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
   const QueryPlan::DAGNodeIndex drop_merged_runs_index =
       execution_plan_->addRelationalOperator(
           new DropTableOperator(
-              *merged_runs_relation,
               query_handle_->query_id(),
+              *merged_runs_relation,
               optimizer_context_->catalog_database(),
               false /* only_drop_blocks */));
   execution_plan_->addDirectDependency(
@@ -1613,10 +1619,10 @@ void ExecutionGenerator::convertTableGenerator(
       physical_tablegen->generator_function_handle()->getProto());
 
   TableGeneratorOperator *op =
-      new TableGeneratorOperator(*output_relation,
+      new TableGeneratorOperator(query_handle_->query_id(),
+                                 *output_relation,
                                  insert_destination_index,
-                                 generator_function_index,
-                                 query_handle_->query_id());
+                                 generator_function_index);
 
   const QueryPlan::DAGNodeIndex tablegen_index =
       execution_plan_->addRelationalOperator(op);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
index a08a476..815c13e 100644
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
@@ -15,6 +15,7 @@
  *   limitations under the License.
  **/
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <vector>
@@ -36,6 +37,10 @@
 namespace quickstep {
 namespace optimizer {
 
+namespace {
+constexpr std::size_t kQueryId = 0;
+}
+
 class ExecutionHeuristicsTest : public ::testing::Test {
  protected:
   virtual void SetUp() {
@@ -75,12 +80,12 @@ class ExecutionHeuristicsTest : public ::testing::Test {
     std::vector<attribute_id> build_attribute_ids;
     build_attribute_ids.push_back(build_attribute_id);
     QueryPlan::DAGNodeIndex build_operator_index =
-        query_plan->addRelationalOperator(new BuildHashOperator(*build_relation,
+        query_plan->addRelationalOperator(new BuildHashOperator(kQueryId,
+                                                                *build_relation,
                                                                 true,
                                                                 build_attribute_ids,
                                                                 false,
-                                                                join_hash_table_index,
-                                                                0  /* dummy query ID */));
+                                                                join_hash_table_index));
     return build_operator_index;
   }
 
@@ -92,17 +97,18 @@ class ExecutionHeuristicsTest : public ::testing::Test {
     std::vector<attribute_id> probe_attribute_ids;
     probe_attribute_ids.push_back(probe_attribute_id);
     QueryPlan::DAGNodeIndex join_operator_index =
-        query_plan->addRelationalOperator(new HashJoinOperator(*build_relation,
-                                                               *probe_relation,
-                                                               true,
-                                                               probe_attribute_ids,
-                                                               false,
-                                                               *probe_relation,
-                                                               0,
-                                                               join_hash_table_index,
-                                                               0,
-                                                               0,
-                                                               0  /* dummy query ID */));
+        query_plan->addRelationalOperator(
+            new HashJoinOperator(kQueryId,
+                                 *build_relation,
+                                 *probe_relation,
+                                 true,
+                                 probe_attribute_ids,
+                                 false,
+                                 *probe_relation,
+                                 0,
+                                 join_hash_table_index,
+                                 0,
+                                 0));
     return join_operator_index;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/AggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.cpp b/relational_operators/AggregationOperator.cpp
index 94ba901..7252541 100644
--- a/relational_operators/AggregationOperator.cpp
+++ b/relational_operators/AggregationOperator.cpp
@@ -39,8 +39,8 @@ bool AggregationOperator::getAllWorkOrders(
       for (const block_id input_block_id : input_relation_block_ids_) {
         container->addNormalWorkOrder(
             new AggregationWorkOrder(
-                input_block_id,
                 query_id_,
+                input_block_id,
                 query_context->getAggregationState(aggr_state_index_)),
             op_index_);
       }
@@ -51,8 +51,8 @@ bool AggregationOperator::getAllWorkOrders(
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
           new AggregationWorkOrder(
-              input_relation_block_ids_[num_workorders_generated_],
               query_id_,
+              input_relation_block_ids_[num_workorders_generated_],
               query_context->getAggregationState(aggr_state_index_)),
           op_index_);
       ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 0fbc381..f340d4e 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -53,17 +53,17 @@ class AggregationOperator : public RelationalOperator {
    * @brief Constructor for aggregating with arbitrary expressions in projection
    *        list.
    *
+   * @param query_id The ID of this query.
    * @param input_relation The relation to perform aggregation over.
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
    * @param aggr_state_index The index of the AggregationState in QueryContext.
-   * @param query_id The ID of this query.
    **/
-  AggregationOperator(const CatalogRelation &input_relation,
+  AggregationOperator(const std::size_t query_id,
+                      const CatalogRelation &input_relation,
                       bool input_relation_is_stored,
-                      const QueryContext::aggregation_state_id aggr_state_index,
-                      const std::size_t query_id)
+                      const QueryContext::aggregation_state_id aggr_state_index)
       : RelationalOperator(query_id),
         input_relation_is_stored_(input_relation_is_stored),
         input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
@@ -109,12 +109,12 @@ class AggregationWorkOrder : public WorkOrder {
   /**
    * @brief Constructor
    *
-   * @param input_block_id The block id.
    * @param query_id The ID of this query.
+   * @param input_block_id The block id.
    * @param state The AggregationState to use.
    **/
-  AggregationWorkOrder(const block_id input_block_id,
-                       const std::size_t query_id,
+  AggregationWorkOrder(const std::size_t query_id,
+                       const block_id input_block_id,
                        AggregationOperationState *state)
       : WorkOrder(query_id),
         input_block_id_(input_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index c6f6f96..9dc4afe 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -69,11 +69,11 @@ bool BuildHashOperator::getAllWorkOrders(
     if (!started_) {
       for (const block_id input_block_id : input_relation_block_ids_) {
         container->addNormalWorkOrder(
-            new BuildHashWorkOrder(input_relation_,
+            new BuildHashWorkOrder(query_id_,
+                                   input_relation_,
                                    join_key_attributes_,
                                    any_join_key_attributes_nullable_,
                                    input_block_id,
-                                   query_id_,
                                    hash_table,
                                    storage_manager),
             op_index_);
@@ -85,11 +85,11 @@ bool BuildHashOperator::getAllWorkOrders(
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
           new BuildHashWorkOrder(
+              query_id_,
               input_relation_,
               join_key_attributes_,
               any_join_key_attributes_nullable_,
               input_relation_block_ids_[num_workorders_generated_],
-              query_id_,
               hash_table,
               storage_manager),
           op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 5a46d8b..50dd7d6 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -59,6 +59,7 @@ class BuildHashOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to build hash table on.
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
@@ -69,14 +70,13 @@ class BuildHashOperator : public RelationalOperator {
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
    *        The HashTable's key Type(s) should be the Type(s) of the
    *        join_key_attributes in input_relation.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  BuildHashOperator(const CatalogRelation &input_relation,
+  BuildHashOperator(const std::size_t query_id,
+                    const CatalogRelation &input_relation,
                     const bool input_relation_is_stored,
                     const std::vector<attribute_id> &join_key_attributes,
                     const bool any_join_key_attributes_nullable,
-                    const QueryContext::join_hash_table_id hash_table_index,
-                    const std::size_t query_id)
+                    const QueryContext::join_hash_table_id hash_table_index)
     : RelationalOperator(query_id),
       input_relation_(input_relation),
       input_relation_is_stored_(input_relation_is_stored),
@@ -131,20 +131,20 @@ class BuildHashWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query.
    * @param input_relation The relation to build hash table on.
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
    * @param build_block_id The block id.
-   * @param query_id The ID of the query.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
    **/
-  BuildHashWorkOrder(const CatalogRelationSchema &input_relation,
+  BuildHashWorkOrder(const std::size_t query_id,
+                     const CatalogRelationSchema &input_relation,
                      const std::vector<attribute_id> &join_key_attributes,
                      const bool any_join_key_attributes_nullable,
                      const block_id build_block_id,
-                     const std::size_t query_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager)
       : WorkOrder(query_id),
@@ -158,20 +158,20 @@ class BuildHashWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query.
    * @param input_relation The relation to build hash table on.
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param query_id The ID of the query.
    * @param build_block_id The block id.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
    **/
-  BuildHashWorkOrder(const CatalogRelationSchema &input_relation,
+  BuildHashWorkOrder(const std::size_t query_id,
+                     const CatalogRelationSchema &input_relation,
                      std::vector<attribute_id> &&join_key_attributes,
                      const bool any_join_key_attributes_nullable,
                      const block_id build_block_id,
-                     const std::size_t query_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager)
       : WorkOrder(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index ede3f02..11a01ae 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -51,14 +51,14 @@ class CreateIndexOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param relation The relation to create index upon.
    * @param index_name The index to create.
-   * @param query_id The ID of the query to which this operator belongs.
    * @param index_description The index_description associated with this index.
    **/
-  CreateIndexOperator(CatalogRelation *relation,
+  CreateIndexOperator(const std::size_t query_id,
+                      CatalogRelation *relation,
                       const std::string &index_name,
-                      const std::size_t query_id,
                       IndexSubBlockDescription &&index_description)  // NOLINT(whitespace/operators)
       : RelationalOperator(query_id),
         relation_(DCHECK_NOTNULL(relation)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 94169ed..15dc9e3 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -54,13 +54,13 @@ bool DeleteOperator::getAllWorkOrders(
     if (!started_) {
       for (const block_id input_block_id : relation_block_ids_) {
         container->addNormalWorkOrder(
-            new DeleteWorkOrder(relation_,
+            new DeleteWorkOrder(query_id_,
+                                relation_,
                                 input_block_id,
                                 predicate,
                                 storage_manager,
                                 op_index_,
                                 scheduler_client_id,
-                                query_id_,
                                 bus),
             op_index_);
       }
@@ -70,13 +70,13 @@ bool DeleteOperator::getAllWorkOrders(
   } else {
     while (num_workorders_generated_ < relation_block_ids_.size()) {
       container->addNormalWorkOrder(
-          new DeleteWorkOrder(relation_,
+          new DeleteWorkOrder(query_id_,
+                              relation_,
                               relation_block_ids_[num_workorders_generated_],
                               predicate,
                               storage_manager,
                               op_index_,
                               scheduler_client_id,
-                              query_id_,
                               bus),
           op_index_);
       ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index ba1f825..c55f585 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -55,18 +55,18 @@ class DeleteOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param relation The relation to perform the DELETE over.
    * @param predicate_index The index of Predicate in QueryContext. All tuples
    *        matching pred will be deleted (If kInvalidPredicateId, then all
    *        tuples will be deleted).
    * @param relation_is_stored If relation is a stored relation and is fully
    *        available to the operator before it can start generating workorders.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  DeleteOperator(const CatalogRelation &relation,
+  DeleteOperator(const std::size_t query_id,
+                 const CatalogRelation &relation,
                  const QueryContext::predicate_id predicate_index,
-                 const bool relation_is_stored,
-                 const std::size_t query_id)
+                 const bool relation_is_stored)
      :  RelationalOperator(query_id),
         relation_(relation),
         predicate_index_(predicate_index),
@@ -122,6 +122,7 @@ class DeleteWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this workorder belongs.
    * @param input_relation The relation to perform the DELETE over.
    * @param input_block_id The block Id.
    * @param predicate All tuples matching \c predicate will be deleted (If
@@ -130,16 +131,15 @@ class DeleteWorkOrder : public WorkOrder {
    * @param delete_operator_index The index of the Delete Operator in the query
    *        plan DAG.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
-   * @param query_id The ID of the query to which this workorder belongs.
    * @param bus A pointer to the TMB.
    **/
-  DeleteWorkOrder(const CatalogRelationSchema &input_relation,
+  DeleteWorkOrder(const std::size_t query_id,
+                  const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const Predicate *predicate,
                   StorageManager *storage_manager,
                   const std::size_t delete_operator_index,
                   const tmb::client_id scheduler_client_id,
-                  const std::size_t query_id,
                   MessageBus *bus)
       : WorkOrder(query_id),
         input_relation_(input_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/DestroyHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.cpp b/relational_operators/DestroyHashOperator.cpp
index c92117a..8aa40b4 100644
--- a/relational_operators/DestroyHashOperator.cpp
+++ b/relational_operators/DestroyHashOperator.cpp
@@ -33,7 +33,7 @@ bool DestroyHashOperator::getAllWorkOrders(
   if (blocking_dependencies_met_ && !work_generated_) {
     work_generated_ = true;
     container->addNormalWorkOrder(
-        new DestroyHashWorkOrder(hash_table_index_, query_id_, query_context),
+        new DestroyHashWorkOrder(query_id_, hash_table_index_, query_context),
         op_index_);
   }
   return work_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 086c279..7d8acb7 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -46,11 +46,11 @@ class DestroyHashOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
-   * @param hash_table_index The index of the JoinHashTable in QueryContext.
    * @param query_id The ID of the query to which this operator belongs.
+   * @param hash_table_index The index of the JoinHashTable in QueryContext.
    **/
-  DestroyHashOperator(const QueryContext::join_hash_table_id hash_table_index,
-                      const std::size_t query_id)
+  DestroyHashOperator(const std::size_t query_id,
+                      const QueryContext::join_hash_table_id hash_table_index)
       : RelationalOperator(query_id),
         hash_table_index_(hash_table_index),
         work_generated_(false) {}
@@ -78,12 +78,12 @@ class DestroyHashWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
-   * @param hash_table_index The index of the JoinHashTable in QueryContext.
    * @param query_id The ID of the query to which this WorkOrder belongs.
+   * @param hash_table_index The index of the JoinHashTable in QueryContext.
    * @param query_context The QueryContext to use.
    **/
-  DestroyHashWorkOrder(const QueryContext::join_hash_table_id hash_table_index,
-                       const std::size_t query_id,
+  DestroyHashWorkOrder(const std::size_t query_id,
+                       const QueryContext::join_hash_table_id hash_table_index,
                        QueryContext *query_context)
       : WorkOrder(query_id),
         hash_table_index_(hash_table_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index 0bbb718..a0a8d6e 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -55,14 +55,14 @@ class DropTableOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
-   * @param relation The relation to drop.
    * @param query_id The ID of the query to which this operator belongs.
+   * @param relation The relation to drop.
    * @param database The databse where to drop \c relation.
    * @param only_drop_blocks If true, only drop the blocks belonging to \c
    *        relation, but leave \c relation in \c database.
    **/
-  DropTableOperator(const CatalogRelation &relation,
-                    const std::size_t query_id,
+  DropTableOperator(const std::size_t query_id,
+                    const CatalogRelation &relation,
                     CatalogDatabase *database,
                     const bool only_drop_blocks = false)
       : RelationalOperator(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index 51e55dc..e8a403f 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -54,16 +54,17 @@ class FinalizeAggregationOperator : public RelationalOperator {
    * @brief Constructor for finalizing aggregation state and writing output
    * tuples.  The actual aggregation is computed by the AggregationOperator.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param aggr_state_index The index of the AggregationState in QueryContext.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert aggregation results.
-   * @param query_id The ID of the query to which this operator belongs.
    */
-  FinalizeAggregationOperator(const QueryContext::aggregation_state_id aggr_state_index,
-                              const CatalogRelation &output_relation,
-                              const QueryContext::insert_destination_id output_destination_index,
-                              const std::size_t query_id)
+  FinalizeAggregationOperator(
+      const std::size_t query_id,
+      const QueryContext::aggregation_state_id aggr_state_index,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index)
       : RelationalOperator(query_id),
         aggr_state_index_(aggr_state_index),
         output_relation_(output_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index d8c7304..b89cfb3 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -290,7 +290,8 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
       if (!started_) {
         for (const block_id probe_block_id : probe_relation_block_ids_) {
           container->addNormalWorkOrder(
-              new JoinWorkOrderClass(build_relation_,
+              new JoinWorkOrderClass(query_id_,
+                                     build_relation_,
                                      probe_relation_,
                                      join_key_attributes_,
                                      any_join_key_attributes_nullable_,
@@ -298,7 +299,6 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
                                      residual_predicate,
                                      selection,
                                      hash_table,
-                                     query_id_,
                                      output_destination,
                                      storage_manager),
               op_index_);
@@ -309,17 +309,18 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
     } else {
       while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
         container->addNormalWorkOrder(
-            new JoinWorkOrderClass(build_relation_,
-                                   probe_relation_,
-                                   join_key_attributes_,
-                                   any_join_key_attributes_nullable_,
-                                   probe_relation_block_ids_[num_workorders_generated_],
-                                   residual_predicate,
-                                   selection,
-                                   hash_table,
-                                   query_id_,
-                                   output_destination,
-                                   storage_manager),
+            new JoinWorkOrderClass(
+                query_id_,
+                build_relation_,
+                probe_relation_,
+                join_key_attributes_,
+                any_join_key_attributes_nullable_,
+                probe_relation_block_ids_[num_workorders_generated_],
+                residual_predicate,
+                selection,
+                hash_table,
+                output_destination,
+                storage_manager),
             op_index_);
         ++num_workorders_generated_;
       }  // end while
@@ -350,6 +351,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
         for (const block_id probe_block_id : probe_relation_block_ids_) {
           container->addNormalWorkOrder(
               new HashOuterJoinWorkOrder(
+                  query_id_,
                   build_relation_,
                   probe_relation_,
                   join_key_attributes_,
@@ -358,7 +360,6 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
                   selection,
                   is_selection_on_build_,
                   hash_table,
-                  query_id_,
                   output_destination,
                   storage_manager),
               op_index_);
@@ -370,6 +371,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
       while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
         container->addNormalWorkOrder(
             new HashOuterJoinWorkOrder(
+                query_id_,
                 build_relation_,
                 probe_relation_,
                 join_key_attributes_,
@@ -378,7 +380,6 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
                 selection,
                 is_selection_on_build_,
                 hash_table,
-                query_id_,
                 output_destination,
                 storage_manager),
             op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 825f360..1d5d4e3 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -85,6 +85,7 @@ class HashJoinOperator : public RelationalOperator {
    *       dependent on the selectivity of the predicates involved. The decision
    *       is left to the query optimizer.
    *
+   * @param query_id The ID of the query.
    * @param build_relation The relation that the hash table was originally
    *        built on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -108,25 +109,25 @@ class HashJoinOperator : public RelationalOperator {
    *        corresponding to the attributes of the relation referred by
    *        output_relation_id. Each Scalar is evaluated for the joined tuples,
    *        and the resulting value is inserted into the join result.
-   * @param query_id The ID of the query.
    * @param is_selection_on_build Whether each selection Scalar is using attributes
    *        from the build relation as input. Can be NULL for inner/semi/anti
    *        joins since this information is not utilized by these joins.
    * @param join_type The type of join corresponding to this operator.
    **/
-  HashJoinOperator(const CatalogRelation &build_relation,
-                   const CatalogRelation &probe_relation,
-                   const bool probe_relation_is_stored,
-                   const std::vector<attribute_id> &join_key_attributes,
-                   const bool any_join_key_attributes_nullable,
-                   const CatalogRelation &output_relation,
-                   const QueryContext::insert_destination_id output_destination_index,
-                   const QueryContext::join_hash_table_id hash_table_index,
-                   const QueryContext::predicate_id residual_predicate_index,
-                   const QueryContext::scalar_group_id selection_index,
-                   const std::size_t query_id,
-                   const std::vector<bool> *is_selection_on_build = nullptr,
-                   const JoinType join_type = JoinType::kInnerJoin)
+  HashJoinOperator(
+      const std::size_t query_id,
+      const CatalogRelation &build_relation,
+      const CatalogRelation &probe_relation,
+      const bool probe_relation_is_stored,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::join_hash_table_id hash_table_index,
+      const QueryContext::predicate_id residual_predicate_index,
+      const QueryContext::scalar_group_id selection_index,
+      const std::vector<bool> *is_selection_on_build = nullptr,
+      const JoinType join_type = JoinType::kInnerJoin)
       : RelationalOperator(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -230,6 +231,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -246,21 +248,21 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashInnerJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                         const CatalogRelationSchema &probe_relation,
-                         const std::vector<attribute_id> &join_key_attributes,
-                         const bool any_join_key_attributes_nullable,
-                         const block_id lookup_block_id,
-                         const Predicate *residual_predicate,
-                         const std::vector<std::unique_ptr<const Scalar>> &selection,
-                         const JoinHashTable &hash_table,
-                         const std::size_t query_id,
-                         InsertDestination *output_destination,
-                         StorageManager *storage_manager)
+  HashInnerJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -276,6 +278,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -292,21 +295,21 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashInnerJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                         const CatalogRelationSchema &probe_relation,
-                         std::vector<attribute_id> &&join_key_attributes,
-                         const bool any_join_key_attributes_nullable,
-                         const block_id lookup_block_id,
-                         const Predicate *residual_predicate,
-                         const std::vector<std::unique_ptr<const Scalar>> &selection,
-                         const JoinHashTable &hash_table,
-                         const std::size_t query_id,
-                         InsertDestination *output_destination,
-                         StorageManager *storage_manager)
+  HashInnerJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      std::vector<attribute_id> &&join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -359,6 +362,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -375,21 +379,21 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashSemiJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                        const CatalogRelationSchema &probe_relation,
-                        const std::vector<attribute_id> &join_key_attributes,
-                        const bool any_join_key_attributes_nullable,
-                        const block_id lookup_block_id,
-                        const Predicate *residual_predicate,
-                        const std::vector<std::unique_ptr<const Scalar>> &selection,
-                        const JoinHashTable &hash_table,
-                        const std::size_t query_id,
-                        InsertDestination *output_destination,
-                        StorageManager *storage_manager)
+  HashSemiJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -405,6 +409,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -421,21 +426,21 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashSemiJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                        const CatalogRelationSchema &probe_relation,
-                        std::vector<attribute_id> &&join_key_attributes,
-                        const bool any_join_key_attributes_nullable,
-                        const block_id lookup_block_id,
-                        const Predicate *residual_predicate,
-                        const std::vector<std::unique_ptr<const Scalar>> &selection,
-                        const JoinHashTable &hash_table,
-                        const std::size_t query_id,
-                        InsertDestination *output_destination,
-                        StorageManager *storage_manager)
+  HashSemiJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      std::vector<attribute_id> &&join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -481,6 +486,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -497,21 +503,21 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashAntiJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                        const CatalogRelationSchema &probe_relation,
-                        const std::vector<attribute_id> &join_key_attributes,
-                        const bool any_join_key_attributes_nullable,
-                        const block_id lookup_block_id,
-                        const Predicate *residual_predicate,
-                        const std::vector<std::unique_ptr<const Scalar>> &selection,
-                        const JoinHashTable &hash_table,
-                        const std::size_t query_id,
-                        InsertDestination *output_destination,
-                        StorageManager *storage_manager)
+  HashAntiJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -527,6 +533,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -543,21 +550,21 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashAntiJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                        const CatalogRelationSchema &probe_relation,
-                        std::vector<attribute_id> &&join_key_attributes,
-                        const bool any_join_key_attributes_nullable,
-                        const block_id lookup_block_id,
-                        const Predicate *residual_predicate,
-                        const std::vector<std::unique_ptr<const Scalar>> &selection,
-                        const JoinHashTable &hash_table,
-                        const std::size_t query_id,
-                        InsertDestination *output_destination,
-                        StorageManager *storage_manager)
+  HashAntiJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      std::vector<attribute_id> &&join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -608,6 +615,7 @@ class HashOuterJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -623,21 +631,21 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
    * @param lookup_block_id The block id of the probe_relation.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashOuterJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                         const CatalogRelationSchema &probe_relation,
-                         const std::vector<attribute_id> &join_key_attributes,
-                         const bool any_join_key_attributes_nullable,
-                         const block_id lookup_block_id,
-                         const std::vector<std::unique_ptr<const Scalar>> &selection,
-                         const std::vector<bool> &is_selection_on_build,
-                         const JoinHashTable &hash_table,
-                         const std::size_t query_id,
-                         InsertDestination *output_destination,
-                         StorageManager *storage_manager)
+  HashOuterJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const std::vector<bool> &is_selection_on_build,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -653,6 +661,7 @@ class HashOuterJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -668,21 +677,21 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashOuterJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                         const CatalogRelationSchema &probe_relation,
-                         std::vector<attribute_id> &&join_key_attributes,
-                         const bool any_join_key_attributes_nullable,
-                         const block_id lookup_block_id,
-                         const std::vector<std::unique_ptr<const Scalar>> &selection,
-                         std::vector<bool> &&is_selection_on_build,
-                         const JoinHashTable &hash_table,
-                         const std::size_t query_id,
-                         InsertDestination *output_destination,
-                         StorageManager *storage_manager)
+  HashOuterJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      std::vector<attribute_id> &&join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      std::vector<bool> &&is_selection_on_build,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index ccef444..51c606d 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -53,16 +53,17 @@ class InsertOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_relation_id The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert the tuple.
    * @param tuple_index The index of the tuple to insert in the QueryContext.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  InsertOperator(const CatalogRelation &output_relation,
-                 const QueryContext::insert_destination_id output_destination_index,
-                 const QueryContext::tuple_id tuple_index,
-                 const std::size_t query_id)
+  InsertOperator(
+      const std::size_t query_id,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::tuple_id tuple_index)
       : RelationalOperator(query_id),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/NestedLoopsJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.cpp b/relational_operators/NestedLoopsJoinOperator.cpp
index 317cc5d..5a47fca 100644
--- a/relational_operators/NestedLoopsJoinOperator.cpp
+++ b/relational_operators/NestedLoopsJoinOperator.cpp
@@ -76,15 +76,17 @@ bool NestedLoopsJoinOperator::getAllWorkOrders(
       for (const block_id left_block_id : left_relation_block_ids_) {
         for (const block_id right_block_id : right_relation_block_ids_) {
           container->addNormalWorkOrder(
-              new NestedLoopsJoinWorkOrder(left_input_relation_,
-                                           right_input_relation_,
-                                           left_block_id,
-                                           right_block_id,
-                                           query_context->getPredicate(join_predicate_index_),
-                                           query_context->getScalarGroup(selection_index_),
-                                           query_id_,
-                                           query_context->getInsertDestination(output_destination_index_),
-                                           storage_manager),
+              new NestedLoopsJoinWorkOrder(
+                  query_id_,
+                  left_input_relation_,
+                  right_input_relation_,
+                  left_block_id,
+                  right_block_id,
+                  query_context->getPredicate(join_predicate_index_),
+                  query_context->getScalarGroup(selection_index_),
+                  query_context->getInsertDestination(
+                      output_destination_index_),
+                  storage_manager),
               op_index_);
         }
       }
@@ -166,15 +168,16 @@ std::size_t NestedLoopsJoinOperator::getAllWorkOrdersHelperBothNotStored(WorkOrd
          right_index < right_max;
          ++right_index) {
       container->addNormalWorkOrder(
-          new NestedLoopsJoinWorkOrder(left_input_relation_,
-                                       right_input_relation_,
-                                       left_relation_block_ids_[left_index],
-                                       right_relation_block_ids_[right_index],
-                                       query_context->getPredicate(join_predicate_index_),
-                                       query_context->getScalarGroup(selection_index_),
-                                       query_id_,
-                                       query_context->getInsertDestination(output_destination_index_),
-                                       storage_manager),
+          new NestedLoopsJoinWorkOrder(
+              query_id_,
+              left_input_relation_,
+              right_input_relation_,
+              left_relation_block_ids_[left_index],
+              right_relation_block_ids_[right_index],
+              query_context->getPredicate(join_predicate_index_),
+              query_context->getScalarGroup(selection_index_),
+              query_context->getInsertDestination(output_destination_index_),
+              storage_manager),
           op_index_);
     }
   }
@@ -201,13 +204,13 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
       for (const block_id left_block_id : left_relation_block_ids_) {
         container->addNormalWorkOrder(
             new NestedLoopsJoinWorkOrder(
+                query_id_,
                 left_input_relation_,
                 right_input_relation_,
                 left_block_id,
                 right_relation_block_ids_[right_index],
                 join_predicate,
                 selection,
-                query_id_,
                 output_destination,
                 storage_manager),
             op_index_);
@@ -221,13 +224,13 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
          ++left_index) {
       for (const block_id right_block_id : right_relation_block_ids_) {
         container->addNormalWorkOrder(
-            new NestedLoopsJoinWorkOrder(left_input_relation_,
+            new NestedLoopsJoinWorkOrder(query_id_,
+                                         left_input_relation_,
                                          right_input_relation_,
                                          left_relation_block_ids_[left_index],
                                          right_block_id,
                                          join_predicate,
                                          selection,
-                                         query_id_,
                                          output_destination,
                                          storage_manager),
             op_index_);