You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ji...@apache.org on 2017/11/17 19:40:41 UTC

[01/27] incubator-quickstep git commit: Printed out the partition info in QueryPlan. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/trace e6c99562b -> aec7623aa (forced update)


Printed out the partition info in QueryPlan.


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

Branch: refs/heads/trace
Commit: 475704ec9510793a70e149b938d02569611c6177
Parents: d85f7a9
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Mon Sep 11 11:57:10 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Thu Sep 14 13:15:10 2017 -0500

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |  1 +
 query_optimizer/OptimizerTree.hpp               | 39 ++++++++++++++++++++
 query_optimizer/logical/CreateTable.cpp         |  5 +++
 query_optimizer/logical/CreateTable.hpp         | 20 ++++++----
 query_optimizer/physical/CreateTable.cpp        |  5 +++
 query_optimizer/physical/CreateTable.hpp        | 20 ++++++----
 .../tests/logical_generator/Create.test         | 15 +++++++-
 .../tests/physical_generator/Create.test        | 26 +++++++++++++
 query_optimizer/tests/resolver/Create.test      | 17 +++++++--
 9 files changed, 128 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/475704ec/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 4ea21b2..5e0db44 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -212,6 +212,7 @@ target_link_libraries(quickstep_queryoptimizer_OptimizerContext
                       quickstep_queryoptimizer_expressions_ExprId
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_OptimizerTree
+                      quickstep_catalog_Catalog_proto
                       quickstep_storage_StorageBlockLayout_proto
                       quickstep_utility_Macros
                       quickstep_utility_TreeStringSerializable)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/475704ec/query_optimizer/OptimizerTree.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/OptimizerTree.hpp b/query_optimizer/OptimizerTree.hpp
index 62df66d..c54ce20 100644
--- a/query_optimizer/OptimizerTree.hpp
+++ b/query_optimizer/OptimizerTree.hpp
@@ -25,6 +25,7 @@
 #include <string>
 #include <vector>
 
+#include "catalog/Catalog.pb.h"
 #include "storage/StorageBlockLayout.pb.h"
 #include "utility/Macros.hpp"
 #include "utility/TreeStringSerializable.hpp"
@@ -283,6 +284,44 @@ OptimizerProtoRepresentation<TreeNodeType>* getOptimizerRepresentationForProto(
   return node.release();
 }
 
+template<class TreeNodeType>
+OptimizerProtoRepresentation<TreeNodeType>* getOptimizerRepresentationForProto(
+    const serialization::PartitionSchemeHeader *partition_header) {
+  if (partition_header == nullptr) {
+    return nullptr;
+  }
+
+  auto node = std::make_unique<OptimizerProtoRepresentation<TreeNodeType>>();
+
+  // Add properties based on the partition type.
+  switch (partition_header->partition_type()) {
+    case serialization::PartitionSchemeHeader::HASH: {
+      node->addProperty("partition_type", "hash");
+      break;
+    }
+    case serialization::PartitionSchemeHeader::RANDOM: {
+      node->addProperty("partition_type", "random");
+      break;
+    }
+    case serialization::PartitionSchemeHeader::RANGE: {
+      node->addProperty("partition_type", "range");
+      // TODO(quickstep-team): display the range boundaries.
+      node->addProperty("range_boundaries", "TODO");
+      break;
+    }
+    default:
+      LOG(FATAL) << "Unrecognized partition type in protobuf message.";
+  }
+  // Every case will specify a partition number and a partition attributes.
+  node->addProperty("num_partitions", partition_header->num_partitions());
+
+  for (int i = 0; i < partition_header->partition_attribute_ids_size(); ++i) {
+    node->addProperty("partition_attr_id", partition_header->partition_attribute_ids(i));
+  }
+
+  return node.release();
+}
+
 /** @} */
 
 }  // namespace optimizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/475704ec/query_optimizer/logical/CreateTable.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CreateTable.cpp b/query_optimizer/logical/CreateTable.cpp
index 111d04b..9977a0f 100644
--- a/query_optimizer/logical/CreateTable.cpp
+++ b/query_optimizer/logical/CreateTable.cpp
@@ -49,6 +49,11 @@ void CreateTable::getFieldStringItems(
     non_container_child_field_names->push_back("block_properties");
     non_container_child_fields->push_back(block_properties_representation_);
   }
+
+  if (partition_scheme_header_proto_representation_) {
+    non_container_child_field_names->push_back("partition_scheme_header");
+    non_container_child_fields->push_back(partition_scheme_header_proto_representation_);
+  }
 }
 
 }  // namespace logical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/475704ec/query_optimizer/logical/CreateTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CreateTable.hpp b/query_optimizer/logical/CreateTable.hpp
index da4325d..b380ac9 100644
--- a/query_optimizer/logical/CreateTable.hpp
+++ b/query_optimizer/logical/CreateTable.hpp
@@ -48,7 +48,7 @@ typedef std::shared_ptr<const CreateTable> CreateTablePtr;
 /**
  * @brief Represents an operation that creates a new table.
  */
-class CreateTable : public Logical {
+class CreateTable final : public Logical {
  public:
   LogicalType getLogicalType() const override { return LogicalType::kCreateTable; }
 
@@ -138,13 +138,17 @@ class CreateTable : public Logical {
         block_properties_(block_properties),
         block_properties_representation_(
             getOptimizerRepresentationForProto<OptimizerTreeBaseNodePtr>(block_properties_.get())),
-        partition_scheme_header_proto_(partition_scheme_header_proto) {}
-
-  std::string relation_name_;
-  std::vector<expressions::AttributeReferencePtr> attributes_;
-  std::shared_ptr<const StorageBlockLayoutDescription> block_properties_;
-  std::shared_ptr<const OptimizerProtoRepresentation<OptimizerTreeBaseNodePtr> > block_properties_representation_;
-  std::shared_ptr<const serialization::PartitionSchemeHeader> partition_scheme_header_proto_;
+        partition_scheme_header_proto_(partition_scheme_header_proto),
+        partition_scheme_header_proto_representation_(
+            getOptimizerRepresentationForProto<OptimizerTreeBaseNodePtr>(partition_scheme_header_proto_.get())) {}
+
+  const std::string relation_name_;
+  const std::vector<expressions::AttributeReferencePtr> attributes_;
+  const std::shared_ptr<const StorageBlockLayoutDescription> block_properties_;
+  const std::shared_ptr<const OptimizerProtoRepresentation<OptimizerTreeBaseNodePtr>> block_properties_representation_;
+  const std::shared_ptr<const serialization::PartitionSchemeHeader> partition_scheme_header_proto_;
+  const std::shared_ptr<const OptimizerProtoRepresentation<OptimizerTreeBaseNodePtr>>
+      partition_scheme_header_proto_representation_;
 
   DISALLOW_COPY_AND_ASSIGN(CreateTable);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/475704ec/query_optimizer/physical/CreateTable.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CreateTable.cpp b/query_optimizer/physical/CreateTable.cpp
index d42eac3..95e6a9d 100644
--- a/query_optimizer/physical/CreateTable.cpp
+++ b/query_optimizer/physical/CreateTable.cpp
@@ -49,6 +49,11 @@ void CreateTable::getFieldStringItems(
     non_container_child_field_names->push_back("block_properties");
     non_container_child_fields->push_back(block_properties_representation_);
   }
+
+  if (partition_scheme_header_proto_representation_) {
+    non_container_child_field_names->push_back("partition_scheme_header");
+    non_container_child_fields->push_back(partition_scheme_header_proto_representation_);
+  }
 }
 
 }  // namespace physical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/475704ec/query_optimizer/physical/CreateTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CreateTable.hpp b/query_optimizer/physical/CreateTable.hpp
index 05eab0d..6ebdfa2 100644
--- a/query_optimizer/physical/CreateTable.hpp
+++ b/query_optimizer/physical/CreateTable.hpp
@@ -49,7 +49,7 @@ typedef std::shared_ptr<const CreateTable> CreateTablePtr;
 /**
  * @brief Creates a table.
  */
-class CreateTable : public Physical {
+class CreateTable final : public Physical {
  public:
   PhysicalType getPhysicalType() const override {
     return PhysicalType::kCreateTable;
@@ -145,13 +145,17 @@ class CreateTable : public Physical {
         block_properties_(block_properties),
         block_properties_representation_(
             getOptimizerRepresentationForProto<OptimizerTreeBaseNodePtr>(block_properties_.get())),
-        partition_scheme_header_proto_(partition_scheme_header_proto) {}
-
-  std::string relation_name_;
-  std::vector<expressions::AttributeReferencePtr> attributes_;
-  std::shared_ptr<const StorageBlockLayoutDescription> block_properties_;
-  std::shared_ptr<const OptimizerProtoRepresentation<OptimizerTreeBaseNodePtr> > block_properties_representation_;
-  std::shared_ptr<const serialization::PartitionSchemeHeader> partition_scheme_header_proto_;
+        partition_scheme_header_proto_(partition_scheme_header_proto),
+        partition_scheme_header_proto_representation_(
+            getOptimizerRepresentationForProto<OptimizerTreeBaseNodePtr>(partition_scheme_header_proto_.get())) {}
+
+  const std::string relation_name_;
+  const std::vector<expressions::AttributeReferencePtr> attributes_;
+  const std::shared_ptr<const StorageBlockLayoutDescription> block_properties_;
+  const std::shared_ptr<const OptimizerProtoRepresentation<OptimizerTreeBaseNodePtr>> block_properties_representation_;
+  const std::shared_ptr<const serialization::PartitionSchemeHeader> partition_scheme_header_proto_;
+  const std::shared_ptr<const OptimizerProtoRepresentation<OptimizerTreeBaseNodePtr>>
+      partition_scheme_header_proto_representation_;
 
   DISALLOW_COPY_AND_ASSIGN(CreateTable);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/475704ec/query_optimizer/tests/logical_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Create.test b/query_optimizer/tests/logical_generator/Create.test
index aac49fb..89b4a99 100644
--- a/query_optimizer/tests/logical_generator/Create.test
+++ b/query_optimizer/tests/logical_generator/Create.test
@@ -52,6 +52,10 @@ CREATE TABLE foo (attr INT) PARTITION BY HASH(attr) PARTITIONS 4;
 --
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
+| +-partition_scheme_header=ProtoDescription
+| | +-Property=ProtoProperty[Property=partition_type,Value=hash]
+| | +-Property=ProtoProperty[Property=num_partitions,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=0]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
 +-output_attributes=
@@ -59,10 +63,19 @@ TopLevelPlan
 ==
 
 CREATE TABLE foo (attr1 INT, attr2 LONG, attr3 FLOAT, attr4 DOUBLE, attr5 CHAR(5), attr6 VARCHAR(4))
-PARTITION BY HASH(attr1, attr2, attr3, attr4, attr5, attr6) PARTITIONS 4;
+PARTITION BY HASH(attr2, attr1, attr3, attr4, attr5, attr6) PARTITIONS 4;
 --
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
+| +-partition_scheme_header=ProtoDescription
+| | +-Property=ProtoProperty[Property=partition_type,Value=hash]
+| | +-Property=ProtoProperty[Property=num_partitions,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=1]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=0]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=2]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=3]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=5]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr1,relation=foo,type=Int]
 |   +-AttributeReference[id=1,name=attr2,relation=foo,type=Long]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/475704ec/query_optimizer/tests/physical_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Create.test b/query_optimizer/tests/physical_generator/Create.test
index 161cc00..a2dd5c0 100644
--- a/query_optimizer/tests/physical_generator/Create.test
+++ b/query_optimizer/tests/physical_generator/Create.test
@@ -122,6 +122,10 @@ CREATE TABLE foo (attr INT) PARTITION BY HASH(attr) PARTITIONS 4;
 [Optimized Logical Plan]
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
+| +-partition_scheme_header=ProtoDescription
+| | +-Property=ProtoProperty[Property=partition_type,Value=hash]
+| | +-Property=ProtoProperty[Property=num_partitions,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=0]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
 +-output_attributes=
@@ -129,6 +133,10 @@ TopLevelPlan
 [Physical Plan]
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
+| +-partition_scheme_header=ProtoDescription
+| | +-Property=ProtoProperty[Property=partition_type,Value=hash]
+| | +-Property=ProtoProperty[Property=num_partitions,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=0]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
 +-output_attributes=
@@ -141,6 +149,15 @@ PARTITION BY HASH(attr1, attr2, attr3, attr4, attr5, attr6) PARTITIONS 4;
 [Optimized Logical Plan]
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
+| +-partition_scheme_header=ProtoDescription
+| | +-Property=ProtoProperty[Property=partition_type,Value=hash]
+| | +-Property=ProtoProperty[Property=num_partitions,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=0]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=1]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=2]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=3]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=5]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr1,relation=foo,type=Int]
 |   +-AttributeReference[id=1,name=attr2,relation=foo,type=Long]
@@ -158,6 +175,15 @@ TopLevelPlan
 [Physical Plan]
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
+| +-partition_scheme_header=ProtoDescription
+| | +-Property=ProtoProperty[Property=partition_type,Value=hash]
+| | +-Property=ProtoProperty[Property=num_partitions,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=0]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=1]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=2]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=3]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=5]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr1,relation=foo,type=Int]
 |   +-AttributeReference[id=1,name=attr2,relation=foo,type=Long]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/475704ec/query_optimizer/tests/resolver/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Create.test b/query_optimizer/tests/resolver/Create.test
index 1372cf4..c216c85 100644
--- a/query_optimizer/tests/resolver/Create.test
+++ b/query_optimizer/tests/resolver/Create.test
@@ -181,9 +181,7 @@ BLOCKPROPERTIES (TYPE compresse...
 CREATE TABLE foo (attr INT, attr2 INT) WITH
 BLOCKPROPERTIES (TYPE compressed_rowstore, COMPRESS 1);
 --
-ERROR: The COMPRESS property must be specified as ALL or a list of attributes. (2 : 1)
-BLOCKPROPERTIES (TYPE compresse...
-^
+[same as above]
 ==
 
 # All specified COMPRESS columns must exist.
@@ -235,6 +233,10 @@ CREATE TABLE foo (attr INT) PARTITION BY HASH(attr) PARTITIONS 4;
 --
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
+| +-partition_scheme_header=ProtoDescription
+| | +-Property=ProtoProperty[Property=partition_type,Value=hash]
+| | +-Property=ProtoProperty[Property=num_partitions,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=0]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
 +-output_attributes=
@@ -246,6 +248,15 @@ PARTITION BY HASH(attr1, attr2, attr3, attr4, attr5, attr6) PARTITIONS 4;
 --
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
+| +-partition_scheme_header=ProtoDescription
+| | +-Property=ProtoProperty[Property=partition_type,Value=hash]
+| | +-Property=ProtoProperty[Property=num_partitions,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=0]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=1]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=2]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=3]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=4]
+| | +-Property=ProtoProperty[Property=partition_attr_id,Value=5]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr1,relation=foo,type=Int]
 |   +-AttributeReference[id=1,name=attr2,relation=foo,type=Long]


[17/27] incubator-quickstep git commit: Added ProbabilityStore class

Posted by ji...@apache.org.
Added ProbabilityStore class

- Used to store probabilities of objects.
- Probabilities are of two kinds: Individual and cumulative.
- All the individual probabilities within the store add up to one.
- Support for finding the object with given cumulative probability.


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

Branch: refs/heads/trace
Commit: 8f094a1c086445b79d6dba36f81326ac06050209
Parents: f820c45
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Sep 29 15:38:42 2017 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Oct 11 10:38:36 2017 -0500

----------------------------------------------------------------------
 query_execution/CMakeLists.txt                  |  13 +
 query_execution/ProbabilityStore.hpp            | 263 +++++++++++++++++++
 .../tests/ProbabilityStore_unittest.cpp         | 106 ++++++++
 3 files changed, 382 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8f094a1c/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 8f797f7..791434a 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -40,6 +40,7 @@ if (ENABLE_DISTRIBUTED)
   add_library(quickstep_queryexecution_PolicyEnforcerDistributed PolicyEnforcerDistributed.cpp PolicyEnforcerDistributed.hpp)
 endif(ENABLE_DISTRIBUTED)
 add_library(quickstep_queryexecution_PolicyEnforcerSingleNode PolicyEnforcerSingleNode.cpp PolicyEnforcerSingleNode.hpp)
+add_library(quickstep_queryexecution_ProbabilityStore ../empty_src.cpp ProbabilityStore.hpp)
 add_library(quickstep_queryexecution_QueryContext QueryContext.cpp QueryContext.hpp)
 add_library(quickstep_queryexecution_QueryContext_proto
             ${queryexecution_QueryContext_proto_srcs}
@@ -201,6 +202,9 @@ target_link_libraries(quickstep_queryexecution_PolicyEnforcerSingleNode
                       quickstep_utility_Macros
                       tmb
                       ${GFLAGS_LIB_NAME})
+target_link_libraries(quickstep_queryexecution_ProbabilityStore
+                      glog
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryexecution_QueryContext
                       glog
                       quickstep_catalog_CatalogDatabaseLite
@@ -372,6 +376,7 @@ target_link_libraries(quickstep_queryexecution
                       quickstep_queryexecution_ForemanSingleNode
                       quickstep_queryexecution_PolicyEnforcerBase
                       quickstep_queryexecution_PolicyEnforcerSingleNode
+                      quickstep_queryexecution_ProbabilityStore
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryContext_proto
                       quickstep_queryexecution_QueryExecutionMessages_proto
@@ -425,6 +430,14 @@ if (ENABLE_DISTRIBUTED)
   add_test(BlockLocator_unittest BlockLocator_unittest)
 endif(ENABLE_DISTRIBUTED)
 
+add_executable(ProbabilityStore_unittest
+        "${CMAKE_CURRENT_SOURCE_DIR}/tests/ProbabilityStore_unittest.cpp")
+target_link_libraries(ProbabilityStore_unittest
+                      gtest
+                      gtest_main
+                      quickstep_queryexecution_ProbabilityStore)
+add_test(ProbabilityStore_unittest ProbabilityStore_unittest)
+
 add_executable(QueryManagerSingleNode_unittest
   "${CMAKE_CURRENT_SOURCE_DIR}/tests/QueryManagerSingleNode_unittest.cpp")
 target_link_libraries(QueryManagerSingleNode_unittest

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8f094a1c/query_execution/ProbabilityStore.hpp
----------------------------------------------------------------------
diff --git a/query_execution/ProbabilityStore.hpp b/query_execution/ProbabilityStore.hpp
new file mode 100644
index 0000000..079f60b
--- /dev/null
+++ b/query_execution/ProbabilityStore.hpp
@@ -0,0 +1,263 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_PROBABILITY_STORE_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_PROBABILITY_STORE_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <random>
+#include <unordered_map>
+#include <vector>
+
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/**
+ * @brief A class that stores the probabilities of objects. We use an integer field
+ *        called "key" to identify each object.
+ *        A probability is expressed as a fraction. All the objects share a common denominator.
+ **/
+class ProbabilityStore {
+ public:
+  /**
+   * @brief Constructor.
+   **/
+  ProbabilityStore()
+      : common_denominator_(1.0), dist_(0.0, 1.0), engine_(std::random_device()()) {}
+
+  /**
+   * @brief Get the number of objects in the store.
+   **/
+  inline std::size_t getNumObjects() const {
+    DCHECK_EQ(individual_probabilities_.size(), cumulative_probabilities_.size());
+    return individual_probabilities_.size();
+  }
+
+  /**
+   * @brief Get the common denominator.
+   */
+  inline std::size_t getDenominator() const {
+    return common_denominator_;
+  }
+
+  /**
+   * @brief Check if an object with a given key is present.
+   * @param key The key of the given object.
+   * @return True if the object is present, false otherwise.
+   */
+  inline bool hasObject(const std::size_t key) const {
+    return (individual_probabilities_.find(key) != individual_probabilities_.end());
+  }
+
+  /**
+   * @brief Add individual (not cumulative) probability for a given object with
+   *        updated denominator.
+   *
+   * @note This function leaves the cumulative probabilities in a consistent
+   *       state. An alternative lazy implementation should be written if cost
+   *       of calculating cumulative probabilities is high.
+   *
+   * @param key The key of the given object.
+   * @param numerator The numerator for the given object.
+   * @param new_denominator The updated denominator for the store.
+   **/
+  void addOrUpdateObjectNewDenominator(const std::size_t key,
+                                       const float numerator,
+                                       const float new_denominator) {
+    CHECK_GT(new_denominator, 0u);
+    DCHECK_LE(numerator, new_denominator);
+    common_denominator_ = new_denominator;
+    addOrUpdateObjectHelper(key, numerator);
+  }
+
+  /**
+   * @brief Add or update multiple objects with a new denominator.
+   * @param keys The keys to be added/updated.
+   * @param numerators The numerators to be added/updated.
+   * @param new_denominator The new denominator.
+   */
+  void addOrUpdateObjectsNewDenominator(
+      const std::vector<std::size_t> &keys,
+      const std::vector<float> &numerators,
+      const float new_denominator) {
+    CHECK_GT(new_denominator, 0u);
+    common_denominator_ = new_denominator;
+    addOrUpdateObjectsHelper(keys, numerators);
+  }
+
+  /**
+   * @brief Remove an object from the store.
+   *
+   * @note This function decrements the denominator with the value of the numerator being removed.
+   *
+   * @param key The key of the object to be removed.
+   **/
+  void removeObject(const std::size_t key) {
+    auto individual_it = individual_probabilities_.find(key);
+    DCHECK(individual_it != individual_probabilities_.end());
+    const float new_denominator = common_denominator_ - individual_it->second;
+    individual_probabilities_.erase(individual_it);
+    common_denominator_ = new_denominator;
+    updateCumulativeProbabilities();
+  }
+
+  /**
+   * @brief Get the individual probability (not cumulative) for an object.
+   *
+   * @param key The key of the object.
+   **/
+  const float getIndividualProbability(const std::size_t key) const {
+    const auto it = individual_probabilities_.find(key);
+    DCHECK(it != individual_probabilities_.end());
+    DCHECK_NE(0.0, common_denominator_);
+    return it->second / common_denominator_;
+  }
+
+  /**
+   * @brief Update the cumulative probabilities.
+   *
+   * @note This function should be called upon if there are any updates,
+   *       additions or deletions to the individual probabilities.
+   * @note An efficient implementation should be written if there are large
+   *       number of objects.
+   **/
+  void updateCumulativeProbabilities() {
+    cumulative_probabilities_.clear();
+    float cumulative_probability = 0;
+    for (const auto p : individual_probabilities_) {
+      cumulative_probability += p.second / common_denominator_;
+      cumulative_probabilities_.emplace_back(p.first,
+                                             cumulative_probability);
+    }
+    if (!cumulative_probabilities_.empty()) {
+      // Adjust the last cumulative probability manually to 1, so that
+      // floating addition related rounding issues are ignored.
+      cumulative_probabilities_.back().updateProbability(1);
+    }
+  }
+
+  /**
+   * @brief Return a uniformly chosen random key.
+   **/
+  inline const std::size_t pickRandomKey() const {
+    return getKeyForProbability(dist_(engine_));
+  }
+
+ private:
+  struct ProbabilityInfo {
+   public:
+    ProbabilityInfo(const std::size_t key, const float probability)
+        : key_(key), probability_(probability) {
+      DCHECK(probability_ <= 1.0);
+    }
+
+    void updateProbability(const float new_probability) {
+      probability_ = new_probability;
+    }
+
+    const std::size_t key_;
+    float probability_;
+  };
+
+  /**
+   * @brief Get a key for a given cumulative probability.
+   *
+   * @param key_cumulative_probability The input cumulative probability.
+   *
+   * @return The object that has a cumulative probability that is greater than
+   *         or equal to the input cumulative probability.
+   **/
+  inline std::size_t getKeyForProbability(
+      const float key_cumulative_probability) const {
+    DCHECK(!cumulative_probabilities_.empty());
+    // It doesn't matter in which order the objects are arranged in the
+    // cumulative_probabilities_ vector.
+    ProbabilityInfo search_key(0, key_cumulative_probability);
+    const auto it = std::upper_bound(
+        cumulative_probabilities_.begin(),
+        cumulative_probabilities_.end(),
+        search_key,
+        [](const ProbabilityInfo &a, const ProbabilityInfo &b) {
+          return a.probability_ < b.probability_;
+        });
+    DCHECK(it != std::end(cumulative_probabilities_));
+    return it->key_;
+  }
+
+  /**
+   * @brief Add individual (not cumulative) probability for a given object.
+   *
+   * @note This function leaves the cumulative probabilities in a consistent
+   *       state. An alternative lazy implementation should be written if cost
+   *       of calculating cumulative probabilities is high.
+   * @note This function may override previously written probability values.
+   *
+   * @param key The key of the given object.
+   * @param numerator The numerator for the given object.
+   **/
+  void addOrUpdateObjectHelper(const std::size_t key,
+                               const float numerator) {
+    DCHECK_LE(numerator, common_denominator_);
+    individual_probabilities_[key] = numerator;
+    updateCumulativeProbabilities();
+  }
+
+  /**
+   * @brief Add individual (not cumulative) probabilities for given objects.
+   *
+   * @note This function leaves the cumulative probabilities in a consistent
+   *       state. An alternative lazy implementation should be written if cost
+   *       of calculating cumulative probabilities is high.
+   * @note This function may override previously written probability values.
+   *
+   * @param keys A vector of keys to be added.
+   * @param numerators The numerators of the given objects.
+   **/
+  void addOrUpdateObjectsHelper(const std::vector<std::size_t> &keys,
+                                const std::vector<float> &numerators) {
+    DCHECK_EQ(keys.size(), numerators.size());
+    for (std::size_t i = 0; i < keys.size(); ++i) {
+      DCHECK_LE(numerators[i], common_denominator_);
+      individual_probabilities_[keys[i]] = numerators[i];
+    }
+    updateCumulativeProbabilities();
+  }
+
+  // Key = key of the object.
+  // Value = Numerator of the object.
+  std::unordered_map<std::size_t, float> individual_probabilities_;
+  std::vector<ProbabilityInfo> cumulative_probabilities_;
+
+  float common_denominator_;
+
+  mutable std::uniform_real_distribution<float> dist_;
+  mutable std::default_random_engine engine_;
+
+  DISALLOW_COPY_AND_ASSIGN(ProbabilityStore);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_PROBABILITY_STORE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8f094a1c/query_execution/tests/ProbabilityStore_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/ProbabilityStore_unittest.cpp b/query_execution/tests/ProbabilityStore_unittest.cpp
new file mode 100644
index 0000000..e333f60
--- /dev/null
+++ b/query_execution/tests/ProbabilityStore_unittest.cpp
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "query_execution/ProbabilityStore.hpp"
+
+#include <cstddef>
+#include <vector>
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+
+TEST(ProbabilityStoreTest, CountTest) {
+  ProbabilityStore store;
+  EXPECT_EQ(0u, store.getNumObjects());
+  const std::size_t kKey = 0;
+  store.addOrUpdateObjectNewDenominator(kKey, 1, 1);
+  EXPECT_EQ(1u, store.getNumObjects());
+  store.removeObject(kKey);
+  EXPECT_EQ(0u, store.getNumObjects());
+
+  std::vector<std::size_t> objects {3, 5, 7, 9};
+  std::vector<float> numerators {1, 2, 2, 5};
+  const std::size_t kNewDenominator = std::accumulate(numerators.begin(), numerators.end(), 0);
+  store.addOrUpdateObjectsNewDenominator(objects, numerators, kNewDenominator);
+
+  EXPECT_EQ(objects.size(), store.getNumObjects());
+}
+
+TEST(ProbabilityStoreTest, IndividualProbabilityTest) {
+  ProbabilityStore store;
+  std::vector<std::size_t> objects {3, 5, 7, 9};
+  std::vector<float> numerators {1, 2, 2, 5};
+  const std::size_t kNewDenominator = std::accumulate(numerators.begin(), numerators.end(), 0);
+  store.addOrUpdateObjectsNewDenominator(objects, numerators, kNewDenominator);
+
+  for (std::size_t object_num = 0; object_num < objects.size(); ++object_num) {
+    EXPECT_EQ(numerators[object_num] / static_cast<float>(kNewDenominator),
+              store.getIndividualProbability(objects[object_num]));
+  }
+}
+
+TEST(ProbabilityStoreTest, PickRandomKeyTest) {
+  ProbabilityStore store;
+  std::vector<std::size_t> objects {3, 5, 7, 9};
+  std::vector<float> numerators {1, 2, 2, 5};
+  const std::size_t kNewDenominator = std::accumulate(numerators.begin(), numerators.end(), 0);
+  store.addOrUpdateObjectsNewDenominator(objects, numerators, kNewDenominator);
+
+  const std::size_t kNumTrials = 10;
+  while (!objects.empty()) {
+    for (std::size_t trial_num = 0; trial_num < kNumTrials; ++trial_num) {
+      const std::size_t picked_key = store.pickRandomKey();
+      const auto it = std::find(objects.begin(), objects.end(), picked_key);
+      EXPECT_TRUE(it != objects.end());
+    }
+    const std::size_t key_to_be_removed = objects.back();
+    store.removeObject(key_to_be_removed);
+    objects.pop_back();
+    EXPECT_EQ(objects.size(), store.getNumObjects());
+  }
+}
+
+TEST(ProbabilityStoreTest, RemoveObjectTest) {
+  ProbabilityStore store;
+  std::vector<std::size_t> objects {3, 5, 7, 9};
+  std::vector<float> numerators {1, 2, 2, 5};
+  const std::size_t kNewDenominator = std::accumulate(numerators.begin(), numerators.end(), 0);
+  store.addOrUpdateObjectsNewDenominator(objects, numerators, kNewDenominator);
+
+  for (std::size_t object_num = 0; object_num < objects.size(); ++object_num) {
+    EXPECT_EQ(numerators[object_num] / static_cast<float>(kNewDenominator),
+              store.getIndividualProbability(objects[object_num]));
+  }
+
+  // Remove last object "9", with numerator 5.
+  store.removeObject(objects.back());
+  objects.pop_back();
+  numerators.pop_back();
+  const float expected_new_denominator =
+      std::accumulate(numerators.begin(), numerators.end(), 0);
+
+  EXPECT_EQ(expected_new_denominator, store.getDenominator());
+  for (std::size_t object_num = 0; object_num < objects.size(); ++object_num) {
+    EXPECT_EQ(numerators[object_num] / static_cast<float>(expected_new_denominator),
+              store.getIndividualProbability(objects[object_num]));
+  }
+}
+
+}  // namespace quickstep


[05/27] incubator-quickstep git commit: Fixed the root path check in the validate_cmakelists script.

Posted by ji...@apache.org.
Fixed the root path check in the validate_cmakelists script.


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

Branch: refs/heads/trace
Commit: bf455e26eb89902731f01928f5eff369a875e5f4
Parents: 8d7284d
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Thu Sep 28 19:36:06 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Thu Sep 28 19:36:06 2017 -0500

----------------------------------------------------------------------
 validate_cmakelists.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bf455e26/validate_cmakelists.py
----------------------------------------------------------------------
diff --git a/validate_cmakelists.py b/validate_cmakelists.py
index 0b2e79d..f5f2f89 100755
--- a/validate_cmakelists.py
+++ b/validate_cmakelists.py
@@ -469,7 +469,7 @@ def main(cmakelists_to_process):
         int: The total number of targets that failed validation because of
             missing or superfluous dependencies.
     """
-    if not os.getcwd().endswith("quickstep"):
+    if not os.path.isfile("validate_cmakelists.py"):
         print("WARNING: you don't appear to be running in the root quickstep "
               "source directory. Don't blame me if something goes wrong.")
     qs_module_dirs = []


[25/27] incubator-quickstep git commit: Quickstep for GRAIL.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 9b77875..aca4c87 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -299,29 +299,30 @@ extern int quickstep_yydebug;
     TOKEN_SMA = 366,
     TOKEN_SMALLINT = 367,
     TOKEN_STDERR = 368,
-    TOKEN_STDOUT = 369,
-    TOKEN_SUBSTRING = 370,
-    TOKEN_TABLE = 371,
-    TOKEN_THEN = 372,
-    TOKEN_TIME = 373,
-    TOKEN_TIMESTAMP = 374,
-    TOKEN_TO = 375,
-    TOKEN_TRUE = 376,
-    TOKEN_TUPLESAMPLE = 377,
-    TOKEN_UNBOUNDED = 378,
-    TOKEN_UNIQUE = 379,
-    TOKEN_UPDATE = 380,
-    TOKEN_USING = 381,
-    TOKEN_VALUES = 382,
-    TOKEN_VARCHAR = 383,
-    TOKEN_WHEN = 384,
-    TOKEN_WHERE = 385,
-    TOKEN_WINDOW = 386,
-    TOKEN_WITH = 387,
-    TOKEN_YEAR = 388,
-    TOKEN_YEARMONTH = 389,
-    TOKEN_EOF = 390,
-    TOKEN_LEX_ERROR = 391
+    TOKEN_STDIN = 369,
+    TOKEN_STDOUT = 370,
+    TOKEN_SUBSTRING = 371,
+    TOKEN_TABLE = 372,
+    TOKEN_THEN = 373,
+    TOKEN_TIME = 374,
+    TOKEN_TIMESTAMP = 375,
+    TOKEN_TO = 376,
+    TOKEN_TRUE = 377,
+    TOKEN_TUPLESAMPLE = 378,
+    TOKEN_UNBOUNDED = 379,
+    TOKEN_UNIQUE = 380,
+    TOKEN_UPDATE = 381,
+    TOKEN_USING = 382,
+    TOKEN_VALUES = 383,
+    TOKEN_VARCHAR = 384,
+    TOKEN_WHEN = 385,
+    TOKEN_WHERE = 386,
+    TOKEN_WINDOW = 387,
+    TOKEN_WITH = 388,
+    TOKEN_YEAR = 389,
+    TOKEN_YEARMONTH = 390,
+    TOKEN_EOF = 391,
+    TOKEN_LEX_ERROR = 392
   };
 #endif
 
@@ -432,7 +433,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 436 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 437 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -467,7 +468,7 @@ int quickstep_yyparse (yyscan_t yyscanner, quickstep::ParseStatement **parsedSta
 #include "SqlLexer_gen.hpp"
 void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string &feature);
 
-#line 471 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 472 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -711,21 +712,21 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  50
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1327
+#define YYLAST   1320
 
 /* YYNTOKENS -- Number of terminals.  */
-#define YYNTOKENS  148
+#define YYNTOKENS  149
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  111
+#define YYNNTS  112
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  300
+#define YYNRULES  302
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  555
+#define YYNSTATES  557
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
 #define YYUNDEFTOK  2
-#define YYMAXUTOK   391
+#define YYMAXUTOK   392
 
 #define YYTRANSLATE(YYX)                                                \
   ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK)
@@ -735,11 +736,11 @@ union yyalloc
 static const yytype_uint8 yytranslate[] =
 {
        0,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-     143,     2,     2,     2,     2,     2,     2,     2,     2,     2,
+     144,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,   147,     2,     2,
-     144,   145,    23,    21,   146,    22,    27,    24,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,     2,     2,   142,
+       2,     2,     2,     2,     2,     2,     2,   148,     2,     2,
+     145,   146,    23,    21,   147,    22,    27,    24,     2,     2,
+       2,     2,     2,     2,     2,     2,     2,     2,     2,   143,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
@@ -773,44 +774,44 @@ static const yytype_uint8 yytranslate[] =
      110,   111,   112,   113,   114,   115,   116,   117,   118,   119,
      120,   121,   122,   123,   124,   125,   126,   127,   128,   129,
      130,   131,   132,   133,   134,   135,   136,   137,   138,   139,
-     140,   141
+     140,   141,   142
 };
 
 #if YYDEBUG
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   644,   644,   648,   652,   656,   660,   663,   670,   673,
-     676,   679,   682,   685,   688,   691,   694,   697,   703,   709,
-     716,   722,   729,   738,   743,   752,   757,   762,   766,   772,
-     777,   780,   783,   788,   791,   794,   797,   800,   803,   806,
-     809,   812,   815,   827,   830,   833,   851,   871,   874,   877,
-     882,   887,   893,   899,   908,   912,   918,   921,   926,   931,
-     936,   943,   950,   954,   960,   963,   968,   971,   976,   979,
-     984,   987,  1006,  1009,  1014,  1018,  1024,  1027,  1030,  1033,
-    1038,  1041,  1044,  1051,  1056,  1067,  1072,  1077,  1081,  1085,
-    1091,  1094,  1100,  1108,  1111,  1114,  1120,  1125,  1130,  1134,
-    1140,  1144,  1147,  1152,  1155,  1160,  1165,  1170,  1174,  1180,
-    1189,  1192,  1197,  1200,  1219,  1224,  1228,  1234,  1240,  1249,
-    1254,  1262,  1268,  1274,  1277,  1280,  1285,  1288,  1293,  1297,
-    1303,  1306,  1309,  1314,  1319,  1324,  1327,  1330,  1335,  1338,
-    1341,  1344,  1347,  1350,  1353,  1356,  1361,  1364,  1369,  1373,
-    1377,  1380,  1384,  1387,  1392,  1395,  1400,  1403,  1408,  1412,
-    1418,  1421,  1426,  1429,  1434,  1437,  1442,  1445,  1464,  1467,
-    1472,  1476,  1482,  1488,  1493,  1496,  1501,  1504,  1509,  1512,
-    1517,  1520,  1525,  1526,  1529,  1534,  1535,  1538,  1543,  1547,
-    1553,  1560,  1563,  1566,  1571,  1574,  1577,  1583,  1586,  1591,
-    1596,  1605,  1610,  1619,  1624,  1627,  1632,  1635,  1640,  1646,
-    1652,  1655,  1658,  1661,  1664,  1667,  1673,  1682,  1685,  1690,
-    1693,  1698,  1701,  1706,  1709,  1712,  1715,  1719,  1723,  1726,
-    1729,  1732,  1735,  1740,  1744,  1748,  1751,  1756,  1761,  1765,
-    1771,  1774,  1779,  1783,  1789,  1794,  1798,  1804,  1809,  1812,
-    1817,  1821,  1827,  1830,  1833,  1836,  1848,  1852,  1871,  1884,
-    1899,  1902,  1905,  1908,  1911,  1914,  1919,  1923,  1929,  1933,
-    1939,  1942,  1947,  1951,  1958,  1961,  1964,  1967,  1970,  1973,
-    1976,  1979,  1982,  1985,  1990,  2001,  2004,  2009,  2012,  2015,
-    2021,  2025,  2031,  2034,  2042,  2045,  2048,  2051,  2057,  2062,
-    2067
+       0,   646,   646,   650,   654,   658,   662,   665,   672,   675,
+     678,   681,   684,   687,   690,   693,   696,   699,   705,   711,
+     718,   724,   731,   740,   745,   754,   759,   764,   768,   774,
+     779,   782,   785,   790,   793,   796,   799,   802,   805,   808,
+     811,   814,   817,   829,   832,   835,   853,   873,   876,   879,
+     884,   889,   895,   901,   910,   914,   920,   923,   928,   933,
+     938,   945,   952,   956,   962,   965,   970,   973,   978,   981,
+     986,   989,  1008,  1011,  1016,  1020,  1026,  1029,  1032,  1035,
+    1040,  1043,  1046,  1053,  1058,  1069,  1074,  1079,  1083,  1087,
+    1093,  1096,  1102,  1110,  1113,  1116,  1122,  1127,  1132,  1136,
+    1142,  1146,  1151,  1155,  1158,  1163,  1166,  1171,  1176,  1181,
+    1185,  1191,  1200,  1203,  1208,  1211,  1230,  1235,  1239,  1245,
+    1251,  1260,  1265,  1273,  1279,  1285,  1288,  1291,  1296,  1299,
+    1304,  1308,  1314,  1317,  1320,  1325,  1330,  1335,  1338,  1341,
+    1346,  1349,  1352,  1355,  1358,  1361,  1364,  1367,  1372,  1375,
+    1380,  1384,  1388,  1391,  1395,  1398,  1403,  1406,  1411,  1414,
+    1419,  1423,  1429,  1432,  1437,  1440,  1445,  1448,  1453,  1456,
+    1475,  1478,  1483,  1487,  1493,  1499,  1504,  1507,  1512,  1515,
+    1520,  1523,  1528,  1531,  1536,  1537,  1540,  1545,  1546,  1549,
+    1554,  1558,  1564,  1571,  1574,  1577,  1582,  1585,  1588,  1594,
+    1597,  1602,  1607,  1616,  1621,  1630,  1635,  1638,  1643,  1646,
+    1651,  1657,  1663,  1666,  1669,  1672,  1675,  1678,  1684,  1693,
+    1696,  1701,  1704,  1709,  1712,  1717,  1720,  1723,  1726,  1730,
+    1734,  1737,  1740,  1743,  1746,  1751,  1755,  1759,  1762,  1767,
+    1772,  1776,  1782,  1785,  1790,  1794,  1800,  1805,  1809,  1815,
+    1820,  1823,  1828,  1832,  1838,  1841,  1844,  1847,  1859,  1863,
+    1882,  1895,  1910,  1913,  1916,  1919,  1922,  1925,  1930,  1934,
+    1940,  1944,  1950,  1953,  1958,  1962,  1969,  1972,  1975,  1978,
+    1981,  1984,  1987,  1990,  1993,  1996,  2001,  2012,  2015,  2020,
+    2023,  2026,  2032,  2036,  2042,  2045,  2053,  2056,  2059,  2062,
+    2068,  2073,  2078
 };
 #endif
 
@@ -846,11 +847,11 @@ static const char *const yytname[] =
   "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_STDERR", "TOKEN_STDOUT", "TOKEN_SUBSTRING", "TOKEN_TABLE",
-  "TOKEN_THEN", "TOKEN_TIME", "TOKEN_TIMESTAMP", "TOKEN_TO", "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_STDERR", "TOKEN_STDIN", "TOKEN_STDOUT", "TOKEN_SUBSTRING",
+  "TOKEN_TABLE", "TOKEN_THEN", "TOKEN_TIME", "TOKEN_TIMESTAMP", "TOKEN_TO",
+  "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",
@@ -863,10 +864,10 @@ static const char *const yytname[] =
   "key_value_list", "key_value", "key_string_value", "key_string_list",
   "key_integer_value", "key_bool_value", "index_type",
   "opt_index_properties", "insert_statement", "copy_statement",
-  "copy_to_target", "opt_copy_params", "update_statement",
-  "delete_statement", "assignment_list", "assignment_item",
-  "set_operation_statement", "opt_priority_clause", "with_clause",
-  "with_list", "with_list_element", "set_operation_union",
+  "copy_from_target", "copy_to_target", "opt_copy_params",
+  "update_statement", "delete_statement", "assignment_list",
+  "assignment_item", "set_operation_statement", "opt_priority_clause",
+  "with_clause", "with_list", "with_list_element", "set_operation_union",
   "set_operation_intersect", "select_query", "opt_all_distinct",
   "selection", "selection_item_commalist", "selection_item", "from_clause",
   "subquery_expression", "opt_sample_clause", "join_type",
@@ -912,16 +913,16 @@ static const yytype_uint16 yytoknum[] =
      360,   361,   362,   363,   364,   365,   366,   367,   368,   369,
      370,   371,   372,   373,   374,   375,   376,   377,   378,   379,
      380,   381,   382,   383,   384,   385,   386,   387,   388,   389,
-     390,   391,    59,    10,    40,    41,    44,    37
+     390,   391,   392,    59,    10,    40,    41,    44,    37
 };
 # endif
 
-#define YYPACT_NINF -410
+#define YYPACT_NINF -404
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-410)))
+  (!!((Yystate) == (-404)))
 
-#define YYTABLE_NINF -139
+#define YYTABLE_NINF -141
 
 #define yytable_value_is_error(Yytable_value) \
   0
@@ -930,62 +931,62 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-     405,  -410,  -410,   -52,    43,   -13,    34,   -38,   116,  -410,
-      48,   231,   231,  -410,   115,   119,  -410,  -410,  -410,  -410,
-    -410,  -410,  -410,  -410,  -410,  -410,   166,    46,    49,  -410,
-     -36,   205,   231,  -410,  -410,    38,   132,   231,   231,   231,
-     231,   231,  -410,  -410,   652,    97,    72,  -410,   203,    95,
-    -410,  -410,  -410,   130,   175,    46,    48,   158,  -410,   130,
-    -410,  -410,  -410,    25,    61,   128,   270,   128,   184,   142,
-     149,  -410,   106,  -410,  -410,   309,   313,  -410,  -410,  -410,
-     743,   153,  -410,   218,  -410,  -410,   189,  -410,  -410,   331,
-    -410,  -410,  -410,  -410,   199,  -410,  -410,   207,   272,   837,
-     347,   305,   211,  -410,  -410,   328,    19,  -410,  -410,   262,
-    -410,  -410,  -410,  -410,  -410,  1019,     5,   231,   231,   220,
-     231,    38,   231,  -410,   130,   373,  -410,   148,   243,  -410,
-    -410,  -410,   244,  -410,   128,  -410,   231,   231,   561,  -410,
-    -410,   245,   231,  -410,  -410,  -410,   561,    52,    17,  -410,
-     400,  -410,   140,   140,  1110,   402,  -410,     1,    29,  -410,
-      20,   149,  1110,  -410,  -410,   231,  1110,  -410,  -410,  -410,
-    -410,  1110,    21,   313,  -410,   231,   323,   -64,  -410,   416,
-    -410,   130,  -410,   167,  -410,   128,   130,    49,  -410,   231,
-     152,   231,   231,   231,  -410,   269,  -410,   169,  1177,   928,
-     220,   470,   426,   433,  -410,  -410,   407,   428,  1188,   303,
-     173,    23,  1110,    64,  -410,  1110,  -410,   388,   306,  -410,
-    -410,  -410,  -410,  -410,  -410,   381,  -410,    66,   310,  -410,
-    -410,     8,   209,   215,  -410,   311,   209,     9,   385,  -410,
-    -410,    19,  -410,   360,  -410,  -410,   315,  1110,  -410,   301,
-     180,   231,  -410,  1110,  -410,   231,  -410,  -410,  -410,   320,
-     384,   386,   329,  -410,  -410,  -410,   196,  -410,  -410,  -410,
-    -410,  -410,    16,   231,   341,   152,   231,  -410,   174,  -410,
-    -410,     2,    81,   561,   561,   206,  -410,  -410,  -410,  -410,
-    -410,  -410,  -410,  -410,  1110,   334,  1110,    58,  -410,   210,
-     335,   349,  1110,    68,  -410,   420,   301,  -410,  -410,  1110,
-     476,  -410,   162,   231,  -410,  -410,   389,  -410,   390,   391,
-     406,    20,  -410,   486,   487,   209,   453,   421,   455,   353,
-     403,  -410,   223,  -410,  1110,  -410,   301,  -410,   561,   357,
-     358,   231,  -410,   231,  -410,  -410,  -410,  -410,  -410,  -410,
-    -410,   231,  -410,  -410,  -410,   225,   477,    86,  -410,   361,
-     366,  -410,   417,   364,  1188,  -410,   430,   231,  -410,  -410,
-     174,  -410,  -410,   433,  -410,  -410,  -410,  1110,   367,   284,
-     837,  -410,   301,   424,  -410,  -410,  1188,  1188,   374,   301,
-    1110,  -410,    33,   -16,  -410,  -410,  -410,  -410,  -410,    20,
-     215,   419,   422,  -410,  1110,   561,   429,  1110,  -410,   475,
-      18,  -410,   301,    31,   231,   231,   227,  -410,   233,  -410,
-     231,  -410,  -410,  -410,  -410,   382,   152,   489,   432,  -410,
-     561,  -410,  -410,   392,  -410,   289,   837,  -410,  1110,   238,
-    -410,  -410,   240,  1188,   301,  -410,   521,  -410,   437,  -410,
-    -410,   393,   426,   498,   456,   393,  1110,  -410,  -410,  -410,
-     522,  -410,   246,   252,  -410,  -410,  -410,   231,  -410,  -410,
-     399,   504,  -410,    32,   231,  1110,   254,   301,  -410,  -410,
-     257,   404,   561,  1110,   540,   412,   408,  -410,   274,    11,
-     442,  -410,   286,   231,   127,  -410,   410,   301,  -410,  -410,
-    -410,   426,   408,  -410,   231,  -410,   412,  -410,  1110,  -410,
-    -410,   460,   454,   447,   458,   549,   231,  -410,   290,  -410,
-    -410,   425,  -410,   528,  -410,  -410,    -6,  -410,  -410,  -410,
-    -410,    12,   431,  -410,   231,   434,  -410,  -410,   495,   462,
-     496,  -410,   231,   293,   360,  -410,  -410,  -410,   299,   473,
-     435,  -410,   563,  -410,  -410
+     613,  -404,  -404,   -19,    63,   -26,    96,    -6,    41,  -404,
+      67,    74,    74,  -404,   194,    81,  -404,  -404,  -404,  -404,
+    -404,  -404,  -404,  -404,  -404,  -404,   148,   -12,   174,  -404,
+     -40,   223,    74,  -404,  -404,    -3,   -17,    74,    74,    74,
+      74,    74,  -404,  -404,   679,   122,   103,  -404,   207,   120,
+    -404,  -404,  -404,   169,   210,   -12,    67,   212,  -404,   169,
+    -404,  -404,  -404,    39,    93,   160,    38,   160,   160,   219,
+     159,   182,  -404,   170,  -404,  -404,   313,   315,  -404,  -404,
+    -404,   758,   181,  -404,   253,  -404,  -404,   197,  -404,  -404,
+     341,  -404,  -404,  -404,  -404,   203,  -404,  -404,   221,   301,
+     837,   368,   320,   248,  -404,  -404,   339,    -8,  -404,  -404,
+     298,  -404,  -404,  -404,  -404,  -404,  1022,     5,    74,    74,
+     259,    74,    -3,    74,  -404,   169,   409,  -404,   186,   201,
+    -404,  -404,  -404,   281,  -404,  -404,  -404,  -404,  -404,    74,
+      74,   600,  -404,  -404,   289,    74,  -404,  -404,  -404,   600,
+      19,    66,  -404,   426,  -404,   155,   155,  1101,   428,  -404,
+     -25,    26,  -404,     8,   182,  1101,  -404,  -404,    74,  1101,
+    -404,  -404,  -404,  -404,  1101,    21,   315,  -404,    74,   363,
+     -68,  -404,   425,  -404,   169,  -404,   109,  -404,   160,   169,
+     174,  -404,    74,   136,    74,    74,    74,   292,  -404,   111,
+     200,   943,   259,   494,   430,   431,  -404,  -404,   552,   419,
+    1180,   294,   141,    15,  1101,   151,  -404,  1101,  -404,   381,
+     300,  -404,  -404,  -404,  -404,  -404,  -404,   376,  -404,   258,
+     302,  -404,  -404,    18,   326,   249,  -404,   304,   326,    58,
+     380,  -404,  -404,    -8,  -404,   350,  -404,  -404,   308,  1101,
+    -404,   288,   206,    74,  -404,  1101,  -404,    74,  -404,  -404,
+    -404,   311,   372,   373,   314,  -404,  -404,  -404,   209,  -404,
+    -404,  -404,  -404,  -404,    53,    74,   328,   136,    74,  -404,
+      97,  -404,  -404,    12,    72,   600,   600,   198,  -404,  -404,
+    -404,  -404,  -404,  -404,  -404,  -404,  1101,   316,  1101,    45,
+    -404,   216,   317,   330,  1101,    55,  -404,   402,   288,  -404,
+    -404,  1101,   459,  -404,   218,    74,  -404,  -404,   370,  -404,
+     371,   374,   385,     8,  -404,   466,   467,   326,   434,   403,
+     435,   333,   386,  -404,   229,  -404,  1101,  -404,   288,  -404,
+     600,   336,   338,    74,  -404,    74,  -404,  -404,  -404,  -404,
+    -404,  -404,  -404,    74,  -404,  -404,  -404,   231,   458,    71,
+    -404,   342,   348,  -404,   398,   347,  1180,  -404,   408,    74,
+    -404,  -404,    97,  -404,  -404,   431,  -404,  -404,  -404,  1101,
+     349,   291,   837,  -404,   288,   404,  -404,  -404,  1180,  1180,
+     360,   288,  1101,  -404,    29,    51,  -404,  -404,  -404,  -404,
+    -404,     8,   249,   394,   405,  -404,  1101,   600,   411,  1101,
+    -404,   468,   113,  -404,   288,    23,    74,    74,   233,  -404,
+     235,  -404,    74,  -404,  -404,  -404,  -404,   366,   136,   474,
+     414,  -404,   600,  -404,  -404,   375,  -404,   329,   837,  -404,
+    1101,   241,  -404,  -404,   243,  1180,   288,  -404,   510,  -404,
+     423,  -404,  -404,   377,   430,   477,   433,   377,  1101,  -404,
+    -404,  -404,   503,  -404,   247,   255,  -404,  -404,  -404,    74,
+    -404,  -404,   378,   483,  -404,    30,    74,  1101,   262,   288,
+    -404,  -404,   264,   382,   600,  1101,   519,   390,   384,  -404,
+     274,    14,   427,  -404,   273,    74,     0,  -404,   387,   288,
+    -404,  -404,  -404,   430,   384,  -404,    74,  -404,   390,  -404,
+    1101,  -404,  -404,   441,   437,   432,   439,   527,    74,  -404,
+     278,  -404,  -404,   399,  -404,   514,  -404,  -404,    47,  -404,
+    -404,  -404,  -404,    22,   406,  -404,    74,   407,  -404,  -404,
+     475,   440,   481,  -404,    74,   282,   350,  -404,  -404,  -404,
+     285,   454,   410,  -404,   548,  -404,  -404
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -993,96 +994,96 @@ static const yytype_int16 yypact[] =
      means the default is an error.  */
 static const yytype_uint16 yydefact[] =
 {
-       0,     6,   300,     0,     0,     0,     0,     0,     0,    18,
-     123,     0,     0,     7,     0,     0,    15,     8,    10,    11,
-      13,    14,     9,    17,    12,    16,     0,   112,   119,   121,
-       0,   298,     0,   292,   293,     0,     0,     0,     0,     0,
-       0,     0,   124,   125,     0,     0,   114,   115,     0,   156,
-       1,     3,     2,     0,     0,   112,   123,     0,   110,     0,
-       5,     4,   299,     0,     0,   103,     0,   103,     0,     0,
-     197,    25,     0,   256,   253,     0,   284,   126,    40,    29,
-       0,     0,    30,    31,    34,    36,     0,    37,    39,     0,
-      41,   252,    35,    38,     0,    32,    33,     0,     0,     0,
-       0,     0,   127,   128,   232,   132,   218,   220,   222,   225,
-     228,   229,   230,   224,   223,     0,   270,     0,     0,     0,
-       0,     0,     0,   111,     0,     0,   120,     0,     0,   100,
-     102,   101,     0,    98,   103,    97,     0,     0,     0,   106,
-     198,     0,     0,    94,   254,   255,     0,     0,   248,   245,
-       0,    43,     0,   257,     0,     0,    44,     0,     0,   259,
-       0,   197,     0,   285,   286,     0,     0,   131,   288,   289,
-     287,     0,     0,     0,   221,     0,     0,   197,   108,     0,
-     116,     0,   117,     0,   290,   103,     0,   118,   113,     0,
-       0,     0,     0,     0,    96,    66,    27,     0,     0,     0,
-       0,     0,   199,   201,   203,   205,     0,   223,     0,    93,
-       0,     0,     0,   248,   242,     0,   246,     0,     0,   262,
-     263,   264,   261,   265,   260,     0,   258,     0,     0,   134,
-     231,     0,     0,   158,   147,   133,   152,   135,   160,   129,
-     130,   217,   219,   174,   226,   271,     0,     0,   233,   250,
-       0,     0,   105,     0,   157,     0,    99,    95,    19,     0,
-       0,     0,     0,    20,    21,    22,     0,    74,    76,    77,
-      78,    79,     0,     0,     0,    64,     0,    42,    56,   204,
-     212,     0,     0,     0,     0,     0,   274,   276,   277,   278,
-     279,   275,   280,   282,     0,     0,     0,     0,   266,     0,
-       0,     0,     0,     0,   243,     0,   249,   241,    45,     0,
-       0,    46,   138,     0,   148,   154,   144,   139,   140,   142,
-       0,     0,   151,     0,     0,   150,     0,   162,     0,     0,
-     176,   234,     0,   235,     0,   107,   109,   291,     0,     0,
-       0,     0,   104,     0,    81,    84,    82,   296,   297,   295,
-     294,     0,    80,    85,   272,     0,   270,     0,    63,    65,
-      68,    28,     0,     0,     0,    47,     0,     0,    49,    55,
-      57,    26,   211,   200,   202,   281,   283,     0,     0,     0,
-       0,   213,   210,     0,   209,   268,     0,     0,     0,   247,
-       0,   240,     0,     0,   153,   155,   145,   141,   143,     0,
-     159,     0,     0,   149,     0,     0,   164,     0,   227,     0,
-     178,   236,   251,     0,     0,     0,     0,    75,     0,    67,
-       0,    86,    87,    88,    89,    90,     0,     0,    70,    48,
-       0,    51,    50,     0,    54,     0,     0,   215,     0,     0,
-     208,   267,     0,     0,   244,   237,     0,   238,     0,   136,
-     137,   161,   163,     0,   166,   175,     0,   181,   180,   173,
-       0,    61,     0,     0,    58,    83,   273,     0,    24,    62,
-       0,     0,    23,     0,     0,     0,     0,   206,   214,   269,
-       0,     0,     0,     0,     0,   168,   177,   188,   191,     0,
-       0,    59,     0,     0,     0,    52,     0,   207,   216,    92,
-     239,   146,   165,   167,     0,   122,   169,   170,     0,   192,
-     193,   194,     0,     0,     0,     0,     0,    91,     0,    72,
-      73,     0,    53,     0,   171,   189,     0,   190,   182,   184,
-     183,     0,     0,    69,     0,     0,   195,   196,     0,     0,
-       0,   179,     0,     0,   174,   185,   187,   186,     0,     0,
-       0,    60,     0,   172,    71
+       0,     6,   302,     0,     0,     0,     0,     0,     0,    18,
+     125,     0,     0,     7,     0,     0,    15,     8,    10,    11,
+      13,    14,     9,    17,    12,    16,     0,   114,   121,   123,
+       0,   300,     0,   294,   295,     0,     0,     0,     0,     0,
+       0,     0,   126,   127,     0,     0,   116,   117,     0,   158,
+       1,     3,     2,     0,     0,   114,   125,     0,   112,     0,
+       5,     4,   301,     0,     0,   105,     0,   105,   105,     0,
+       0,   199,    25,     0,   258,   255,     0,   286,   128,    40,
+      29,     0,     0,    30,    31,    34,    36,     0,    37,    39,
+       0,    41,   254,    35,    38,     0,    32,    33,     0,     0,
+       0,     0,     0,   129,   130,   234,   134,   220,   222,   224,
+     227,   230,   231,   232,   226,   225,     0,   272,     0,     0,
+       0,     0,     0,     0,   113,     0,     0,   122,     0,     0,
+     102,   104,   103,     0,    98,   100,   101,    96,    97,     0,
+       0,     0,   108,   200,     0,     0,    94,   256,   257,     0,
+       0,   250,   247,     0,    43,     0,   259,     0,     0,    44,
+       0,     0,   261,     0,   199,     0,   287,   288,     0,     0,
+     133,   290,   291,   289,     0,     0,     0,   223,     0,     0,
+     199,   110,     0,   118,     0,   119,     0,   292,   105,     0,
+     120,   115,     0,     0,     0,     0,     0,    66,    27,     0,
+       0,     0,     0,     0,   201,   203,   205,   207,     0,   225,
+       0,    93,     0,     0,     0,   250,   244,     0,   248,     0,
+       0,   264,   265,   266,   263,   267,   262,     0,   260,     0,
+       0,   136,   233,     0,     0,   160,   149,   135,   154,   137,
+     162,   131,   132,   219,   221,   176,   228,   273,     0,     0,
+     235,   252,     0,     0,   107,     0,   159,     0,    99,    95,
+      19,     0,     0,     0,     0,    20,    21,    22,     0,    74,
+      76,    77,    78,    79,     0,     0,     0,    64,     0,    42,
+      56,   206,   214,     0,     0,     0,     0,     0,   276,   278,
+     279,   280,   281,   277,   282,   284,     0,     0,     0,     0,
+     268,     0,     0,     0,     0,     0,   245,     0,   251,   243,
+      45,     0,     0,    46,   140,     0,   150,   156,   146,   141,
+     142,   144,     0,     0,   153,     0,     0,   152,     0,   164,
+       0,     0,   178,   236,     0,   237,     0,   109,   111,   293,
+       0,     0,     0,     0,   106,     0,    81,    84,    82,   298,
+     299,   297,   296,     0,    80,    85,   274,     0,   272,     0,
+      63,    65,    68,    28,     0,     0,     0,    47,     0,     0,
+      49,    55,    57,    26,   213,   202,   204,   283,   285,     0,
+       0,     0,     0,   215,   212,     0,   211,   270,     0,     0,
+       0,   249,     0,   242,     0,     0,   155,   157,   147,   143,
+     145,     0,   161,     0,     0,   151,     0,     0,   166,     0,
+     229,     0,   180,   238,   253,     0,     0,     0,     0,    75,
+       0,    67,     0,    86,    87,    88,    89,    90,     0,     0,
+      70,    48,     0,    51,    50,     0,    54,     0,     0,   217,
+       0,     0,   210,   269,     0,     0,   246,   239,     0,   240,
+       0,   138,   139,   163,   165,     0,   168,   177,     0,   183,
+     182,   175,     0,    61,     0,     0,    58,    83,   275,     0,
+      24,    62,     0,     0,    23,     0,     0,     0,     0,   208,
+     216,   271,     0,     0,     0,     0,     0,   170,   179,   190,
+     193,     0,     0,    59,     0,     0,     0,    52,     0,   209,
+     218,    92,   241,   148,   167,   169,     0,   124,   171,   172,
+       0,   194,   195,   196,     0,     0,     0,     0,     0,    91,
+       0,    72,    73,     0,    53,     0,   173,   191,     0,   192,
+     184,   186,   185,     0,     0,    69,     0,     0,   197,   198,
+       0,     0,     0,   181,     0,     0,   176,   187,   189,   188,
+       0,     0,     0,    60,     0,   174,    71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -410,  -410,  -410,  -410,  -410,  -410,  -410,  -410,  -148,  -410,
-     376,   216,  -410,  -410,  -264,  -410,  -410,  -410,  -410,  -410,
-    -410,  -409,   236,  -410,  -410,  -410,  -410,  -410,  -410,  -410,
-    -410,    85,    -3,  -410,  -410,  -410,   325,  -410,   526,  -410,
-    -410,   467,   217,   464,   -43,   533,  -410,  -410,   436,  -410,
-    -114,  -410,  -410,  -186,   192,  -184,    -9,  -410,  -410,  -410,
-    -410,  -410,  -410,  -410,    89,    55,  -410,  -410,  -410,  -410,
-    -410,  -410,   109,    92,  -410,  -410,    79,  -410,  -142,   318,
-     322,   409,   -42,   441,   439,   501,  -154,  -410,  -410,  -410,
-    -410,   398,  -410,   469,   413,  -232,  -195,   449,  -324,  -410,
-    -129,  -410,  -410,  -410,  -410,  -410,  -141,    -4,  -410,  -410,
-    -410
+    -404,  -404,  -404,  -404,  -404,  -404,  -404,  -404,  -150,  -404,
+     359,   188,  -404,  -404,  -267,  -404,  -404,  -404,  -404,  -404,
+    -404,  -403,   227,  -404,  -404,  -404,  -404,  -404,  -404,  -404,
+    -404,  -404,    17,    -5,  -404,  -404,  -404,   324,  -404,   523,
+    -404,  -404,   460,   238,   455,   -29,   525,  -404,  -404,   417,
+    -404,  -102,  -404,  -404,  -194,   183,  -193,    -9,  -404,  -404,
+    -404,  -404,  -404,  -404,  -404,    77,    42,  -404,  -404,  -404,
+    -404,  -404,  -404,   102,    79,  -404,  -404,    43,  -404,  -147,
+     305,   306,   392,   -35,   422,   420,   479,  -158,  -404,  -404,
+    -404,  -404,   383,  -404,   445,   388,  -236,  -204,   443,  -340,
+    -404,  -130,  -404,  -404,  -404,  -404,  -404,  -144,    -4,  -404,
+    -404,  -404
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
-      -1,    14,    15,    16,    17,    18,    19,    20,   196,   197,
-     100,   369,   370,   371,   263,   359,   360,   274,   428,   472,
-     521,   266,   267,   268,   269,   270,   271,   425,   468,    21,
-      22,    65,   133,    23,    24,   177,   178,    25,    58,    26,
-      46,    47,   157,    28,    29,    44,   101,   102,   103,   161,
-     104,   325,   320,   233,   234,   314,   315,   235,   327,   406,
-     454,   485,   505,   506,   507,   329,   330,   410,   459,   460,
-     515,   541,   486,   487,   511,   527,   139,   140,   202,   203,
-     204,   205,   206,   106,   107,   108,   109,   110,   111,   112,
-     213,   214,   148,   149,   217,   250,   113,   225,   299,   209,
-     114,   355,   296,   115,   166,   171,   183,   116,   353,    30,
-      31
+      -1,    14,    15,    16,    17,    18,    19,    20,   198,   199,
+     101,   371,   372,   373,   265,   361,   362,   276,   430,   474,
+     523,   268,   269,   270,   271,   272,   273,   427,   470,    21,
+      22,    67,    65,   134,    23,    24,   180,   181,    25,    58,
+      26,    46,    47,   160,    28,    29,    44,   102,   103,   104,
+     164,   105,   327,   322,   235,   236,   316,   317,   237,   329,
+     408,   456,   487,   507,   508,   509,   331,   332,   412,   461,
+     462,   517,   543,   488,   489,   513,   529,   142,   143,   204,
+     205,   206,   207,   208,   107,   108,   109,   110,   111,   112,
+     113,   215,   216,   151,   152,   219,   252,   114,   227,   301,
+     211,   115,   357,   298,   116,   169,   174,   186,   117,   355,
+      30,    31
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
@@ -1090,276 +1091,276 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      36,   210,   105,    48,   211,   182,   236,    45,    49,   207,
-     283,   358,    33,   298,    34,   332,   126,   207,   512,   538,
-      33,   344,    34,   345,    33,    33,    34,    34,    63,   143,
-      56,   283,   175,    68,    69,    70,    71,    72,   147,   283,
-     283,   258,   168,   169,   346,   312,   232,    33,   323,    34,
-     163,   164,   322,   446,   163,   164,   127,   158,   492,   281,
-     536,   513,   539,   442,   135,    37,   129,    56,   383,    32,
-     207,   138,   207,   163,   164,    56,    42,   236,   215,    59,
-     537,   347,   251,    40,   518,   128,   280,   163,   164,   163,
-     164,   285,   286,   287,   288,   289,   290,   291,   292,   293,
-     294,   167,   163,   164,    60,    39,    43,    61,    38,    48,
-     348,   349,   227,   179,    49,    50,   184,   232,   186,   480,
-     105,    67,    10,   421,   457,   215,   422,   423,   361,   447,
-     458,   194,   195,   198,   249,   400,   324,   310,   184,   514,
-     540,   403,   350,   257,   354,   302,   229,   372,   439,   176,
-     384,   146,   231,   176,   207,   207,   237,    10,   295,   282,
-     351,   240,   469,    64,   231,   243,   170,   236,   244,   431,
-     303,   245,   451,   306,   230,   455,   461,   495,   445,   130,
-     131,   381,   256,    57,   362,   198,   212,   264,   265,   272,
-     390,   441,   298,   219,   189,   190,   413,   259,   212,    41,
-     416,   519,   424,    66,   476,   249,   185,   232,    62,   207,
-     418,   336,   117,    33,    53,    34,   220,    27,   118,   363,
-      10,    35,   260,   375,   376,   377,   230,   237,    49,   364,
-     221,   222,    49,   520,   316,    33,   119,    34,   141,   120,
-     238,   317,   313,    55,    10,   236,    54,   179,   298,   318,
-     142,   337,   379,   223,   382,   261,   252,    64,   122,    51,
-     389,    52,   125,   452,   437,   132,   365,   392,   352,   356,
-     121,   319,   198,   462,   463,   134,   207,   366,   224,   136,
-      10,   262,   367,   378,   138,   232,   137,   316,   473,   191,
-     192,   466,   412,   438,   317,   163,   164,   150,   475,  -138,
-     151,   207,   318,   368,   395,   163,   164,   394,   509,    49,
-     163,   164,   254,   255,   275,   276,   144,   237,   301,   255,
-     145,    49,   163,   164,   319,   333,   334,    33,    73,    34,
-      74,   510,    33,   152,    34,   435,   153,   184,   249,   272,
-     501,   342,   343,   154,    75,    76,   246,   184,   444,   163,
-     164,   155,   159,   207,   156,   385,   386,   162,    78,    79,
-     172,   165,   249,   433,   181,   249,    80,    81,   411,   334,
-     419,   420,   464,   255,    82,    83,   160,    84,   465,   255,
-     188,   247,    85,   478,   334,   479,   386,    86,   193,   208,
-      87,   490,   255,   543,   249,   237,   477,   491,   255,   498,
-     334,   548,   499,   386,    88,    89,     1,   218,     2,   228,
-     184,   184,    90,   273,   488,    91,   356,   285,   286,   287,
-     288,   289,   290,   291,   292,   293,   294,   253,   163,   164,
-      92,   517,   343,   497,   283,   533,   343,     3,   549,   255,
-      93,   488,   284,    94,   551,   255,    95,    96,   297,   300,
-     307,   308,   309,     4,     5,   311,    97,   321,   326,   328,
-     331,     6,    98,   272,   338,     7,   488,    99,   248,   339,
-     496,   340,   357,   341,    33,    73,    34,    74,   380,   387,
-     199,   388,   391,   393,   295,     8,   396,   397,   398,   272,
-     399,    75,    76,   401,   402,   404,   405,   407,   408,   409,
-     523,   414,   415,   427,   175,    78,    79,   426,   430,   429,
-       9,   436,   532,    80,    81,   432,   440,   456,   443,    10,
-     449,    82,    83,   450,    84,   453,   467,   470,   481,    85,
-     184,   471,   482,   200,    86,    11,   474,    87,   184,   334,
-     483,   489,    12,   493,   484,    13,   494,   503,   504,   500,
-     516,    88,    89,   526,   508,   522,   528,   529,   531,    90,
-     530,   535,    91,   545,   547,    33,    73,    34,    74,   534,
-     554,   199,   546,   552,   278,   542,   335,    92,   544,   417,
-     553,   123,    75,    76,    10,   180,   434,    93,   187,   124,
-      94,   448,   502,    95,    96,   524,    78,    79,   239,   550,
-     525,   373,   226,    97,    80,    81,   374,   241,   279,    98,
-     242,   304,    82,    83,   201,    84,   174,   216,     0,     0,
-      85,     0,     0,     0,   200,    86,   305,     0,    87,     0,
+      36,   212,   213,    48,    56,   238,   300,    45,    49,   106,
+     360,   209,    33,   334,    34,   171,   172,    56,   185,   209,
+     285,   514,    33,   285,    34,    33,    56,    34,    63,   540,
+     127,   285,   178,    69,    70,    71,    72,    73,   285,   314,
+     166,   167,   260,   135,   146,   324,   150,   166,   167,   444,
+     166,   167,    37,    68,    66,   385,   283,    33,   346,    34,
+     347,   234,   137,   138,   515,   161,   494,    33,   141,    34,
+     128,   209,   541,   209,   521,   238,   166,   167,    33,   253,
+      34,   348,   287,   288,   289,   290,   291,   292,   293,   294,
+     295,   296,   520,   166,   167,    42,    38,   325,   130,   129,
+     282,    60,   170,    32,    61,   482,   522,   364,   423,    64,
+      48,   424,   425,   538,   182,    49,    40,   187,   349,   189,
+     448,   231,   229,    64,    41,    43,    57,   217,   363,   402,
+     106,   234,    10,   539,   405,   197,   200,   386,   304,   188,
+     173,   187,   365,   516,   251,   356,   441,   350,   351,   297,
+     179,   542,   366,   233,   214,   209,   209,   136,   374,   239,
+     259,   471,   433,   233,   242,   238,   245,    39,   284,   463,
+     453,   246,   232,   457,   247,   447,   497,    10,   392,   305,
+     352,   261,   308,   258,   443,   300,   326,   426,   200,   367,
+     266,   267,   274,   415,    50,   383,    53,   449,   353,   418,
+     368,   149,   478,   179,    59,   369,   262,   240,   221,   420,
+     209,   131,   217,   132,   251,   377,   378,   379,   232,   459,
+     338,   234,    51,   254,    52,   460,    62,   370,    54,   239,
+      49,   222,   192,   193,    49,    79,    80,   118,    27,   263,
+     120,   300,    35,   238,    82,   223,   224,   194,   195,   182,
+     119,    83,    84,   339,    85,   256,   257,   277,   278,    86,
+     454,   381,    10,   384,    55,   121,   264,    88,   225,   391,
+     354,   358,   464,   465,   200,   380,   394,   209,   439,   166,
+     167,    89,   279,    10,    10,   475,   214,   303,   257,    91,
+     318,   122,   468,   123,   226,   166,   167,   319,   133,   234,
+     440,   414,   209,   144,   140,   320,   397,    93,   511,   166,
+     167,    49,   166,   167,   139,   145,   126,    94,   141,   239,
+     147,   318,   148,    49,    96,    97,   153,   321,   319,   312,
+      33,   512,    34,  -140,    98,   154,   320,   503,   477,   187,
+      99,   274,   155,    33,   437,    34,   156,   251,   157,   187,
+     166,   167,   335,   336,   209,   344,   345,   446,   321,   315,
+     166,   167,   387,   388,   396,   435,   158,    33,    74,    34,
+      75,   251,   168,   162,   251,   413,   336,   421,   422,   466,
+     257,   467,   257,   159,    76,    77,   248,   480,   336,   481,
+     388,   163,   545,   492,   257,   165,   175,   239,    79,    80,
+     550,   493,   257,   251,   184,   479,    81,    82,   500,   336,
+     501,   388,   187,   187,    83,    84,   191,    85,   358,   519,
+     345,   249,    86,   490,   535,   345,   196,    87,   551,   257,
+      88,   553,   257,   220,   210,   230,   255,   275,   285,   299,
+     286,   302,   499,   309,    89,    90,   310,   311,   313,   330,
+     490,   323,    91,   328,   333,    92,   340,   341,   342,   343,
+     359,   382,   389,   390,   393,   274,   395,   398,   399,   401,
+      93,   400,   498,   403,   404,   490,   406,   409,   407,   410,
+      94,   416,   411,   417,    95,   178,   429,    96,    97,   428,
+     431,   274,   432,   434,   438,   451,   442,    98,    33,    74,
+      34,    75,   525,    99,   201,   445,   452,   455,   100,   250,
+     458,   469,   472,   473,   534,    76,    77,   483,   484,   485,
+     476,   486,   491,   495,   336,   496,   505,   506,   502,    79,
+      80,   510,   187,   524,   528,   518,   533,    81,    82,   530,
+     187,   532,   531,   547,   536,    83,    84,   537,    85,   549,
+     548,   544,   546,    86,   554,   556,   555,   202,    87,   280,
+     436,    88,   287,   288,   289,   290,   291,   292,   293,   294,
+     295,   296,   419,   166,   167,    89,    90,   337,   124,   183,
+     190,   125,   241,    91,   450,   526,    92,   504,   552,   527,
+     375,   243,   376,   281,   244,   177,   218,     0,   306,   228,
+       0,    93,     0,   307,    33,    74,    34,    75,    10,     0,
+     201,    94,     0,     0,     1,    95,     2,     0,    96,    97,
+       0,    76,    77,     0,     0,     0,     0,     0,    98,   297,
+       0,     0,     0,     0,    99,    79,    80,     0,     0,   203,
+       0,     0,     0,    81,    82,     3,     0,     0,     0,     0,
+       0,    83,    84,     0,    85,     0,     0,     0,     0,    86,
+       0,     4,     5,   202,    87,     0,     0,    88,     0,     6,
+       0,     0,     0,     7,     0,     0,     0,     0,     0,     0,
+       0,    89,    90,    33,    74,    34,    75,     0,     0,    91,
+       0,     0,    92,     8,     0,     0,     0,     0,     0,     0,
+      76,    77,    78,     0,     0,     0,     0,    93,     0,     0,
+       0,     0,     0,     0,    79,    80,     0,    94,     9,     0,
+       0,    95,    81,    82,    96,    97,     0,    10,     0,     0,
+      83,    84,     0,    85,    98,     0,     0,     0,    86,     0,
+      99,     0,     0,    87,    11,   203,    88,     0,     0,     0,
+       0,    12,     0,     0,    13,     0,     0,     0,     0,     0,
+      89,    90,    33,    74,    34,    75,     0,     0,    91,     0,
+       0,    92,     0,     0,     0,     0,     0,     0,     0,    76,
+      77,     0,     0,     0,     0,     0,    93,     0,     0,     0,
+       0,     0,     0,    79,    80,     0,    94,     0,     0,     0,
+      95,    81,    82,    96,    97,     0,     0,     0,     0,    83,
+      84,     0,    85,    98,     0,     0,     0,    86,     0,    99,
+       0,     0,    87,     0,   100,    88,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,    89,
+      90,    33,    74,    34,    75,     0,     0,    91,     0,     0,
+      92,     0,     0,     0,     0,     0,     0,     0,    76,    77,
+       0,     0,     0,     0,     0,    93,     0,     0,     0,     0,
+       0,     0,    79,    80,     0,    94,     0,     0,     0,    95,
+      81,    82,    96,    97,     0,     0,     0,     0,    83,    84,
+       0,    85,    98,   149,     0,     0,    86,     0,    99,     0,
+       0,    87,     0,   100,    88,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,    89,    90,
+       0,     0,     0,     0,     0,     0,    91,     0,     0,    92,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,    88,    89,     0,     0,     0,     0,     0,     0,
-      90,     0,     0,    91,     0,     0,    33,    73,    34,    74,
-       0,     0,     0,     0,     0,     0,     0,     0,    92,     0,
-       0,     0,     0,    75,    76,    77,     0,     0,    93,     0,
-       0,    94,     0,     0,    95,    96,     0,    78,    79,     0,
-       0,     0,     0,     0,    97,    80,    81,     0,     0,     0,
-      98,     0,     0,    82,    83,   201,    84,     0,     0,     0,
-       0,    85,     0,     0,     0,     0,    86,     0,     0,    87,
+       0,     0,     0,     0,    93,     0,     0,    33,    74,    34,
+      75,    10,     0,     0,    94,     0,     0,     0,    95,     0,
+       0,    96,    97,     0,    76,    77,     0,     0,     0,     0,
+       0,    98,     0,     0,     0,     0,     0,    99,    79,    80,
+       0,     0,   100,     0,     0,     0,    81,    82,     0,     0,
+       0,     0,     0,     0,    83,    84,     0,    85,     0,     0,
+       0,     0,    86,     0,     0,     0,   202,    87,     0,     0,
+      88,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,    89,    90,    33,    74,    34,    75,
+       0,     0,    91,     0,     0,    92,     0,     0,     0,     0,
+       0,     0,     0,    76,   176,     0,     0,     0,     0,     0,
+      93,     0,     0,     0,     0,     0,     0,    79,    80,     0,
+      94,     0,     0,     0,    95,    81,    82,    96,    97,     0,
+       0,     0,     0,    83,    84,     0,    85,    98,     0,     0,
+       0,    86,     0,    99,     0,     0,    87,     0,   203,    88,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,    88,    89,     0,     0,     0,     0,     0,
-       0,    90,     0,     0,    91,     0,     0,    33,    73,    34,
-      74,     0,     0,     0,     0,     0,     0,     0,     0,    92,
-       0,     0,     0,     0,    75,    76,     0,     0,     0,    93,
-       0,     0,    94,     0,     0,    95,    96,     0,    78,    79,
-       0,     0,     0,     0,     0,    97,    80,    81,     0,     0,
-       0,    98,     0,     0,    82,    83,    99,    84,     0,     0,
-       0,     0,    85,     0,     0,     0,     0,    86,     0,     0,
-      87,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    88,    89,     0,     0,     0,     0,
-       0,     0,    90,     0,     0,    91,     0,     0,     0,     0,
-       0,    33,    73,    34,    74,     0,     0,     0,     0,     0,
-      92,     0,     0,     0,     0,     0,     0,     0,    75,    76,
-      93,     0,     0,    94,     0,     0,    95,    96,     0,     0,
-       0,     0,    78,    79,     0,     0,    97,   146,     0,     0,
-      80,    81,    98,     0,     0,     0,     0,    99,    82,    83,
-       0,    84,     0,     0,     0,     0,    85,     0,     0,     0,
-       0,    86,     0,     0,    87,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    88,    89,
-       0,     0,     0,     0,     0,     0,    90,     0,     0,    91,
-       0,     0,    33,    73,    34,    74,     0,     0,     0,     0,
-       0,     0,     0,     0,    92,     0,     0,     0,     0,    75,
-      76,    10,     0,     0,    93,     0,     0,    94,     0,     0,
-      95,    96,     0,    78,    79,     0,     0,     0,     0,     0,
-      97,    80,    81,     0,     0,     0,    98,     0,     0,    82,
-      83,    99,    84,     0,     0,     0,     0,    85,     0,     0,
-       0,   200,    86,     0,     0,    87,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,     0,    88,
-      89,     0,     0,     0,     0,     0,     0,    90,     0,     0,
-      91,     0,     0,    33,    73,    34,    74,     0,     0,     0,
-       0,     0,     0,     0,     0,    92,     0,     0,     0,     0,
-      75,   173,     0,     0,     0,    93,     0,     0,    94,     0,
-       0,    95,    96,     0,    78,    79,     0,     0,     0,     0,
-       0,    97,    80,    81,     0,     0,     0,    98,     0,     0,
-      82,    83,   201,    84,     0,     0,     0,     0,    85,     0,
-       0,     0,     0,    86,     0,     0,    87,     0,     0,     0,
+       0,     0,     0,    89,    90,    33,    74,    34,    75,     0,
+       0,    91,     0,     0,    92,     0,     0,     0,     0,     0,
+       0,     0,    76,    77,     0,     0,     0,     0,     0,    93,
+       0,     0,     0,     0,     0,     0,    79,    80,     0,    94,
+       0,     0,     0,    95,    81,    82,    96,    97,     0,     0,
+       0,     0,    83,    84,     0,    85,    98,     0,     0,     0,
+      86,     0,    99,     0,     0,    87,     0,   100,    88,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-      88,    89,     0,     0,     0,     0,     0,     0,    90,     0,
-       0,    91,     0,     0,    33,    73,    34,    74,     0,     0,
-       0,     0,     0,     0,     0,     0,    92,     0,     0,     0,
-       0,    75,    76,     0,     0,     0,    93,     0,     0,    94,
-       0,     0,    95,    96,     0,    78,    79,     0,     0,     0,
-       0,     0,    97,    80,    81,     0,     0,     0,    98,     0,
-       0,    82,    83,    99,    84,     0,     0,     0,     0,    85,
-       0,     0,     0,     0,    86,     0,     0,    87,     0,     0,
+       0,     0,    89,    90,     0,    74,     0,    75,     0,     0,
+      91,     0,     0,    92,     0,     0,     0,     0,     0,     0,
+       0,    76,   176,     0,     0,     0,     0,     0,    93,     0,
+       0,     0,     0,     0,     0,    79,    80,     0,    94,     0,
+       0,     0,    95,     0,    82,    96,    97,     0,     0,     0,
+       0,    83,    84,     0,    85,    98,     0,     0,     0,    86,
+       0,    99,     0,     0,     0,     0,   100,    88,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    88,    89,    73,     0,    74,     0,     0,     0,    90,
-       0,     0,    91,     0,     0,     0,     0,     0,     0,    75,
-     173,     0,    78,    79,     0,     0,     0,    92,     0,     0,
-       0,    81,     0,    78,    79,     0,     0,    93,    82,    83,
-      94,    84,    81,    95,    96,     0,    85,     0,     0,    82,
-      83,     0,    84,    97,    87,     0,     0,    85,     0,    98,
-       0,     0,     0,     0,    99,    87,     0,     0,    88,   277,
-       0,     0,     0,     0,     0,     0,    90,     0,     0,    88,
-      89,     0,     0,     0,     0,     0,     0,    90,     0,     0,
-      91,     0,     0,     0,    92,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    93,    92,     0,     0,     0,     0,
-      95,    96,     0,     0,     0,    93,     0,     0,     0,     0,
-      97,    95,    96,     0,     0,     0,    98,     0,     0,     0,
-       0,    97,     0,     0,     0,     0,     0,    98
+       0,    89,    90,     0,     0,     0,     0,     0,     0,    91,
+       0,     0,    92,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    93,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    94,     0,     0,
+       0,     0,     0,     0,    96,    97,     0,     0,     0,     0,
+       0,     0,     0,     0,    98,     0,     0,     0,     0,     0,
+      99
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   142,    44,    12,   146,   119,   160,    11,    12,   138,
-       8,   275,     4,   208,     6,   247,    59,   146,     7,     7,
-       4,     5,     6,     7,     4,     4,     6,     6,    32,    72,
-      29,     8,    27,    37,    38,    39,    40,    41,    80,     8,
-       8,   189,    23,    24,    28,   231,   160,     4,    39,     6,
-      21,    22,   236,    69,    21,    22,    31,    99,   467,   201,
-      66,    50,    50,   387,    67,    78,     5,    29,    10,   121,
-     199,   135,   201,    21,    22,    29,    28,   231,    61,    30,
-      86,    65,   146,   121,   493,    60,   200,    21,    22,    21,
-      22,    10,    11,    12,    13,    14,    15,    16,    17,    18,
-      19,   105,    21,    22,   140,    71,    58,   143,   121,   118,
-      94,    95,   154,   117,   118,     0,   120,   231,   122,   443,
-     162,    36,   114,    37,   106,    61,    40,    41,   276,   145,
-     112,   134,   136,   137,   176,   321,   127,    71,   142,   128,
-     128,   325,   126,   186,   273,   122,   145,   145,   380,   144,
-      92,   134,   144,   144,   283,   284,   160,   114,    77,   201,
-     144,   165,   426,   125,   144,   144,   147,   321,   172,   364,
-     212,   175,   404,   215,   145,   407,   145,   145,   145,   118,
-     119,   295,   185,   137,    10,   189,   134,   191,   192,   193,
-     122,   386,   387,    53,    46,    47,   338,    45,   134,    83,
-     341,    74,   116,    71,   436,   247,   121,   321,     3,   338,
-     351,   253,   115,     4,    48,     6,    76,     0,   146,    45,
-     114,     4,    70,    17,    18,    19,   145,   231,   232,    55,
-      90,    91,   236,   106,    72,     4,    33,     6,   132,   144,
-     161,    79,    33,    26,   114,   399,    80,   251,   443,    87,
-     144,   255,   294,   113,   296,   103,   177,   125,    83,   140,
-     302,   142,   104,   405,   378,   137,    92,   309,   272,   273,
-      53,   109,   276,   414,   415,     5,   405,   103,   138,    95,
-     114,   129,   108,    77,   135,   399,   144,    72,   430,    46,
-      47,   420,   334,     9,    79,    21,    22,   144,     9,    84,
-      82,   430,    87,   129,   313,    21,    22,   145,    34,   313,
-      21,    22,   145,   146,   145,   146,     7,   321,   145,   146,
-       7,   325,    21,    22,   109,   145,   146,     4,     5,     6,
-       7,    57,     4,   144,     6,   377,     5,   341,   380,   343,
-     482,   145,   146,   144,    21,    22,    23,   351,   390,    21,
-      22,   144,     5,   482,    82,   145,   146,   146,    35,    36,
-      98,    33,   404,   367,   144,   407,    43,    44,   145,   146,
-     145,   146,   145,   146,    51,    52,    71,    54,   145,   146,
-       7,    58,    59,   145,   146,   145,   146,    64,   144,   144,
-      67,   145,   146,   534,   436,   399,   438,   145,   146,   145,
-     146,   542,   145,   146,    81,    82,     1,     7,     3,     7,
-     414,   415,    89,   144,   456,    92,   420,    10,    11,    12,
-      13,    14,    15,    16,    17,    18,    19,    11,    21,    22,
-     107,   145,   146,   475,     8,   145,   146,    32,   145,   146,
-     117,   483,     9,   120,   145,   146,   123,   124,    20,   146,
-      62,   145,    71,    48,    49,   145,   133,   146,    73,    99,
-     145,    56,   139,   467,   144,    60,   508,   144,   145,    85,
-     474,    85,   131,   144,     4,     5,     6,     7,   144,   144,
-      10,   132,    62,     7,    77,    80,    97,    97,    97,   493,
-      84,    21,    22,     7,     7,    42,    75,    42,   145,    96,
-     504,   144,   144,   137,    27,    35,    36,   146,   144,    92,
-     105,   144,   516,    43,    44,    85,    92,    42,   144,   114,
-     101,    51,    52,   101,    54,    96,   144,    38,     7,    59,
-     534,    99,    95,    63,    64,   130,   144,    67,   542,   146,
-      42,    19,   137,   144,    88,   140,    42,     7,   136,   145,
-     108,    81,    82,    93,   146,   145,   102,   110,     9,    89,
-     102,    33,    92,    68,    68,     4,     5,     6,     7,   144,
-       7,    10,   110,   100,   198,   144,   251,   107,   144,   343,
-     145,    55,    21,    22,   114,   118,   370,   117,   124,    56,
-     120,   399,   483,   123,   124,   506,    35,    36,   162,   544,
-     508,   283,   153,   133,    43,    44,   284,   166,   199,   139,
-     171,   213,    51,    52,   144,    54,   115,   148,    -1,    -1,
-      59,    -1,    -1,    -1,    63,    64,   213,    -1,    67,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,
-      89,    -1,    -1,    92,    -1,    -1,     4,     5,     6,     7,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,
-      -1,    -1,    -1,    21,    22,    23,    -1,    -1,   117,    -1,
-      -1,   120,    -1,    -1,   123,   124,    -1,    35,    36,    -1,
-      -1,    -1,    -1,    -1,   133,    43,    44,    -1,    -1,    -1,
-     139,    -1,    -1,    51,    52,   144,    54,    -1,    -1,    -1,
-      -1,    59,    -1,    -1,    -1,    -1,    64,    -1,    -1,    67,
+       4,   145,   149,    12,    29,   163,   210,    11,    12,    44,
+     277,   141,     4,   249,     6,    23,    24,    29,   120,   149,
+       8,     7,     4,     8,     6,     4,    29,     6,    32,     7,
+      59,     8,    27,    37,    38,    39,    40,    41,     8,   233,
+      21,    22,   192,     5,    73,   238,    81,    21,    22,   389,
+      21,    22,    78,    36,    71,    10,   203,     4,     5,     6,
+       7,   163,    67,    68,    50,   100,   469,     4,   136,     6,
+      31,   201,    50,   203,    74,   233,    21,    22,     4,   147,
+       6,    28,    10,    11,    12,    13,    14,    15,    16,    17,
+      18,    19,   495,    21,    22,    28,   122,    39,     5,    60,
+     202,   141,   106,   122,   144,   445,   106,    10,    37,   126,
+     119,    40,    41,    66,   118,   119,   122,   121,    65,   123,
+      69,   146,   157,   126,    83,    58,   138,    61,   278,   323,
+     165,   233,   114,    86,   327,   139,   140,    92,   123,   122,
+     148,   145,    45,   129,   179,   275,   382,    94,    95,    77,
+     145,   129,    55,   145,   135,   285,   286,   119,   146,   163,
+     189,   428,   366,   145,   168,   323,   145,    71,   203,   146,
+     406,   175,   146,   409,   178,   146,   146,   114,   123,   214,
+     127,    45,   217,   188,   388,   389,   128,   116,   192,    92,
+     194,   195,   196,   340,     0,   297,    48,   146,   145,   343,
+     103,   135,   438,   145,    30,   108,    70,   164,    53,   353,
+     340,   118,    61,   120,   249,    17,    18,    19,   146,   106,
+     255,   323,   141,   180,   143,   112,     3,   130,    80,   233,
+     234,    76,    46,    47,   238,    35,    36,   115,     0,   103,
+      33,   445,     4,   401,    44,    90,    91,    46,    47,   253,
+     147,    51,    52,   257,    54,   146,   147,   146,   147,    59,
+     407,   296,   114,   298,    26,   145,   130,    67,   113,   304,
+     274,   275,   416,   417,   278,    77,   311,   407,   380,    21,
+      22,    81,    82,   114,   114,   432,   135,   146,   147,    89,
+      72,    53,   422,    83,   139,    21,    22,    79,   138,   401,
+       9,   336,   432,   133,   145,    87,   315,   107,    34,    21,
+      22,   315,    21,    22,    95,   145,   104,   117,   136,   323,
+       7,    72,     7,   327,   124,   125,   145,   109,    79,    71,
+       4,    57,     6,    84,   134,    82,    87,   484,     9,   343,
+     140,   345,   145,     4,   379,     6,     5,   382,   145,   353,
+      21,    22,   146,   147,   484,   146,   147,   392,   109,    33,
+      21,    22,   146,   147,   146,   369,   145,     4,     5,     6,
+       7,   406,    33,     5,   409,   146,   147,   146,   147,   146,
+     147,   146,   147,    82,    21,    22,    23,   146,   147,   146,
+     147,    71,   536,   146,   147,   147,    98,   401,    35,    36,
+     544,   146,   147,   438,   145,   440,    43,    44,   146,   147,
+     146,   147,   416,   417,    51,    52,     7,    54,   422,   146,
+     147,    58,    59,   458,   146,   147,   145,    64,   146,   147,
+      67,   146,   147,     7,   145,     7,    11,   145,     8,    20,
+       9,   147,   477,    62,    81,    82,   146,    71,   146,    99,
+     485,   147,    89,    73,   146,    92,   145,    85,    85,   145,
+     132,   145,   145,   133,    62,   469,     7,    97,    97,    84,
+     107,    97,   476,     7,     7,   510,    42,    42,    75,   146,
+     117,   145,    96,   145,   121,    27,   138,   124,   125,   147,
+      92,   495,   145,    85,   145,   101,    92,   134,     4,     5,
+       6,     7,   506,   140,    10,   145,   101,    96,   145,   146,
+      42,   145,    38,    99,   518,    21,    22,     7,    95,    42,
+     145,    88,    19,   145,   147,    42,     7,   137,   146,    35,
+      36,   147,   536,   146,    93,   108,     9,    43,    44,   102,
+     544,   102,   110,    68,   145,    51,    52,    33,    54,    68,
+     110,   145,   145,    59,   100,     7,   146,    63,    64,   200,
+     372,    67,    10,    11,    12,    13,    14,    15,    16,    17,
+      18,    19,   345,    21,    22,    81,    82,   253,    55,   119,
+     125,    56,   165,    89,   401,   508,    92,   485,   546,   510,
+     285,   169,   286,   201,   174,   116,   151,    -1,   215,   156,
+      -1,   107,    -1,   215,     4,     5,     6,     7,   114,    -1,
+      10,   117,    -1,    -1,     1,   121,     3,    -1,   124,   125,
+      -1,    21,    22,    -1,    -1,    -1,    -1,    -1,   134,    77,
+      -1,    -1,    -1,    -1,   140,    35,    36,    -1,    -1,   145,
+      -1,    -1,    -1,    43,    44,    32,    -1,    -1,    -1,    -1,
+      -1,    51,    52,    -1,    54,    -1,    -1,    -1,    -1,    59,
+      -1,    48,    49,    63,    64,    -1,    -1,    67,    -1,    56,
+      -1,    -1,    -1,    60,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    81,    82,     4,     5,     6,     7,    -1,    -1,    89,
+      -1,    -1,    92,    80,    -1,    -1,    -1,    -1,    -1,    -1,
+      21,    22,    23,    -1,    -1,    -1,    -1,   107,    -1,    -1,
+      -1,    -1,    -1,    -1,    35,    36,    -1,   117,   105,    -1,
+      -1,   121,    43,    44,   124,   125,    -1,   114,    -1,    -1,
+      51,    52,    -1,    54,   134,    -1,    -1,    -1,    59,    -1,
+     140,    -1,    -1,    64,   131,   145,    67,    -1,    -1,    -1,
+      -1,   138,    -1,    -1,   141,    -1,    -1,    -1,    -1,    -1,
+      81,    82,     4,     5,     6,     7,    -1,    -1,    89,    -1,
+      -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    21,
+      22,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,
+      -1,    -1,    -1,    35,    36,    -1,   117,    -1,    -1,    -1,
+     121,    43,    44,   124,   125,    -1,    -1,    -1,    -1,    51,
+      52,    -1,    54,   134,    -1,    -1,    -1,    59,    -1,   140,
+      -1,    -1,    64,    -1,   145,    67,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,
+      82,     4,     5,     6,     7,    -1,    -1,    89,    -1,    -1,
+      92,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    21,    22,
+      -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,
+      -1,    -1,    35,    36,    -1,   117,    -1,    -1,    -1,   121,
+      43,    44,   124,   125,    -1,    -1,    -1,    -1,    51,    52,
+      -1,    54,   134,   135,    -1,    -1,    59,    -1,   140,    -1,
+      -1,    64,    -1,   145,    67,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
+      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,
-      -1,    89,    -1,    -1,    92,    -1,    -1,     4,     5,     6,
-       7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,
-      -1,    -1,    -1,    -1,    21,    22,    -1,    -1,    -1,   117,
-      -1,    -1,   120,    -1,    -1,   123,   124,    -1,    35,    36,
-      -1,    -1,    -1,    -1,    -1,   133,    43,    44,    -1,    -1,
-      -1,   139,    -1,    -1,    51,    52,   144,    54,    -1,    -1,
-      -1,    -1,    59,    -1,    -1,    -1,    -1,    64,    -1,    -1,
+      -1,    -1,    -1,    -1,   107,    -1,    -1,     4,     5,     6,
+       7,   114,    -1,    -1,   117,    -1,    -1,    -1,   121,    -1,
+      -1,   124,   125,    -1,    21,    22,    -1,    -1,    -1,    -1,
+      -1,   134,    -1,    -1,    -1,    -1,    -1,   140,    35,    36,
+      -1,    -1,   145,    -1,    -1,    -1,    43,    44,    -1,    -1,
+      -1,    -1,    -1,    -1,    51,    52,    -1,    54,    -1,    -1,
+      -1,    -1,    59,    -1,    -1,    -1,    63,    64,    -1,    -1,
       67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    81,    82,     4,     5,     6,     7,
       -1,    -1,    89,    -1,    -1,    92,    -1,    -1,    -1,    -1,
-      -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
-     107,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    21,    22,
-     117,    -1,    -1,   120,    -1,    -1,   123,   124,    -1,    -1,
-      -1,    -1,    35,    36,    -1,    -1,   133,   134,    -1,    -1,
-      43,    44,   139,    -1,    -1,    -1,    -1,   144,    51,    52,
-      -1,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,
-      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,    -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    21,
-      22,   114,    -1,    -1,   117,    -1,    -1,   120,    -1,    -1,
-     123,   124,    -1,    35,    36,    -1,    -1,    -1,    -1,    -1,
-     133,    43,    44,    -1,    -1,    -1,   139,    -1,    -1,    51,
-      52,   144,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,
-      -1,    63,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,
-      82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,
-      92,    -1,    -1,     4,     5,     6,     7,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,
-      21,    22,    -1,    -1,    -1,   117,    -1,    -1,   120,    -1,
-      -1,   123,   124,    -1,    35,    36,    -1,    -1,    -1,    -1,
-      -1,   133,    43,    44,    -1,    -1,    -1,   139,    -1,    -1,
-      51,    52,   144,    54,    -1,    -1,    -1,    -1,    59,    -1,
-      -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,    -1,
+      -1,    -1,    -1,    21,    22,    -1,    -1,    -1,    -1,    -1,
+     107,    -1,    -1,    -1,    -1,    -1,    -1,    35,    36,    -1,
+     117,    -1,    -1,    -1,   121,    43,    44,   124,   125,    -1,
+      -1,    -1,    -1,    51,    52,    -1,    54,   134,    -1,    -1,
+      -1,    59,    -1,   140,    -1,    -1,    64,    -1,   145,    67,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    81,    82,     4,     5,     6,     7,    -1,
+      -1,    89,    -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    21,    22,    -1,    -1,    -1,    -1,    -1,   107,
+      -1,    -1,    -1,    -1,    -1,    -1,    35,    36,    -1,   117,
+      -1,    -1,    -1,   121,    43,    44,   124,   125,    -1,    -1,
+      -1,    -1,    51,    52,    -1,    54,   134,    -1,    -1,    -1,
+      59,    -1,   140,    -1,    -1,    64,    -1,   145,    67,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,
-      -1,    92,    -1,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,
-      -1,    21,    22,    -1,    -1,    -1,   117,    -1,    -1,   120,
-      -1,    -1,   123,   124,    -1,    35,    36,    -1,    -1,    -1,
-      -1,    -1,   133,    43,    44,    -1,    -1,    -1,   139,    -1,
-      -1,    51,    52,   144,    54,    -1,    -1,    -1,    -1,    59,
-      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
+      -1,    -1,    81,    82,    -1,     5,    -1,     7,    -1,    -1,
+      89,    -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    21,    22,    -1,    -1,    -1,    -1,    -1,   107,    -1,
+      -1,    -1,    -1,    -1,    -1,    35,    36,    -1,   117,    -1,
+      -1,    -1,   121,    -1,    44,   124,   125,    -1,    -1,    -1,
+      -1,    51,    52,    -1,    54,   134,    -1,    -1,    -1,    59,
+      -1,   140,    -1,    -1,    -1,    -1,   145,    67,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    81,    82,     5,    -1,     7,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,    21,
-      22,    -1,    35,    36,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    44,    -1,    35,    36,    -1,    -1,   117,    51,    52,
-     120,    54,    44,   123,   124,    -1,    59,    -1,    -1,    51,
-      52,    -1,    54,   133,    67,    -1,    -1,    59,    -1,   139,
-      -1,    -1,    -1,    -1,   144,    67,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    81,
-      82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,
-      92,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,   117,   107,    -1,    -1,    -1,    -1,
-     123,   124,    -1,    -1,    -1,   117,    -1,    -1,    -1,    -1,
-     133,   123,   124,    -1,    -1,    -1,   139,    -1,    -1,    -1,
-      -1,   133,    -1,    -1,    -1,    -1,    -1,   139
+      -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,
+      -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,   117,    -1,    -1,
+      -1,    -1,    -1,    -1,   124,   125,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,   134,    -1,    -1,    -1,    -1,    -1,
+     140
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
@@ -1367,97 +1368,97 @@ static const yytype_int16 yycheck[] =
 static const yytype_uint16 yystos[] =
 {
        0,     1,     3,    32,    48,    49,    56,    60,    80,   105,
-     114,   130,   137,   140,   149,   150,   151,   152,   153,   154,
-     155,   177,   178,   181,   182,   185,   187,   190,   191,   192,
-     257,   258,   121,     4,     6,   190,   255,    78,   121,    71,
-     121,    83,    28,    58,   193,   255,   188,   189,   204,   255,
-       0,   140,   142,    48,    80,   190,    29,   137,   186,    30,
-     140,   143,     3,   255,   125,   179,    71,   179,   255,   255,
-     255,   255,   255,     5,     7,    21,    22,    23,    35,    36,
-      43,    44,    51,    52,    54,    59,    64,    67,    81,    82,
-      89,    92,   107,   117,   120,   123,   124,   133,   139,   144,
-     158,   194,   195,   196,   198,   230,   231,   232,   233,   234,
-     235,   236,   237,   244,   248,   251,   255,   115,   146,    33,
-     144,   190,    83,   186,   193,   104,   192,    31,    60,     5,
-     118,   119,   137,   180,     5,   180,    95,   144,   135,   224,
-     225,   132,   144,   192,     7,     7,   134,   230,   240,   241,
-     144,    82,   144,     5,   144,   144,    82,   190,   230,     5,
-      71,   197,   146,    21,    22,    33,   252,   255,    23,    24,
-     147,   253,    98,    22,   233,    27,   144,   183,   184,   255,
-     189,   144,   198,   254,   255,   179,   255,   191,     7,    46,
-      47,    46,    47,   144,   180,   255,   156,   157,   255,    10,
-      63,   144,   226,   227,   228,   229,   230,   248,   144,   247,
-     254,   226,   134,   238,   239,    61,   241,   242,     7,    53,
-      76,    90,    91,   113,   138,   245,   245,   230,     7,   145,
-     145,   144,   198,   201,   202,   205,   234,   255,   224,   196,
-     255,   231,   232,   144,   255,   255,    23,    58,   145,   230,
-     243,   146,   224,    11,   145,   146,   180,   192,   156,    45,
-      70,   103,   129,   162,   255,   255,   169,   170,   171,   172,
-     173,   174,   255,   144,   165,   145,   146,    82,   158,   229,
-     198,   226,   230,     8,     9,    10,    11,    12,    13,    14,
-      15,    16,    17,    18,    19,    77,   250,    20,   244,   246,
-     146,   145,   122,   230,   239,   242,   230,    62,   145,    71,
-      71,   145,   201,    33,   203,   204,    72,    79,    87,   109,
-     200,   146,   203,    39,   127,   199,    73,   206,    99,   213,
-     214,   145,   243,   145,   146,   184,   230,   255,   144,    85,
-      85,   144,   145,   146,     5,     7,    28,    65,    94,    95,
-     126,   144,   255,   256,   248,   249,   255,   131,   162,   163,
-     164,   156,    10,    45,    55,    92,   103,   108,   129,   159,
-     160,   161,   145,   227,   228,    17,    18,    19,    77,   230,
-     144,   198,   230,    10,    92,   145,   146,   144,   132,   230,
-     122,    62,   230,     7,   145,   204,    97,    97,    97,    84,
-     201,     7,     7,   203,    42,    75,   207,    42,   145,    96,
-     215,   145,   230,   226,   144,   144,   254,   170,   254,   145,
-     146,    37,    40,    41,   116,   175,   146,   137,   166,    92,
-     144,   244,    85,   255,   159,   230,   144,   198,     9,   243,
-      92,   244,   246,   144,   230,   145,    69,   145,   202,   101,
-     101,   243,   226,    96,   208,   243,    42,   106,   112,   216,
-     217,   145,   254,   254,   145,   145,   248,   144,   176,   162,
-      38,    99,   167,   226,   144,     9,   243,   230,   145,   145,
-     246,     7,    95,    42,    88,   209,   220,   221,   230,    19,
-     145,   145,   169,   144,    42,   145,   255,   230,   145,   145,
-     145,   226,   220,     7,   136,   210,   211,   212,   146,    34,
-      57,   222,     7,    50,   128,   218,   108,   145,   169,    74,
-     106,   168,   145,   255,   212,   221,    93,   223,   102,   110,
-     102,     9,   255,   145,   144,    33,    66,    86,     7,    50,
-     128,   219,   144,   254,   144,    68,   110,    68,   254,   145,
-     213,   145,   100,   145,     7
+     114,   131,   138,   141,   150,   151,   152,   153,   154,   155,
+     156,   178,   179,   183,   184,   187,   189,   192,   193,   194,
+     259,   260,   122,     4,     6,   192,   257,    78,   122,    71,
+     122,    83,    28,    58,   195,   257,   190,   191,   206,   257,
+       0,   141,   143,    48,    80,   192,    29,   138,   188,    30,
+     141,   144,     3,   257,   126,   181,    71,   180,   181,   257,
+     257,   257,   257,   257,     5,     7,    21,    22,    23,    35,
+      36,    43,    44,    51,    52,    54,    59,    64,    67,    81,
+      82,    89,    92,   107,   117,   121,   124,   125,   134,   140,
+     145,   159,   196,   197,   198,   200,   232,   233,   234,   235,
+     236,   237,   238,   239,   246,   250,   253,   257,   115,   147,
+      33,   145,   192,    83,   188,   195,   104,   194,    31,    60,
+       5,   118,   120,   138,   182,     5,   119,   182,   182,    95,
+     145,   136,   226,   227,   133,   145,   194,     7,     7,   135,
+     232,   242,   243,   145,    82,   145,     5,   145,   145,    82,
+     192,   232,     5,    71,   199,   147,    21,    22,    33,   254,
+     257,    23,    24,   148,   255,    98,    22,   235,    27,   145,
+     185,   186,   257,   191,   145,   200,   256,   257,   181,   257,
+     193,     7,    46,    47,    46,    47,   145,   257,   157,   158,
+     257,    10,    63,   145,   228,   229,   230,   231,   232,   250,
+     145,   249,   256,   228,   135,   240,   241,    61,   243,   244,
+       7,    53,    76,    90,    91,   113,   139,   247,   247,   232,
+       7,   146,   146,   145,   200,   203,   204,   207,   236,   257,
+     226,   198,   257,   233,   234,   145,   257,   257,    23,    58,
+     146,   232,   245,   147,   226,    11,   146,   147,   182,   194,
+     157,    45,    70,   103,   130,   163,   257,   257,   170,   171,
+     172,   173,   174,   175,   257,   145,   166,   146,   147,    82,
+     159,   231,   200,   228,   232,     8,     9,    10,    11,    12,
+      13,    14,    15,    16,    17,    18,    19,    77,   252,    20,
+     246,   248,   147,   146,   123,   232,   241,   244,   232,    62,
+     146,    71,    71,   146,   203,    33,   205,   206,    72,    79,
+      87,   109,   202,   147,   205,    39,   128,   201,    73,   208,
+      99,   215,   216,   146,   245,   146,   147,   186,   232,   257,
+     145,    85,    85,   145,   146,   147,     5,     7,    28,    65,
+      94,    95,   127,   145,   257,   258,   250,   251,   257,   132,
+     163,   164,   165,   157,    10,    45,    55,    92,   103,   108,
+     130,   160,   161,   162,   146,   229,   230,    17,    18,    19,
+      77,   232,   145,   200,   232,    10,    92,   146,   147,   145,
+     133,   232,   123,    62,   232,     7,   146,   206,    97,    97,
+      97,    84,   203,     7,     7,   205,    42,    75,   209,    42,
+     146,    96,   217,   146,   232,   228,   145,   145,   256,   171,
+     256,   146,   147,    37,    40,    41,   116,   176,   147,   138,
+     167,    92,   145,   246,    85,   257,   160,   232,   145,   200,
+       9,   245,    92,   246,   248,   145,   232,   146,    69,   146,
+     204,   101,   101,   245,   228,    96,   210,   245,    42,   106,
+     112,   218,   219,   146,   256,   256,   146,   146,   250,   145,
+     177,   163,    38,    99,   168,   228,   145,     9,   245,   232,
+     146,   146,   248,     7,    95,    42,    88,   211,   222,   223,
+     232,    19,   146,   146,   170,   145,    42,   146,   257,   232,
+     146,   146,   146,   228,   222,     7,   137,   212,   213,   214,
+     147,    34,    57,   224,     7,    50,   129,   220,   108,   146,
+     170,    74,   106,   169,   146,   257,   214,   223,    93,   225,
+     102,   110,   102,     9,   257,   146,   145,    33,    66,    86,
+       7,    50,   129,   221,   145,   256,   145,    68,   110,    68,
+     256,   146,   215,   146,   100,   146,     7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
 static const yytype_uint16 yyr1[] =
 {
-       0,   148,   149,   149,   149,   149,   149,   149,   150,   150,
-     150,   150,   150,   150,   150,   150,   150,   150,   151,   152,
-     152,   152,   152,   153,   154,   155,   156,   157,   157,   158,
-     158,   158,   158,   158,   158,   158,   158,   158,   158,   158,
-     158,   158,   158,   158,   158,   158,   158,   159,   159,   159,
-     159,   159,   159,   159,   160,   160,   161,   161,   162,   162,
-     162,   162,   163,   163,   164,   164,   165,   165,   166,   166,
-     167,   167,   168,   168,   169,   169,   170,   170,   170,   170,
-     171,   171,   171,   172,   173,   174,   175,   175,   175,   175,
-     176,   176,   177,   177,   177,   177,   178,   178,   178,   178,
-     179,   179,   179,   180,   180,   181,   182,   183,   183,   184,
-     185,   185,   186,   186,   187,   188,   188,   189,   190,   190,
-     191,   191,   192,   193,   193,   193,   194,   194,   195,   195,
-     196,   196,   196,   197,   198,   199,   199,   199,   200,   200,
-     200,   200,   200,   200,   200,   200,   201,   201,   202,   202,
-     202,   202,   202,   202,   203,   203,   204,   204,   205,   205,
-     206,   206,   207,   207,   208,   208,   209,   209,   210,   210,
-     211,   211,   212,   213,   214,   214,   215,   215,   216,   216,
-     217,   217,   218,   218,   218,   219,   219,   219,   220,   220,
-     221,   222,   222,   222,   223,   223,   223,   224,   224,   225,
-     226,   226,   227,   227,   228,   228,   229,   229,   229,   229,
-     229,   229,   229,   229,   229,   229,   229,   230,   230,   231,
-     231,   232,   232,   233,   233,   233,   233,   233,   233,   233,
-     233,   233,   233,   234,   234,   234,   234,   235,   236,   236,
-     237,   237,   238,   238,   239,   240,   240,   241,   242,   242,
-     243,   243,   244,   244,   244,   244,   244,   244,   244,   244,
-     245,   245,   245,   245,   245,   245,   246,   246,   247,   247,
-     248,   248,   249,   249,   250,   250,   250,   250,   250,   250,
-     250,   250,   250,   250,   251,   252,   252,   253,   253,   253,
-     254,   254,   255,   255,   256,   256,   256,   256,   257,   258,
-     258
+       0,   149,   150,   150,   150,   150,   150,   150,   151,   151,
+     151,   151,   151,   151,   151,   151,   151,   151,   152,   153,
+     153,   153,   153,   154,   155,   156,   157,   158,   158,   159,
+     159,   159,   159,   159,   159,   159,   159,   159,   159,   159,
+     159,   159,   159,   159,   159,   159,   159,   160,   160,   160,
+     160,   160,   160,   160,   161,   161,   162,   162,   163,   163,
+     163,   163,   164,   164,   165,   165,   166,   166,   167,   167,
+     168,   168,   169,   169,   170,   170,   171,   171,   171,   171,
+     172,   172,   172,   173,   174,   175,   176,   176,   176,   176,
+     177,   177,   178,   178,   178,   178,   179,   179,   179,   179,
+     180,   180,   181,   181,   181,   182,   182,   183,   184,   185,
+     185,   186,   187,   187,   188,   188,   189,   190,   190,   191,
+     192,   192,   193,   193,   194,   195,   195,   195,   196,   196,
+     197,   197,   198,   198,   198,   199,   200,   201,   201,   201,
+     202,   202,   202,   202,   202,   202,   202,   202,   203,   203,
+     204,   204,   204,   204,   204,   204,   205,   205,   206,   206,
+     207,   207,   208,   208,   209,   209,   210,   210,   211,   211,
+     212,   212,   213,   213,   214,   215,   216,   216,   217,   217,
+     218,   218,   219,   219,   220,   220,   220,   221,   221,   221,
+     222,   222,   223,   224,   224,   224,   225,   225,   225,   226,
+     226,   227,   228,   228,   229,   229,   230,   230,   231,   231,
+     231,   231,   231,   231,   231,   231,   231,   231,   231,   232,
+     232,   233,   233,   234,   234,   235,   235,   235,   235,   235,
+     235,   235,   235,   235,   235,   236,   236,   236,   236,   237,
+     238,   238,   239,   239,   240,   240,   241,   242,   242,   243,
+     244,   244,   245,   245,   246,   246,   246,   246,   246,   246,
+     246,   246,   247,   247,   247,   247,   247,   247,   248,   248,
+     249,   249,   250,   250,   251,   251,   252,   252,   252,   252,
+     252,   252,   252,   252,   252,   252,   253,   254,   254,   255,
+     255,   255,   256,   256,   257,   257,   258,   258,   258,   258,
+     259,   260,   260
 };
 
   /* YYR2[YYN] -- Number of symbols on the right hand side of rule YYN.  */
@@ -1472,28 +1473,28 @@ static const yytype_uint8 yyr2[] =
       10,     4,     3,     1,     0,     1,     0,     3,     0,     5,
        0,     8,     1,     1,     1,     3,     1,     1,     1,     1,
        2,     2,     2,     4,     2,     2,     1,     1,     1,     1,
-       0,     3,    10,     5,     4,     5,     5,     4,     4,     5,
-       2,     2,     2,     0,     4,     5,     4,     3,     1,     3,
-       2,     3,     0,     3,     2,     1,     3,     3,     4,     1,
-       3,     1,    10,     0,     1,     1,     1,     1,     1,     3,
-       3,     2,     1,     2,     3,     0,     3,     3,     0,     1,
-       1,     2,     1,     2,     1,     2,     6,     1,     2,     3,
-       2,     2,     1,     3,     1,     2,     1,     4,     1,     3,
-       0,     3,     0,     2,     0,     3,     0,     2,     0,     1,
-       1,     2,     6,     3,     0,     3,     0,     3,     0,     5,
-       1,     1,     2,     2,     2,     2,     2,     2,     1,     3,
-       3,     0,     1,     1,     0,     2,     2,     0,     1,     2,
-       3,     1,     3,     1,     2,     1,     5,     6,     4,     3,
-       3,     3,     2,     3,     5,     4,     6,     3,     1,     3,
-       1,     2,     1,     1,     1,     1,     3,     5,     1,     1,
-       1,     3,     1,     3,     4,     4,     5,     6,     6,     8,
-       5,     4,     1,     2,     4,     1,     2,     4,     0,     2,
-       1,     3,     1,     1,     2,     2,     1,     2,     3,     2,
-       1,     1,     1,     1,     1,     1,     1,     3,     3,     5,
-       1,     3,     1,     3,     1,     1,     1,     1,     1,     1,
-       1,     2,     1,     2,     1,     1,     1,     1,     1,     1,
-       1,     3,     1,     1,     1,     1,     1,     1,     2,     2,
-       0
+       0,     3,    10,     5,     4,     5,     4,     4,     4,     5,
+       2,     2,     2,     2,     2,     0,     4,     5,     4,     3,
+       1,     3,     2,     3,     0,     3,     2,     1,     3,     3,
+       4,     1,     3,     1,    10,     0,     1,     1,     1,     1,
+       1,     3,     3,     2,     1,     2,     3,     0,     3,     3,
+       0,     1,     1,     2,     1,     2,     1,     2,     6,     1,
+       2,     3,     2,     2,     1,     3,     1,     2,     1,     4,
+       1,     3,     0,     3,     0,     2,     0,     3,     0,     2,
+       0,     1,     1,     2,     6,     3,     0,     3,     0,     3,
+       0,     5,     1,     1,     2,     2,     2,     2,     2,     2,
+       1,     3,     3,     0,     1,     1,     0,     2,     2,     0,
+       1,     2,     3,     1,     3,     1,     2,     1,     5,     6,
+       4,     3,     3,     3,     2,     3,     5,     4,     6,     3,
+       1,     3,     1,     2,     1,     1,     1,     1,     3,     5,
+       1,     1,     1,     3,     1,     3,     4,     4,     5,     6,
+       6,     8,     5,     4,     1,     2,     4,     1,     2,     4,
+       0,     2,     1,     3,     1,     1,     2,     2,     1,     2,
+       3,     2,     1,     1,     1,     1,     1,     1,     1,     3,
+       3,     5,     1,     3,     1,     3,     1,     1,     1,     1,
+       1,     1,     1,     2,     1,     2,     1,     1,     1,     1,
+       1,     1,     1,     3,     1,     1,     1,     1,     1,     1,
+       2,     2,     0
 };
 
 
@@ -1990,1081 +1991,1091 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
   switch (yytype)
     {
           case 3: /* TOKEN_COMMAND  */
-#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 637 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2000 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2001 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 4: /* TOKEN_NAME  */
-#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 637 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2010 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2011 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 5: /* TOKEN_STRING_SINGLE_QUOTED  */
-#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 637 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2020 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2021 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 6: /* TOKEN_STRING_DOUBLE_QUOTED  */
-#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 637 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2030 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2031 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 7: /* TOKEN_UNSIGNED_NUMVAL  */
-#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 637 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).numeric_literal_value_) != nullptr) {
     delete ((*yyvaluep).numeric_literal_value_);
   }
 }
-#line 2040 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 

<TRUNCATED>


[18/27] incubator-quickstep git commit: Added Vector Aggregation support in the distributed version.

Posted by ji...@apache.org.
Added Vector Aggregation support in the distributed version.


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

Branch: refs/heads/trace
Commit: e79b520ec919fbe101ad72978c02216eeeeb6ca6
Parents: 8f094a1
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Fri Aug 4 17:03:34 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Thu Oct 12 11:44:44 2017 -0500

----------------------------------------------------------------------
 .../FinalizeAggregationOperator.cpp             | 31 ++++++++++----------
 .../InitializeAggregationOperator.cpp           | 23 +++++++--------
 relational_operators/WorkOrderFactory.cpp       |  2 --
 3 files changed, 27 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e79b520e/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 68d0ef4..92fc7f6 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -67,28 +67,29 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
   return true;
 }
 
-// TODO(quickstep-team) : Think about how the number of partitions could be
-// accessed in this function. Until then, we can't use partitioned aggregation
-// finalization with the distributed version.
 bool FinalizeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
   if (started_) {
     return true;
   }
 
   for (partition_id part_id = 0; part_id < num_partitions_; ++part_id) {
-    serialization::WorkOrder *proto = new serialization::WorkOrder;
-    proto->set_work_order_type(serialization::FINALIZE_AGGREGATION);
-    proto->set_query_id(query_id_);
-    proto->SetExtension(serialization::FinalizeAggregationWorkOrder::aggr_state_index,
-                        aggr_state_index_);
-    proto->SetExtension(serialization::FinalizeAggregationWorkOrder::partition_id,
-                        part_id);
-    proto->SetExtension(serialization::FinalizeAggregationWorkOrder::state_partition_id,
-                        0u);
-    proto->SetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index,
-                        output_destination_index_);
+    for (std::size_t state_part_id = 0;
+         state_part_id < aggr_state_num_partitions_;
+         ++state_part_id) {
+      serialization::WorkOrder *proto = new serialization::WorkOrder;
+      proto->set_work_order_type(serialization::FINALIZE_AGGREGATION);
+      proto->set_query_id(query_id_);
+      proto->SetExtension(serialization::FinalizeAggregationWorkOrder::aggr_state_index,
+                          aggr_state_index_);
+      proto->SetExtension(serialization::FinalizeAggregationWorkOrder::partition_id,
+                          part_id);
+      proto->SetExtension(serialization::FinalizeAggregationWorkOrder::state_partition_id,
+                          state_part_id);
+      proto->SetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index,
+                          output_destination_index_);
 
-    container->addWorkOrderProto(proto, op_index_);
+      container->addWorkOrderProto(proto, op_index_);
+    }
   }
 
   started_ = true;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e79b520e/relational_operators/InitializeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeAggregationOperator.cpp b/relational_operators/InitializeAggregationOperator.cpp
index 39a6fb4..89dfd7e 100644
--- a/relational_operators/InitializeAggregationOperator.cpp
+++ b/relational_operators/InitializeAggregationOperator.cpp
@@ -64,26 +64,25 @@ bool InitializeAggregationOperator::getAllWorkOrders(
   return true;
 }
 
-// TODO(quickstep-team) : Think about how the number of partitions could be
-// accessed in this function. Until then, we can't use partitioned aggregation
-// initialization with the distributed version.
 bool InitializeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
-  LOG(FATAL) << "Not supported";
-
   if (started_) {
     return true;
   }
 
   for (partition_id part_id = 0; part_id < num_partitions_; ++part_id) {
-    serialization::WorkOrder *proto = new serialization::WorkOrder;
-    proto->set_work_order_type(serialization::INITIALIZE_AGGREGATION);
-    proto->set_query_id(query_id_);
+    for (std::size_t state_part_id = 0;
+         state_part_id < aggr_state_num_init_partitions_;
+         ++state_part_id) {
+      serialization::WorkOrder *proto = new serialization::WorkOrder;
+      proto->set_work_order_type(serialization::INITIALIZE_AGGREGATION);
+      proto->set_query_id(query_id_);
 
-    proto->SetExtension(serialization::InitializeAggregationWorkOrder::aggr_state_index, aggr_state_index_);
-    proto->SetExtension(serialization::InitializeAggregationWorkOrder::partition_id, part_id);
-    proto->SetExtension(serialization::InitializeAggregationWorkOrder::state_partition_id, 0u);
+      proto->SetExtension(serialization::InitializeAggregationWorkOrder::aggr_state_index, aggr_state_index_);
+      proto->SetExtension(serialization::InitializeAggregationWorkOrder::partition_id, part_id);
+      proto->SetExtension(serialization::InitializeAggregationWorkOrder::state_partition_id, state_part_id);
 
-    container->addWorkOrderProto(proto, op_index_);
+      container->addWorkOrderProto(proto, op_index_);
+    }
   }
   started_ = true;
   return true;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e79b520e/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 25cc81a..3a991bd 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -237,8 +237,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
 
       LOG(INFO) << "Creating FinalizeAggregationWorkOrder (Partition " << part_id << ") for Query " << query_id
                 << " in Shiftboss " << shiftboss_index;
-      // TODO(quickstep-team): Handle inner-table partitioning in the distributed
-      // setting.
       return new FinalizeAggregationWorkOrder(
           query_id,
           part_id,


[21/27] incubator-quickstep git commit: Support Multiple Tuple Inserts

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index f6b5247..142059d 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -198,6 +198,7 @@ union YYSTYPE
   quickstep::NumericParseLiteralValue *numeric_literal_value_;
   quickstep::ParseLiteralValue *literal_value_;
   quickstep::PtrList<quickstep::ParseScalarLiteral> *literal_value_list_;
+  quickstep::PtrList<quickstep::PtrList<quickstep::ParseScalarLiteral>> *literal_value_list_multiple_;
 
   quickstep::ParseExpression *expression_;
 
@@ -288,7 +289,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 292 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 293 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 372d576..14d8949 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1461,70 +1461,72 @@ void ExecutionGenerator::convertInsertTuple(
       *catalog_database_->getRelationById(
           input_relation_info->relation->getID());
 
-  // Construct the tuple proto to be inserted.
-  const QueryContext::tuple_id tuple_index = query_context_proto_->tuples_size();
+  for (const std::vector<expressions::ScalarLiteralPtr> &tuple : physical_plan->column_values()) {
+    // Construct the tuple proto to be inserted.
+    const QueryContext::tuple_id tuple_index = query_context_proto_->tuples_size();
 
-  S::Tuple *tuple_proto = query_context_proto_->add_tuples();
-  for (const E::ScalarLiteralPtr &literal : physical_plan->column_values()) {
-    tuple_proto->add_attribute_values()->CopyFrom(literal->value().getProto());
-  }
+    S::Tuple *tuple_proto = query_context_proto_->add_tuples();
+    for (const E::ScalarLiteralPtr &literal : tuple) {
+      tuple_proto->add_attribute_values()->CopyFrom(literal->value().getProto());
+    }
 
-  // FIXME(qzeng): A better way is using a traits struct to look up whether a storage
-  //               block supports ad-hoc insertion instead of hard-coding the block types.
-  const StorageBlockLayout &storage_block_layout =
-      input_relation.getDefaultStorageBlockLayout();
-  if (storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
-      TupleStorageSubBlockDescription::COMPRESSED_COLUMN_STORE ||
-      storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
-            TupleStorageSubBlockDescription::COMPRESSED_PACKED_ROW_STORE) {
-    THROW_SQL_ERROR() << "INSERT statement is not supported for the relation "
-                      << input_relation.getName()
-                      << ", because its storage blocks do not support ad-hoc insertion";
-  }
+    // FIXME(qzeng): A better way is using a traits struct to look up whether a storage
+    //               block supports ad-hoc insertion instead of hard-coding the block types.
+    const StorageBlockLayout &storage_block_layout =
+        input_relation.getDefaultStorageBlockLayout();
+    if (storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
+        TupleStorageSubBlockDescription::COMPRESSED_COLUMN_STORE ||
+        storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
+              TupleStorageSubBlockDescription::COMPRESSED_PACKED_ROW_STORE) {
+      THROW_SQL_ERROR() << "INSERT statement is not supported for the relation "
+                        << input_relation.getName()
+                        << ", because its storage blocks do not support ad-hoc insertion";
+    }
 
-  // Create InsertDestination proto.
-  const QueryContext::insert_destination_id insert_destination_index =
-      query_context_proto_->insert_destinations_size();
-  S::InsertDestination *insert_destination_proto = query_context_proto_->add_insert_destinations();
+    // Create InsertDestination proto.
+    const QueryContext::insert_destination_id insert_destination_index =
+        query_context_proto_->insert_destinations_size();
+    S::InsertDestination *insert_destination_proto = query_context_proto_->add_insert_destinations();
 
-  insert_destination_proto->set_relation_id(input_relation.getID());
-  insert_destination_proto->mutable_layout()->MergeFrom(
-      input_relation.getDefaultStorageBlockLayout().getDescription());
+    insert_destination_proto->set_relation_id(input_relation.getID());
+    insert_destination_proto->mutable_layout()->MergeFrom(
+        input_relation.getDefaultStorageBlockLayout().getDescription());
 
-  if (input_relation.hasPartitionScheme()) {
-    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::PARTITION_AWARE);
-    insert_destination_proto->MutableExtension(S::PartitionAwareInsertDestination::partition_scheme)
-        ->MergeFrom(input_relation.getPartitionScheme()->getProto());
-  } else {
-    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
+    if (input_relation.hasPartitionScheme()) {
+      insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::PARTITION_AWARE);
+      insert_destination_proto->MutableExtension(S::PartitionAwareInsertDestination::partition_scheme)
+          ->MergeFrom(input_relation.getPartitionScheme()->getProto());
+    } else {
+      insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
 
-    const vector<block_id> blocks(input_relation.getBlocksSnapshot());
-    for (const block_id block : blocks) {
-      insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+      const vector<block_id> blocks(input_relation.getBlocksSnapshot());
+      for (const block_id block : blocks) {
+        insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+      }
     }
-  }
 
-  const QueryPlan::DAGNodeIndex insert_operator_index =
-      execution_plan_->addRelationalOperator(
-          new InsertOperator(query_handle_->query_id(),
-                             input_relation,
-                             insert_destination_index,
-                             tuple_index));
-  insert_destination_proto->set_relational_op_index(insert_operator_index);
+    const QueryPlan::DAGNodeIndex insert_operator_index =
+        execution_plan_->addRelationalOperator(
+            new InsertOperator(query_handle_->query_id(),
+                               input_relation,
+                               insert_destination_index,
+                               tuple_index));
+    insert_destination_proto->set_relational_op_index(insert_operator_index);
 
-  CatalogRelation *mutable_relation =
-      catalog_database_->getRelationByIdMutable(input_relation.getID());
-  const QueryPlan::DAGNodeIndex save_blocks_index =
-      execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
-  if (!input_relation_info->isStoredRelation()) {
-    execution_plan_->addDirectDependency(insert_operator_index,
-                                         input_relation_info->producer_operator_index,
-                                         true /* is_pipeline_breaker */);
+    CatalogRelation *mutable_relation =
+        catalog_database_->getRelationByIdMutable(input_relation.getID());
+    const QueryPlan::DAGNodeIndex save_blocks_index =
+        execution_plan_->addRelationalOperator(
+            new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
+    if (!input_relation_info->isStoredRelation()) {
+      execution_plan_->addDirectDependency(insert_operator_index,
+                                           input_relation_info->producer_operator_index,
+                                           true /* is_pipeline_breaker */);
+    }
+    execution_plan_->addDirectDependency(save_blocks_index,
+                                         insert_operator_index,
+                                         false /* is_pipeline_breaker */);
   }
-  execution_plan_->addDirectDependency(save_blocks_index,
-                                       insert_operator_index,
-                                       false /* is_pipeline_breaker */);
 }
 
 void ExecutionGenerator::convertInsertSelection(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/logical/InsertTuple.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/InsertTuple.cpp b/query_optimizer/logical/InsertTuple.cpp
index e5ffa35..e2ce196 100644
--- a/query_optimizer/logical/InsertTuple.cpp
+++ b/query_optimizer/logical/InsertTuple.cpp
@@ -41,8 +41,10 @@ void InsertTuple::getFieldStringItems(
   non_container_child_field_names->push_back("input");
   non_container_child_fields->push_back(input_);
 
-  container_child_field_names->push_back("column_values");
-  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(column_values_));
+  for (const auto &column_values : column_values_) {
+    container_child_field_names->push_back("column_values");
+    container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(column_values));
+  }
 }
 
 }  // namespace logical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/logical/InsertTuple.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/InsertTuple.hpp b/query_optimizer/logical/InsertTuple.hpp
index fd0301e..dd35510 100644
--- a/query_optimizer/logical/InsertTuple.hpp
+++ b/query_optimizer/logical/InsertTuple.hpp
@@ -61,7 +61,7 @@ class InsertTuple : public Logical {
   /**
    * @return Column values to be used to compose a new tuple.
    */
-  const std::vector<expressions::ScalarLiteralPtr>& column_values() const {
+  const std::vector<std::vector<expressions::ScalarLiteralPtr>>& column_values() const {
     return column_values_;
   }
 
@@ -83,12 +83,12 @@ class InsertTuple : public Logical {
    * @brief Creates an InsertTuple logical node.
    *
    * @param input The input produces the relation to insert the tuple to.
-   * @param column_values The column values of the tuple to be inserted.
+   * @param column_values The column values of the tuples to be inserted.
    * @return An immutable InsertTuple node.
    */
   static InsertTuplePtr Create(
       const LogicalPtr &input,
-      const std::vector<expressions::ScalarLiteralPtr> &column_values) {
+      const std::vector<std::vector<expressions::ScalarLiteralPtr>> &column_values) {
     return InsertTuplePtr(new InsertTuple(input, column_values));
   }
 
@@ -103,13 +103,13 @@ class InsertTuple : public Logical {
 
  private:
   InsertTuple(const LogicalPtr &input,
-              const std::vector<expressions::ScalarLiteralPtr> &column_values)
+              const std::vector<std::vector<expressions::ScalarLiteralPtr>> &column_values)
       : input_(input), column_values_(column_values) {
     addChild(input_);
   }
 
   LogicalPtr input_;
-  std::vector<expressions::ScalarLiteralPtr> column_values_;
+  std::vector<std::vector<expressions::ScalarLiteralPtr>> column_values_;
 
   DISALLOW_COPY_AND_ASSIGN(InsertTuple);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/physical/InsertTuple.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/InsertTuple.cpp b/query_optimizer/physical/InsertTuple.cpp
index 3085389..b209aa0 100644
--- a/query_optimizer/physical/InsertTuple.cpp
+++ b/query_optimizer/physical/InsertTuple.cpp
@@ -40,8 +40,10 @@ void InsertTuple::getFieldStringItems(
   non_container_child_field_names->push_back("input");
   non_container_child_fields->push_back(input_);
 
-  container_child_field_names->push_back("column_values");
-  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(column_values_));
+  for (const auto &column_values : column_values_) {
+    container_child_field_names->push_back("column_values");
+    container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(column_values));
+  }
 }
 
 }  // namespace physical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/physical/InsertTuple.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/InsertTuple.hpp b/query_optimizer/physical/InsertTuple.hpp
index 40f2582..10c7c5b 100644
--- a/query_optimizer/physical/InsertTuple.hpp
+++ b/query_optimizer/physical/InsertTuple.hpp
@@ -69,7 +69,7 @@ class InsertTuple : public Physical {
   /**
    * @return Column values to be used to compose a new tuple.
    */
-  const std::vector<expressions::ScalarLiteralPtr>& column_values() const {
+  const std::vector<std::vector<expressions::ScalarLiteralPtr>>& column_values() const {
     return column_values_;
   }
 
@@ -103,7 +103,7 @@ class InsertTuple : public Physical {
    */
   static InsertTuplePtr Create(
       const PhysicalPtr &input,
-      const std::vector<expressions::ScalarLiteralPtr> &column_values) {
+      const std::vector<std::vector<expressions::ScalarLiteralPtr>> &column_values) {
     return InsertTuplePtr(new InsertTuple(input, column_values));
   }
 
@@ -118,13 +118,13 @@ class InsertTuple : public Physical {
 
  private:
   InsertTuple(const PhysicalPtr &input,
-              const std::vector<expressions::ScalarLiteralPtr> &column_values)
+              const std::vector<std::vector<expressions::ScalarLiteralPtr>> &column_values)
       : input_(input), column_values_(column_values) {
     addChild(input_);
   }
 
   PhysicalPtr input_;
-  std::vector<expressions::ScalarLiteralPtr> column_values_;
+  std::vector<std::vector<expressions::ScalarLiteralPtr>> column_values_;
 
   DISALLOW_COPY_AND_ASSIGN(InsertTuple);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 2991568..0b6dc22 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -1060,73 +1060,81 @@ L::LogicalPtr Resolver::resolveInsertTuple(
   // Resolve column values.
   const std::vector<E::AttributeReferencePtr> relation_attributes =
       input_logical->getOutputAttributes();
-  const PtrList<ParseScalarLiteral> &parse_column_values =
+  const PtrList<PtrList<ParseScalarLiteral>> &parse_column_values_list =
       insert_statement.getLiteralValues();
-  DCHECK_GT(parse_column_values.size(), 0u);
 
-  if (parse_column_values.size() > relation_attributes.size()) {
-    THROW_SQL_ERROR_AT(insert_statement.relation_name())
-        << "The relation " << insert_statement.relation_name()->value()
-        << " has " << std::to_string(relation_attributes.size())
-        << " columns, but " << std::to_string(parse_column_values.size())
-        << " values are provided";
-  }
+  std::vector<std::vector<E::ScalarLiteralPtr>> resolved_column_values_list;
+  DCHECK_GT(parse_column_values_list.size(), 0u);
 
-  std::vector<E::ScalarLiteralPtr> resolved_column_values;
-  std::vector<E::AttributeReferencePtr>::size_type aid = 0;
-  for (const ParseScalarLiteral &parse_literal_value : parse_column_values) {
-    E::ScalarLiteralPtr resolved_literal_value;
-    ExpressionResolutionInfo expr_resolution_info(
-        name_resolver,
-        "INSERT statement" /* clause_name */,
-        nullptr /* select_list_info */);
-    // When resolving the literal, use the attribute's Type as a hint.
-    CHECK(E::SomeScalarLiteral::MatchesWithConditionalCast(
-        resolveExpression(parse_literal_value,
-                          &(relation_attributes[aid]->getValueType()),
-                          &expr_resolution_info),
-        &resolved_literal_value));
-
-    // Check that the resolved Type is safely coercible to the attribute's
-    // Type.
-    if (!relation_attributes[aid]->getValueType().isSafelyCoercibleFrom(
-            resolved_literal_value->getValueType())) {
-      THROW_SQL_ERROR_AT(&parse_literal_value)
-          << "The assigned value for the column "
-          << relation_attributes[aid]->attribute_name() << " has the type "
-          << resolved_literal_value->getValueType().getName()
-          << ", which cannot be safely coerced to the column's type "
-          << relation_attributes[aid]->getValueType().getName();
-    }
+  for (const PtrList<ParseScalarLiteral> &parse_column_values : parse_column_values_list) {
+    DCHECK_GT(parse_column_values.size(), 0u);
 
-    // If the Type is not exactly right (but is safely coercible), coerce it.
-    if (!resolved_literal_value->getValueType().equals(
-            relation_attributes[aid]->getValueType())) {
-      resolved_literal_value = E::ScalarLiteral::Create(
-          relation_attributes[aid]->getValueType().coerceValue(
-              resolved_literal_value->value(),
-              resolved_literal_value->getValueType()),
-          relation_attributes[aid]->getValueType());
+    if (parse_column_values.size() > relation_attributes.size()) {
+      THROW_SQL_ERROR_AT(insert_statement.relation_name())
+          << "The relation " << insert_statement.relation_name()->value()
+          << " has " << std::to_string(relation_attributes.size())
+          << " columns, but " << std::to_string(parse_column_values.size())
+          << " values are provided";
     }
 
-    resolved_column_values.push_back(resolved_literal_value);
-    ++aid;
-  }
+    std::vector<E::ScalarLiteralPtr> resolved_column_values;
+    std::vector<E::AttributeReferencePtr>::size_type aid = 0;
+    for (const ParseScalarLiteral &parse_literal_value : parse_column_values) {
+      E::ScalarLiteralPtr resolved_literal_value;
+      ExpressionResolutionInfo expr_resolution_info(
+          name_resolver,
+          "INSERT statement" /* clause_name */,
+          nullptr /* select_list_info */);
+      // When resolving the literal, use the attribute's Type as a hint.
+      CHECK(E::SomeScalarLiteral::MatchesWithConditionalCast(
+          resolveExpression(parse_literal_value,
+                            &(relation_attributes[aid]->getValueType()),
+                            &expr_resolution_info),
+          &resolved_literal_value));
+
+      // Check that the resolved Type is safely coercible to the attribute's
+      // Type.
+      if (!relation_attributes[aid]->getValueType().isSafelyCoercibleFrom(
+              resolved_literal_value->getValueType())) {
+        THROW_SQL_ERROR_AT(&parse_literal_value)
+            << "The assigned value for the column "
+            << relation_attributes[aid]->attribute_name() << " has the type "
+            << resolved_literal_value->getValueType().getName()
+            << ", which cannot be safely coerced to the column's type "
+            << relation_attributes[aid]->getValueType().getName();
+      }
 
-  while (aid < relation_attributes.size()) {
-    if (!relation_attributes[aid]->getValueType().isNullable()) {
-      THROW_SQL_ERROR_AT(insert_statement.relation_name())
-          << "Must assign a non-NULL value to column "
-          << relation_attributes[aid]->attribute_name();
+      // If the Type is not exactly right (but is safely coercible), coerce it.
+      if (!resolved_literal_value->getValueType().equals(
+              relation_attributes[aid]->getValueType())) {
+        resolved_literal_value = E::ScalarLiteral::Create(
+            relation_attributes[aid]->getValueType().coerceValue(
+                resolved_literal_value->value(),
+                resolved_literal_value->getValueType()),
+            relation_attributes[aid]->getValueType());
+      }
+
+      resolved_column_values.push_back(resolved_literal_value);
+      ++aid;
+    }
+
+    while (aid < relation_attributes.size()) {
+      if (!relation_attributes[aid]->getValueType().isNullable()) {
+        THROW_SQL_ERROR_AT(insert_statement.relation_name())
+            << "Must assign a non-NULL value to column "
+            << relation_attributes[aid]->attribute_name();
+      }
+      // Create a NULL value.
+      resolved_column_values.push_back(E::ScalarLiteral::Create(
+          relation_attributes[aid]->getValueType().makeNullValue(),
+          relation_attributes[aid]->getValueType()));
+      ++aid;
     }
-    // Create a NULL value.
-    resolved_column_values.push_back(E::ScalarLiteral::Create(
-        relation_attributes[aid]->getValueType().makeNullValue(),
-        relation_attributes[aid]->getValueType()));
-    ++aid;
+
+    resolved_column_values_list.push_back(std::move(resolved_column_values));
   }
 
-  return L::InsertTuple::Create(input_logical, resolved_column_values);
+  return L::InsertTuple::Create(input_logical, resolved_column_values_list);
 }
 
 L::LogicalPtr Resolver::resolveUpdate(


[10/27] incubator-quickstep git commit: Fixed the distributed version due to query execution engine simplification.

Posted by ji...@apache.org.
Fixed the distributed version due to query execution engine simplification.


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

Branch: refs/heads/trace
Commit: e496cb58e10d32de9dc83d69ece84df3f5b62747
Parents: 0898a77
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Fri Oct 6 22:33:02 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Fri Oct 6 22:33:02 2017 -0500

----------------------------------------------------------------------
 query_execution/QueryManagerDistributed.cpp | 24 +++++++++---------------
 relational_operators/WorkOrderFactory.cpp   |  4 ++--
 2 files changed, 11 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e496cb58/query_execution/QueryManagerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.cpp b/query_execution/QueryManagerDistributed.cpp
index 30a1396..97b451f 100644
--- a/query_execution/QueryManagerDistributed.cpp
+++ b/query_execution/QueryManagerDistributed.cpp
@@ -70,8 +70,11 @@ QueryManagerDistributed::QueryManagerDistributed(QueryHandle *query_handle,
   // Collect all the workorders from all the non-blocking relational operators in the DAG.
   for (const dag_node_index index : non_dependent_operators_) {
     if (!fetchNormalWorkOrders(index)) {
-      DCHECK(!checkRebuildRequired(index) || initiateRebuild(index));
-      markOperatorFinished(index);
+      if (checkRebuildRequired(index)) {
+        initiateRebuild(index);
+      } else {
+        markOperatorFinished(index);
+      }
     }
   }
 
@@ -201,21 +204,12 @@ void QueryManagerDistributed::processInitiateRebuildResponseMessage(const dag_no
                                                                     const std::size_t shiftboss_index) {
   query_exec_state_->updateRebuildStatus(op_index, num_rebuild_work_orders, shiftboss_index);
 
-  if (!query_exec_state_->hasRebuildFinished(op_index, num_shiftbosses_)) {
-    // Wait for the rebuild work orders to finish.
-    return;
+  if (query_exec_state_->hasRebuildFinished(op_index, num_shiftbosses_)) {
+    // No needs for rebuilds, or the rebuild has finished.
+    markOperatorFinished(op_index);
   }
 
-  // No needs for rebuilds, or the rebuild has finished.
-  markOperatorFinished(op_index);
-
-  for (const std::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)) {
-      fetchNormalWorkOrders(dependent_op_index);
-    }
-  }
+  // Wait for the rebuild work orders to finish.
 }
 
 bool QueryManagerDistributed::initiateRebuild(const dag_node_index index) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e496cb58/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 5baa21b..25cc81a 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -741,7 +741,7 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
                  proto.GetExtension(serialization::DeleteWorkOrder::predicate_index)) &&
              proto.HasExtension(serialization::DeleteWorkOrder::block_id) &&
              proto.HasExtension(serialization::DeleteWorkOrder::operator_index) &&
-             proto.GetExtension(serialization::DeleteWorkOrder::partition_id);
+             proto.HasExtension(serialization::DeleteWorkOrder::partition_id);
     }
     case serialization::DESTROY_AGGREGATION_STATE: {
       return proto.HasExtension(serialization::DestroyAggregationStateWorkOrder::aggr_state_index) &&
@@ -1033,7 +1033,7 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
                  proto.GetExtension(serialization::UpdateWorkOrder::update_group_index)) &&
              proto.HasExtension(serialization::UpdateWorkOrder::operator_index) &&
              proto.HasExtension(serialization::UpdateWorkOrder::block_id) &&
-             proto.GetExtension(serialization::UpdateWorkOrder::partition_id);
+             proto.HasExtension(serialization::UpdateWorkOrder::partition_id);
     }
     case serialization::WINDOW_AGGREGATION: {
       return proto.HasExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index) &&


[08/27] incubator-quickstep git commit: Optimized the mod operation in HashPartition.

Posted by ji...@apache.org.
Optimized the mod operation in HashPartition.


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

Branch: refs/heads/trace
Commit: 7fb7a775568b49178af80f15383b641bc1b267bf
Parents: 1b2698d
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Fri Sep 29 15:37:14 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Fri Sep 29 18:03:21 2017 -0500

----------------------------------------------------------------------
 catalog/PartitionSchemeHeader.hpp | 18 +++++++++++++-----
 1 file changed, 13 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7fb7a775/catalog/PartitionSchemeHeader.hpp
----------------------------------------------------------------------
diff --git a/catalog/PartitionSchemeHeader.hpp b/catalog/PartitionSchemeHeader.hpp
index d34ca1f..35d4081 100644
--- a/catalog/PartitionSchemeHeader.hpp
+++ b/catalog/PartitionSchemeHeader.hpp
@@ -187,7 +187,8 @@ class HashPartitionSchemeHeader final : public PartitionSchemeHeader {
    **/
   HashPartitionSchemeHeader(const std::size_t num_partitions,
                             PartitionAttributeIds &&attributes)  // NOLINT(whitespace/operators)
-      : PartitionSchemeHeader(PartitionType::kHash, num_partitions, std::move(attributes)) {
+      : PartitionSchemeHeader(PartitionType::kHash, num_partitions, std::move(attributes)),
+        is_power_of_two_(!(num_partitions & (num_partitions - 1))) {
   }
 
   /**
@@ -199,13 +200,20 @@ class HashPartitionSchemeHeader final : public PartitionSchemeHeader {
   partition_id getPartitionId(
       const PartitionValues &value_of_attributes) const override {
     DCHECK_EQ(partition_attribute_ids_.size(), value_of_attributes.size());
-    // TODO(gerald): Optimize for the case where the number of partitions is a
-    // power of 2. We can just mask out the lower-order hash bits rather than
-    // doing a division operation.
-    return HashCompositeKey(value_of_attributes) % num_partitions_;
+    return getPartitionId(HashCompositeKey(value_of_attributes));
   }
 
  private:
+  partition_id getPartitionId(const std::size_t hash_code) const {
+    if (is_power_of_two_) {
+      return hash_code & (num_partitions_ - 1);
+    }
+
+    return (hash_code >= num_partitions_) ? hash_code % num_partitions_
+                                          : hash_code;
+  }
+
+  const bool is_power_of_two_;
   DISALLOW_COPY_AND_ASSIGN(HashPartitionSchemeHeader);
 };
 


[14/27] incubator-quickstep git commit: Added a new set API for TupleIdSequence.

Posted by ji...@apache.org.
Added a new set API for TupleIdSequence.


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

Branch: refs/heads/trace
Commit: 69fd94b8917c53e5a7a3e5899382c6ba12cf1c2b
Parents: a61b03d
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Thu Sep 28 19:28:30 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Mon Oct 9 13:14:07 2017 -0500

----------------------------------------------------------------------
 expressions/scalar/ScalarCaseExpression.cpp     |   2 +-
 relational_operators/HashJoinOperator.cpp       |   2 +-
 storage/AggregationOperationState.cpp           |   3 +-
 storage/BloomFilterIndexSubBlock.cpp            |   2 +-
 storage/CMakeLists.txt                          |   1 +
 storage/CSBTreeIndexSubBlock.cpp                |  22 +-
 ...ompressedColumnStoreTupleStorageSubBlock.cpp |  36 +--
 ...ressedPackedRowStoreTupleStorageSubBlock.cpp |  36 +--
 storage/InsertDestination.hpp                   |   4 +-
 storage/SMAIndexSubBlock.cpp                    |   2 +-
 storage/StorageBlock.cpp                        |   9 +-
 storage/TupleIdSequence.hpp                     |  15 ++
 storage/TupleStorageSubBlock.cpp                |   2 +-
 utility/BitVector.hpp                           |  16 ++
 utility/lip_filter/LIPFilterAdaptiveProber.hpp  |   4 +-
 utility/tests/BitVector_unittest.cpp            | 226 +++++++++----------
 16 files changed, 206 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/expressions/scalar/ScalarCaseExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarCaseExpression.cpp b/expressions/scalar/ScalarCaseExpression.cpp
index 00a7710..6847425 100644
--- a/expressions/scalar/ScalarCaseExpression.cpp
+++ b/expressions/scalar/ScalarCaseExpression.cpp
@@ -319,7 +319,7 @@ ColumnVectorPtr ScalarCaseExpression::getAllValuesForJoin(
                                                 *right_accessor,
                                                 right_relation_id,
                                                 check_pair.second)) {
-        current_case_positions->set(pos, true);
+        current_case_positions->set(pos);
         current_case_matches.emplace_back(check_pair);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index b07e4cb..4083bd3 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -758,7 +758,7 @@ void HashSemiJoinWorkOrder::executeWithResidualPredicate() {
                                                       *probe_accessor,
                                                       probe_relation_id,
                                                       hash_match.second)) {
-        filter.set(hash_match.second, true);
+        filter.set(hash_match.second);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 0f4795f..73f1983 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -579,8 +579,7 @@ void AggregationOperationState::aggregateBlockHashTableImplPartitioned(
           accessor->getTupleWithAttributes(group_by_key_ids));
       const std::size_t curr_tuple_partition_id =
           curr_tuple->getTupleHash() % num_partitions;
-      partition_membership[curr_tuple_partition_id]->set(
-          accessor->getCurrentPosition(), true);
+      partition_membership[curr_tuple_partition_id]->set(accessor->getCurrentPosition());
     }
 
     // Aggregate each partition.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/BloomFilterIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/BloomFilterIndexSubBlock.cpp b/storage/BloomFilterIndexSubBlock.cpp
index 4351c05..1af3872 100644
--- a/storage/BloomFilterIndexSubBlock.cpp
+++ b/storage/BloomFilterIndexSubBlock.cpp
@@ -206,7 +206,7 @@ TupleIdSequence* BloomFilterIndexSubBlock::getMatchesForPredicate(
     } else {
       for (tuple_id tid = 0; tid <= tuple_store_.getMaxTupleID(); ++tid) {
         if (tuple_store_.hasTupleWithID(tid)) {
-          tuple_sequence->set(tid, true);
+          tuple_sequence->set(tid);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 92a3292..fb09e49 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -1065,6 +1065,7 @@ target_link_libraries(quickstep_storage_ThreadPrivateCompactKeyHashTable
                       quickstep_utility_Macros
                       quickstep_utility_ScopedBuffer)
 target_link_libraries(quickstep_storage_TupleIdSequence
+                      glog
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_BitVector
                       quickstep_utility_Macros)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/CSBTreeIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/CSBTreeIndexSubBlock.cpp b/storage/CSBTreeIndexSubBlock.cpp
index dd75467..cc18e89 100644
--- a/storage/CSBTreeIndexSubBlock.cpp
+++ b/storage/CSBTreeIndexSubBlock.cpp
@@ -1791,7 +1791,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateEqualPredicate(
           // End of matches.
           return matches.release();
         }
-        matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_), true);
+        matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_));
       }
       key_ptr += key_tuple_id_pair_length_bytes_;
     }
@@ -1824,7 +1824,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateNotEqualPredicate(
                                + sizeof(NodeHeader)
                                + key_length_bytes_;
     for (uint16_t entry_num = 0; entry_num < num_keys; ++entry_num) {
-      matches->set(*reinterpret_cast<const tuple_id*>(tuple_id_ptr), true);
+      matches->set(*reinterpret_cast<const tuple_id*>(tuple_id_ptr));
       tuple_id_ptr += key_tuple_id_pair_length_bytes_;
     }
     search_node = getRightSiblingOfLeafNode(search_node);
@@ -1841,7 +1841,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateNotEqualPredicate(
       if (!equal_found) {
         if (key_less_literal_comparator.compareDataPtrsInl(key_ptr, literal)) {
           // key < literal
-          matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_), true);
+          matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_));
         } else {
           equal_found = true;
         }
@@ -1854,7 +1854,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateNotEqualPredicate(
           for (uint16_t subsequent_num = entry_num;
                subsequent_num < num_keys;
                ++subsequent_num) {
-            matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_), true);
+            matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_));
             key_ptr += key_tuple_id_pair_length_bytes_;
           }
           past_equal = true;
@@ -1877,7 +1877,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateNotEqualPredicate(
                                + sizeof(NodeHeader)
                                + key_length_bytes_;
     for (uint16_t entry_num = 0; entry_num < num_keys; ++entry_num) {
-      matches->set(*reinterpret_cast<const tuple_id*>(tuple_id_ptr), true);
+      matches->set(*reinterpret_cast<const tuple_id*>(tuple_id_ptr));
       tuple_id_ptr += key_tuple_id_pair_length_bytes_;
     }
     search_node = getRightSiblingOfLeafNode(search_node);
@@ -1910,7 +1910,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateLessPredicate(
                                + sizeof(NodeHeader)
                                + key_length_bytes_;
     for (uint16_t entry_num = 0; entry_num < num_keys; ++entry_num) {
-      matches->set(*reinterpret_cast<const tuple_id*>(tuple_id_ptr), true);
+      matches->set(*reinterpret_cast<const tuple_id*>(tuple_id_ptr));
       tuple_id_ptr += key_tuple_id_pair_length_bytes_;
     }
     search_node = getRightSiblingOfLeafNode(search_node);
@@ -1927,7 +1927,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateLessPredicate(
         if (!equal_found) {
           if (key_less_literal_comparator.compareDataPtrsInl(key_ptr, literal)) {
             // key < literal
-            matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_), true);
+            matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_));
           } else {
             equal_found = true;
           }
@@ -1938,7 +1938,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateLessPredicate(
             // literal < key
             return matches.release();
           } else {
-            matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_), true);
+            matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_));
           }
         }
 
@@ -1954,7 +1954,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateLessPredicate(
       for (uint16_t entry_num = 0; entry_num < num_keys; ++entry_num) {
         if (key_less_literal_comparator.compareDataPtrsInl(key_ptr, literal)) {
           // key < literal
-          matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_), true);
+          matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_));
         } else {
           return matches.release();
         }
@@ -2001,7 +2001,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateGreaterPredicate(
       if (match_found) {
         // Fill in the matching entries from this leaf.
         for (uint16_t match_num = entry_num; match_num < num_keys; ++match_num) {
-          matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_), true);
+          matches->set(*reinterpret_cast<const tuple_id*>(key_ptr + key_length_bytes_));
           key_ptr += key_tuple_id_pair_length_bytes_;
         }
         break;
@@ -2024,7 +2024,7 @@ TupleIdSequence* CSBTreeIndexSubBlock::evaluateGreaterPredicate(
                                + sizeof(NodeHeader)
                                + key_length_bytes_;
     for (uint16_t entry_num = 0; entry_num < num_keys; ++entry_num) {
-      matches->set(*reinterpret_cast<const tuple_id*>(tuple_id_ptr), true);
+      matches->set(*reinterpret_cast<const tuple_id*>(tuple_id_ptr));
       tuple_id_ptr += key_tuple_id_pair_length_bytes_;
     }
     search_node = getRightSiblingOfLeafNode(search_node);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/CompressedColumnStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/CompressedColumnStoreTupleStorageSubBlock.cpp b/storage/CompressedColumnStoreTupleStorageSubBlock.cpp
index 3bd0c3a..cc23445 100644
--- a/storage/CompressedColumnStoreTupleStorageSubBlock.cpp
+++ b/storage/CompressedColumnStoreTupleStorageSubBlock.cpp
@@ -538,7 +538,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getNotEqualCodesExcl
                ++tid) {
             if ((code != static_cast<const uint8_t*>(attr_stripe)[tid])
                 && (null_code != static_cast<const uint8_t*>(attr_stripe)[tid])) {
-              matches->set(tid, true);
+              matches->set(tid);
             }
           }
           break;
@@ -548,7 +548,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getNotEqualCodesExcl
                ++tid) {
             if ((code != static_cast<const uint16_t*>(attr_stripe)[tid])
                 && (null_code != static_cast<const uint16_t*>(attr_stripe)[tid])) {
-              matches->set(tid, true);
+              matches->set(tid);
             }
           }
           break;
@@ -558,7 +558,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getNotEqualCodesExcl
                ++tid) {
             if ((code != static_cast<const uint16_t*>(attr_stripe)[tid])
                 && (null_code != static_cast<const uint16_t*>(attr_stripe)[tid])) {
-              matches->set(tid, true);
+              matches->set(tid);
             }
           }
           break;
@@ -578,7 +578,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getNotEqualCodesExcl
                ++filter_it) {
             if ((code != static_cast<const uint8_t*>(attr_stripe)[*filter_it])
                 && (null_code != static_cast<const uint8_t*>(attr_stripe)[*filter_it])) {
-              matches->set(*filter_it, true);
+              matches->set(*filter_it);
             }
           }
           break;
@@ -588,7 +588,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getNotEqualCodesExcl
                ++filter_it) {
             if ((code != static_cast<const uint16_t*>(attr_stripe)[*filter_it])
                 && (null_code != static_cast<const uint16_t*>(attr_stripe)[*filter_it])) {
-              matches->set(*filter_it, true);
+              matches->set(*filter_it);
             }
           }
           break;
@@ -598,7 +598,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getNotEqualCodesExcl
                ++filter_it) {
             if ((code != static_cast<const uint16_t*>(attr_stripe)[*filter_it])
                 && (null_code != static_cast<const uint16_t*>(attr_stripe)[*filter_it])) {
-              matches->set(*filter_it, true);
+              matches->set(*filter_it);
             }
           }
           break;
@@ -677,7 +677,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesInRange(
                ++tid) {
             if (range.first <= (static_cast<const uint8_t*>(attr_stripe)[tid])
                 && (static_cast<const uint8_t*>(attr_stripe)[tid] < range.second)) {
-              matches->set(tid, true);
+              matches->set(tid);
             }
           }
           break;
@@ -687,7 +687,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesInRange(
                ++tid) {
             if (range.first <= (static_cast<const uint16_t*>(attr_stripe)[tid])
                 && (static_cast<const uint16_t*>(attr_stripe)[tid] < range.second)) {
-              matches->set(tid, true);
+              matches->set(tid);
             }
           }
           break;
@@ -697,7 +697,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesInRange(
                ++tid) {
             if (range.first <= (static_cast<const uint32_t*>(attr_stripe)[tid])
                 && (static_cast<const uint32_t*>(attr_stripe)[tid] < range.second)) {
-              matches->set(tid, true);
+              matches->set(tid);
             }
           }
           break;
@@ -717,7 +717,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesInRange(
                ++filter_it) {
             if (range.first <= (static_cast<const uint8_t*>(attr_stripe)[*filter_it])
                 && (static_cast<const uint8_t*>(attr_stripe)[*filter_it] < range.second)) {
-              matches->set(*filter_it, true);
+              matches->set(*filter_it);
             }
           }
           break;
@@ -727,7 +727,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesInRange(
                ++filter_it) {
             if (range.first <= (static_cast<const uint16_t*>(attr_stripe)[*filter_it])
                 && (static_cast<const uint16_t*>(attr_stripe)[*filter_it] < range.second)) {
-              matches->set(*filter_it, true);
+              matches->set(*filter_it);
             }
           }
           break;
@@ -737,7 +737,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesInRange(
                ++filter_it) {
             if (range.first <= (static_cast<const uint32_t*>(attr_stripe)[*filter_it])
                 && (static_cast<const uint32_t*>(attr_stripe)[*filter_it] < range.second)) {
-              matches->set(*filter_it, true);
+              matches->set(*filter_it);
             }
           }
           break;
@@ -924,7 +924,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesSatisfyingCo
              tid < *static_cast<const tuple_id*>(sub_block_memory_);
              ++tid) {
           if (comp(code, static_cast<const uint8_t*>(attr_stripe)[tid])) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -933,7 +933,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesSatisfyingCo
              tid < *static_cast<const tuple_id*>(sub_block_memory_);
              ++tid) {
           if (comp(code, static_cast<const uint16_t*>(attr_stripe)[tid])) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -942,7 +942,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesSatisfyingCo
              tid < *static_cast<const tuple_id*>(sub_block_memory_);
              ++tid) {
           if (comp(code, static_cast<const uint32_t*>(attr_stripe)[tid])) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -961,7 +961,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesSatisfyingCo
              filter_it != filter->end();
              ++filter_it) {
           if (comp(code, static_cast<const uint8_t*>(attr_stripe)[*filter_it])) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -970,7 +970,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesSatisfyingCo
              filter_it != filter->end();
              ++filter_it) {
           if (comp(code, static_cast<const uint16_t*>(attr_stripe)[*filter_it])) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -979,7 +979,7 @@ TupleIdSequence* CompressedColumnStoreTupleStorageSubBlock::getCodesSatisfyingCo
              filter_it != filter->end();
              ++filter_it) {
           if (comp(code, static_cast<const uint32_t*>(attr_stripe)[*filter_it])) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp b/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp
index d362f98..f087293 100644
--- a/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp
+++ b/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp
@@ -403,7 +403,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getNotEqualCodesE
              ++tid, attr_location += tuple_length_bytes_) {
           if ((code != *reinterpret_cast<const uint8_t*>(attr_location))
               && (null_code != *reinterpret_cast<const uint8_t*>(attr_location))) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -413,7 +413,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getNotEqualCodesE
              ++tid, attr_location += tuple_length_bytes_) {
           if ((code != *reinterpret_cast<const uint16_t*>(attr_location))
               && (null_code != *reinterpret_cast<const uint16_t*>(attr_location))) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -423,7 +423,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getNotEqualCodesE
              ++tid, attr_location += tuple_length_bytes_) {
           if ((code != *reinterpret_cast<const uint32_t*>(attr_location))
               && (null_code != *reinterpret_cast<const uint32_t*>(attr_location))) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -444,7 +444,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getNotEqualCodesE
           const void *local_attr_location = attr_location + (*filter_it) * tuple_length_bytes_;
           if ((code != *reinterpret_cast<const uint8_t*>(local_attr_location))
               && (null_code != *reinterpret_cast<const uint8_t*>(local_attr_location))) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -455,7 +455,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getNotEqualCodesE
           const void *local_attr_location = attr_location + (*filter_it) * tuple_length_bytes_;
           if ((code != *reinterpret_cast<const uint16_t*>(local_attr_location))
               && (null_code != *reinterpret_cast<const uint16_t*>(local_attr_location))) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -466,7 +466,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getNotEqualCodesE
           const void *local_attr_location = attr_location + (*filter_it) * tuple_length_bytes_;
           if ((code != *reinterpret_cast<const uint32_t*>(local_attr_location))
               && (null_code != *reinterpret_cast<const uint32_t*>(local_attr_location))) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -513,7 +513,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesInRange(
              ++tid, attr_location += tuple_length_bytes_) {
           if (range.first <= (*reinterpret_cast<const uint8_t*>(attr_location))
               && (*reinterpret_cast<const uint8_t*>(attr_location) < range.second)) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -523,7 +523,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesInRange(
              ++tid, attr_location += tuple_length_bytes_) {
           if (range.first <= (*reinterpret_cast<const uint16_t*>(attr_location))
               && (*reinterpret_cast<const uint16_t*>(attr_location) < range.second)) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -533,7 +533,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesInRange(
              ++tid, attr_location += tuple_length_bytes_) {
           if (range.first <= (*reinterpret_cast<const uint32_t*>(attr_location))
               && (*reinterpret_cast<const uint32_t*>(attr_location) < range.second)) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -554,7 +554,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesInRange(
           const void *local_attr_location = attr_location + (*filter_it) * tuple_length_bytes_;
           if (range.first <= (*reinterpret_cast<const uint8_t*>(local_attr_location))
               && (*reinterpret_cast<const uint8_t*>(local_attr_location) < range.second)) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -565,7 +565,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesInRange(
           const void *local_attr_location = attr_location + (*filter_it) * tuple_length_bytes_;
           if (range.first <= (*reinterpret_cast<const uint16_t*>(local_attr_location))
               && (*reinterpret_cast<const uint16_t*>(local_attr_location) < range.second)) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -576,7 +576,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesInRange(
           const void *local_attr_location = attr_location + (*filter_it) * tuple_length_bytes_;
           if (range.first <= (*reinterpret_cast<const uint32_t*>(local_attr_location))
               && (*reinterpret_cast<const uint32_t*>(local_attr_location) < range.second)) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -638,7 +638,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesSatisfyin
              tid < *static_cast<const tuple_id*>(sub_block_memory_);
              ++tid, attr_location += tuple_length_bytes_) {
           if (comp(code, *reinterpret_cast<const uint8_t*>(attr_location))) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -647,7 +647,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesSatisfyin
              tid < *static_cast<const tuple_id*>(sub_block_memory_);
              ++tid, attr_location += tuple_length_bytes_) {
           if (comp(code, *reinterpret_cast<const uint16_t*>(attr_location))) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -656,7 +656,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesSatisfyin
              tid < *static_cast<const tuple_id*>(sub_block_memory_);
              ++tid, attr_location += tuple_length_bytes_) {
           if (comp(code, *reinterpret_cast<const uint32_t*>(attr_location))) {
-            matches->set(tid, true);
+            matches->set(tid);
           }
         }
         break;
@@ -676,7 +676,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesSatisfyin
              ++filter_it) {
           const void *local_attr_location = attr_location + (*filter_it) * tuple_length_bytes_;
           if (comp(code, *reinterpret_cast<const uint8_t*>(local_attr_location))) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -686,7 +686,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesSatisfyin
              ++filter_it) {
           const void *local_attr_location = attr_location + (*filter_it) * tuple_length_bytes_;
           if (comp(code, *reinterpret_cast<const uint16_t*>(local_attr_location))) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;
@@ -696,7 +696,7 @@ TupleIdSequence* CompressedPackedRowStoreTupleStorageSubBlock::getCodesSatisfyin
              ++filter_it) {
           const void *local_attr_location = attr_location + (*filter_it) * tuple_length_bytes_;
           if (comp(code, *reinterpret_cast<const uint32_t*>(local_attr_location))) {
-            matches->set(*filter_it, true);
+            matches->set(*filter_it);
           }
         }
         break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index a0a7bc2..ab791b1 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -646,7 +646,7 @@ class PartitionAwareInsertDestination : public InsertDestination {
 
     if (partition_attr_ids.empty()) {
       while (accessor->next()) {
-        (*partition_membership)[input_partition_id_]->set(accessor->getCurrentPosition(), true);
+        (*partition_membership)[input_partition_id_]->set(accessor->getCurrentPosition());
       }
     } else {
       PartitionSchemeHeader::PartitionValues values(partition_attr_ids.size());
@@ -655,7 +655,7 @@ class PartitionAwareInsertDestination : public InsertDestination {
           values[i] = accessor->getTypedValue(partition_attr_ids[i]);
         }
         (*partition_membership)[partition_scheme_header_->getPartitionId(values)]->set(
-            accessor->getCurrentPosition(), true);
+            accessor->getCurrentPosition());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/SMAIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/SMAIndexSubBlock.cpp b/storage/SMAIndexSubBlock.cpp
index 3b3b879..81cf6c0 100644
--- a/storage/SMAIndexSubBlock.cpp
+++ b/storage/SMAIndexSubBlock.cpp
@@ -685,7 +685,7 @@ TupleIdSequence* SMAIndexSubBlock::getMatchesForPredicate(
       } else {
         for (tuple_id tid = 0; tid <= tuple_store_.getMaxTupleID(); ++tid) {
           if (tuple_store_.hasTupleWithID(tid)) {
-            tidseq->set(tid, true);
+            tidseq->set(tid);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 31f1db2..e3e47d1 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -347,8 +347,7 @@ void StorageBlock::sample(const bool is_block_sample,
         sequence_mask->set(tuple_index_mapping.find(random_number) ==
                                    tuple_index_mapping.end()
                                ? random_number
-                               : tuple_index_mapping[random_number],
-                           true);
+                               : tuple_index_mapping[random_number]);
         tuple_index_mapping[random_number] = sequence->length() - (n + 1);
     }
 
@@ -461,7 +460,7 @@ StorageBlock::UpdateResult StorageBlock::update(
         tuple_store_->setAttributeValueInPlaceTyped(*match_it, update_it->first, update_it->second);
       }
 
-      in_place_ids.set(*match_it, true);
+      in_place_ids.set(*match_it);
     } else {
       // Make a copy of the tuple with the updated values.
       std::vector<TypedValue> updated_tuple_values;
@@ -479,7 +478,7 @@ StorageBlock::UpdateResult StorageBlock::update(
       }
 
       relocation_buffer.emplace_back(std::move(updated_tuple_values));
-      relocate_ids.set(*match_it, true);
+      relocate_ids.set(*match_it);
     }
   }
 
@@ -526,7 +525,7 @@ StorageBlock::UpdateResult StorageBlock::update(
             rebuild_all = true;
           } else {
             // Only bother adding 'reinsert_id' to 'in_place_ids' if not rebuilding.
-            in_place_ids.set(reinsert_result.inserted_id, true);
+            in_place_ids.set(reinsert_result.inserted_id);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/TupleIdSequence.hpp
----------------------------------------------------------------------
diff --git a/storage/TupleIdSequence.hpp b/storage/TupleIdSequence.hpp
index 5047270..8c3a074 100644
--- a/storage/TupleIdSequence.hpp
+++ b/storage/TupleIdSequence.hpp
@@ -28,6 +28,8 @@
 #include "utility/BitVector.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 /** \addtogroup Storage
@@ -155,6 +157,19 @@ class TupleIdSequence {
   }
 
   /**
+   * @brief Set a particular tuple ID as being on or off in this sequence.
+   *
+   * @param on If true, tuple should be part of this sequence. If false,
+   *        remove tuple from this sequence (if it was present).
+   **/
+  inline void set(const tuple_id tuple) {
+    DCHECK_GE(tuple, 0);
+    DCHECK_LT(static_cast<std::size_t>(tuple), internal_bitvector_.size());
+
+    internal_bitvector_.setBit(tuple);
+  }
+
+  /**
    * @brief Set a range of tuple IDs all at once.
    *
    * @param first_tuple The first ID to set.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/storage/TupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/TupleStorageSubBlock.cpp b/storage/TupleStorageSubBlock.cpp
index c5d482f..f2eef49 100644
--- a/storage/TupleStorageSubBlock.cpp
+++ b/storage/TupleStorageSubBlock.cpp
@@ -65,7 +65,7 @@ TupleIdSequence* TupleStorageSubBlock::getExistenceMap() const {
   } else {
     for (tuple_id tid = 0; tid <= max_tid; ++tid) {
       if (hasTupleWithID(tid)) {
-        existing_tuples->set(tid, true);
+        existing_tuples->set(tid);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/utility/BitVector.hpp
----------------------------------------------------------------------
diff --git a/utility/BitVector.hpp b/utility/BitVector.hpp
index c404b7e..4472407 100644
--- a/utility/BitVector.hpp
+++ b/utility/BitVector.hpp
@@ -279,6 +279,18 @@ class BitVector {
   }
 
   /**
+   * @brief Set the value of a single bit.
+   *
+   * @param bit_num The desired bit in this BitVector.
+   **/
+  inline void setBit(const std::size_t bit_num) {
+    DCHECK(!short_version_) << "Not implemented.";
+    DCHECK_LT(bit_num, num_bits_);
+
+    setBitRegularVersion(bit_num);
+  }
+
+  /**
    * @brief Set the value of a range of bits simulaneously.
    *
    * @param start_bit_num The first bit whose value should be set.
@@ -918,6 +930,10 @@ class BitVector {
     data_array_[index_pos_in_data_array] |= op_value;
   }
 
+  inline void setBitRegularVersion(const std::size_t bit_num) {
+    data_array_[bit_num >> kHigherOrderShift] |= (TopBit<std::size_t>() >> (bit_num & kLowerOrderMask));
+  }
+
   template <typename VectorType>
   inline void setBitShortVersion(const std::size_t bit_num, bool value) {
     if (value) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/utility/lip_filter/LIPFilterAdaptiveProber.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterAdaptiveProber.hpp b/utility/lip_filter/LIPFilterAdaptiveProber.hpp
index e1a75d6..fb826da 100644
--- a/utility/lip_filter/LIPFilterAdaptiveProber.hpp
+++ b/utility/lip_filter/LIPFilterAdaptiveProber.hpp
@@ -147,7 +147,7 @@ class LIPFilterAdaptiveProber {
 
       const std::uint32_t num_hits = filterBatch(accessor, &batch, batch_size);
       for (std::uint32_t i = 0; i < num_hits; ++i) {
-        matches->set(batch[i], true);
+        matches->set(batch[i]);
       }
 
       batch_start += batch_size;
@@ -181,7 +181,7 @@ class LIPFilterAdaptiveProber {
 
       const std::uint32_t num_hits = filterBatch(accessor, &batch, batch_size);
       for (std::uint32_t i = 0; i < num_hits; ++i) {
-        matches->set(batch[i], true);
+        matches->set(batch[i]);
       }
 
       num_tuples_left -= batch_size;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/69fd94b8/utility/tests/BitVector_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/BitVector_unittest.cpp b/utility/tests/BitVector_unittest.cpp
index 774b830..053f405 100644
--- a/utility/tests/BitVector_unittest.cpp
+++ b/utility/tests/BitVector_unittest.cpp
@@ -357,14 +357,14 @@ TYPED_TEST(BitVectorTest, SetAndGetTest) {
   big_bit_vector->clear();
 
   // Set some bits, particularly around potential boundaries between size_t strides.
-  big_bit_vector->setBit(0, true);
-  big_bit_vector->setBit(5, true);
-  big_bit_vector->setBit(31, true);
-  big_bit_vector->setBit(32, true);
-  big_bit_vector->setBit(63, true);
-  big_bit_vector->setBit(64, true);
-  big_bit_vector->setBit(127, true);
-  big_bit_vector->setBit(128, true);
+  big_bit_vector->setBit(0);
+  big_bit_vector->setBit(5);
+  big_bit_vector->setBit(31);
+  big_bit_vector->setBit(32);
+  big_bit_vector->setBit(63);
+  big_bit_vector->setBit(64);
+  big_bit_vector->setBit(127);
+  big_bit_vector->setBit(128);
 
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
     if ((i == 0) || (i == 5) || (i == 31) || (i == 32) || (i == 63) || (i == 64) || (i == 127) || (i == 128)) {
@@ -454,7 +454,7 @@ TYPED_TEST(BitVectorTest, SetAndGetTest) {
 
   // Set all the bits.
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
-    big_bit_vector->setBit(i, true);
+    big_bit_vector->setBit(i);
   }
   EXPECT_EQ(TestFixture::kBigBitSize, big_bit_vector->onesCount());
   EXPECT_EQ(TestFixture::kBigBitSize, big_bit_vector->firstZero());
@@ -487,10 +487,10 @@ TYPED_TEST(BitVectorTest, AssignFromTest) {
   std::unique_ptr<TypeParam> big_bit_vector(this->createBitVector(TestFixture::kBigBitSize));
   big_bit_vector->clear();
 
-  big_bit_vector->setBit(2, true);
-  big_bit_vector->setBit(4, true);
-  big_bit_vector->setBit(42, true);
-  big_bit_vector->setBit(77, true);
+  big_bit_vector->setBit(2);
+  big_bit_vector->setBit(4);
+  big_bit_vector->setBit(42);
+  big_bit_vector->setBit(77);
 
   std::unique_ptr<TypeParam> big_bit_vector_copy(this->createBitVector(TestFixture::kBigBitSize));
   big_bit_vector_copy->assignFrom(*big_bit_vector);
@@ -693,14 +693,14 @@ TYPED_TEST(BitVectorTest, ShiftTailForwardTest) {
   std::unique_ptr<TypeParam> big_bit_vector(this->createBitVector(TestFixture::kBigBitSize));
   big_bit_vector->clear();
 
-  big_bit_vector->setBit(11, true);
-  big_bit_vector->setBit(13, true);
-  big_bit_vector->setBit(27, true);
-  big_bit_vector->setBit(42, true);
-  big_bit_vector->setBit(84, true);
-  big_bit_vector->setBit(88, true);
-  big_bit_vector->setBit(91, true);
-  big_bit_vector->setBit(123, true);
+  big_bit_vector->setBit(11);
+  big_bit_vector->setBit(13);
+  big_bit_vector->setBit(27);
+  big_bit_vector->setBit(42);
+  big_bit_vector->setBit(84);
+  big_bit_vector->setBit(88);
+  big_bit_vector->setBit(91);
+  big_bit_vector->setBit(123);
 
   big_bit_vector->shiftTailForward(15, 70);
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
@@ -714,16 +714,16 @@ TYPED_TEST(BitVectorTest, ShiftTailForwardTest) {
 
   // Also try a relatively small shift within the same word.
   big_bit_vector->clear();
-  big_bit_vector->setBit(11, true);
-  big_bit_vector->setBit(13, true);
-  big_bit_vector->setBit(27, true);
-  big_bit_vector->setBit(42, true);
-  big_bit_vector->setBit(45, true);
-  big_bit_vector->setBit(51, true);
-  big_bit_vector->setBit(84, true);
-  big_bit_vector->setBit(88, true);
-  big_bit_vector->setBit(91, true);
-  big_bit_vector->setBit(123, true);
+  big_bit_vector->setBit(11);
+  big_bit_vector->setBit(13);
+  big_bit_vector->setBit(27);
+  big_bit_vector->setBit(42);
+  big_bit_vector->setBit(45);
+  big_bit_vector->setBit(51);
+  big_bit_vector->setBit(84);
+  big_bit_vector->setBit(88);
+  big_bit_vector->setBit(91);
+  big_bit_vector->setBit(123);
 
   big_bit_vector->shiftTailForward(43, 7);
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
@@ -738,18 +738,18 @@ TYPED_TEST(BitVectorTest, ShiftTailForwardTest) {
 
   // Align the shift distance to size_t.
   big_bit_vector->clear();
-  big_bit_vector->setBit(11, true);
-  big_bit_vector->setBit(13, true);
-  big_bit_vector->setBit(27, true);
-  big_bit_vector->setBit(42, true);
-  big_bit_vector->setBit(45, true);
-  big_bit_vector->setBit(51, true);
-  big_bit_vector->setBit(84, true);
-  big_bit_vector->setBit(88, true);
-  big_bit_vector->setBit(91, true);
-  big_bit_vector->setBit(123, true);
-  big_bit_vector->setBit(128, true);
-  big_bit_vector->setBit(137, true);
+  big_bit_vector->setBit(11);
+  big_bit_vector->setBit(13);
+  big_bit_vector->setBit(27);
+  big_bit_vector->setBit(42);
+  big_bit_vector->setBit(45);
+  big_bit_vector->setBit(51);
+  big_bit_vector->setBit(84);
+  big_bit_vector->setBit(88);
+  big_bit_vector->setBit(91);
+  big_bit_vector->setBit(123);
+  big_bit_vector->setBit(128);
+  big_bit_vector->setBit(137);
 
   big_bit_vector->shiftTailForward(43, 64);
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
@@ -764,18 +764,18 @@ TYPED_TEST(BitVectorTest, ShiftTailForwardTest) {
 
   // Exactly align both the tail start and the shift distance to size_t.
   big_bit_vector->clear();
-  big_bit_vector->setBit(11, true);
-  big_bit_vector->setBit(13, true);
-  big_bit_vector->setBit(27, true);
-  big_bit_vector->setBit(42, true);
-  big_bit_vector->setBit(45, true);
-  big_bit_vector->setBit(51, true);
-  big_bit_vector->setBit(84, true);
-  big_bit_vector->setBit(88, true);
-  big_bit_vector->setBit(91, true);
-  big_bit_vector->setBit(123, true);
-  big_bit_vector->setBit(128, true);
-  big_bit_vector->setBit(137, true);
+  big_bit_vector->setBit(11);
+  big_bit_vector->setBit(13);
+  big_bit_vector->setBit(27);
+  big_bit_vector->setBit(42);
+  big_bit_vector->setBit(45);
+  big_bit_vector->setBit(51);
+  big_bit_vector->setBit(84);
+  big_bit_vector->setBit(88);
+  big_bit_vector->setBit(91);
+  big_bit_vector->setBit(123);
+  big_bit_vector->setBit(128);
+  big_bit_vector->setBit(137);
 
   big_bit_vector->shiftTailForward(64, 64);
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
@@ -867,14 +867,14 @@ TYPED_TEST(BitVectorTest, ShiftTailBackwardTest) {
   std::unique_ptr<TypeParam> big_bit_vector(this->createBitVector(TestFixture::kBigBitSize));
   big_bit_vector->clear();
 
-  big_bit_vector->setBit(11, true);
-  big_bit_vector->setBit(13, true);
-  big_bit_vector->setBit(27, true);
-  big_bit_vector->setBit(42, true);
-  big_bit_vector->setBit(84, true);
-  big_bit_vector->setBit(88, true);
-  big_bit_vector->setBit(91, true);
-  big_bit_vector->setBit(123, true);
+  big_bit_vector->setBit(11);
+  big_bit_vector->setBit(13);
+  big_bit_vector->setBit(27);
+  big_bit_vector->setBit(42);
+  big_bit_vector->setBit(84);
+  big_bit_vector->setBit(88);
+  big_bit_vector->setBit(91);
+  big_bit_vector->setBit(123);
 
   big_bit_vector->shiftTailBackward(15, 70);
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
@@ -889,16 +889,16 @@ TYPED_TEST(BitVectorTest, ShiftTailBackwardTest) {
 
   // Also try a relatively small shift within the same word.
   big_bit_vector->clear();
-  big_bit_vector->setBit(11, true);
-  big_bit_vector->setBit(13, true);
-  big_bit_vector->setBit(27, true);
-  big_bit_vector->setBit(42, true);
-  big_bit_vector->setBit(45, true);
-  big_bit_vector->setBit(51, true);
-  big_bit_vector->setBit(84, true);
-  big_bit_vector->setBit(88, true);
-  big_bit_vector->setBit(91, true);
-  big_bit_vector->setBit(123, true);
+  big_bit_vector->setBit(11);
+  big_bit_vector->setBit(13);
+  big_bit_vector->setBit(27);
+  big_bit_vector->setBit(42);
+  big_bit_vector->setBit(45);
+  big_bit_vector->setBit(51);
+  big_bit_vector->setBit(84);
+  big_bit_vector->setBit(88);
+  big_bit_vector->setBit(91);
+  big_bit_vector->setBit(123);
 
   big_bit_vector->shiftTailBackward(43, 7);
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
@@ -913,18 +913,18 @@ TYPED_TEST(BitVectorTest, ShiftTailBackwardTest) {
 
   // Align the shift distance to size_t.
   big_bit_vector->clear();
-  big_bit_vector->setBit(11, true);
-  big_bit_vector->setBit(13, true);
-  big_bit_vector->setBit(27, true);
-  big_bit_vector->setBit(42, true);
-  big_bit_vector->setBit(45, true);
-  big_bit_vector->setBit(51, true);
-  big_bit_vector->setBit(84, true);
-  big_bit_vector->setBit(88, true);
-  big_bit_vector->setBit(91, true);
-  big_bit_vector->setBit(123, true);
-  big_bit_vector->setBit(128, true);
-  big_bit_vector->setBit(137, true);
+  big_bit_vector->setBit(11);
+  big_bit_vector->setBit(13);
+  big_bit_vector->setBit(27);
+  big_bit_vector->setBit(42);
+  big_bit_vector->setBit(45);
+  big_bit_vector->setBit(51);
+  big_bit_vector->setBit(84);
+  big_bit_vector->setBit(88);
+  big_bit_vector->setBit(91);
+  big_bit_vector->setBit(123);
+  big_bit_vector->setBit(128);
+  big_bit_vector->setBit(137);
 
   big_bit_vector->shiftTailBackward(43, 64);
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
@@ -940,18 +940,18 @@ TYPED_TEST(BitVectorTest, ShiftTailBackwardTest) {
 
   // Exactly align both the tail start and the shift distance to size_t.
   big_bit_vector->clear();
-  big_bit_vector->setBit(11, true);
-  big_bit_vector->setBit(13, true);
-  big_bit_vector->setBit(27, true);
-  big_bit_vector->setBit(42, true);
-  big_bit_vector->setBit(45, true);
-  big_bit_vector->setBit(51, true);
-  big_bit_vector->setBit(84, true);
-  big_bit_vector->setBit(88, true);
-  big_bit_vector->setBit(91, true);
-  big_bit_vector->setBit(123, true);
-  big_bit_vector->setBit(128, true);
-  big_bit_vector->setBit(137, true);
+  big_bit_vector->setBit(11);
+  big_bit_vector->setBit(13);
+  big_bit_vector->setBit(27);
+  big_bit_vector->setBit(42);
+  big_bit_vector->setBit(45);
+  big_bit_vector->setBit(51);
+  big_bit_vector->setBit(84);
+  big_bit_vector->setBit(88);
+  big_bit_vector->setBit(91);
+  big_bit_vector->setBit(123);
+  big_bit_vector->setBit(128);
+  big_bit_vector->setBit(137);
 
   big_bit_vector->shiftTailBackward(64, 64);
   for (size_t i = 0; i < TestFixture::kBigBitSize; ++i) {
@@ -1027,14 +1027,14 @@ TYPED_TEST(BitVectorTest, RebindTest) {
 
   // Set some bits, particularly around potential boundaries between size_t
   // strides.
-  big_bit_vector->setBit(0, true);
-  big_bit_vector->setBit(5, true);
-  big_bit_vector->setBit(31, true);
-  big_bit_vector->setBit(32, true);
-  big_bit_vector->setBit(63, true);
-  big_bit_vector->setBit(64, true);
-  big_bit_vector->setBit(127, true);
-  big_bit_vector->setBit(128, true);
+  big_bit_vector->setBit(0);
+  big_bit_vector->setBit(5);
+  big_bit_vector->setBit(31);
+  big_bit_vector->setBit(32);
+  big_bit_vector->setBit(63);
+  big_bit_vector->setBit(64);
+  big_bit_vector->setBit(127);
+  big_bit_vector->setBit(128);
 
   // Destroy the old BitVector and create a new one bound to the same memory.
   ASSERT_EQ(1u, this->allocated_chunks_.size());
@@ -1093,19 +1093,19 @@ TYPED_TEST(BitVectorTest, AnyTest) {
   big_bit_vector->clear();
   EXPECT_FALSE(big_bit_vector->any());
 
-  big_bit_vector->setBit(0, true);
+  big_bit_vector->setBit(0);
   EXPECT_TRUE(big_bit_vector->any());
   big_bit_vector->setBit(0, false);
   EXPECT_FALSE(big_bit_vector->any());
 
-  big_bit_vector->setBit(TestFixture::kBigBitSize - 1, true);
+  big_bit_vector->setBit(TestFixture::kBigBitSize - 1);
   EXPECT_TRUE(big_bit_vector->any());
   big_bit_vector->setBit(TestFixture::kBigBitSize - 1, false);
   EXPECT_FALSE(big_bit_vector->any());
 
-  big_bit_vector->setBit(TestFixture::kBigBitSize / 2, true);
+  big_bit_vector->setBit(TestFixture::kBigBitSize / 2);
   EXPECT_TRUE(big_bit_vector->any());
-  big_bit_vector->setBit(TestFixture::kBigBitSize / 2 + 1, true);
+  big_bit_vector->setBit(TestFixture::kBigBitSize / 2 + 1);
   EXPECT_TRUE(big_bit_vector->any());
   big_bit_vector->setBit(TestFixture::kBigBitSize / 2, false);
   EXPECT_TRUE(big_bit_vector->any());
@@ -1150,21 +1150,21 @@ TYPED_TEST(BitVectorTest, AllTest) {
 
   big_bit_vector->setBit(0, false);
   EXPECT_FALSE(big_bit_vector->all());
-  big_bit_vector->setBit(0, true);
+  big_bit_vector->setBit(0);
   EXPECT_TRUE(big_bit_vector->all());
 
   big_bit_vector->setBit(TestFixture::kBigBitSize - 1, false);
   EXPECT_FALSE(big_bit_vector->all());
-  big_bit_vector->setBit(TestFixture::kBigBitSize - 1, true);
+  big_bit_vector->setBit(TestFixture::kBigBitSize - 1);
   EXPECT_TRUE(big_bit_vector->all());
 
   big_bit_vector->setBit(TestFixture::kBigBitSize / 2, false);
   EXPECT_FALSE(big_bit_vector->all());
   big_bit_vector->setBit(TestFixture::kBigBitSize / 2 + 1, false);
   EXPECT_FALSE(big_bit_vector->all());
-  big_bit_vector->setBit(TestFixture::kBigBitSize / 2, true);
+  big_bit_vector->setBit(TestFixture::kBigBitSize / 2);
   EXPECT_FALSE(big_bit_vector->all());
-  big_bit_vector->setBit(TestFixture::kBigBitSize / 2 + 1, true);
+  big_bit_vector->setBit(TestFixture::kBigBitSize / 2 + 1);
   EXPECT_TRUE(big_bit_vector->all());
 }
 



[16/27] incubator-quickstep git commit: Removed unused argument always_mark_full.

Posted by ji...@apache.org.
Removed unused argument always_mark_full.


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

Branch: refs/heads/trace
Commit: f820c45ee56a9e74671fb1c22c5e6b7e13471b5d
Parents: ffb8e05
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Thu Oct 5 17:18:05 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Tue Oct 10 14:14:29 2017 -0500

----------------------------------------------------------------------
 storage/InsertDestination.cpp          | 73 ++++++++++-------------------
 storage/InsertDestination.hpp          | 21 ++++-----
 storage/InsertDestinationInterface.hpp | 12 ++---
 3 files changed, 36 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f820c45e/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 8821019..c2b44d8 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -199,7 +199,7 @@ void InsertDestination::insertTupleInBatch(const Tuple &tuple) {
   returnBlock(std::move(output_block), false);
 }
 
-void InsertDestination::bulkInsertTuples(ValueAccessor *accessor, bool always_mark_full) {
+void InsertDestination::bulkInsertTuples(ValueAccessor *accessor, const bool always_mark_full) {
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {  // NOLINT(build/c++11)
@@ -207,23 +207,17 @@ void InsertDestination::bulkInsertTuples(ValueAccessor *accessor, bool always_ma
     while (!accessor->iterationFinished()) {
       MutableBlockReference output_block = this->getBlockForInsertion();
       // FIXME(chasseur): Deal with TupleTooLargeForBlock exception.
-      if (output_block->bulkInsertTuples(accessor) == 0) {
-        // output_block is full.
-        this->returnBlock(std::move(output_block), true);
-      } else {
-        // Bulk insert into output_block was successful. output_block
-        // will be rebuilt when there won't be any more insertions to it.
-        this->returnBlock(std::move(output_block),
-                          always_mark_full || !accessor->iterationFinished());
-      }
+      const auto num_tuples_inserted = output_block->bulkInsertTuples(accessor);
+      this->returnBlock(std::move(output_block),
+                        num_tuples_inserted == 0 || !accessor->iterationFinished() ||
+                            always_mark_full);
     }
   });
 }
 
 void InsertDestination::bulkInsertTuplesWithRemappedAttributes(
     const std::vector<attribute_id> &attribute_map,
-    ValueAccessor *accessor,
-    bool always_mark_full) {
+    ValueAccessor *accessor) {
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {  // NOLINT(build/c++11)
@@ -231,17 +225,10 @@ void InsertDestination::bulkInsertTuplesWithRemappedAttributes(
     while (!accessor->iterationFinished()) {
       MutableBlockReference output_block = this->getBlockForInsertion();
       // FIXME(chasseur): Deal with TupleTooLargeForBlock exception.
-      if (output_block->bulkInsertTuplesWithRemappedAttributes(
-              attribute_map,
-              accessor) == 0) {
-        // output_block is full.
-        this->returnBlock(std::move(output_block), true);
-      } else {
-        // Bulk insert into output_block was successful. output_block
-        // will be rebuilt when there won't be any more insertions to it.
-        this->returnBlock(std::move(output_block),
-                          always_mark_full || !accessor->iterationFinished());
-      }
+      const auto num_tuples_inserted =
+          output_block->bulkInsertTuplesWithRemappedAttributes(attribute_map, accessor);
+      this->returnBlock(std::move(output_block),
+                        num_tuples_inserted == 0 || !accessor->iterationFinished());
     }
   });
 }
@@ -267,8 +254,7 @@ void removeGapOnlyAccessors(
 }
 
 void InsertDestination::bulkInsertTuplesFromValueAccessors(
-    const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map,
-    bool always_mark_full) {
+    const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map) {
   // Handle pathological corner case where there are no accessors
   if (accessor_attribute_map.size() == 0)
     return;
@@ -323,9 +309,7 @@ void InsertDestination::bulkInsertTuplesFromValueAccessors(
 
     // Update the header for output_block and then return it.
     output_block->bulkInsertPartialTuplesFinalize(num_tuples_inserted);
-    const bool mark_full = always_mark_full
-                           || !first_accessor->iterationFinishedVirtual();
-    this->returnBlock(std::move(output_block), mark_full);
+    this->returnBlock(std::move(output_block), !first_accessor->iterationFinishedVirtual());
   }
 }
 
@@ -606,7 +590,7 @@ void PartitionAwareInsertDestination::insertTupleInBatch(const Tuple &tuple) {
   returnBlockInPartition(std::move(output_block), false, part_id);
 }
 
-void PartitionAwareInsertDestination::bulkInsertTuples(ValueAccessor *accessor, bool always_mark_full) {
+void PartitionAwareInsertDestination::bulkInsertTuples(ValueAccessor *accessor, const bool always_mark_full) {
   const std::size_t num_partitions = partition_scheme_header_->getNumPartitions();
 
   InvokeOnAnyValueAccessor(
@@ -639,29 +623,24 @@ void PartitionAwareInsertDestination::bulkInsertTuples(ValueAccessor *accessor,
       adapter[partition]->beginIteration();
       while (!adapter[partition]->iterationFinished()) {
         MutableBlockReference output_block = this->getBlockForInsertionInPartition(partition);
-        if (output_block->bulkInsertTuples(adapter[partition].get()) == 0) {
-          this->returnBlockInPartition(std::move(output_block), true, partition);
-        } else {
-          // Bulk insert into output_block was successful. output_block
-          // will be rebuilt when there won't be any more insertions to it.
-          this->returnBlockInPartition(std::move(output_block),
-                                       always_mark_full || !adapter[partition]->iterationFinished(),
-                                       partition);
-        }
+        const auto num_tuples_inserted = output_block->bulkInsertTuples(adapter[partition].get());
+        this->returnBlockInPartition(std::move(output_block),
+                                     num_tuples_inserted == 0 || !adapter[partition]->iterationFinished() ||
+                                         always_mark_full,
+                                     partition);
       }
     }
   });
 }
 
 void PartitionAwareInsertDestination::bulkInsertTuplesWithRemappedAttributes(
-    const std::vector<attribute_id> &attribute_map, ValueAccessor *accessor, bool always_mark_full) {
+    const std::vector<attribute_id> &attribute_map, ValueAccessor *accessor) {
   const std::size_t num_partitions = partition_scheme_header_->getNumPartitions();
 
   InvokeOnAnyValueAccessor(
       accessor,
       [this,
        &attribute_map,
-       &always_mark_full,
        &num_partitions](auto *accessor) -> void {  // NOLINT(build/c++11)
     std::vector<std::unique_ptr<TupleIdSequence>> partition_membership(num_partitions);
 
@@ -688,15 +667,11 @@ void PartitionAwareInsertDestination::bulkInsertTuplesWithRemappedAttributes(
       adapter[partition]->beginIteration();
       while (!adapter[partition]->iterationFinished()) {
         MutableBlockReference output_block = this->getBlockForInsertionInPartition(partition);
-        if (output_block->bulkInsertTuplesWithRemappedAttributes(attribute_map, adapter[partition].get()) == 0) {
-          this->returnBlockInPartition(std::move(output_block), true, partition);
-        } else {
-          // Bulk insert into output_block was successful. output_block
-          // will be rebuilt when there won't be any more insertions to it.
-          this->returnBlockInPartition(std::move(output_block),
-                                       always_mark_full || !adapter[partition]->iterationFinished(),
-                                       partition);
-        }
+        const auto num_tuple_inserted =
+            output_block->bulkInsertTuplesWithRemappedAttributes(attribute_map, adapter[partition].get());
+        this->returnBlockInPartition(std::move(output_block),
+                                     num_tuple_inserted == 0 || !adapter[partition]->iterationFinished(),
+                                     partition);
       }
     }
   });

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f820c45e/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index ab791b1..3b39aeb 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -156,16 +156,15 @@ class InsertDestination : public InsertDestinationInterface {
 
   void insertTupleInBatch(const Tuple &tuple) override;
 
-  void bulkInsertTuples(ValueAccessor *accessor, bool always_mark_full = false) override;
+  void bulkInsertTuples(ValueAccessor *accessor,
+                        const bool always_mark_full = false) override;
 
   void bulkInsertTuplesWithRemappedAttributes(
       const std::vector<attribute_id> &attribute_map,
-      ValueAccessor *accessor,
-      bool always_mark_full = false) override;
+      ValueAccessor *accessor) override;
 
   void bulkInsertTuplesFromValueAccessors(
-      const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map,
-      bool always_mark_full = false) override;
+      const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map) override;
 
   void insertTuplesFromVector(std::vector<Tuple>::const_iterator begin,
                               std::vector<Tuple>::const_iterator end) override;
@@ -363,8 +362,7 @@ class AlwaysCreateBlockInsertDestination : public InsertDestination {
   }
 
   void bulkInsertTuplesFromValueAccessors(
-      const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map,
-      bool always_mark_full = false) override {
+      const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map) override {
     LOG(FATAL) << "bulkInsertTuplesFromValueAccessors is not implemented for AlwaysCreateBlockInsertDestination";
   }
 
@@ -551,16 +549,15 @@ class PartitionAwareInsertDestination : public InsertDestination {
 
   void insertTupleInBatch(const Tuple &tuple) override;
 
-  void bulkInsertTuples(ValueAccessor *accessor, bool always_mark_full = false) override;
+  void bulkInsertTuples(ValueAccessor *accessor,
+                        const bool always_mark_full = false) override;
 
   void bulkInsertTuplesWithRemappedAttributes(
       const std::vector<attribute_id> &attribute_map,
-      ValueAccessor *accessor,
-      bool always_mark_full = false) override;
+      ValueAccessor *accessor) override;
 
   void bulkInsertTuplesFromValueAccessors(
-      const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map,
-      bool always_mark_full = false) override {
+      const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map) override {
     LOG(FATAL) << "bulkInsertTuplesFromValueAccessors is not implemented for PartitionAwareInsertDestination";
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f820c45e/storage/InsertDestinationInterface.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestinationInterface.hpp b/storage/InsertDestinationInterface.hpp
index b8c584b..7b87a98 100644
--- a/storage/InsertDestinationInterface.hpp
+++ b/storage/InsertDestinationInterface.hpp
@@ -104,7 +104,7 @@ class InsertDestinationInterface {
    *        insertion from ValueAccessor even when partially full.
    **/
   virtual void bulkInsertTuples(ValueAccessor *accessor,
-                                bool always_mark_full = false) = 0;
+                                const bool always_mark_full = false) = 0;
 
   /**
    * @brief Bulk-insert tuples from a ValueAccessor with differently-ordered
@@ -115,13 +115,10 @@ class InsertDestinationInterface {
    *        corresponding attributes which should be read from accessor.
    * @param accessor A ValueAccessor whose tuples will by inserted into blocks
    *        from this InsertDestination.
-   * @param always_mark_full If \c true, always mark the blocks full after
-   *        insertion from ValueAccessor even when partially full.
    **/
   virtual void bulkInsertTuplesWithRemappedAttributes(
       const std::vector<attribute_id> &attribute_map,
-      ValueAccessor *accessor,
-      bool always_mark_full = false) = 0;
+      ValueAccessor *accessor) = 0;
 
   /**
    * @brief Bulk-insert tuples from one or more ValueAccessors
@@ -137,12 +134,9 @@ class InsertDestinationInterface {
    *        is the attribute_id "n" in corresponding input value accessor.
    *        Set the i-th element to kInvalidCatalogId if it doesn't come from
    *        the corresponding value accessor.
-   * @param always_mark_full If \c true, always mark the blocks full after
-   *        insertion from ValueAccessor even when partially full.
    **/
   virtual void bulkInsertTuplesFromValueAccessors(
-      const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map,
-      bool always_mark_full = false) = 0;
+      const std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> &accessor_attribute_map) = 0;
 
   /**
    * @brief Insert tuples from a range of Tuples in a vector.


[20/27] incubator-quickstep git commit: Fixed the include path for farmhash.

Posted by ji...@apache.org.
Fixed the include path for farmhash.


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

Branch: refs/heads/trace
Commit: 79bfcf9ed294477a24823b00bd814df0de54ee5e
Parents: b5130fe
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Fri Oct 13 16:07:51 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Fri Oct 13 16:07:51 2017 -0500

----------------------------------------------------------------------
 CMakeLists.txt       | 1 +
 types/TypedValue.hpp | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/79bfcf9e/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index e0d020b..071f8fc 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -635,6 +635,7 @@ endif()
 
 # Add required cmake-controlled third-party libraries (farmhash, gflags, glog, and re2).
 add_subdirectory ("${THIRD_PARTY_SOURCE_DIR}/farmhash" "${CMAKE_CURRENT_BINARY_DIR}/third_party/farmhash")
+include_directories("${THIRD_PARTY_SOURCE_DIR}")
 
 add_subdirectory ("${THIRD_PARTY_SOURCE_DIR}/gflags" "${CMAKE_CURRENT_BINARY_DIR}/third_party/gflags")
 include_directories("${CMAKE_CURRENT_BINARY_DIR}/third_party/gflags/include")

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/79bfcf9e/types/TypedValue.hpp
----------------------------------------------------------------------
diff --git a/types/TypedValue.hpp b/types/TypedValue.hpp
index 0ba3d53..3075061 100644
--- a/types/TypedValue.hpp
+++ b/types/TypedValue.hpp
@@ -34,7 +34,7 @@
 #include "utility/HashPair.hpp"
 #include "utility/Macros.hpp"
 
-#include "third_party/src/farmhash/farmhash.h"
+#include "farmhash/farmhash.h"
 
 #include "glog/logging.h"
 


[26/27] incubator-quickstep git commit: Quickstep for GRAIL.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index 4800cde..628b0ce 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -592,8 +592,8 @@ static void yynoreturn yy_fatal_error ( const char* msg , yyscan_t yyscanner );
 	yyg->yy_hold_char = *yy_cp; \
 	*yy_cp = '\0'; \
 	yyg->yy_c_buf_p = yy_cp;
-#define YY_NUM_RULES 164
-#define YY_END_OF_BUFFER 165
+#define YY_NUM_RULES 165
+#define YY_END_OF_BUFFER 166
 /* This struct is not used in this scanner,
    but its presence is necessary. */
 struct yy_trans_info
@@ -601,72 +601,72 @@ struct yy_trans_info
 	flex_int32_t yy_verify;
 	flex_int32_t yy_nxt;
 	};
-static const flex_int16_t yy_accept[589] =
+static const flex_int16_t yy_accept[591] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
-        0,    0,  165,    2,    2,  163,  163,  162,  161,  163,
-      140,  136,  139,  136,  136,  159,  132,  129,  133,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  137,    4,    5,    5,    3,  155,
-      155,  152,  156,  156,  150,  157,  157,  154,    1,  162,
-      130,  160,  159,  159,  159,    0,  134,  131,  135,  158,
-      158,  158,  158,   10,  158,  158,  158,   22,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  138,
-
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,   58,   67,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,   81,   82,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  113,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,    4,    5,    3,  155,  151,  156,
-      149,  149,  141,  143,  144,  145,  146,  147,  148,  149,
-      157,  153,  160,  159,    0,  159,    6,    7,  158,    9,
-       11,  158,  158,   15,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,   33,  158,  158,  158,  158,
-
-      158,  158,  158,  158,   43,  158,  158,  158,  158,  158,
-      158,   50,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,   62,  158,   69,  158,  158,  158,  158,  158,  158,
-      158,   77,  158,   80,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,   98,  158,  158,
-      103,  104,  158,  158,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,  158,  141,
-      143,  142,  158,  158,  158,  158,  158,  158,  158,   20,
-       23,  158,  158,  158,   28,  158,  158,  158,   31,  158,
-      158,  158,   37,  158,  158,   41,   42,  158,  158,  158,
-
-      158,  158,  158,  158,   52,   53,  158,   55,  158,   57,
-      158,  158,  158,  158,   66,   68,   70,   71,   72,  158,
-       74,  158,  158,   78,  158,  158,   85,  158,  158,  158,
-      158,  158,   92,  158,   94,  158,  158,  158,  100,  158,
-      158,  158,  158,  158,  158,  158,  158,  110,  111,  114,
-      158,  158,  158,  158,  158,  158,  158,  158,  123,  158,
-      158,  126,  127,  141,  142,    8,  158,  158,  158,  158,
-      158,  158,  158,   25,  158,  158,  158,  158,  158,  158,
-      158,  158,  158,  158,  158,  158,  158,  158,   46,   47,
-       48,  158,  158,   54,  158,   59,   60,  158,  158,  158,
-
-       73,  158,   76,   79,   83,   84,  158,  158,  158,  158,
-      158,   93,  158,  158,   97,  158,  158,  158,  158,  158,
-      158,  158,  109,  158,  158,  158,  117,  158,  158,  120,
-      158,  158,  124,  158,  158,  158,  158,   14,  158,  158,
-      158,  158,  158,   26,  158,   29,  158,  158,  158,  158,
-      158,   36,  158,  158,   40,   44,  158,  158,  158,   56,
-       61,  158,  158,  158,   75,  158,  158,  158,  158,  158,
-      158,   96,  158,  101,  102,  158,  106,  107,  158,  158,
-      158,  158,  118,  119,  121,  158,  125,  158,  158,   13,
-      158,  158,  158,  158,  158,  158,   21,   30,  158,   34,
-
-       35,  158,  158,   45,  158,   51,   63,  158,  158,  158,
-       88,  158,   90,  158,  158,  158,  158,  158,  158,  158,
-      158,  122,  158,  158,  158,  158,  158,  158,  158,  158,
-       32,  158,   39,  158,  158,   65,  158,  158,   91,  158,
-      158,  105,  158,  158,  158,  158,  158,   12,  158,  158,
-      158,  158,   24,  158,  158,   49,   64,   86,   89,  158,
-      158,  108,  112,  158,  116,  128,   16,  158,  158,  158,
-       27,   38,   87,   95,  158,  158,  158,   18,   19,  158,
-      115,  158,  158,  158,   99,  158,   17,    0
+        0,    0,  166,    2,    2,  164,  164,  163,  162,  164,
+      141,  137,  140,  137,  137,  160,  133,  130,  134,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  138,    4,    5,    5,    3,  156,
+      156,  153,  157,  157,  151,  158,  158,  155,    1,  163,
+      131,  161,  160,  160,  160,    0,  135,  132,  136,  159,
+      159,  159,  159,   10,  159,  159,  159,   22,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  139,
+
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,   58,   67,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,   81,   82,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  114,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,    4,    5,    3,  156,  152,  157,
+      150,  150,  142,  144,  145,  146,  147,  148,  149,  150,
+      158,  154,  161,  160,    0,  160,    6,    7,  159,    9,
+       11,  159,  159,   15,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,   33,  159,  159,  159,  159,
+
+      159,  159,  159,  159,   43,  159,  159,  159,  159,  159,
+      159,   50,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,   62,  159,   69,  159,  159,  159,  159,  159,  159,
+      159,   77,  159,   80,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,   98,  159,  159,
+      103,  104,  159,  159,  159,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,  142,
+      144,  143,  159,  159,  159,  159,  159,  159,  159,   20,
+       23,  159,  159,  159,   28,  159,  159,  159,   31,  159,
+      159,  159,   37,  159,  159,   41,   42,  159,  159,  159,
+
+      159,  159,  159,  159,   52,   53,  159,   55,  159,   57,
+      159,  159,  159,  159,   66,   68,   70,   71,   72,  159,
+       74,  159,  159,   78,  159,  159,   85,  159,  159,  159,
+      159,  159,   92,  159,   94,  159,  159,  159,  100,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  111,  112,
+      115,  159,  159,  159,  159,  159,  159,  159,  159,  124,
+      159,  159,  127,  128,  142,  143,    8,  159,  159,  159,
+      159,  159,  159,  159,   25,  159,  159,  159,  159,  159,
+      159,  159,  159,  159,  159,  159,  159,  159,  159,   46,
+       47,   48,  159,  159,   54,  159,   59,   60,  159,  159,
+
+      159,   73,  159,   76,   79,   83,   84,  159,  159,  159,
+      159,  159,   93,  159,  159,   97,  159,  159,  159,  159,
+      159,  107,  159,  159,  110,  159,  159,  159,  118,  159,
+      159,  121,  159,  159,  125,  159,  159,  159,  159,   14,
+      159,  159,  159,  159,  159,   26,  159,   29,  159,  159,
+      159,  159,  159,   36,  159,  159,   40,   44,  159,  159,
+      159,   56,   61,  159,  159,  159,   75,  159,  159,  159,
+      159,  159,  159,   96,  159,  101,  102,  159,  106,  108,
+      159,  159,  159,  159,  119,  120,  122,  159,  126,  159,
+      159,   13,  159,  159,  159,  159,  159,  159,   21,   30,
+
+      159,   34,   35,  159,  159,   45,  159,   51,   63,  159,
+      159,  159,   88,  159,   90,  159,  159,  159,  159,  159,
+      159,  159,  159,  123,  159,  159,  159,  159,  159,  159,
+      159,  159,   32,  159,   39,  159,  159,   65,  159,  159,
+       91,  159,  159,  105,  159,  159,  159,  159,  159,   12,
+      159,  159,  159,  159,   24,  159,  159,   49,   64,   86,
+       89,  159,  159,  109,  113,  159,  117,  129,   16,  159,
+      159,  159,   27,   38,   87,   95,  159,  159,  159,   18,
+       19,  159,  116,  159,  159,  159,   99,  159,   17,    0
     } ;
 
 static const YY_CHAR yy_ec[256] =
@@ -713,27 +713,27 @@ static const YY_CHAR yy_meta[72] =
         8
     } ;
 
-static const flex_int16_t yy_base[604] =
+static const flex_int16_t yy_base[606] =
     {   0,
         0,    1,   46,    0,  117,  162,    2,    3,  127,  128,
-        6,   10,  147, 1316, 1316,    0, 1316,   13, 1316,  130,
-     1316, 1316, 1316,  129,    6,  129,    4, 1316,   28,  124,
+        6,   10,  147, 1318, 1318,    0, 1318,   13, 1318,  130,
+     1318, 1318, 1318,  129,    6,  129,    4, 1318,   28,  124,
       159,  213,  165,  167,  263,   92,  158,  163,   96,  107,
       214,  160,  186,  219,  221,  155,  281,  274,  325,  257,
-      186,  209,    0,  219, 1316,   27,    4,   19,    0,    0,
+      186,  209,    0,  219, 1318,   27,    4,   19,    0,    0,
         0,   17,    0,    0,  389,    0,    0,    8,    0,   22,
-     1316,    0,  293,  325,  343,   18, 1316, 1316, 1316,    0,
+     1318,    0,  293,  325,  343,   18, 1318, 1318, 1318,    0,
       223,  265,  234,  242,  260,  292,  288,    0,  299,  330,
-      337,  324,  334,  324,  325,  380,  325,  331,  346, 1316,
+      337,  324,  334,  324,  325,  380,  325,  331,  346, 1318,
 
       348,  364,  378,  376,  371,  378,  382,  386,  390,  389,
       386,  385,  435,    0,  402,  389,  400,  435,  433,  431,
       433,  436,  431,  440,  447,    0,  452,  437,  453,  441,
       442,  456,  453,  449,  465,  457,  444,  494,  468,  495,
       500,  501,  499,  492,    0,  486,  492,  507,  506,  502,
-      500,  508,  501,  516,    0,   29,    0,    0, 1316,    0,
-     1316, 1316,   22,   24, 1316, 1316, 1316, 1316, 1316,    0,
-        0, 1316,    0,  524,   26,   28,    0,    0,  517,    0,
+      500,  508,  501,  516,    0,   29,    0,    0, 1318,    0,
+     1318, 1318,   22,   24, 1318, 1318, 1318, 1318, 1318,    0,
+        0, 1318,    0,  524,   26,   28,    0,    0,  517,    0,
       518,  501,  516,  504,  545,  525,  531,  552,  536,  542,
       537,  562,  544,  547,  561,    0,  558,  567,  564,  567,
 
@@ -742,126 +742,126 @@ static const flex_int16_t yy_base[604] =
       614,  615,  607,    0,  602,  603,  619,  616,  619,  606,
       608,    0,  617,    0,  626,  627,  615,  614,  634,  635,
       626,  620,  636,  633,  641,  659,  657,  652,  658,  671,
-        0,  665,  673,  660,  668,  668,  678,  679,  673,  671,
-      672,  689,  677,  671,  692,  683,  692,  690,  681,   30,
-      125,    0,  685,  690,  705,  708,  718,  718,  718,    0,
-      733,  724,  723,  717,    0,  718,  722,  736,  722,  730,
-      723,  725,  741,  738,  736,    0,    0,  729,  749,  748,
-
-      734,  735,  741,  748,    0,    0,  743,    0,  747,    0,
-      738,  750,  762,  774,    0,    0,    0,    0,    0,  767,
-        0,  769,  785,  775,  777,  778,    0,  789,  794,  795,
-      800,  784,    0,  798,    0,  786,  781,  786,    0,  803,
-      794,  808,  800,  795,  793,  795,  812,    0,  800,    0,
-      815,  805,  824,  818,  825,  840,  845,  843,    0,  847,
-      838,    0,  841,  131, 1316,    0,  852,  852,  838,  858,
-      844,  855,  859,    0,  850,  847,  861,  864,  856,  862,
-      871,  861,  870,  863,  864,  879,  877,  894,    0,    0,
-        0,  880,  898,    0,  901,    0,    0,  889,  905,  892,
-
-        0,  907,    0,    0,    0,    0,  895,  902,  913,  900,
-      910,    0,  915,  905,    0,  917,  919,  904,  918,  910,
-      909,  912,    0,  911,  914,  921,    0,  931,  937,    0,
-      935,  954,    0,  938,  948,  957,  953,    0,  946,  951,
-      969,  963,  953,    0,  973,    0,  970,  956,  964,  966,
-      959,    0,  976,  978,    0,    0,  962,  976,  972,    0,
-        0,  969,  983,  988,    0,  982,  973,  985,  975,  992,
-      999,    0, 1007,    0,    0, 1007,    0,    0, 1015, 1024,
-     1025, 1023,    0,    0,    0, 1010,    0, 1016, 1017,    0,
-     1023, 1018, 1021, 1023, 1031, 1028,    0,    0, 1033,    0,
-
-        0, 1030, 1020,    0, 1029,    0,    0, 1041, 1033, 1031,
-        0, 1033,    0, 1024, 1048, 1043, 1038, 1056, 1058, 1064,
-     1074,    0, 1062, 1076, 1070, 1069, 1070, 1068, 1071, 1076,
-        0, 1077,    0, 1085, 1073,    0, 1080, 1088,    0, 1091,
-     1084,    0, 1091, 1085, 1086, 1099, 1096,    0, 1098, 1102,
-     1097, 1105,    0, 1096, 1121,    0,    0, 1110,    0, 1116,
-     1128,    0,    0, 1128,    0,    0,    0, 1123, 1137, 1125,
-        0,    0,    0,    0, 1124, 1141, 1127,    0,    0, 1143,
-        0, 1140, 1132, 1146,    0, 1133,    0, 1316, 1198, 1208,
-     1218, 1228, 1238, 1242, 1245, 1251, 1261, 1271, 1281, 1291,
-
-     1301, 1306, 1308
+        0,  665,  673,  660,  668,  669,  679,  680,  674,  672,
+      674,  689,  678,  674,  693,  683,  694,  691,  685,   30,
+      125,    0,  686,  696,  717,  709,  724,  720,  721,    0,
+      734,  725,  724,  718,    0,  720,  723,  737,  723,  731,
+      724,  726,  742,  739,  737,    0,    0,  730,  752,  749,
+
+      735,  736,  742,  750,    0,    0,  745,    0,  748,    0,
+      744,  762,  763,  780,    0,    0,    0,    0,    0,  769,
+        0,  772,  785,  775,  777,  778,    0,  790,  795,  796,
+      801,  785,    0,  799,    0,  787,  782,  787,    0,  804,
+      797,  809,  801,  796,  801,  796,  799,  815,    0,  807,
+        0,  833,  818,  831,  826,  830,  844,  847,  845,    0,
+      849,  841,    0,  844,  131, 1318,    0,  854,  854,  840,
+      860,  846,  857,  861,    0,  852,  851,  865,  866,  858,
+      864,  874,  865,  873,  870,  882,  892,  884,  902,    0,
+        0,    0,  885,  902,    0,  903,    0,    0,  891,  907,
+
+      895,    0,  910,    0,    0,    0,    0,  896,  904,  915,
+      902,  912,    0,  917,  907,    0,  919,  923,  908,  920,
+      912,    0,  911,  915,    0,  915,  917,  928,    0,  949,
+      950,    0,  942,  962,    0,  943,  952,  959,  955,    0,
+      948,  954,  972,  965,  955,    0,  975,    0,  972,  958,
+      966,  968,  961,    0,  978,  982,    0,    0,  966,  978,
+      974,    0,    0,  971,  986,  992,    0,  985,  980, 1003,
+      988,  999, 1007,    0, 1012,    0,    0, 1011,    0,    0,
+     1017, 1026, 1027, 1026,    0,    0,    0, 1013,    0, 1018,
+     1019,    0, 1025, 1020, 1023, 1025, 1033, 1030,    0,    0,
+
+     1035,    0,    0, 1034, 1024,    0, 1031,    0,    0, 1043,
+     1035, 1034,    0, 1037,    0, 1027, 1055, 1061, 1051, 1063,
+     1066, 1069, 1078,    0, 1064, 1078, 1072, 1072, 1073, 1070,
+     1073, 1078,    0, 1079,    0, 1087, 1075,    0, 1082, 1090,
+        0, 1093, 1088,    0, 1095, 1087, 1088, 1101, 1099,    0,
+     1102, 1105, 1104, 1123,    0, 1109, 1128,    0,    0, 1118,
+        0, 1121, 1132,    0,    0, 1130,    0,    0,    0, 1125,
+     1139, 1128,    0,    0,    0,    0, 1127, 1143, 1129,    0,
+        0, 1145,    0, 1142, 1134, 1148,    0, 1135,    0, 1318,
+     1200, 1210, 1220, 1230, 1240, 1244, 1247, 1253, 1263, 1273,
+
+     1283, 1293, 1303, 1308, 1310
     } ;
 
-static const flex_int16_t yy_def[604] =
+static const flex_int16_t yy_def[606] =
     {   0,
-      589,  589,  588,    3,  590,  590,  591,  591,  592,  592,
-      593,  593,  588,  588,  588,  594,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  588,  588,  588,  588,  596,  597,
-      597,  588,  598,  598,  599,  600,  600,  588,  594,  588,
-      588,  601,  588,  588,  588,  588,  588,  588,  588,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  588,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  588,  588,  596,  597,  588,  598,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  602,
-      600,  588,  601,  588,  588,  588,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  588,
-      588,  603,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  588,  588,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  595,    0,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-
-      588,  588,  588
+      591,  591,  590,    3,  592,  592,  593,  593,  594,  594,
+      595,  595,  590,  590,  590,  596,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  590,  590,  590,  590,  598,  599,
+      599,  590,  600,  600,  601,  602,  602,  590,  596,  590,
+      590,  603,  590,  590,  590,  590,  590,  590,  590,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  590,
+
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  590,  590,  598,  599,  590,  600,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  604,
+      602,  590,  603,  590,  590,  590,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  590,
+      590,  605,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  590,  590,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  597,  597,    0,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+
+      590,  590,  590,  590,  590
     } ;
 
-static const flex_int16_t yy_nxt[1388] =
+static const flex_int16_t yy_nxt[1390] =
     {   0,
-      588,  155,   15,   15,   61,   61,  156,  156,   67,   62,
+      590,  155,   15,   15,   61,   61,  156,  156,   67,   62,
        62,   68,   67,  172,   70,   68,   70,   73,   73,   77,
        78,  156,  156,   70,  159,   70,  175,  175,  155,  176,
       176,  156,  156,  270,  271,  271,  271,  176,  176,  176,
-      176,  364,  271,   79,   16,   16,   17,   18,   19,   18,
+      176,  365,  271,   79,   16,   16,   17,   18,   19,   18,
        20,   21,   22,   23,   22,   24,   25,   26,   26,   17,
        27,   28,   29,   30,   31,   32,   33,   34,   35,   36,
        37,   38,   39,   40,   41,   42,   43,   44,   45,   46,
@@ -872,45 +872,45 @@ static const flex_int16_t yy_nxt[1388] =
        48,   49,   50,   51,   52,   53,   54,   17,   56,   57,
        58,   17,   17,   17,   17,   17,  110,  115,  116,   64,
        64,   17,   17,   17,   62,   62,  271,  271,   72,   74,
-       75,   75,  271,  271,   81,   71,  588,  588,  588,  588,
-       76,  588,   82,  588,   83,  110,  115,  116,  588,   84,
+       75,   75,  271,  271,   81,   71,  590,  590,  590,  590,
+       76,  590,   82,  590,   83,  110,  115,  116,  590,   84,
        17,   17,   17,   56,   57,   58,   17,   17,   17,   17,
        17,   65,   65,   81,  100,  111,   17,   17,   17,   76,
        85,   82,   95,   83,   86,  121,   96,   87,   84,  112,
-       97,  122,  133,  113,  588,  101,   98,  102,  114,   99,
+       97,  122,  133,  113,  590,  101,   98,  102,  114,   99,
 
-       88,  588,  588,  151,  111,   17,   17,  103,  588,   85,
-      588,   95,  588,   86,  121,   96,   87,  123,  112,   97,
+       88,  590,  590,  151,  111,   17,   17,  103,  590,   85,
+      590,   95,  590,   86,  121,   96,   87,  123,  112,   97,
       122,  133,  113,  124,  101,   98,  102,  114,   99,   88,
        89,  117,  151,  152,  153,  118,  103,   90,  130,  119,
       154,  125,  131,  177,   91,  120,  123,   92,   93,  126,
-       94,  588,  124,  127,  180,  132,  128,  129,  588,   89,
-      117,  181,  152,  153,  118,  588,   90,  130,  119,  154,
-      125,  131,  177,   91,  120,  588,   92,   93,  126,   94,
-      104,  588,  127,  180,  132,  128,  129,  148,  105,  149,
-      181,  106,  150,  178,  107,  138,  182,  108,  134,  588,
-
-      109,  179,  135,  139,   73,   73,  136,  588,  588,  104,
-      140,  141,  137,  588,   76,  183,  148,  105,  149,  185,
+       94,  590,  124,  127,  180,  132,  128,  129,  590,   89,
+      117,  181,  152,  153,  118,  590,   90,  130,  119,  154,
+      125,  131,  177,   91,  120,  590,   92,   93,  126,   94,
+      104,  590,  127,  180,  132,  128,  129,  148,  105,  149,
+      181,  106,  150,  178,  107,  138,  182,  108,  134,  590,
+
+      109,  179,  135,  139,   73,   73,  136,  590,  590,  104,
+      140,  141,  137,  590,   76,  183,  148,  105,  149,  185,
       106,  150,  178,  107,  138,  182,  108,  134,  184,  109,
-      179,  135,  139,  588,  186,  136,  174,  174,  588,  140,
+      179,  135,  139,  590,  186,  136,  174,  174,  590,  140,
       141,  137,  142,   76,  183,  192,   76,  187,  185,  143,
       144,  188,  193,   74,   75,   75,  145,  184,  194,  146,
       201,  195,  147,  186,   76,  189,  196,  190,  202,  191,
-      588,  142,  588,  588,  192,   76,  187,  203,  143,  144,
-      188,  193,  588,  204,  205,  145,  588,  194,  146,  201,
+      590,  142,  590,  590,  192,   76,  187,  203,  143,  144,
+      188,  193,  590,  204,  205,  145,  590,  194,  146,  201,
       195,  147,  162,   76,  189,  196,  190,  202,  191,  197,
 
       163,  164,  198,  206,  208,  209,  203,  165,  199,  210,
       211,  166,  204,  205,  207,  200,  212,  213,  214,  167,
-      215,  216,  218,  168,  217,  169,  588,  223,  197,  170,
+      215,  216,  218,  168,  217,  169,  590,  223,  197,  170,
       224,  198,  206,  208,  209,  225,  165,  199,  210,  211,
-      166,  588,  588,  207,  200,  212,  213,  214,  167,  215,
+      166,  590,  590,  207,  200,  212,  213,  214,  167,  215,
       216,  218,  168,  217,  169,  219,  223,  226,  170,  224,
       227,  229,  228,  230,  225,  220,  231,  232,  233,  234,
       221,  222,  235,  236,  237,  238,  239,  240,  242,  243,
       247,  241,  244,  248,  219,  252,  226,  245,  246,  227,
-      229,  228,  230,  588,  220,  231,  232,  233,  234,  221,
+      229,  228,  230,  590,  220,  231,  232,  233,  234,  221,
 
       222,  235,  236,  237,  238,  239,  240,  242,  243,  247,
       241,  244,  248,  249,  252,  253,  245,  246,  254,  255,
@@ -932,37 +932,37 @@ static const flex_int16_t yy_nxt[1388] =
       311,  335,  336,  312,  313,  314,  316,  317,  318,  319,
       320,  321,  322,  323,  324,  315,  325,  326,  327,  328,
       337,  338,  329,  330,  331,  333,  332,  339,  334,  341,
-      335,  336,  342,  343,  344,  346,  347,  340,  348,  349,
+      335,  336,  342,  343,  344,  347,  348,  340,  345,  349,
 
-      350,  351,  352,  353,  345,  354,  355,  356,  357,  337,
-      338,  358,  361,  359,  362,  363,  339,  360,  341,  366,
-      367,  342,  343,  344,  346,  347,  368,  348,  349,  350,
-      351,  352,  353,  345,  354,  355,  356,  357,  369,  370,
-      358,  361,  359,  362,  363,  371,  360,  372,  366,  367,
+      350,  351,  352,  353,  346,  354,  356,  355,  357,  337,
+      338,  358,  359,  360,  362,  363,  339,  361,  341,  364,
+      367,  342,  343,  344,  347,  348,  368,  345,  349,  350,
+      351,  352,  353,  346,  354,  356,  355,  357,  369,  370,
+      358,  359,  360,  362,  363,  371,  361,  372,  364,  367,
       373,  374,  375,  376,  377,  368,  378,  379,  380,  381,
       382,  383,  384,  385,  386,  387,  388,  369,  370,  389,
       390,  391,  392,  393,  371,  394,  372,  395,  396,  373,
       374,  375,  376,  377,  397,  378,  379,  380,  381,  382,
       383,  384,  385,  386,  387,  388,  398,  399,  389,  390,
 
-      391,  392,  393,  401,  394,  402,  395,  396,  400,  403,
-      404,  405,  406,  397,  407,  408,  409,  410,  411,  412,
+      391,  392,  393,  400,  394,  402,  395,  396,  403,  404,
+      405,  406,  407,  397,  401,  408,  409,  410,  411,  412,
       413,  414,  415,  416,  417,  398,  399,  418,  419,  420,
-      421,  422,  401,  423,  402,  424,  425,  400,  403,  404,
-      405,  406,  426,  407,  408,  409,  410,  411,  412,  413,
+      421,  422,  400,  423,  402,  424,  425,  403,  404,  405,
+      406,  407,  426,  401,  408,  409,  410,  411,  412,  413,
       414,  415,  416,  417,  427,  428,  418,  419,  420,  421,
       422,  429,  423,  430,  424,  425,  431,  432,  433,  434,
-      435,  426,  436,  437,  438,  439,  440,  442,  443,  441,
-      444,  445,  446,  427,  428,  447,  448,  449,  450,  451,
+      435,  426,  436,  437,  438,  439,  440,  441,  442,  444,
+      445,  443,  446,  427,  428,  447,  448,  449,  450,  451,
       429,  452,  430,  453,  454,  431,  432,  433,  434,  435,
 
-      455,  436,  437,  438,  439,  440,  442,  443,  441,  444,
-      445,  446,  456,  457,  447,  448,  449,  450,  451,  458,
-      452,  459,  453,  454,  460,  461,  462,  463,  465,  455,
-      464,  466,  467,  468,  469,  470,  471,  472,  473,  474,
+      455,  436,  437,  438,  439,  440,  441,  442,  444,  445,
+      443,  446,  456,  457,  447,  448,  449,  450,  451,  458,
+      452,  459,  453,  454,  460,  461,  462,  463,  464,  455,
+      465,  467,  468,  466,  469,  470,  471,  472,  473,  474,
       475,  456,  457,  476,  477,  478,  479,  480,  458,  481,
-      459,  482,  483,  460,  461,  462,  463,  465,  484,  464,
-      466,  467,  468,  469,  470,  471,  472,  473,  474,  475,
+      459,  482,  483,  460,  461,  462,  463,  464,  484,  465,
+      467,  468,  466,  469,  470,  471,  472,  473,  474,  475,
       485,  486,  476,  477,  478,  479,  480,  487,  481,  488,
       482,  483,  489,  490,  491,  492,  493,  484,  494,  495,
       496,  497,  498,  499,  500,  501,  502,  503,  504,  485,
@@ -985,33 +985,33 @@ static const flex_int16_t yy_nxt[1388] =
       559,  560,  561,  562,  572,  573,  563,  564,  565,  566,
       567,  574,  568,  575,  569,  570,  576,  577,  578,  579,
       580,  571,  581,  582,  583,  584,  585,  586,  587,  588,
-      588,  588,  588,  572,  573,  588,  588,  588,  588,  588,
-      574,  588,  575,  588,  588,  576,  577,  578,  579,  580,
-      588,  581,  582,  583,  584,  585,  586,  587,   14,   14,
-
-       14,   14,   14,   14,   14,   14,   14,   14,   59,   59,
-       59,   59,   59,   59,   59,   59,   59,   59,   60,   60,
-       60,   60,   60,   60,   60,   60,   60,   60,   63,   63,
-       63,   63,   63,   63,   63,   63,   63,   63,   66,   66,
-       66,   66,   66,   66,   66,   66,   66,   66,   69,   69,
-       80,   80,   80,  588,   80,  157,  157,  157,  157,  588,
-      157,  158,  158,  158,  588,  158,  158,  158,  158,  158,
-      158,  160,  160,  160,  588,  160,  160,  160,  160,  588,
-      160,  161,  161,  161,  161,  161,  161,  161,  161,  161,
-      161,  171,  171,  588,  171,  171,  171,  171,  171,  171,
-
-      171,  173,  588,  173,  173,  173,  173,  173,  173,  173,
-      173,  272,  272,  365,  365,   13,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588
+      589,  590,  590,  572,  573,  590,  590,  590,  590,  590,
+      574,  590,  575,  590,  590,  576,  577,  578,  579,  580,
+      590,  581,  582,  583,  584,  585,  586,  587,  588,  589,
+
+       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,  590,   80,  157,  157,  157,
+      157,  590,  157,  158,  158,  158,  590,  158,  158,  158,
+      158,  158,  158,  160,  160,  160,  590,  160,  160,  160,
+      160,  590,  160,  161,  161,  161,  161,  161,  161,  161,
+      161,  161,  161,  171,  171,  590,  171,  171,  171,  171,
+
+      171,  171,  171,  173,  590,  173,  173,  173,  173,  173,
+      173,  173,  173,  272,  272,  366,  366,   13,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590
     } ;
 
-static const flex_int16_t yy_chk[1388] =
+static const flex_int16_t yy_chk[1390] =
     {   0,
         0,  155,    1,    2,    7,    8,   57,   57,   11,    7,
         8,   11,   12,   68,   18,   12,   18,   25,   25,   27,
@@ -1028,7 +1028,7 @@ static const flex_int16_t yy_chk[1388] =
         3,    3,    3,    3,    3,    3,    3,    5,    5,    5,
         5,    5,    5,    5,    5,    5,   36,   39,   40,    9,
        10,    5,    5,    5,    9,   10,  271,  271,   24,   26,
-       26,   26,  364,  364,   30,   20,   13,    0,    0,    0,
+       26,   26,  365,  365,   30,   20,   13,    0,    0,    0,
        26,    0,   30,    0,   30,   36,   39,   40,    0,   30,
         5,    5,    6,    6,    6,    6,    6,    6,    6,    6,
         6,    9,   10,   30,   34,   37,    6,    6,    6,   26,
@@ -1088,87 +1088,87 @@ static const flex_int16_t yy_chk[1388] =
       219,  244,  245,  220,  221,  222,  223,  225,  226,  227,
       228,  229,  230,  231,  233,  222,  235,  236,  237,  238,
       246,  247,  239,  240,  241,  242,  241,  248,  243,  249,
-      244,  245,  250,  252,  253,  254,  255,  248,  256,  257,
-
-      258,  259,  260,  261,  253,  261,  262,  263,  264,  246,
-      247,  265,  267,  266,  268,  269,  248,  266,  249,  273,
-      274,  250,  252,  253,  254,  255,  275,  256,  257,  258,
-      259,  260,  261,  253,  261,  262,  263,  264,  276,  277,
-      265,  267,  266,  268,  269,  278,  266,  279,  273,  274,
-      281,  282,  283,  284,  286,  275,  287,  288,  289,  290,
-      291,  292,  293,  294,  295,  298,  299,  276,  277,  300,
-      301,  302,  303,  304,  278,  307,  279,  309,  311,  281,
-      282,  283,  284,  286,  312,  287,  288,  289,  290,  291,
-      292,  293,  294,  295,  298,  299,  313,  314,  300,  301,
-
-      302,  303,  304,  320,  307,  322,  309,  311,  314,  323,
-      324,  325,  326,  312,  328,  329,  330,  331,  332,  334,
-      336,  337,  338,  340,  341,  313,  314,  342,  343,  344,
-      345,  346,  320,  347,  322,  349,  351,  314,  323,  324,
-      325,  326,  352,  328,  329,  330,  331,  332,  334,  336,
-      337,  338,  340,  341,  353,  354,  342,  343,  344,  345,
-      346,  355,  347,  356,  349,  351,  357,  358,  360,  361,
-      363,  352,  367,  368,  369,  370,  371,  372,  373,  371,
-      375,  376,  377,  353,  354,  378,  379,  380,  381,  382,
-      355,  383,  356,  384,  385,  357,  358,  360,  361,  363,
-
-      386,  367,  368,  369,  370,  371,  372,  373,  371,  375,
-      376,  377,  387,  388,  378,  379,  380,  381,  382,  392,
-      383,  393,  384,  385,  395,  398,  399,  400,  402,  386,
-      400,  407,  408,  409,  410,  411,  413,  414,  416,  417,
-      418,  387,  388,  419,  420,  421,  422,  424,  392,  425,
-      393,  426,  428,  395,  398,  399,  400,  402,  429,  400,
-      407,  408,  409,  410,  411,  413,  414,  416,  417,  418,
-      431,  432,  419,  420,  421,  422,  424,  434,  425,  435,
-      426,  428,  436,  437,  439,  440,  441,  429,  442,  443,
-      445,  447,  448,  449,  450,  451,  453,  454,  457,  431,
-
-      432,  458,  459,  462,  463,  464,  434,  466,  435,  467,
-      468,  436,  437,  439,  440,  441,  469,  442,  443,  445,
-      447,  448,  449,  450,  451,  453,  454,  457,  470,  471,
-      458,  459,  462,  463,  464,  473,  466,  476,  467,  468,
-      479,  480,  481,  482,  486,  469,  488,  489,  491,  492,
-      493,  494,  495,  496,  499,  502,  503,  470,  471,  505,
-      508,  509,  510,  512,  473,  514,  476,  515,  516,  479,
-      480,  481,  482,  486,  517,  488,  489,  491,  492,  493,
-      494,  495,  496,  499,  502,  503,  518,  519,  505,  508,
-      509,  510,  512,  520,  514,  521,  515,  516,  523,  524,
-
-      525,  526,  527,  517,  528,  529,  530,  532,  534,  535,
-      537,  538,  540,  541,  543,  518,  519,  544,  545,  546,
-      547,  549,  520,  550,  521,  551,  552,  523,  524,  525,
-      526,  527,  554,  528,  529,  530,  532,  534,  535,  537,
-      538,  540,  541,  543,  555,  558,  544,  545,  546,  547,
-      549,  560,  550,  561,  551,  552,  564,  568,  569,  570,
-      575,  554,  576,  577,  580,  582,  583,  584,  586,    0,
-        0,    0,    0,  555,  558,    0,    0,    0,    0,    0,
-      560,    0,  561,    0,    0,  564,  568,  569,  570,  575,
-        0,  576,  577,  580,  582,  583,  584,  586,  589,  589,
-
-      589,  589,  589,  589,  589,  589,  589,  589,  590,  590,
-      590,  590,  590,  590,  590,  590,  590,  590,  591,  591,
-      591,  591,  591,  591,  591,  591,  591,  591,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  593,  593,
-      593,  593,  593,  593,  593,  593,  593,  593,  594,  594,
-      595,  595,  595,    0,  595,  596,  596,  596,  596,    0,
-      596,  597,  597,  597,    0,  597,  597,  597,  597,  597,
-      597,  598,  598,  598,    0,  598,  598,  598,  598,    0,
-      598,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  600,  600,    0,  600,  600,  600,  600,  600,  600,
-
-      600,  601,    0,  601,  601,  601,  601,  601,  601,  601,
-      601,  602,  602,  603,  603,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588,  588,  588,  588,
-      588,  588,  588,  588,  588,  588,  588
+      244,  245,  250,  252,  253,  254,  255,  248,  253,  256,
+
+      257,  258,  259,  260,  253,  261,  262,  261,  263,  246,
+      247,  264,  265,  266,  267,  268,  248,  266,  249,  269,
+      273,  250,  252,  253,  254,  255,  274,  253,  256,  257,
+      258,  259,  260,  253,  261,  262,  261,  263,  275,  276,
+      264,  265,  266,  267,  268,  277,  266,  278,  269,  273,
+      279,  281,  282,  283,  284,  274,  286,  287,  288,  289,
+      290,  291,  292,  293,  294,  295,  298,  275,  276,  299,
+      300,  301,  302,  303,  277,  304,  278,  307,  309,  279,
+      281,  282,  283,  284,  311,  286,  287,  288,  289,  290,
+      291,  292,  293,  294,  295,  298,  312,  313,  299,  300,
+
+      301,  302,  303,  314,  304,  320,  307,  309,  322,  323,
+      324,  325,  326,  311,  314,  328,  329,  330,  331,  332,
+      334,  336,  337,  338,  340,  312,  313,  341,  342,  343,
+      344,  345,  314,  346,  320,  347,  348,  322,  323,  324,
+      325,  326,  350,  314,  328,  329,  330,  331,  332,  334,
+      336,  337,  338,  340,  352,  353,  341,  342,  343,  344,
+      345,  354,  346,  355,  347,  348,  356,  357,  358,  359,
+      361,  350,  362,  364,  368,  369,  370,  371,  372,  373,
+      374,  372,  376,  352,  353,  377,  378,  379,  380,  381,
+      354,  382,  355,  383,  384,  356,  357,  358,  359,  361,
+
+      385,  362,  364,  368,  369,  370,  371,  372,  373,  374,
+      372,  376,  386,  387,  377,  378,  379,  380,  381,  388,
+      382,  389,  383,  384,  393,  394,  396,  399,  400,  385,
+      401,  403,  408,  401,  409,  410,  411,  412,  414,  415,
+      417,  386,  387,  418,  419,  420,  421,  423,  388,  424,
+      389,  426,  427,  393,  394,  396,  399,  400,  428,  401,
+      403,  408,  401,  409,  410,  411,  412,  414,  415,  417,
+      430,  431,  418,  419,  420,  421,  423,  433,  424,  434,
+      426,  427,  436,  437,  438,  439,  441,  428,  442,  443,
+      444,  445,  447,  449,  450,  451,  452,  453,  455,  430,
+
+      431,  456,  459,  460,  461,  464,  433,  465,  434,  466,
+      468,  436,  437,  438,  439,  441,  469,  442,  443,  444,
+      445,  447,  449,  450,  451,  452,  453,  455,  470,  471,
+      456,  459,  460,  461,  464,  472,  465,  473,  466,  468,
+      475,  478,  481,  482,  483,  469,  484,  488,  490,  491,
+      493,  494,  495,  496,  497,  498,  501,  470,  471,  504,
+      505,  507,  510,  511,  472,  512,  473,  514,  516,  475,
+      478,  481,  482,  483,  517,  484,  488,  490,  491,  493,
+      494,  495,  496,  497,  498,  501,  518,  519,  504,  505,
+      507,  510,  511,  520,  512,  521,  514,  516,  522,  523,
+
+      525,  526,  527,  517,  528,  529,  530,  531,  532,  534,
+      536,  537,  539,  540,  542,  518,  519,  543,  545,  546,
+      547,  548,  520,  549,  521,  551,  552,  522,  523,  525,
+      526,  527,  553,  528,  529,  530,  531,  532,  534,  536,
+      537,  539,  540,  542,  554,  556,  543,  545,  546,  547,
+      548,  557,  549,  560,  551,  552,  562,  563,  566,  570,
+      571,  553,  572,  577,  578,  579,  582,  584,  585,  586,
+      588,    0,    0,  554,  556,    0,    0,    0,    0,    0,
+      557,    0,  560,    0,    0,  562,  563,  566,  570,  571,
+        0,  572,  577,  578,  579,  582,  584,  585,  586,  588,
+
+      591,  591,  591,  591,  591,  591,  591,  591,  591,  591,
+      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      594,  594,  594,  594,  594,  594,  594,  594,  594,  594,
+      595,  595,  595,  595,  595,  595,  595,  595,  595,  595,
+      596,  596,  597,  597,  597,    0,  597,  598,  598,  598,
+      598,    0,  598,  599,  599,  599,    0,  599,  599,  599,
+      599,  599,  599,  600,  600,  600,    0,  600,  600,  600,
+      600,    0,  600,  601,  601,  601,  601,  601,  601,  601,
+      601,  601,  601,  602,  602,    0,  602,  602,  602,  602,
+
+      602,  602,  602,  603,    0,  603,  603,  603,  603,  603,
+      603,  603,  603,  604,  604,  605,  605,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590,  590,
+      590,  590,  590,  590,  590,  590,  590,  590,  590
     } ;
 
 /* Table of booleans, true if rule could match eol. */
-static const flex_int32_t yy_rule_can_match_eol[165] =
+static const flex_int32_t yy_rule_can_match_eol[166] =
     {   0,
 0, 1, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
@@ -1177,8 +1177,8 @@ static const flex_int32_t yy_rule_can_match_eol[165] =
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
-    0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 1, 1, 0, 0, 
-    0, 1, 0, 0, 0,     };
+    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 1, 1, 0, 
+    0, 0, 1, 0, 0, 0,     };
 
 /* The intent behind this definition is that it'll catch
  * any uses of REJECT which flex missed.
@@ -1613,13 +1613,13 @@ yy_match:
 			while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state )
 				{
 				yy_current_state = (int) yy_def[yy_current_state];
-				if ( yy_current_state >= 589 )
+				if ( yy_current_state >= 591 )
 					yy_c = yy_meta[yy_c];
 				}
 			yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
 			++yy_cp;
 			}
-		while ( yy_current_state != 588 );
+		while ( yy_current_state != 590 );
 		yy_cp = yyg->yy_last_accepting_cpos;
 		yy_current_state = yyg->yy_last_accepting_state;
 
@@ -2213,122 +2213,122 @@ return TOKEN_STDERR;
 case 107:
 YY_RULE_SETUP
 #line 273 "../SqlLexer.lpp"
-return TOKEN_STDOUT;
+return TOKEN_STDIN;
 	YY_BREAK
 case 108:
 YY_RULE_SETUP
 #line 274 "../SqlLexer.lpp"
-return TOKEN_SUBSTRING;
+return TOKEN_STDOUT;
 	YY_BREAK
 case 109:
 YY_RULE_SETUP
 #line 275 "../SqlLexer.lpp"
-return TOKEN_TABLE;
+return TOKEN_SUBSTRING;
 	YY_BREAK
 case 110:
 YY_RULE_SETUP
 #line 276 "../SqlLexer.lpp"
-return TOKEN_THEN;
+return TOKEN_TABLE;
 	YY_BREAK
 case 111:
 YY_RULE_SETUP
 #line 277 "../SqlLexer.lpp"
-return TOKEN_TIME;
+return TOKEN_THEN;
 	YY_BREAK
 case 112:
 YY_RULE_SETUP
 #line 278 "../SqlLexer.lpp"
-return TOKEN_TIMESTAMP;
+return TOKEN_TIME;
 	YY_BREAK
 case 113:
 YY_RULE_SETUP
 #line 279 "../SqlLexer.lpp"
-return TOKEN_TO;
+return TOKEN_TIMESTAMP;
 	YY_BREAK
 case 114:
 YY_RULE_SETUP
 #line 280 "../SqlLexer.lpp"
-return TOKEN_TRUE;
+return TOKEN_TO;
 	YY_BREAK
 case 115:
 YY_RULE_SETUP
 #line 281 "../SqlLexer.lpp"
-return TOKEN_TUPLESAMPLE;
+return TOKEN_TRUE;
 	YY_BREAK
 case 116:
 YY_RULE_SETUP
 #line 282 "../SqlLexer.lpp"
-return TOKEN_UNBOUNDED;
+return TOKEN_TUPLESAMPLE;
 	YY_BREAK
 case 117:
 YY_RULE_SETUP
 #line 283 "../SqlLexer.lpp"
-return TOKEN_UNION;
+return TOKEN_UNBOUNDED;
 	YY_BREAK
 case 118:
 YY_RULE_SETUP
 #line 284 "../SqlLexer.lpp"
-return TOKEN_UNIQUE;
+return TOKEN_UNION;
 	YY_BREAK
 case 119:
 YY_RULE_SETUP
 #line 285 "../SqlLexer.lpp"
-return TOKEN_UPDATE;
+return TOKEN_UNIQUE;
 	YY_BREAK
 case 120:
 YY_RULE_SETUP
 #line 286 "../SqlLexer.lpp"
-return TOKEN_USING;
+return TOKEN_UPDATE;
 	YY_BREAK
 case 121:
 YY_RULE_SETUP
 #line 287 "../SqlLexer.lpp"
-return TOKEN_VALUES;
+return TOKEN_USING;
 	YY_BREAK
 case 122:
 YY_RULE_SETUP
 #line 288 "../SqlLexer.lpp"
-return TOKEN_VARCHAR;
+return TOKEN_VALUES;
 	YY_BREAK
 case 123:
 YY_RULE_SETUP
 #line 289 "../SqlLexer.lpp"
-return TOKEN_WHEN;
+return TOKEN_VARCHAR;
 	YY_BREAK
 case 124:
 YY_RULE_SETUP
 #line 290 "../SqlLexer.lpp"
-return TOKEN_WHERE;
+return TOKEN_WHEN;
 	YY_BREAK
 case 125:
 YY_RULE_SETUP
 #line 291 "../SqlLexer.lpp"
-return TOKEN_WINDOW;
+return TOKEN_WHERE;
 	YY_BREAK
 case 126:
 YY_RULE_SETUP
 #line 292 "../SqlLexer.lpp"
-return TOKEN_WITH;
+return TOKEN_WINDOW;
 	YY_BREAK
 case 127:
 YY_RULE_SETUP
 #line 293 "../SqlLexer.lpp"
-return TOKEN_YEAR;
+return TOKEN_WITH;
 	YY_BREAK
 case 128:
 YY_RULE_SETUP
 #line 294 "../SqlLexer.lpp"
-return TOKEN_YEARMONTH;
+return TOKEN_YEAR;
 	YY_BREAK
 case 129:
 YY_RULE_SETUP
-#line 296 "../SqlLexer.lpp"
-return TOKEN_EQ;
+#line 295 "../SqlLexer.lpp"
+return TOKEN_YEARMONTH;
 	YY_BREAK
 case 130:
 YY_RULE_SETUP
 #line 297 "../SqlLexer.lpp"
-return TOKEN_NEQ;
+return TOKEN_EQ;
 	YY_BREAK
 case 131:
 YY_RULE_SETUP
@@ -2338,56 +2338,61 @@ return TOKEN_NEQ;
 case 132:
 YY_RULE_SETUP
 #line 299 "../SqlLexer.lpp"
-return TOKEN_LT;
+return TOKEN_NEQ;
 	YY_BREAK
 case 133:
 YY_RULE_SETUP
 #line 300 "../SqlLexer.lpp"
-return TOKEN_GT;
+return TOKEN_LT;
 	YY_BREAK
 case 134:
 YY_RULE_SETUP
 #line 301 "../SqlLexer.lpp"
-return TOKEN_LEQ;
+return TOKEN_GT;
 	YY_BREAK
 case 135:
 YY_RULE_SETUP
 #line 302 "../SqlLexer.lpp"
-return TOKEN_GEQ;
+return TOKEN_LEQ;
 	YY_BREAK
 case 136:
 YY_RULE_SETUP
-#line 304 "../SqlLexer.lpp"
-return yytext[0];
+#line 303 "../SqlLexer.lpp"
+return TOKEN_GEQ;
 	YY_BREAK
 case 137:
 YY_RULE_SETUP
 #line 305 "../SqlLexer.lpp"
 return yytext[0];
 	YY_BREAK
+case 138:
+YY_RULE_SETUP
+#line 306 "../SqlLexer.lpp"
+return yytext[0];
+	YY_BREAK
 /**
     * Quoted strings. Prefacing a string with an 'e' or 'E' causes escape
     * sequences to be processed (as in PostgreSQL).
     **/
-case 138:
+case 139:
 YY_RULE_SETUP
-#line 311 "../SqlLexer.lpp"
+#line 312 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED_ESCAPED);
   }
 	YY_BREAK
-case 139:
+case 140:
 YY_RULE_SETUP
-#line 316 "../SqlLexer.lpp"
+#line 317 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED);
   }
 	YY_BREAK
-case 140:
+case 141:
 YY_RULE_SETUP
-#line 321 "../SqlLexer.lpp"
+#line 322 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_DOUBLE_QUOTED);
@@ -2399,7 +2404,7 @@ YY_RULE_SETUP
 case YY_STATE_EOF(CONDITION_STRING_SINGLE_QUOTED):
 case YY_STATE_EOF(CONDITION_STRING_SINGLE_QUOTED_ESCAPED):
 case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
-#line 330 "../SqlLexer.lpp"
+#line 331 "../SqlLexer.lpp"
 {
     delete yylval->string_value_;
     BEGIN(INITIAL);
@@ -2410,9 +2415,9 @@ case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
 
 /* Process escape sequences. */
 
-case 141:
+case 142:
 YY_RULE_SETUP
-#line 340 "../SqlLexer.lpp"
+#line 341 "../SqlLexer.lpp"
 {
     /* Octal code */
     unsigned int code;
@@ -2426,9 +2431,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 142:
+case 143:
 YY_RULE_SETUP
-#line 352 "../SqlLexer.lpp"
+#line 353 "../SqlLexer.lpp"
 {
     /* Hexadecimal code */
     unsigned int code;
@@ -2436,9 +2441,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 143:
+case 144:
 YY_RULE_SETUP
-#line 358 "../SqlLexer.lpp"
+#line 359 "../SqlLexer.lpp"
 {
     /* A numeric escape sequence that isn't correctly specified. */
     delete yylval->string_value_;
@@ -2447,58 +2452,58 @@ YY_RULE_SETUP
     return TOKEN_LEX_ERROR;
   }
 	YY_BREAK
-case 144:
+case 145:
 YY_RULE_SETUP
-#line 365 "../SqlLexer.lpp"
+#line 366 "../SqlLexer.lpp"
 {
     /* Backspace */
     yylval->string_value_->push_back('\b');
   }
 	YY_BREAK
-case 145:
+case 146:
 YY_RULE_SETUP
-#line 369 "../SqlLexer.lpp"
+#line 370 "../SqlLexer.lpp"
 {
     /* Form-feed */
     yylval->string_value_->push_back('\f');
   }
 	YY_BREAK
-case 146:
+case 147:
 YY_RULE_SETUP
-#line 373 "../SqlLexer.lpp"
+#line 374 "../SqlLexer.lpp"
 {
     /* Newline */
     yylval->string_value_->push_back('\n');
   }
 	YY_BREAK
-case 147:
+case 148:
 YY_RULE_SETUP
-#line 377 "../SqlLexer.lpp"
+#line 378 "../SqlLexer.lpp"
 {
     /* Carriage-return */
     yylval->string_value_->push_back('\r');
   }
 	YY_BREAK
-case 148:
+case 149:
 YY_RULE_SETUP
-#line 381 "../SqlLexer.lpp"
+#line 382 "../SqlLexer.lpp"
 {
     /* Horizontal Tab */
     yylval->string_value_->push_back('\t');
   }
 	YY_BREAK
-case 149:
-/* rule 149 can match eol */
+case 150:
+/* rule 150 can match eol */
 YY_RULE_SETUP
-#line 385 "../SqlLexer.lpp"
+#line 386 "../SqlLexer.lpp"
 {
     /* Any other character (including actual newline or carriage return) */
     yylval->string_value_->push_back(yytext[1]);
   }
 	YY_BREAK
-case 150:
+case 151:
 YY_RULE_SETUP
-#line 389 "../SqlLexer.lpp"
+#line 390 "../SqlLexer.lpp"
 {
     /* This should only be encountered right before an EOF. */
     delete yylval->string_value_;
@@ -2509,17 +2514,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 151:
+case 152:
 YY_RULE_SETUP
-#line 399 "../SqlLexer.lpp"
+#line 400 "../SqlLexer.lpp"
 {
     /* Two quotes in a row become a single quote (this is specified by the SQL standard). */
     yylval->string_value_->push_back('\'');
   }
 	YY_BREAK
-case 152:
+case 153:
 YY_RULE_SETUP
-#line 403 "../SqlLexer.lpp"
+#line 404 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2528,17 +2533,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 153:
+case 154:
 YY_RULE_SETUP
-#line 411 "../SqlLexer.lpp"
+#line 412 "../SqlLexer.lpp"
 {
     /* Two quotes in a row become a single quote (this is specified by the SQL standard). */
     yylval->string_value_->push_back('"');
   }
 	YY_BREAK
-case 154:
+case 155:
 YY_RULE_SETUP
-#line 415 "../SqlLexer.lpp"
+#line 416 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2546,94 +2551,94 @@ YY_RULE_SETUP
   }
 	YY_BREAK
 
-case 155:
-/* rule 155 can match eol */
+case 156:
+/* rule 156 can match eol */
 YY_RULE_SETUP
-#line 422 "../SqlLexer.lpp"
+#line 423 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 156:
-/* rule 156 can match eol */
+case 157:
+/* rule 157 can match eol */
 YY_RULE_SETUP
-#line 427 "../SqlLexer.lpp"
+#line 428 "../SqlLexer.lpp"
 {
   /* Scan up to a quote or escape sequence. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 157:
-/* rule 157 can match eol */
+case 158:
+/* rule 158 can match eol */
 YY_RULE_SETUP
-#line 432 "../SqlLexer.lpp"
+#line 433 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
 
-case 158:
+case 159:
 YY_RULE_SETUP
-#line 438 "../SqlLexer.lpp"
+#line 439 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(
         yylloc->first_line, yylloc->first_column, std::string(yytext, yyleng));
     return TOKEN_NAME;
   }
 	YY_BREAK
-case 159:
+case 160:
 YY_RULE_SETUP
-#line 444 "../SqlLexer.lpp"
+#line 445 "../SqlLexer.lpp"
 {
     yylval->numeric_literal_value_ = new quickstep::NumericParseLiteralValue(
         yylloc->first_line, yylloc->first_column, yytext);
     return TOKEN_UNSIGNED_NUMVAL;
   }
 	YY_BREAK
-case 160:
+case 161:
 YY_RULE_SETUP
-#line 450 "../SqlLexer.lpp"
+#line 451 "../SqlLexer.lpp"
 /* comment */
 	YY_BREAK
-case 161:
-/* rule 161 can match eol */
+case 162:
+/* rule 162 can match eol */
 YY_RULE_SETUP
-#line 452 "../SqlLexer.lpp"
+#line 453 "../SqlLexer.lpp"
 { yycolumn = 0; }
 	YY_BREAK
-case 162:
+case 163:
 YY_RULE_SETUP
-#line 454 "../SqlLexer.lpp"
+#line 455 "../SqlLexer.lpp"
 ; /* ignore white space */
 	YY_BREAK
 /* CONDITION_SQL */
 case YY_STATE_EOF(INITIAL):
 case YY_STATE_EOF(CONDITION_COMMAND):
 case YY_STATE_EOF(CONDITION_SQL):
-#line 458 "../SqlLexer.lpp"
+#line 459 "../SqlLexer.lpp"
 {
   /* All conditions except for mutli-state string extracting conditions. */
   BEGIN(INITIAL);
   return TOKEN_EOF;
 }
 	YY_BREAK
-case 163:
+case 164:
 YY_RULE_SETUP
-#line 464 "../SqlLexer.lpp"
+#line 465 "../SqlLexer.lpp"
 {
   BEGIN(INITIAL);
   quickstep_yyerror(NULL, yyscanner, NULL, "illegal character");
   return TOKEN_LEX_ERROR;
 }
 	YY_BREAK
-case 164:
+case 165:
 YY_RULE_SETUP
-#line 470 "../SqlLexer.lpp"
+#line 471 "../SqlLexer.lpp"
 YY_FATAL_ERROR( "flex scanner jammed" );
 	YY_BREAK
-#line 2636 "SqlLexer_gen.cpp"
+#line 2641 "SqlLexer_gen.cpp"
 
 	case YY_END_OF_BUFFER:
 		{
@@ -2931,7 +2936,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 		while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state )
 			{
 			yy_current_state = (int) yy_def[yy_current_state];
-			if ( yy_current_state >= 589 )
+			if ( yy_current_state >= 591 )
 				yy_c = yy_meta[yy_c];
 			}
 		yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
@@ -2960,11 +2965,11 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 	while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state )
 		{
 		yy_current_state = (int) yy_def[yy_current_state];
-		if ( yy_current_state >= 589 )
+		if ( yy_current_state >= 591 )
 			yy_c = yy_meta[yy_c];
 		}
 	yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
-	yy_is_jam = (yy_current_state == 588);
+	yy_is_jam = (yy_current_state == 590);
 
 	(void)yyg;
 	return yy_is_jam ? 0 : yy_current_state;
@@ -3794,6 +3799,6 @@ void yyfree (void * ptr , yyscan_t yyscanner)
 
 #define YYTABLES_NAME "yytables"
 
-#line 470 "../SqlLexer.lpp"
+#line 471 "../SqlLexer.lpp"
 
 

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


[04/27] incubator-quickstep git commit: Simplified the work order generation.

Posted by ji...@apache.org.
Simplified the work order generation.


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

Branch: refs/heads/trace
Commit: 8d7284decb7ebf5c0eaac232f39027ddd8bf6144
Parents: 77960a4
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Mon Aug 21 19:51:55 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Fri Sep 22 13:43:08 2017 -0500

----------------------------------------------------------------------
 query_execution/CMakeLists.txt                  |   2 -
 query_execution/ForemanDistributed.cpp          |   5 +-
 query_execution/ForemanSingleNode.cpp           |  16 +--
 query_execution/QueryManagerBase.cpp            | 136 ++++++++-----------
 query_execution/QueryManagerBase.hpp            |  79 ++---------
 query_execution/QueryManagerDistributed.cpp     |  54 +++-----
 query_execution/QueryManagerDistributed.hpp     |   3 +-
 query_execution/QueryManagerSingleNode.cpp      |  58 ++++----
 query_execution/QueryManagerSingleNode.hpp      |   7 +-
 query_execution/WorkOrdersContainer.hpp         |   1 +
 .../tests/QueryManagerSingleNode_unittest.cpp   |  58 ++++----
 11 files changed, 152 insertions(+), 267 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 5c750f0..9394c00 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -119,7 +119,6 @@ if (ENABLE_DISTRIBUTED)
                         quickstep_storage_StorageBlockInfo
                         quickstep_storage_StorageManager
                         quickstep_threading_ThreadUtil
-                        quickstep_utility_EqualsAnyConstant
                         quickstep_utility_Macros
                         tmb
                         ${GFLAGS_LIB_NAME})
@@ -135,7 +134,6 @@ target_link_libraries(quickstep_queryexecution_ForemanSingleNode
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage
                       quickstep_threading_ThreadUtil
-                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros
                       tmb
                       ${GFLAGS_LIB_NAME})

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/ForemanDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.cpp b/query_execution/ForemanDistributed.cpp
index 942f383..82cc624 100644
--- a/query_execution/ForemanDistributed.cpp
+++ b/query_execution/ForemanDistributed.cpp
@@ -48,7 +48,6 @@
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
 #include "threading/ThreadUtil.hpp"
-#include "utility/EqualsAnyConstant.hpp"
 
 #include "glog/logging.h"
 
@@ -233,9 +232,7 @@ void ForemanDistributed::run() {
 }
 
 bool ForemanDistributed::canCollectNewMessages(const tmb::message_type_id message_type) {
-  return !QUICKSTEP_EQUALS_ANY_CONSTANT(message_type,
-                                        kCatalogRelationNewBlockMessage,
-                                        kWorkOrderFeedbackMessage);
+  return message_type != kCatalogRelationNewBlockMessage;
 }
 
 bool ForemanDistributed::isAggregationRelatedWorkOrder(const S::WorkOrderMessage &proto,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index 1501408..d66f1f5 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -33,7 +33,6 @@
 #include "query_execution/WorkerDirectory.hpp"
 #include "query_execution/WorkerMessage.hpp"
 #include "threading/ThreadUtil.hpp"
-#include "utility/EqualsAnyConstant.hpp"
 #include "utility/Macros.hpp"
 
 #include "gflags/gflags.h"
@@ -179,18 +178,13 @@ void ForemanSingleNode::run() {
 }
 
 bool ForemanSingleNode::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 {
+  if (message_type == kCatalogRelationNewBlockMessage) {
     return false;
   }
+
+  // If the least loaded worker has only one pending work order, we should
+  // collect new messages and dispatch them.
+  return (worker_directory_->getLeastLoadedWorker().second <= FLAGS_min_load_per_worker);
 }
 
 void ForemanSingleNode::dispatchWorkerMessages(const vector<unique_ptr<WorkerMessage>> &messages) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/QueryManagerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.cpp b/query_execution/QueryManagerBase.cpp
index 565c6ad..374c96d 100644
--- a/query_execution/QueryManagerBase.cpp
+++ b/query_execution/QueryManagerBase.cpp
@@ -50,7 +50,9 @@ QueryManagerBase::QueryManagerBase(QueryHandle *query_handle)
       num_operators_in_dag_(query_dag_->size()),
       output_consumers_(num_operators_in_dag_),
       blocking_dependencies_(num_operators_in_dag_),
-      query_exec_state_(new QueryExecutionState(num_operators_in_dag_)) {
+      query_exec_state_(new QueryExecutionState(num_operators_in_dag_)),
+      blocking_dependents_(num_operators_in_dag_),
+      non_blocking_dependencies_(num_operators_in_dag_) {
   if (FLAGS_visualize_execution_dag) {
     dag_visualizer_ =
         std::make_unique<quickstep::ExecutionDAGVisualizer>(query_handle_->getQueryPlan());
@@ -66,16 +68,22 @@ QueryManagerBase::QueryManagerBase(QueryHandle *query_handle)
       query_exec_state_->setRebuildRequired(node_index);
     }
 
+    if (query_dag_->getDependencies(node_index).empty()) {
+      non_dependent_operators_.push_back(node_index);
+    }
+
     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 a pipeline-breaker. Streaming of blocks is not possible
         // between these two operators.
-        blocking_dependencies_[dependent_op_index].push_back(node_index);
+        blocking_dependencies_[dependent_op_index].insert(node_index);
+        blocking_dependents_[node_index].push_back(dependent_op_index);
       } else {
         // The link is not a pipeline-breaker. Streaming of blocks is possible
         // between these two operators.
+        non_blocking_dependencies_[dependent_op_index].insert(node_index);
         output_consumers_[node_index].push_back(dependent_op_index);
       }
     }
@@ -102,6 +110,12 @@ void QueryManagerBase::processFeedbackMessage(
   RelationalOperator *op =
       query_dag_->getNodePayloadMutable(op_index);
   op->receiveFeedbackMessage(msg);
+
+  if (query_exec_state_->hasDoneGenerationWorkOrders(op_index)) {
+    return;
+  }
+
+  fetchNormalWorkOrders(op_index);
 }
 
 void QueryManagerBase::processWorkOrderCompleteMessage(
@@ -109,97 +123,32 @@ void QueryManagerBase::processWorkOrderCompleteMessage(
     const partition_id part_id) {
   query_exec_state_->decrementNumQueuedWorkOrders(op_index);
 
-  // Check if new work orders are available and fetch them if so.
-  fetchNormalWorkOrders(op_index);
+  if (!checkNormalExecutionOver(op_index)) {
+    // Normal execution under progress for this operator.
+    return;
+  }
 
   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.
+    DCHECK(!checkRebuildInitiated(op_index));
+    if (!initiateRebuild(op_index)) {
+      // Rebuild under progress.
+      return;
     }
-  } else if (checkOperatorExecutionOver(op_index)) {
-    // Rebuild not required for this operator and its normal execution is
-    // complete.
-    markOperatorFinished(op_index);
+    // Rebuild initiated and completed right away.
   }
 
-  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);
-    }
-  }
+  markOperatorFinished(op_index);
 }
 
 void QueryManagerBase::processRebuildWorkOrderCompleteMessage(const dag_node_index op_index,
                                                               const partition_id part_id) {
   query_exec_state_->decrementNumRebuildWorkOrders(op_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);
-      }
-    }
-  }
-}
-
-void QueryManagerBase::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.
+  if (!checkRebuildOver(op_index)) {
     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);
-        }
-      }
-    }
-  }
+  markOperatorFinished(op_index);
 }
 
 void QueryManagerBase::processDataPipelineMessage(const dag_node_index op_index,
@@ -214,23 +163,44 @@ void QueryManagerBase::processDataPipelineMessage(const dag_node_index op_index,
     query_dag_->getNodePayloadMutable(consumer_index)->feedInputBlock(block, rel_id, part_id);
     // Because of the streamed input just fed, check if there are any new
     // WorkOrders available and if so, fetch them.
-    fetchNormalWorkOrders(consumer_index);
+    if (checkAllBlockingDependenciesMet(consumer_index)) {
+      fetchNormalWorkOrders(consumer_index);
+    }
   }
 }
 
 void QueryManagerBase::markOperatorFinished(const dag_node_index index) {
   query_exec_state_->setExecutionFinished(index);
 
+  for (const dag_node_index dependent_op_index : blocking_dependents_[index]) {
+    blocking_dependencies_[dependent_op_index].erase(index);
+  }
+
+  for (const dag_node_index dependent_op_index : output_consumers_[index]) {
+    non_blocking_dependencies_[dependent_op_index].erase(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);
+      // Signal dependent operator that current operator is done feeding input blocks.
+      query_dag_->getNodePayloadMutable(dependent_op_index)->doneFeedingInputBlocks(output_rel);
+    }
+
+    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.
+      if (!fetchNormalWorkOrders(dependent_op_index) &&
+          non_blocking_dependencies_[dependent_op_index].empty() &&
+          checkNormalExecutionOver(dependent_op_index) &&
+          (!checkRebuildRequired(dependent_op_index) || initiateRebuild(dependent_op_index))) {
+        markOperatorFinished(dependent_op_index);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/QueryManagerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.hpp b/query_execution/QueryManagerBase.hpp
index 78d67cc..366ab61 100644
--- a/query_execution/QueryManagerBase.hpp
+++ b/query_execution/QueryManagerBase.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <unordered_set>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
@@ -165,56 +166,20 @@ class QueryManagerBase {
 
  protected:
   /**
-   * @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.
-   **/
-  void processOperator(const dag_node_index index,
-                       const bool recursively_check_dependents);
-
-  /**
    * @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.
+   *        2. For all the dependents of this operator, 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.
+   *        3. Check if all of their blocking dependencies are met. If so
+   *        fetch normal work orders.
    *
    * @param index The index of the given relational operator in the DAG.
    **/
   void markOperatorFinished(const dag_node_index 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.
    *
@@ -229,27 +194,7 @@ class QueryManagerBase {
    **/
   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 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 {
-    return this->checkNormalExecutionOver(index) &&
-           (!checkRebuildRequired(index) || this->checkRebuildOver(index));
+    return blocking_dependencies_[node_index].empty();
   }
 
   /**
@@ -295,7 +240,9 @@ class QueryManagerBase {
   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::vector<std::unordered_set<dag_node_index>> blocking_dependencies_;
+
+  std::vector<dag_node_index> non_dependent_operators_;
 
   std::unique_ptr<QueryExecutionState> query_exec_state_;
 
@@ -338,6 +285,10 @@ class QueryManagerBase {
    **/
   virtual bool checkRebuildOver(const dag_node_index index) const = 0;
 
+  // For all nodes, store their pipeline breaking dependents (if any).
+  std::vector<std::vector<dag_node_index>> blocking_dependents_;
+  std::vector<std::unordered_set<dag_node_index>> non_blocking_dependencies_;
+
   DISALLOW_COPY_AND_ASSIGN(QueryManagerBase);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/QueryManagerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.cpp b/query_execution/QueryManagerDistributed.cpp
index 1144e9f..30a1396 100644
--- a/query_execution/QueryManagerDistributed.cpp
+++ b/query_execution/QueryManagerDistributed.cpp
@@ -67,10 +67,11 @@ QueryManagerDistributed::QueryManagerDistributed(QueryHandle *query_handle,
       bus_(bus),
       normal_workorder_protos_container_(
           new WorkOrderProtosContainer(num_operators_in_dag_)) {
-  // Collect all the workorders from all the relational operators in the DAG.
-  for (dag_node_index index = 0; index < num_operators_in_dag_; ++index) {
-    if (checkAllBlockingDependenciesMet(index)) {
-      processOperator(index, false);
+  // Collect all the workorders from all the non-blocking relational operators in the DAG.
+  for (const dag_node_index index : non_dependent_operators_) {
+    if (!fetchNormalWorkOrders(index)) {
+      DCHECK(!checkRebuildRequired(index) || initiateRebuild(index));
+      markOperatorFinished(index);
     }
   }
 
@@ -177,35 +178,22 @@ serialization::WorkOrderMessage* QueryManagerDistributed::getNextWorkOrderMessag
 }
 
 bool QueryManagerDistributed::fetchNormalWorkOrders(const dag_node_index index) {
-  bool generated_new_workorder_protos = 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_workorder_protos_before =
-        normal_workorder_protos_container_->getNumWorkOrderProtos(index);
-    const bool done_generation =
-        query_dag_->getNodePayloadMutable(index)
-            ->getAllWorkOrderProtos(normal_workorder_protos_container_.get());
-    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_workorder_protos =
-        (num_pending_workorder_protos_before <
-         normal_workorder_protos_container_->getNumWorkOrderProtos(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.
+  DCHECK(checkAllBlockingDependenciesMet(index));
+  DCHECK(!query_exec_state_->hasDoneGenerationWorkOrders(index));
+
+  const size_t num_pending_workorder_protos_before =
+      normal_workorder_protos_container_->getNumWorkOrderProtos(index);
+  const bool done_generation =
+      query_dag_->getNodePayloadMutable(index)
+          ->getAllWorkOrderProtos(normal_workorder_protos_container_.get());
+  if (done_generation) {
+    query_exec_state_->setDoneGenerationWorkOrders(index);
   }
-  return generated_new_workorder_protos;
+
+  return (num_pending_workorder_protos_before < normal_workorder_protos_container_->getNumWorkOrderProtos(index));
 }
 
 void QueryManagerDistributed::processInitiateRebuildResponseMessage(const dag_node_index op_index,
@@ -225,7 +213,7 @@ void QueryManagerDistributed::processInitiateRebuildResponseMessage(const dag_no
        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);
+      fetchNormalWorkOrders(dependent_op_index);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/QueryManagerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.hpp b/query_execution/QueryManagerDistributed.hpp
index a021fdd..8d870c6 100644
--- a/query_execution/QueryManagerDistributed.hpp
+++ b/query_execution/QueryManagerDistributed.hpp
@@ -250,8 +250,7 @@ class QueryManagerDistributed final : public QueryManagerBase {
 
  private:
   bool checkNormalExecutionOver(const dag_node_index index) const override {
-    return (checkAllDependenciesMet(index) &&
-            !normal_workorder_protos_container_->hasWorkOrderProto(index) &&
+    return (!normal_workorder_protos_container_->hasWorkOrderProto(index) &&
             query_exec_state_->getNumQueuedWorkOrders(index) == 0 &&
             query_exec_state_->hasDoneGenerationWorkOrders(index));
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/QueryManagerSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerSingleNode.cpp b/query_execution/QueryManagerSingleNode.cpp
index 82a0de6..2c9f673 100644
--- a/query_execution/QueryManagerSingleNode.cpp
+++ b/query_execution/QueryManagerSingleNode.cpp
@@ -61,10 +61,11 @@ QueryManagerSingleNode::QueryManagerSingleNode(
       workorders_container_(
           new WorkOrdersContainer(num_operators_in_dag_, num_numa_nodes)),
       database_(static_cast<const CatalogDatabase&>(*catalog_database)) {
-  // Collect all the workorders from all the relational operators in the DAG.
-  for (dag_node_index index = 0; index < num_operators_in_dag_; ++index) {
-    if (checkAllBlockingDependenciesMet(index)) {
-      processOperator(index, false);
+  // Collect all the workorders from all the non-blocking relational operators in the DAG.
+  for (const dag_node_index index : non_dependent_operators_) {
+    if (!fetchNormalWorkOrders(index)) {
+      DCHECK(!checkRebuildRequired(index) || initiateRebuild(index));
+      markOperatorFinished(index);
     }
   }
 }
@@ -87,38 +88,25 @@ WorkerMessage* QueryManagerSingleNode::getNextWorkerMessage(
 }
 
 bool QueryManagerSingleNode::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));
+  // 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.
+  DCHECK(checkAllBlockingDependenciesMet(index));
+  DCHECK(!query_exec_state_->hasDoneGenerationWorkOrders(index));
+
+  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);
   }
-  return generated_new_workorders;
+
+  return (num_pending_workorders_before < workorders_container_->getNumNormalWorkOrders(index));
 }
 
 bool QueryManagerSingleNode::initiateRebuild(const dag_node_index index) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/QueryManagerSingleNode.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerSingleNode.hpp b/query_execution/QueryManagerSingleNode.hpp
index f9d038b..a726bbc 100644
--- a/query_execution/QueryManagerSingleNode.hpp
+++ b/query_execution/QueryManagerSingleNode.hpp
@@ -99,8 +99,7 @@ class QueryManagerSingleNode final : public QueryManagerBase {
 
  private:
   bool checkNormalExecutionOver(const dag_node_index index) const override {
-    return (checkAllDependenciesMet(index) &&
-            !workorders_container_->hasNormalWorkOrder(index) &&
+    return (!workorders_container_->hasNormalWorkOrder(index) &&
             query_exec_state_->getNumQueuedWorkOrders(index) == 0 &&
             query_exec_state_->hasDoneGenerationWorkOrders(index));
   }
@@ -108,8 +107,8 @@ class QueryManagerSingleNode final : public QueryManagerBase {
   bool initiateRebuild(const dag_node_index index) override;
 
   bool checkRebuildOver(const dag_node_index index) const override {
-    return query_exec_state_->hasRebuildInitiated(index) &&
-           !workorders_container_->hasRebuildWorkOrder(index) &&
+    DCHECK(query_exec_state_->hasRebuildInitiated(index));
+    return !workorders_container_->hasRebuildWorkOrder(index) &&
            (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/WorkOrdersContainer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkOrdersContainer.hpp b/query_execution/WorkOrdersContainer.hpp
index e8d5ff8..3c2d9bf 100644
--- a/query_execution/WorkOrdersContainer.hpp
+++ b/query_execution/WorkOrdersContainer.hpp
@@ -542,6 +542,7 @@ class WorkOrdersContainer {
 
   DISALLOW_COPY_AND_ASSIGN(WorkOrdersContainer);
 };
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d7284de/query_execution/tests/QueryManagerSingleNode_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManagerSingleNode_unittest.cpp b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
index 19b42ac..dd3f472 100644
--- a/query_execution/tests/QueryManagerSingleNode_unittest.cpp
+++ b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
@@ -353,14 +353,14 @@ TEST_F(QueryManagerTest, 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.
+  // called 3 times and 3 work orders will be returned, i.e., 2 calls to
+  // getAllWorkOrders() insert 2 WorkOrder and return false, and the last will
+  // insert 1 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));
+      query_plan_->addRelationalOperator(new MockOperator(true, false, 3, 3));
 
   const MockOperator &op = static_cast<const MockOperator &>(
       query_plan_->getQueryPlanDAG().getNodePayload(id));
@@ -378,7 +378,7 @@ TEST_F(QueryManagerTest, SingleNodeDAGDynamicWorkOrdersTest) {
     unique_ptr<WorkerMessage> worker_message;
     worker_message.reset(query_manager_->getNextWorkerMessage(id, -1));
 
-    EXPECT_TRUE(worker_message != nullptr);
+    ASSERT_TRUE(worker_message != nullptr);
     EXPECT_EQ(WorkerMessage::WorkerMessageType::kWorkOrder,
               worker_message->getType());
     EXPECT_EQ(id, worker_message->getRelationalOpIndex());
@@ -391,6 +391,7 @@ TEST_F(QueryManagerTest, SingleNodeDAGDynamicWorkOrdersTest) {
     if (i < 2) {
       // Send a message to QueryManager upon workorder completion.
       EXPECT_FALSE(placeWorkOrderCompleteMessage(id));
+      query_manager_->fetchNormalWorkOrders(id);
     } else {
       // Send a message to QueryManager upon workorder completion.
       // Last event.
@@ -511,7 +512,7 @@ TEST_F(QueryManagerTest, TwoNodesDAGPipeLinkTest) {
   const QueryPlan::DAGNodeIndex id1 =
       query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
   const QueryPlan::DAGNodeIndex id2 =
-      query_plan_->addRelationalOperator(new MockOperator(true, true, 3));
+      query_plan_->addRelationalOperator(new MockOperator(true, true, 2));
 
   // Create a non-blocking link.
   query_plan_->addDirectDependency(id2, id1, false);
@@ -531,7 +532,7 @@ TEST_F(QueryManagerTest, TwoNodesDAGPipeLinkTest) {
   EXPECT_EQ(1, op1.getNumWorkOrders());
   EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlock));
 
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
+  EXPECT_EQ(0, 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));
@@ -562,7 +563,7 @@ TEST_F(QueryManagerTest, TwoNodesDAGPipeLinkTest) {
   EXPECT_EQ(1, op2.getNumCalls(MockOperator::kFeedInputBlock));
 
   // A call to op2's getAllWorkOrders because of the streamed input.
-  EXPECT_EQ(2, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
+  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(1, op2.getNumWorkOrders());
 
   // Place a message of a workorder completion of op1 on Foreman's input queue.
@@ -573,7 +574,7 @@ TEST_F(QueryManagerTest, TwoNodesDAGPipeLinkTest) {
   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.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(2, op2.getNumWorkOrders());
 
   worker_message.reset(query_manager_->getNextWorkerMessage(id2, -1));
@@ -620,7 +621,7 @@ TEST_F(QueryManagerTest, TwoNodesDAGPartiallyFilledBlocksTest) {
   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));
+      query_plan_->addRelationalOperator(new MockOperator(true, true, 2, 1));
 
   // Create a non-blocking link.
   query_plan_->addDirectDependency(id2, id1, false);
@@ -670,7 +671,7 @@ TEST_F(QueryManagerTest, TwoNodesDAGPartiallyFilledBlocksTest) {
   EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(1, op1.getNumWorkOrders());
 
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
+  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(0, op2.getNumWorkOrders());
 
   unique_ptr<WorkerMessage> worker_message;
@@ -704,7 +705,7 @@ TEST_F(QueryManagerTest, TwoNodesDAGPartiallyFilledBlocksTest) {
   EXPECT_FALSE(placeRebuildWorkOrderCompleteMessage(id1));
   // Based on the streamed input, op2's getAllWorkOrders should produce a
   // workorder.
-  EXPECT_EQ(3, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
+  EXPECT_EQ(2, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(1, op2.getNumWorkOrders());
 
   worker_message.reset(query_manager_->getNextWorkerMessage(id2, -1));
@@ -734,16 +735,14 @@ TEST_F(QueryManagerTest, MultipleNodesNoOutputTest) {
   // When an operator produces workorders but no output, the QueryManager 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));
+      query_plan_->addRelationalOperator(new MockOperator(true, false));
     } else {
-      ids[i] = query_plan_->addRelationalOperator(new MockOperator(true, true));
+      query_plan_->addRelationalOperator(new MockOperator(true, true));
     }
-    VLOG(3) << ids[i];
+    VLOG(3) << i;
   }
 
   /**
@@ -753,46 +752,47 @@ TEST_F(QueryManagerTest, MultipleNodesNoOutputTest) {
    *
    **/
   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]))
+    query_plan_->addDirectDependency(i + 1, i, false);
+    static_cast<MockOperator*>(query_plan_->getQueryPlanDAGMutable()->getNodePayloadMutable(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])));
+    operators.push_back(static_cast<const MockOperator*>(&query_plan_->getQueryPlanDAG().getNodePayload(i)));
   }
 
   constructQueryManager();
 
   // operators[0] should have produced a workorder by now.
+  EXPECT_EQ(1, operators[0]->getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(1, operators[0]->getNumWorkOrders());
 
   unique_ptr<WorkerMessage> worker_message;
-  worker_message.reset(query_manager_->getNextWorkerMessage(ids[0], -1));
+  worker_message.reset(query_manager_->getNextWorkerMessage(0, -1));
 
   EXPECT_TRUE(worker_message != nullptr);
   EXPECT_EQ(WorkerMessage::WorkerMessageType::kWorkOrder,
             worker_message->getType());
 
-  EXPECT_EQ(ids[0], worker_message->getRelationalOpIndex());
+  EXPECT_EQ(0, worker_message->getRelationalOpIndex());
 
   delete worker_message->getWorkOrder();
 
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(ids[0]));
-  EXPECT_FALSE(getOperatorFinishedStatus(ids[0]));
+  EXPECT_EQ(1, getNumWorkOrdersInExecution(0));
+  EXPECT_FALSE(getOperatorFinishedStatus(0));
 
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    EXPECT_EQ(1, operators[ids[i]]->getNumCalls(MockOperator::kGetAllWorkOrders));
+  for (QueryPlan::DAGNodeIndex i = 1; i < kNumNodes; ++i) {
+    EXPECT_EQ(0, operators[i]->getNumCalls(MockOperator::kGetAllWorkOrders));
   }
 
   // Send a message to QueryManager upon workorder (generated by operators[0])
   // completion.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(ids[0]));
+  EXPECT_TRUE(placeWorkOrderCompleteMessage(0));
 
   for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    EXPECT_EQ(0, getNumWorkOrdersInExecution(ids[i]));
-    EXPECT_TRUE(getOperatorFinishedStatus(ids[i]));
+    EXPECT_EQ(0, getNumWorkOrdersInExecution(i));
+    EXPECT_TRUE(getOperatorFinishedStatus(i));
     if (i < kNumNodes - 1) {
       EXPECT_EQ(1, operators[i + 1]->getNumCalls(MockOperator::kDoneFeedingInputBlocks));
     }



[06/27] incubator-quickstep git commit: Bug fix in LockManager loop

Posted by ji...@apache.org.
Bug fix in LockManager loop

- Added a false condition for acquire lock
- Added clarifying comment.


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

Branch: refs/heads/trace
Commit: 9cbb930b5adb589f7a2ba8140d8d59227c9a570e
Parents: bf455e2
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Wed Sep 27 10:02:10 2017 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Fri Sep 29 10:42:41 2017 -0500

----------------------------------------------------------------------
 transaction/LockManager.cpp | 15 ++++++++-------
 1 file changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9cbb930b/transaction/LockManager.cpp
----------------------------------------------------------------------
diff --git a/transaction/LockManager.cpp b/transaction/LockManager.cpp
index 2a3760f..c917b4b 100644
--- a/transaction/LockManager.cpp
+++ b/transaction/LockManager.cpp
@@ -80,21 +80,22 @@ void LockManager::run() {
         if (request.getRequestType() == RequestType::kReleaseLocks) {
           CHECK(releaseAllLocks(request.getTransactionId()))
               << "Unexpected condition occured.";
-
         } else if (acquireLock(request.getTransactionId(),
                                request.getResourceId(),
                                request.getAccessMode())) {
+          // Lock has been acquired.
           LOG(INFO) << "Transaction "
                     << std::to_string(request.getTransactionId())
-                    << " is waiting " + request.getResourceId().toString();
+                    << " acquired " + request.getResourceId().toString();
 
-            inner_pending_requests_.push(request);
+          permitted_requests_.push(request);
         } else {
-            LOG(INFO) << "Transaction "
-                      << std::to_string(request.getTransactionId())
-                      << " acquired " + request.getResourceId().toString();
+          // We are unable to acquire lock at this point.
+          LOG(INFO) << "Transaction "
+                    << std::to_string(request.getTransactionId())
+                    << " is waiting " + request.getResourceId().toString();
 
-            permitted_requests_.push(request);
+          inner_pending_requests_.push(request);
         }
       }
     }


[24/27] incubator-quickstep git commit: Quickstep for GRAIL.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index 142059d..96c649d 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -156,29 +156,30 @@ extern int quickstep_yydebug;
     TOKEN_SMA = 366,
     TOKEN_SMALLINT = 367,
     TOKEN_STDERR = 368,
-    TOKEN_STDOUT = 369,
-    TOKEN_SUBSTRING = 370,
-    TOKEN_TABLE = 371,
-    TOKEN_THEN = 372,
-    TOKEN_TIME = 373,
-    TOKEN_TIMESTAMP = 374,
-    TOKEN_TO = 375,
-    TOKEN_TRUE = 376,
-    TOKEN_TUPLESAMPLE = 377,
-    TOKEN_UNBOUNDED = 378,
-    TOKEN_UNIQUE = 379,
-    TOKEN_UPDATE = 380,
-    TOKEN_USING = 381,
-    TOKEN_VALUES = 382,
-    TOKEN_VARCHAR = 383,
-    TOKEN_WHEN = 384,
-    TOKEN_WHERE = 385,
-    TOKEN_WINDOW = 386,
-    TOKEN_WITH = 387,
-    TOKEN_YEAR = 388,
-    TOKEN_YEARMONTH = 389,
-    TOKEN_EOF = 390,
-    TOKEN_LEX_ERROR = 391
+    TOKEN_STDIN = 369,
+    TOKEN_STDOUT = 370,
+    TOKEN_SUBSTRING = 371,
+    TOKEN_TABLE = 372,
+    TOKEN_THEN = 373,
+    TOKEN_TIME = 374,
+    TOKEN_TIMESTAMP = 375,
+    TOKEN_TO = 376,
+    TOKEN_TRUE = 377,
+    TOKEN_TUPLESAMPLE = 378,
+    TOKEN_UNBOUNDED = 379,
+    TOKEN_UNIQUE = 380,
+    TOKEN_UPDATE = 381,
+    TOKEN_USING = 382,
+    TOKEN_VALUES = 383,
+    TOKEN_VARCHAR = 384,
+    TOKEN_WHEN = 385,
+    TOKEN_WHERE = 386,
+    TOKEN_WINDOW = 387,
+    TOKEN_WITH = 388,
+    TOKEN_YEAR = 389,
+    TOKEN_YEARMONTH = 390,
+    TOKEN_EOF = 391,
+    TOKEN_LEX_ERROR = 392
   };
 #endif
 
@@ -289,7 +290,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 293 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 294 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 14d8949..3bca344 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1212,10 +1212,13 @@ void ExecutionGenerator::convertCopyFrom(
         ->MergeFrom(output_relation->getPartitionScheme()->getProto());
   } else {
     insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
-
-    const vector<block_id> blocks(output_relation->getBlocksSnapshot());
-    for (const block_id block : blocks) {
-      insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+    const StorageBlockLayout &layout = output_relation->getDefaultStorageBlockLayout();
+    const auto sub_block_type = layout.getDescription().tuple_store_description().sub_block_type();
+    if (sub_block_type != TupleStorageSubBlockDescription::COMPRESSED_COLUMN_STORE) {
+      const vector<block_id> blocks(output_relation->getBlocksSnapshot());
+      for (const block_id block : blocks) {
+        insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+      }
     }
   }
 
@@ -1880,7 +1883,6 @@ void ExecutionGenerator::convertAggregate(
       use_parallel_initialization = true;
       aggr_state_num_partitions = CalculateNumFinalizationPartitionsForCollisionFreeVectorTable(max_num_groups);
 
-      DCHECK(!group_by_aggrs_info.empty());
       CalculateCollisionFreeAggregationInfo(max_num_groups, group_by_aggrs_info,
                                             aggr_state_proto->mutable_collision_free_vector_info());
     } else {

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 0b6dc22..17198c2 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -1019,10 +1019,13 @@ L::LogicalPtr Resolver::resolveInsertSelection(
     if (destination_type.equals(selection_type)) {
       cast_expressions.emplace_back(selection_attributes[aid]);
     } else {
-      // TODO(jianqiao): implement Cast operation for non-numeric types.
+      // TODO(jianqiao): Implement Cast operation for non-numeric types.
+      // TODO(jianqiao): We temporarily disable the safely-coercible check for
+      // tricks that work around "argmin". Will switch it back once the "Cast"
+      // function is supported.
       if (destination_type.getSuperTypeID() == Type::SuperTypeID::kNumeric &&
           selection_type.getSuperTypeID() == Type::SuperTypeID::kNumeric &&
-          destination_type.isSafelyCoercibleFrom(selection_type)) {
+          destination_type.isCoercibleFrom(selection_type)) {
         // Add cast operation
         const E::AttributeReferencePtr attr = selection_attributes[aid];
         const E::ExpressionPtr cast_expr =
@@ -1038,7 +1041,7 @@ L::LogicalPtr Resolver::resolveInsertSelection(
             << insert_statement.relation_name()->value() << "."
             << destination_attributes[aid]->attribute_name() << " has type "
             << selection_attributes[aid]->getValueType().getName()
-            << ", which cannot be safely coerced to the column's type "
+            << ", which cannot be coerced to the column's type "
             << destination_attributes[aid]->getValueType().getName();
      }
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/relational_operators/TableExportOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableExportOperator.cpp b/relational_operators/TableExportOperator.cpp
index f6a73bf..cb90d72 100644
--- a/relational_operators/TableExportOperator.cpp
+++ b/relational_operators/TableExportOperator.cpp
@@ -120,6 +120,8 @@ void TableExportOperator::receiveFeedbackMessage(
     } else if (lo_file_name == "$stderr") {
       file_ = stderr;
     } else {
+      DCHECK(!file_name_.empty());
+      DCHECK_EQ('@', file_name_.front());
       file_ = std::fopen(file_name_.substr(1).c_str(), "wb");
       // TODO(quickstep-team): Decent handling of exceptions at query runtime.
       if (file_ == nullptr) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index 66137d8..88b7214 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -57,6 +57,7 @@
 #include "types/containers/Tuple.hpp"
 #include "utility/BulkIoConfiguration.hpp"
 #include "utility/Glob.hpp"
+#include "utility/ScopedBuffer.hpp"
 
 #include "gflags/gflags.h"
 #include "glog/logging.h"
@@ -110,19 +111,36 @@ bool TextScanOperator::getAllWorkOrders(
     const tmb::client_id scheduler_client_id,
     tmb::MessageBus *bus) {
   DCHECK(query_context != nullptr);
+  DCHECK(!file_pattern_.empty());
 
-  const std::vector<std::string> files = utility::file::GlobExpand(file_pattern_);
-
-  CHECK_NE(files.size(), 0u)
-      << "No files matched '" << file_pattern_ << "'. Exiting.";
+  if (work_generated_) {
+    return true;
+  }
 
   InsertDestination *output_destination =
       query_context->getInsertDestination(output_destination_index_);
 
-  if (work_generated_) {
+  if (file_pattern_ == "$stdin") {
+    container->addNormalWorkOrder(
+        new TextScanWorkOrder(query_id_,
+                              file_pattern_,
+                              0,
+                              -1 /* text_segment_size */,
+                              options_->getDelimiter(),
+                              options_->escapeStrings(),
+                              output_destination),
+        op_index_);
+    work_generated_ = true;
     return true;
   }
 
+  DCHECK_EQ('@', file_pattern_.front());
+  const std::vector<std::string> files =
+      utility::file::GlobExpand(file_pattern_.substr(1));
+
+  CHECK_NE(files.size(), 0u)
+      << "No files matched '" << file_pattern_ << "'. Exiting.";
+
   for (const std::string &file : files) {
 #ifdef QUICKSTEP_HAVE_UNISTD
     // Check file permissions before trying to open it.
@@ -221,6 +239,12 @@ serialization::WorkOrder* TextScanOperator::createWorkOrderProto(
 }
 
 void TextScanWorkOrder::execute() {
+  DCHECK(!filename_.empty());
+  if (filename_ == "$stdin") {
+    executeInputStream();
+    return;
+  }
+
   const CatalogRelationSchema &relation = output_destination_->getRelation();
   std::vector<Tuple> tuples;
   bool is_faulty;
@@ -436,6 +460,76 @@ void TextScanWorkOrder::execute() {
   output_destination_->bulkInsertTuples(&column_vectors);
 }
 
+void TextScanWorkOrder::executeInputStream() {
+  std::string data;
+  const int len = std::ftell(stdin);
+  if (len >= 0) {
+    ScopedBuffer buffer(len, false);
+    std::rewind(stdin);
+    std::fread(buffer.get(), 1, len, stdin);
+    data = std::string(static_cast<char*>(buffer.get()), len);
+  } else {
+    std::unique_ptr<std::ostringstream> oss = std::make_unique<std::ostringstream>();
+    *oss << std::cin.rdbuf();
+    data = oss->str();
+    oss.reset();
+  }
+
+  if (data.back() != '\n') {
+    data.push_back('\n');
+  }
+
+  const CatalogRelationSchema &relation = output_destination_->getRelation();
+  std::vector<Tuple> tuples;
+  std::vector<TypedValue> row_tuple;
+  bool is_faulty;
+
+  const char *row_ptr = data.c_str();
+  const char *end_ptr = row_ptr + data.length();
+
+  while (row_ptr < end_ptr) {
+    if (*row_ptr == '\r' || *row_ptr == '\n') {
+      // Skip empty lines.
+      ++row_ptr;
+    } else {
+      row_tuple = parseRow(&row_ptr, relation, &is_faulty);
+      if (is_faulty) {
+        // Skip faulty rows
+        LOG(INFO) << "Faulty row found. Hence switching to next row.";
+      } else {
+        // Convert vector returned to tuple only when a valid row is encountered.
+        tuples.emplace_back(Tuple(std::move(row_tuple)));
+      }
+    }
+  }
+
+  // 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));
+      }
+      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;
+  }
+
+  // Bulk insert the tuples.
+  output_destination_->bulkInsertTuples(&column_vectors);
+}
+
 std::vector<TypedValue> TextScanWorkOrder::parseRow(const char **row_ptr,
                                                     const CatalogRelationSchema &relation,
                                                     bool *is_faulty) const {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index 01c559c..30462d7 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -225,6 +225,8 @@ class TextScanWorkOrder : public WorkOrder {
   void execute() override;
 
  private:
+  void executeInputStream();
+
   /**
    * @brief Extract a field string starting at \p *field_ptr. This method also
    *        expands escape sequences if \p process_escape_sequences_ is true.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 73f1983..9d58107 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -393,8 +393,7 @@ bool AggregationOperationState::ProtoIsValid(
       }
 
       const S::CollisionFreeVectorInfo &proto_collision_free_vector_info = proto.collision_free_vector_info();
-      if (!proto_collision_free_vector_info.IsInitialized() ||
-          proto_collision_free_vector_info.state_offsets_size() != group_by_expressions_size) {
+      if (!proto_collision_free_vector_info.IsInitialized()) {
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/utility/ExecutionDAGVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/ExecutionDAGVisualizer.cpp b/utility/ExecutionDAGVisualizer.cpp
index 8059ef3..f009a72 100644
--- a/utility/ExecutionDAGVisualizer.cpp
+++ b/utility/ExecutionDAGVisualizer.cpp
@@ -234,9 +234,6 @@ void ExecutionDAGVisualizer::bindProfilingStats(
         std::max(time_end[relop_index], workorder_end_time);
     time_elapsed[relop_index] += (workorder_end_time - workorder_start_time);
 
-    if (workorders_count.find(relop_index) == workorders_count.end()) {
-      workorders_count[relop_index] = 0;
-    }
     ++workorders_count[relop_index];
     if (mean_time_per_workorder.find(relop_index) ==
         mean_time_per_workorder.end()) {
@@ -292,8 +289,7 @@ void ExecutionDAGVisualizer::bindProfilingStats(
       node_info.labels.emplace_back(
           "effective concurrency: " + FormatDigits(concurrency, 2));
 
-      DCHECK(workorders_count.find(node_index) != workorders_count.end());
-      const std::size_t workorders_count_for_node = workorders_count.at(node_index);
+      const std::size_t workorders_count_for_node = workorders_count[node_index];
       if (workorders_count_for_node > 0) {
         mean_time_per_workorder[node_index] =
             mean_time_per_workorder[node_index] /


[12/27] incubator-quickstep git commit: Removed the virtual function call in InvokeOnAnyValueAccessor.

Posted by ji...@apache.org.
Removed the virtual function call in InvokeOnAnyValueAccessor.


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

Branch: refs/heads/trace
Commit: 79710ca6c6b75410bf2c26b4646acbfc5d554d7c
Parents: 696a783
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Fri Oct 6 14:34:21 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Mon Oct 9 11:37:16 2017 -0500

----------------------------------------------------------------------
 storage/SplitRowStoreTupleStorageSubBlock.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/79710ca6/storage/SplitRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreTupleStorageSubBlock.cpp b/storage/SplitRowStoreTupleStorageSubBlock.cpp
index 5060208..9f5a839 100644
--- a/storage/SplitRowStoreTupleStorageSubBlock.cpp
+++ b/storage/SplitRowStoreTupleStorageSubBlock.cpp
@@ -343,7 +343,7 @@ tuple_id SplitRowStoreTupleStorageSubBlock::bulkInsertPartialTuplesImpl(
 
   InvokeOnAnyValueAccessor(
       accessor,
-      [&](auto *accessor) -> void {  // NOLINT(build/c++11
+      [&](auto *accessor) -> void {  // NOLINT(build/c++11)
     BitVector<true> tuple_null_bitmap(tuple_slot, num_null_attrs_);
     const std::size_t nullmap_size = BitVector<true>::BytesNeeded(num_null_attrs_);
 
@@ -410,7 +410,7 @@ tuple_id SplitRowStoreTupleStorageSubBlock::bulkInsertPartialTuplesImpl(
           max_num_tuples_to_insert += additional_tuples_insert;
         }
       }
-    } while (fill_to_capacity && !accessor->iterationFinishedVirtual() &&
+    } while (fill_to_capacity && !accessor->iterationFinished() &&
              num_tuples_inserted < max_num_tuples_to_insert);
   });
 


[11/27] incubator-quickstep git commit: Fixed a flaky case in Catalog test.

Posted by ji...@apache.org.
Fixed a flaky case in Catalog test.


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

Branch: refs/heads/trace
Commit: 696a783e5d8adb3ca62ca9044a8d7ccd89f67b3a
Parents: e496cb5
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Sun Oct 8 14:20:41 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Mon Oct 9 11:34:50 2017 -0500

----------------------------------------------------------------------
 catalog/tests/Catalog_unittest.cpp | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/696a783e/catalog/tests/Catalog_unittest.cpp
----------------------------------------------------------------------
diff --git a/catalog/tests/Catalog_unittest.cpp b/catalog/tests/Catalog_unittest.cpp
index f761026..7b121fa 100644
--- a/catalog/tests/Catalog_unittest.cpp
+++ b/catalog/tests/Catalog_unittest.cpp
@@ -552,13 +552,15 @@ TEST_F(CatalogTest, CatalogIndexTest) {
   IndexSubBlockDescription index_description;
   index_description.set_sub_block_type(IndexSubBlockDescription::CSB_TREE);
   index_description.add_indexed_attribute_ids(rel->getAttributeByName("attr_idx1")->getID());
+  IndexSubBlockDescription index_description_copy;
+  index_description_copy.MergeFrom(index_description);
 
   EXPECT_TRUE(rel->addIndex("idx1", std::move(index_description)));
   EXPECT_TRUE(rel->hasIndexWithName("idx1"));
   // Adding an index with duplicate name should return false.
-  EXPECT_FALSE(rel->addIndex("idx1", std::move(index_description)));
+  EXPECT_FALSE(rel->addIndex("idx1", IndexSubBlockDescription()));
   // Adding an index of same type with different name on the same attribute should return false.
-  EXPECT_FALSE(rel->addIndex("idx2", std::move(index_description)));
+  EXPECT_FALSE(rel->addIndex("idx2", std::move(index_description_copy)));
 
   index_description.Clear();
   index_description.set_sub_block_type(IndexSubBlockDescription::CSB_TREE);


[07/27] incubator-quickstep git commit: Created a class to track execution statistics

Posted by ji...@apache.org.
Created a class to track execution statistics

- Stats are maintained for active operators in the query.


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

Branch: refs/heads/trace
Commit: 1b2698d2225bfab59fb675da5f92a2285dd5650c
Parents: 9cbb930
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Wed Sep 27 14:55:16 2017 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Fri Sep 29 12:10:52 2017 -0500

----------------------------------------------------------------------
 query_execution/CMakeLists.txt     |   5 +
 query_execution/ExecutionStats.hpp | 211 ++++++++++++++++++++++++++++++++
 2 files changed, 216 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b2698d2/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 9394c00..8f797f7 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -29,6 +29,7 @@ if (ENABLE_DISTRIBUTED)
   add_library(quickstep_queryexecution_BlockLocator BlockLocator.cpp BlockLocator.hpp)
   add_library(quickstep_queryexecution_BlockLocatorUtil BlockLocatorUtil.cpp BlockLocatorUtil.hpp)
 endif(ENABLE_DISTRIBUTED)
+add_library(quickstep_queryexecution_ExecutionStats ../empty_src.cpp ExecutionStats.hpp)
 add_library(quickstep_queryexecution_ForemanBase ../empty_src.cpp ForemanBase.hpp)
 if (ENABLE_DISTRIBUTED)
   add_library(quickstep_queryexecution_ForemanDistributed ForemanDistributed.cpp ForemanDistributed.hpp)
@@ -123,6 +124,9 @@ if (ENABLE_DISTRIBUTED)
                         tmb
                         ${GFLAGS_LIB_NAME})
 endif(ENABLE_DISTRIBUTED)
+target_link_libraries(quickstep_queryexecution_ExecutionStats
+                      glog
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryexecution_ForemanSingleNode
                       glog
                       quickstep_queryexecution_AdmitRequestMessage
@@ -363,6 +367,7 @@ target_link_libraries(quickstep_queryexecution_WorkerSelectionPolicy
 add_library(quickstep_queryexecution ../empty_src.cpp QueryExecutionModule.hpp)
 target_link_libraries(quickstep_queryexecution
                       quickstep_queryexecution_AdmitRequestMessage
+                      quickstep_queryexecution_ExecutionStats
                       quickstep_queryexecution_ForemanBase
                       quickstep_queryexecution_ForemanSingleNode
                       quickstep_queryexecution_PolicyEnforcerBase

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b2698d2/query_execution/ExecutionStats.hpp
----------------------------------------------------------------------
diff --git a/query_execution/ExecutionStats.hpp b/query_execution/ExecutionStats.hpp
new file mode 100644
index 0000000..8d19651
--- /dev/null
+++ b/query_execution/ExecutionStats.hpp
@@ -0,0 +1,211 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_EXECUTION_STATS_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_EXECUTION_STATS_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <deque>
+#include <memory>
+#include <unordered_map>
+#include <utility>
+
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup QueryExecution
+ *  @{
+ */
+
+/**
+ * @brief Record the execution stats of a query.
+ *
+ * @note The time is measured in microseconds.
+ **/
+class ExecutionStats {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param max_entries The maximum number of entries we remember for each
+   *        operator.
+   **/
+  explicit ExecutionStats(const std::size_t max_entries)
+      : max_entries_(max_entries) {}
+
+  /**
+   * @brief Get the number of active operators in stats.
+   **/
+  std::size_t getNumActiveOperators() const {
+    return active_operators_.size();
+  }
+
+  /**
+   * @brief Check if there are stats present for at least one active operator.
+   **/
+  inline bool hasStats() const {
+    for (auto it = active_operators_.begin(); it != active_operators_.end(); ++it) {
+      if (it->second->hasStatsForOperator()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * @brief Get the current stats for the query.
+   *
+   * @return A pair - 1st element is total time, 2nd element is total number of
+   *         WorkOrders for the whole query.
+   **/
+  std::pair<std::uint64_t, std::uint64_t> getCurrentStatsForQuery() const {
+    std::uint64_t total_time = 0;
+    std::uint64_t num_workorders = 0;
+    for (auto it = active_operators_.begin(); it != active_operators_.end(); ++it) {
+      auto operator_stats = getCurrentStatsForOperator(it->first);
+      total_time += operator_stats.first;
+      num_workorders += operator_stats.second;
+    }
+    return std::make_pair(total_time, num_workorders);
+  }
+
+  /**
+   * @brief Get the average work order time for the query.
+   */
+  double getAverageWorkOrderTimeForQuery() const {
+    auto result = getCurrentStatsForQuery();
+    if (result.second != 0) {
+      return result.first / static_cast<double>(result.second);
+    }
+    return 0.0;
+  }
+
+  /**
+   * @brief Get the current stats for the given operator.
+   * @param operator_id The ID of the operator.
+   * @return A pair - 1st element is total time, 2nd element is total number of
+   *         WorkOrders for the operator.
+   */
+  std::pair<std::uint64_t, std::uint64_t> getCurrentStatsForOperator(const std::size_t operator_id) const {
+    if (hasOperator(operator_id)) {
+      DCHECK(active_operators_.at(operator_id) != nullptr);
+      return active_operators_.at(operator_id)->getStats();
+    }
+    return std::make_pair(0, 0);
+  }
+
+  double getAverageWorkOrderTimeForOperator(const std::size_t operator_id) const {
+    auto result = getCurrentStatsForOperator(operator_id);
+    if (result.second != 0) {
+      return result.first / static_cast<double>(result.second);
+    }
+    return 0.0;
+  }
+
+  /**
+   * @brief Add a new entry to stats.
+   *
+   * @param value The value to be added.
+   * @param operator_index The operator index which the value belongs to.
+   **/
+  void addEntry(const std::size_t value, const std::size_t operator_index) {
+    if (!hasOperator(operator_index)) {
+      // This is the first entry for the given operator.
+      // Create the OperatorStats object for this operator.
+      active_operators_[operator_index] =
+          std::make_unique<OperatorStats>(max_entries_);
+    }
+    active_operators_[operator_index]->addEntry(value);
+  }
+
+  /**
+   * @brief Remove the operator with given index. This should be called only
+   *        when the given operator finishes its execution.
+   **/
+  void removeOperator(const std::size_t operator_index) {
+    DCHECK(hasOperator(operator_index));
+    active_operators_.erase(operator_index);
+  }
+
+ private:
+  /**
+   * @brief Stats for an operator within the query.
+   *
+   * @note We remember only the last N entries for the operator.
+   **/
+  class OperatorStats {
+   public:
+    /**
+     * @brief Constructor.
+     *
+     * @param max_entries The maximum number of entries we remember. Typically
+     *        these are the last N (=max_entries) entries.
+     **/
+    explicit OperatorStats(const std::size_t max_entries) : max_entries_(max_entries) {}
+
+    inline std::pair<std::uint64_t, std::size_t> getStats() const {
+      return std::make_pair(std::accumulate(times_.begin(), times_.end(), 0),
+                            times_.size());
+    }
+
+    inline void addEntry(const std::uint64_t time_value) {
+      if (times_.size() == max_entries_) {
+        times_.pop_front();
+      }
+      times_.push_back(time_value);
+      DCHECK_LE(times_.size(), max_entries_);
+    }
+
+    inline bool hasStatsForOperator() const {
+      return !times_.empty();
+    }
+
+   private:
+    const std::size_t max_entries_;
+    // Times are measured in microseconds.
+    std::deque<std::uint64_t> times_;
+
+    DISALLOW_COPY_AND_ASSIGN(OperatorStats);
+  };
+
+  /**
+   * @brief Check if the operator with given index is present in the stats.
+   **/
+  inline bool hasOperator(const std::size_t operator_index) const {
+    return active_operators_.find(operator_index) != active_operators_.end();
+  }
+
+  const std::size_t max_entries_;
+
+  std::unordered_map<std::size_t, std::unique_ptr<OperatorStats>>
+      active_operators_;
+
+  DISALLOW_COPY_AND_ASSIGN(ExecutionStats);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_EXECUTION_STATS_HPP_


[27/27] incubator-quickstep git commit: Quickstep for GRAIL.

Posted by ji...@apache.org.
Quickstep for GRAIL.


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

Branch: refs/heads/trace
Commit: aec7623aa7782c7dc9aa0633b53c9da85634a4a0
Parents: 0fe838d
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sat Sep 2 23:06:37 2017 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Fri Nov 17 13:24:22 2017 -0600

----------------------------------------------------------------------
 CMakeLists.txt                                  |    4 +-
 catalog/CatalogRelationStatistics.hpp           |    9 +
 cli/CommandExecutor.cpp                         |  249 +-
 cli/Constants.hpp                               |    2 +
 cli/IOInterface.hpp                             |    9 +-
 cli/LocalIO.hpp                                 |    4 +
 cli/NetworkCli.proto                            |    1 +
 cli/NetworkCliClient.hpp                        |    7 +-
 cli/NetworkCliClientMain.cpp                    |   30 +-
 cli/NetworkIO.cpp                               |    5 +
 cli/NetworkIO.hpp                               |   33 +-
 cli/QuickstepCli.cpp                            |    3 +-
 cli/quickstep/NetworkCliOuterClass.java         | 1388 +++++++
 cli/tests/NetworkIO_unittest.cpp                |    2 +-
 .../aggregation/AggregateFunctionMax.cpp        |    4 +-
 .../aggregation/AggregateFunctionMin.cpp        |    4 +-
 parser/SqlLexer.lpp                             |    1 +
 parser/SqlParser.ypp                            |   15 +-
 parser/preprocessed/SqlLexer_gen.cpp            |  861 ++---
 parser/preprocessed/SqlLexer_gen.hpp            |    2 +-
 parser/preprocessed/SqlParser_gen.cpp           | 3436 +++++++++---------
 parser/preprocessed/SqlParser_gen.hpp           |   49 +-
 query_optimizer/ExecutionGenerator.cpp          |   12 +-
 .../cost_model/StarSchemaSimpleCostModel.cpp    |    4 +-
 query_optimizer/resolver/Resolver.cpp           |    9 +-
 relational_operators/TableExportOperator.cpp    |    2 +
 relational_operators/TextScanOperator.cpp       |  104 +-
 relational_operators/TextScanOperator.hpp       |    2 +
 storage/AggregationOperationState.cpp           |    3 +-
 utility/ExecutionDAGVisualizer.cpp              |    6 +-
 30 files changed, 4009 insertions(+), 2251 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 071f8fc..cb8e9f4 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -291,7 +291,7 @@ else()
   # Treat warnings as errors, 'cause we hardcore.
   CHECK_CXX_COMPILER_FLAG("-Werror" COMPILER_HAS_WERROR)
   if (COMPILER_HAS_WERROR)
-    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror")
+    # set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror")
   endif()
 
   # Clang reports such warning when using Protoc 3.0 beta.
@@ -585,7 +585,7 @@ if(USE_TCMALLOC)
   CHECK_CXX_COMPILER_FLAG("-Wno-return-type-c-linkage"
                           COMPILER_HAS_WNO_RETURN_TYPE_C_LINKAGE)
   if (COMPILER_HAS_WNO_RETURN_TYPE_C_LINKAGE)
-    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-return-type-c-linkage")
+    # set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-return-type-c-linkage")
   endif()
 endif()
 

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/IOInterface.hpp
----------------------------------------------------------------------
diff --git a/cli/IOInterface.hpp b/cli/IOInterface.hpp
index dc0d5b2..ec125f9 100644
--- a/cli/IOInterface.hpp
+++ b/cli/IOInterface.hpp
@@ -40,14 +40,19 @@ class IOHandle {
   virtual ~IOHandle() {}
 
   /**
+   * @return A file handle for standard input.
+   */
+  virtual FILE* in() = 0;
+
+  /**
    * @return A file handle for standard output.
    */
-  virtual FILE *out() = 0;
+  virtual FILE* out() = 0;
 
   /**
    * @return A file handle for error output.
    */
-  virtual FILE *err() = 0;
+  virtual FILE* err() = 0;
 
   virtual std::string getCommand() const = 0;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/LocalIO.hpp
----------------------------------------------------------------------
diff --git a/cli/LocalIO.hpp b/cli/LocalIO.hpp
index 4dc3b3f..42501d2 100644
--- a/cli/LocalIO.hpp
+++ b/cli/LocalIO.hpp
@@ -46,6 +46,10 @@ class LocalIOHandle final : public IOHandle {
 
   ~LocalIOHandle() override {}
 
+  FILE *in() override {
+    return stdin;
+  }
+
   FILE *out() override {
     return stdout;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/NetworkCli.proto
----------------------------------------------------------------------
diff --git a/cli/NetworkCli.proto b/cli/NetworkCli.proto
index e065c7c..1558d01 100644
--- a/cli/NetworkCli.proto
+++ b/cli/NetworkCli.proto
@@ -25,6 +25,7 @@ service NetworkCli {
 
 message QueryRequest {
   string query = 1;
+  string data = 2;
 }
 
 message QueryResponse {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/NetworkCliClient.hpp
----------------------------------------------------------------------
diff --git a/cli/NetworkCliClient.hpp b/cli/NetworkCliClient.hpp
index 7affeff..5c11cb2 100644
--- a/cli/NetworkCliClient.hpp
+++ b/cli/NetworkCliClient.hpp
@@ -53,9 +53,10 @@ class NetworkCliClient {
    * @param user_query A SQL statement or command to be executed on the server.
    * @return The text of the server's response.
    */
-  std::string Query(const std::string &user_query) {
+  std::string Query(const std::string &user_query, const std::string &data) {
     QueryRequest request;
     request.set_query(user_query);
+    request.set_data(data);
     QueryResponse response;
 
     Status status = SendQuery(request, &response);
@@ -63,8 +64,8 @@ class NetworkCliClient {
     if (status.ok()) {
       return HandleQueryResponse(response);
     } else {
-      LOG(WARNING) << "RPC call failed with code " << status.error_code()
-                   << " and message: " << status.error_message();
+      std::cout << "RPC call failed with code " << status.error_code()
+                << " and message: " << status.error_message() << "\n";
       return "RPC failed";
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/NetworkCliClientMain.cpp
----------------------------------------------------------------------
diff --git a/cli/NetworkCliClientMain.cpp b/cli/NetworkCliClientMain.cpp
index c55819b..164d81e 100644
--- a/cli/NetworkCliClientMain.cpp
+++ b/cli/NetworkCliClientMain.cpp
@@ -38,22 +38,42 @@ int main(int argc, char **argv) {
   gflags::ParseCommandLineFlags(&argc, &argv, true);
   grpc_init();
 
+  grpc::ChannelArguments grpc_args;
+  grpc_args.SetMaxSendMessageSize(quickstep::FLAGS_cli_network_max_message_length);
+  grpc_args.SetMaxReceiveMessageSize(quickstep::FLAGS_cli_network_max_message_length);
+
   // Attempts to send a single query retrieved from stdin to the Quickstep Server.
   NetworkCliClient qs_client(
-    grpc::CreateChannel(quickstep::NetworkIO::GetAddress(),
-                        grpc::InsecureChannelCredentials()));
+    grpc::CreateCustomChannel(quickstep::NetworkIO::GetAddress(),
+                              grpc::InsecureChannelCredentials(),
+                              grpc_args));
 
   // Read stdin until EOF, then we use a Line reader to divide query into parts.
   std::cin >> std::noskipws;
   std::istream_iterator<char> it(std::cin), end;
-  std::string user_queries(it, end);
+  std::string input(it, end);
+
+  if (input.empty()) {
+    return 0;
+  }
+
+  // Temporary hack for transmitting data in a separate channel ...
+  std::string queries;
+  std::string data;
+  if (input.front() == 0) {
+    const std::size_t r = input.find(static_cast<char>(0), 1);
+    queries = input.substr(1, r - 1);
+    data = input.substr(r + 1);
+  } else {
+    queries = std::move(input);
+  }
 
   LineReaderBuffered linereader;
-  linereader.setBuffer(user_queries);
+  linereader.setBuffer(queries);
   while (!linereader.bufferEmpty()) {
     std::string query = linereader.getNextCommand();
     if (!query.empty()) {
-      std::cout << qs_client.Query(query) << std::endl;
+      std::cout << qs_client.Query(query, data) << std::endl;
     }
   }
   return 0;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/NetworkIO.cpp
----------------------------------------------------------------------
diff --git a/cli/NetworkIO.cpp b/cli/NetworkIO.cpp
index f0acc65..cbc815e 100644
--- a/cli/NetworkIO.cpp
+++ b/cli/NetworkIO.cpp
@@ -47,10 +47,15 @@ DEFINE_string(cli_network_ip, "0.0.0.0",
                 "if the cli is set to use the network mode (--mode=network). Defaults to "
                 "the address of localhost.");
 
+DEFINE_int32(cli_network_max_message_length, 1073741824,
+             "The maximum message length transferred through grpc");
+
 NetworkIO::NetworkIO() {
   grpc::ServerBuilder builder;
   builder.AddListeningPort(GetAddress(), grpc::InsecureServerCredentials());
   builder.RegisterService(&service_);
+  builder.SetMaxReceiveMessageSize(FLAGS_cli_network_max_message_length);
+  builder.SetMaxSendMessageSize(FLAGS_cli_network_max_message_length);
   server_ = builder.BuildAndStart();
   LOG(INFO) << "Listening on " << GetAddress();
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/NetworkIO.hpp
----------------------------------------------------------------------
diff --git a/cli/NetworkIO.hpp b/cli/NetworkIO.hpp
index 3693f88..e55c206 100644
--- a/cli/NetworkIO.hpp
+++ b/cli/NetworkIO.hpp
@@ -47,6 +47,7 @@ using grpc::Status;
 namespace quickstep {
 DECLARE_int32(cli_network_port);
 DECLARE_string(cli_network_ip);
+DECLARE_int32(cli_network_max_message_length);
 namespace networkio_internal {
 
 /**
@@ -80,8 +81,9 @@ class RequestState {
    * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    */
   void waitForResponse() {
-    while (!response_ready_)
+    while (!response_ready_) {
       condition_->await();
+    }
   }
 
   /**
@@ -95,16 +97,16 @@ class RequestState {
   }
 
   /**
-   * @return The producer's query for Quickstep to process.
+   * @return The producer's request for Quickstep to process.
    */
-  std::string getRequest() const {
-    return request_.query();
+  const QueryRequest& getRequest() const {
+    return request_;
   }
 
   /**
    * @return The response message from Quickstep.
    */
-  QueryResponse getResponse() const {
+  const QueryResponse& getResponse() const {
     DCHECK(response_ready_);
     return response_message_;
   }
@@ -212,12 +214,21 @@ class NetworkCliServiceImpl final : public NetworkCli::Service {
 class NetworkIOHandle final : public IOHandle {
  public:
   explicit NetworkIOHandle(RequestState* state)
-      : request_state_(state) {}
+      : request_state_(state) {
+    const std::string &data = request_state_->getRequest().data();
+    if (!data.empty()) {
+      std::fwrite(data.c_str(), 1, data.length(), in_stream_.file());
+    }
+  }
 
   ~NetworkIOHandle() override {
-      // All the commands from the last network interaction have completed, return our response.
-      // This signals to the producer thread that the interaction is complete.
-      request_state_->responseReady(out_stream_.str(), err_stream_.str());
+    // All the commands from the last network interaction have completed, return our response.
+    // This signals to the producer thread that the interaction is complete.
+    request_state_->responseReady(out_stream_.str(), err_stream_.str());
+  }
+
+  FILE* in() override {
+    return in_stream_.file();
   }
 
   FILE* out() override {
@@ -229,11 +240,11 @@ class NetworkIOHandle final : public IOHandle {
   }
 
   std::string getCommand() const override {
-    return request_state_->getRequest();
+    return request_state_->getRequest().query();
   }
 
  private:
-  MemStream out_stream_, err_stream_;
+  MemStream in_stream_, out_stream_, err_stream_;
   RequestState *request_state_;
 
   DISALLOW_COPY_AND_ASSIGN(NetworkIOHandle);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index e8ca56c..5db5dfc 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -305,8 +305,9 @@ int main(int argc, char* argv[]) {
   for (;;) {
     string *command_string = new string();
     std::unique_ptr<quickstep::IOHandle> io_handle(io->getNextIOHandle());
+    ScopedReassignment<FILE*> reassign_stdin(&stdin, io_handle->in());
     ScopedReassignment<FILE*> reassign_stdout(&stdout, io_handle->out());
-    ScopedReassignment<FILE*> reassign_stderr(&stderr, io_handle->err());
+//    ScopedReassignment<FILE*> reassign_stderr(&stderr, io_handle->err());
 
     *command_string = io_handle->getCommand();
     LOG(INFO) << "Command received: " << *command_string;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/quickstep/NetworkCliOuterClass.java
----------------------------------------------------------------------
diff --git a/cli/quickstep/NetworkCliOuterClass.java b/cli/quickstep/NetworkCliOuterClass.java
new file mode 100644
index 0000000..6f4e833
--- /dev/null
+++ b/cli/quickstep/NetworkCliOuterClass.java
@@ -0,0 +1,1388 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NetworkCli.proto
+
+package quickstep;
+
+public final class NetworkCliOuterClass {
+  private NetworkCliOuterClass() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistryLite registry) {
+  }
+
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+    registerAllExtensions(
+        (com.google.protobuf.ExtensionRegistryLite) registry);
+  }
+  public interface QueryRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:quickstep.QueryRequest)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>string query = 1;</code>
+     */
+    java.lang.String getQuery();
+    /**
+     * <code>string query = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getQueryBytes();
+
+    /**
+     * <code>string data = 2;</code>
+     */
+    java.lang.String getData();
+    /**
+     * <code>string data = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getDataBytes();
+  }
+  /**
+   * Protobuf type {@code quickstep.QueryRequest}
+   */
+  public  static final class QueryRequest extends
+      com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:quickstep.QueryRequest)
+      QueryRequestOrBuilder {
+  private static final long serialVersionUID = 0L;
+    // Use QueryRequest.newBuilder() to construct.
+    private QueryRequest(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private QueryRequest() {
+      query_ = "";
+      data_ = "";
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private QueryRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownFieldProto3(
+                  input, unknownFields, extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              java.lang.String s = input.readStringRequireUtf8();
+
+              query_ = s;
+              break;
+            }
+            case 18: {
+              java.lang.String s = input.readStringRequireUtf8();
+
+              data_ = s;
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              quickstep.NetworkCliOuterClass.QueryRequest.class, quickstep.NetworkCliOuterClass.QueryRequest.Builder.class);
+    }
+
+    public static final int QUERY_FIELD_NUMBER = 1;
+    private volatile java.lang.Object query_;
+    /**
+     * <code>string query = 1;</code>
+     */
+    public java.lang.String getQuery() {
+      java.lang.Object ref = query_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        query_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>string query = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getQueryBytes() {
+      java.lang.Object ref = query_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        query_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int DATA_FIELD_NUMBER = 2;
+    private volatile java.lang.Object data_;
+    /**
+     * <code>string data = 2;</code>
+     */
+    public java.lang.String getData() {
+      java.lang.Object ref = data_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        data_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>string data = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getDataBytes() {
+      java.lang.Object ref = data_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        data_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (!getQueryBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, query_);
+      }
+      if (!getDataBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, data_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (!getQueryBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, query_);
+      }
+      if (!getDataBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, data_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof quickstep.NetworkCliOuterClass.QueryRequest)) {
+        return super.equals(obj);
+      }
+      quickstep.NetworkCliOuterClass.QueryRequest other = (quickstep.NetworkCliOuterClass.QueryRequest) obj;
+
+      boolean result = true;
+      result = result && getQuery()
+          .equals(other.getQuery());
+      result = result && getData()
+          .equals(other.getData());
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      hash = (37 * hash) + QUERY_FIELD_NUMBER;
+      hash = (53 * hash) + getQuery().hashCode();
+      hash = (37 * hash) + DATA_FIELD_NUMBER;
+      hash = (53 * hash) + getData().hashCode();
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(
+        java.nio.ByteBuffer data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(
+        java.nio.ByteBuffer data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(quickstep.NetworkCliOuterClass.QueryRequest prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code quickstep.QueryRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:quickstep.QueryRequest)
+        quickstep.NetworkCliOuterClass.QueryRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                quickstep.NetworkCliOuterClass.QueryRequest.class, quickstep.NetworkCliOuterClass.QueryRequest.Builder.class);
+      }
+
+      // Construct using quickstep.NetworkCliOuterClass.QueryRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        query_ = "";
+
+        data_ = "";
+
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryRequest_descriptor;
+      }
+
+      public quickstep.NetworkCliOuterClass.QueryRequest getDefaultInstanceForType() {
+        return quickstep.NetworkCliOuterClass.QueryRequest.getDefaultInstance();
+      }
+
+      public quickstep.NetworkCliOuterClass.QueryRequest build() {
+        quickstep.NetworkCliOuterClass.QueryRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public quickstep.NetworkCliOuterClass.QueryRequest buildPartial() {
+        quickstep.NetworkCliOuterClass.QueryRequest result = new quickstep.NetworkCliOuterClass.QueryRequest(this);
+        result.query_ = query_;
+        result.data_ = data_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          java.lang.Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, java.lang.Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          java.lang.Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof quickstep.NetworkCliOuterClass.QueryRequest) {
+          return mergeFrom((quickstep.NetworkCliOuterClass.QueryRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(quickstep.NetworkCliOuterClass.QueryRequest other) {
+        if (other == quickstep.NetworkCliOuterClass.QueryRequest.getDefaultInstance()) return this;
+        if (!other.getQuery().isEmpty()) {
+          query_ = other.query_;
+          onChanged();
+        }
+        if (!other.getData().isEmpty()) {
+          data_ = other.data_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        quickstep.NetworkCliOuterClass.QueryRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (quickstep.NetworkCliOuterClass.QueryRequest) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+
+      private java.lang.Object query_ = "";
+      /**
+       * <code>string query = 1;</code>
+       */
+      public java.lang.String getQuery() {
+        java.lang.Object ref = query_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          query_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>string query = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getQueryBytes() {
+        java.lang.Object ref = query_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          query_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>string query = 1;</code>
+       */
+      public Builder setQuery(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        query_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string query = 1;</code>
+       */
+      public Builder clearQuery() {
+        
+        query_ = getDefaultInstance().getQuery();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string query = 1;</code>
+       */
+      public Builder setQueryBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        query_ = value;
+        onChanged();
+        return this;
+      }
+
+      private java.lang.Object data_ = "";
+      /**
+       * <code>string data = 2;</code>
+       */
+      public java.lang.String getData() {
+        java.lang.Object ref = data_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          data_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>string data = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getDataBytes() {
+        java.lang.Object ref = data_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          data_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>string data = 2;</code>
+       */
+      public Builder setData(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        data_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string data = 2;</code>
+       */
+      public Builder clearData() {
+        
+        data_ = getDefaultInstance().getData();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string data = 2;</code>
+       */
+      public Builder setDataBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        data_ = value;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFieldsProto3(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:quickstep.QueryRequest)
+    }
+
+    // @@protoc_insertion_point(class_scope:quickstep.QueryRequest)
+    private static final quickstep.NetworkCliOuterClass.QueryRequest DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new quickstep.NetworkCliOuterClass.QueryRequest();
+    }
+
+    public static quickstep.NetworkCliOuterClass.QueryRequest getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<QueryRequest>
+        PARSER = new com.google.protobuf.AbstractParser<QueryRequest>() {
+      public QueryRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+          return new QueryRequest(input, extensionRegistry);
+      }
+    };
+
+    public static com.google.protobuf.Parser<QueryRequest> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<QueryRequest> getParserForType() {
+      return PARSER;
+    }
+
+    public quickstep.NetworkCliOuterClass.QueryRequest getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface QueryResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:quickstep.QueryResponse)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>string query_result = 1;</code>
+     */
+    java.lang.String getQueryResult();
+    /**
+     * <code>string query_result = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getQueryResultBytes();
+
+    /**
+     * <code>string error_result = 2;</code>
+     */
+    java.lang.String getErrorResult();
+    /**
+     * <code>string error_result = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getErrorResultBytes();
+  }
+  /**
+   * Protobuf type {@code quickstep.QueryResponse}
+   */
+  public  static final class QueryResponse extends
+      com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:quickstep.QueryResponse)
+      QueryResponseOrBuilder {
+  private static final long serialVersionUID = 0L;
+    // Use QueryResponse.newBuilder() to construct.
+    private QueryResponse(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private QueryResponse() {
+      queryResult_ = "";
+      errorResult_ = "";
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private QueryResponse(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownFieldProto3(
+                  input, unknownFields, extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              java.lang.String s = input.readStringRequireUtf8();
+
+              queryResult_ = s;
+              break;
+            }
+            case 18: {
+              java.lang.String s = input.readStringRequireUtf8();
+
+              errorResult_ = s;
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              quickstep.NetworkCliOuterClass.QueryResponse.class, quickstep.NetworkCliOuterClass.QueryResponse.Builder.class);
+    }
+
+    public static final int QUERY_RESULT_FIELD_NUMBER = 1;
+    private volatile java.lang.Object queryResult_;
+    /**
+     * <code>string query_result = 1;</code>
+     */
+    public java.lang.String getQueryResult() {
+      java.lang.Object ref = queryResult_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        queryResult_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>string query_result = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getQueryResultBytes() {
+      java.lang.Object ref = queryResult_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        queryResult_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int ERROR_RESULT_FIELD_NUMBER = 2;
+    private volatile java.lang.Object errorResult_;
+    /**
+     * <code>string error_result = 2;</code>
+     */
+    public java.lang.String getErrorResult() {
+      java.lang.Object ref = errorResult_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        errorResult_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>string error_result = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getErrorResultBytes() {
+      java.lang.Object ref = errorResult_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        errorResult_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (!getQueryResultBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, queryResult_);
+      }
+      if (!getErrorResultBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, errorResult_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (!getQueryResultBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, queryResult_);
+      }
+      if (!getErrorResultBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, errorResult_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof quickstep.NetworkCliOuterClass.QueryResponse)) {
+        return super.equals(obj);
+      }
+      quickstep.NetworkCliOuterClass.QueryResponse other = (quickstep.NetworkCliOuterClass.QueryResponse) obj;
+
+      boolean result = true;
+      result = result && getQueryResult()
+          .equals(other.getQueryResult());
+      result = result && getErrorResult()
+          .equals(other.getErrorResult());
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      hash = (37 * hash) + QUERY_RESULT_FIELD_NUMBER;
+      hash = (53 * hash) + getQueryResult().hashCode();
+      hash = (37 * hash) + ERROR_RESULT_FIELD_NUMBER;
+      hash = (53 * hash) + getErrorResult().hashCode();
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(
+        java.nio.ByteBuffer data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(
+        java.nio.ByteBuffer data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static quickstep.NetworkCliOuterClass.QueryResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(quickstep.NetworkCliOuterClass.QueryResponse prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code quickstep.QueryResponse}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:quickstep.QueryResponse)
+        quickstep.NetworkCliOuterClass.QueryResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                quickstep.NetworkCliOuterClass.QueryResponse.class, quickstep.NetworkCliOuterClass.QueryResponse.Builder.class);
+      }
+
+      // Construct using quickstep.NetworkCliOuterClass.QueryResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        queryResult_ = "";
+
+        errorResult_ = "";
+
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return quickstep.NetworkCliOuterClass.internal_static_quickstep_QueryResponse_descriptor;
+      }
+
+      public quickstep.NetworkCliOuterClass.QueryResponse getDefaultInstanceForType() {
+        return quickstep.NetworkCliOuterClass.QueryResponse.getDefaultInstance();
+      }
+
+      public quickstep.NetworkCliOuterClass.QueryResponse build() {
+        quickstep.NetworkCliOuterClass.QueryResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public quickstep.NetworkCliOuterClass.QueryResponse buildPartial() {
+        quickstep.NetworkCliOuterClass.QueryResponse result = new quickstep.NetworkCliOuterClass.QueryResponse(this);
+        result.queryResult_ = queryResult_;
+        result.errorResult_ = errorResult_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          java.lang.Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, java.lang.Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          java.lang.Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof quickstep.NetworkCliOuterClass.QueryResponse) {
+          return mergeFrom((quickstep.NetworkCliOuterClass.QueryResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(quickstep.NetworkCliOuterClass.QueryResponse other) {
+        if (other == quickstep.NetworkCliOuterClass.QueryResponse.getDefaultInstance()) return this;
+        if (!other.getQueryResult().isEmpty()) {
+          queryResult_ = other.queryResult_;
+          onChanged();
+        }
+        if (!other.getErrorResult().isEmpty()) {
+          errorResult_ = other.errorResult_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        quickstep.NetworkCliOuterClass.QueryResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (quickstep.NetworkCliOuterClass.QueryResponse) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+
+      private java.lang.Object queryResult_ = "";
+      /**
+       * <code>string query_result = 1;</code>
+       */
+      public java.lang.String getQueryResult() {
+        java.lang.Object ref = queryResult_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          queryResult_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>string query_result = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getQueryResultBytes() {
+        java.lang.Object ref = queryResult_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          queryResult_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>string query_result = 1;</code>
+       */
+      public Builder setQueryResult(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        queryResult_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string query_result = 1;</code>
+       */
+      public Builder clearQueryResult() {
+        
+        queryResult_ = getDefaultInstance().getQueryResult();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string query_result = 1;</code>
+       */
+      public Builder setQueryResultBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        queryResult_ = value;
+        onChanged();
+        return this;
+      }
+
+      private java.lang.Object errorResult_ = "";
+      /**
+       * <code>string error_result = 2;</code>
+       */
+      public java.lang.String getErrorResult() {
+        java.lang.Object ref = errorResult_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          errorResult_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>string error_result = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getErrorResultBytes() {
+        java.lang.Object ref = errorResult_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          errorResult_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>string error_result = 2;</code>
+       */
+      public Builder setErrorResult(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        errorResult_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string error_result = 2;</code>
+       */
+      public Builder clearErrorResult() {
+        
+        errorResult_ = getDefaultInstance().getErrorResult();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string error_result = 2;</code>
+       */
+      public Builder setErrorResultBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        errorResult_ = value;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFieldsProto3(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:quickstep.QueryResponse)
+    }
+
+    // @@protoc_insertion_point(class_scope:quickstep.QueryResponse)
+    private static final quickstep.NetworkCliOuterClass.QueryResponse DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new quickstep.NetworkCliOuterClass.QueryResponse();
+    }
+
+    public static quickstep.NetworkCliOuterClass.QueryResponse getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<QueryResponse>
+        PARSER = new com.google.protobuf.AbstractParser<QueryResponse>() {
+      public QueryResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+          return new QueryResponse(input, extensionRegistry);
+      }
+    };
+
+    public static com.google.protobuf.Parser<QueryResponse> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<QueryResponse> getParserForType() {
+      return PARSER;
+    }
+
+    public quickstep.NetworkCliOuterClass.QueryResponse getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_quickstep_QueryRequest_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_quickstep_QueryRequest_fieldAccessorTable;
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_quickstep_QueryResponse_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_quickstep_QueryResponse_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static  com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\020NetworkCli.proto\022\tquickstep\"+\n\014QueryRe" +
+      "quest\022\r\n\005query\030\001 \001(\t\022\014\n\004data\030\002 \001(\t\";\n\rQu" +
+      "eryResponse\022\024\n\014query_result\030\001 \001(\t\022\024\n\014err" +
+      "or_result\030\002 \001(\t2N\n\nNetworkCli\022@\n\tSendQue" +
+      "ry\022\027.quickstep.QueryRequest\032\030.quickstep." +
+      "QueryResponse\"\000b\006proto3"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public com.google.protobuf.ExtensionRegistry assignDescriptors(
+              com.google.protobuf.Descriptors.FileDescriptor root) {
+            descriptor = root;
+            return null;
+          }
+        };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+    internal_static_quickstep_QueryRequest_descriptor =
+      getDescriptor().getMessageTypes().get(0);
+    internal_static_quickstep_QueryRequest_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_quickstep_QueryRequest_descriptor,
+        new java.lang.String[] { "Query", "Data", });
+    internal_static_quickstep_QueryResponse_descriptor =
+      getDescriptor().getMessageTypes().get(1);
+    internal_static_quickstep_QueryResponse_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_quickstep_QueryResponse_descriptor,
+        new java.lang.String[] { "QueryResult", "ErrorResult", });
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/cli/tests/NetworkIO_unittest.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/NetworkIO_unittest.cpp b/cli/tests/NetworkIO_unittest.cpp
index 9dbd63e..9c4bfc4 100644
--- a/cli/tests/NetworkIO_unittest.cpp
+++ b/cli/tests/NetworkIO_unittest.cpp
@@ -68,7 +68,7 @@ class TestNetworkIO {
     CHECK(current_request_ == nullptr);
     current_request_ = service_.waitForRequest();
     EXPECT_EQ(current_request_->getCanceled(), false);
-    return current_request_->getRequest();
+    return current_request_->getRequest().query();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/expressions/aggregation/AggregateFunctionMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionMax.cpp b/expressions/aggregation/AggregateFunctionMax.cpp
index cc04bf4..6ceacc9 100644
--- a/expressions/aggregation/AggregateFunctionMax.cpp
+++ b/expressions/aggregation/AggregateFunctionMax.cpp
@@ -50,7 +50,9 @@ const Type* AggregateFunctionMax::resultTypeForArgumentTypes(
     return nullptr;
   }
 
-  return &(argument_types.front()->getNullableVersion());
+  // FIXME(jianqiao): The result type can be nullable when it is NOT a group-by
+  // aggregation.
+  return argument_types.front();
 }
 
 AggregationHandle* AggregateFunctionMax::createHandle(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/expressions/aggregation/AggregateFunctionMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionMin.cpp b/expressions/aggregation/AggregateFunctionMin.cpp
index 9fa93de..d88169f 100644
--- a/expressions/aggregation/AggregateFunctionMin.cpp
+++ b/expressions/aggregation/AggregateFunctionMin.cpp
@@ -50,7 +50,9 @@ const Type* AggregateFunctionMin::resultTypeForArgumentTypes(
     return nullptr;
   }
 
-  return &(argument_types.front()->getNullableVersion());
+  // FIXME(jianqiao): The result type can be nullable when it is NOT a group-by
+  // aggregation.
+  return argument_types.front();
 }
 
 AggregationHandle* AggregateFunctionMin::createHandle(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index d818d0b..d6b57bf 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -270,6 +270,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "sma"              return TOKEN_SMA;
   "smallint"         return TOKEN_SMALLINT;
   "stderr"           return TOKEN_STDERR;
+  "stdin"            return TOKEN_STDIN;
   "stdout"           return TOKEN_STDOUT;
   "substring"        return TOKEN_SUBSTRING;
   "table"            return TOKEN_TABLE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aec7623a/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index ba69b3d..1a12512 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -341,6 +341,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_SMA;
 %token TOKEN_SMALLINT;
 %token TOKEN_STDERR;
+%token TOKEN_STDIN;
 %token TOKEN_STDOUT;
 %token TOKEN_SUBSTRING;
 %token TOKEN_TABLE;
@@ -371,6 +372,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
   datetime_unit
   index_type
   partition_type
+  copy_from_target
   copy_to_target
 
 %type <boolean_value_>
@@ -1117,10 +1119,10 @@ insert_statement:
   ;
 
 copy_statement:
-  TOKEN_COPY any_name TOKEN_FROM TOKEN_STRING_SINGLE_QUOTED opt_copy_params {
+  TOKEN_COPY any_name copy_from_target opt_copy_params {
     $$ = new quickstep::ParseStatementCopy(@1.first_line, @1.first_column,
                                            quickstep::ParseStatementCopy::kFrom,
-                                           $2, $4, $5);
+                                           $2, $3, $4);
   }
   | TOKEN_COPY any_name copy_to_target opt_copy_params {
     $$ = new quickstep::ParseStatementCopy(@1.first_line, @1.first_column,
@@ -1136,6 +1138,15 @@ copy_statement:
                                            $3, $1, $4, $5);
   };
 
+copy_from_target:
+  TOKEN_FROM TOKEN_STRING_SINGLE_QUOTED {
+    $$ = new quickstep::ParseString($2->line_number(), $2->column_number(), "@" + $2->value());
+    delete $2;
+  }
+  | TOKEN_FROM TOKEN_STDIN {
+    $$ = new quickstep::ParseString(@2.first_line, @2.first_column, "$stdin");
+  };
+
 copy_to_target:
   TOKEN_TO TOKEN_STRING_SINGLE_QUOTED {
     $$ = new quickstep::ParseString($2->line_number(), $2->column_number(), "@" + $2->value());


[09/27] incubator-quickstep git commit: Fixed the root path check in the cyclic_dependency.py.

Posted by ji...@apache.org.
Fixed the root path check in the cyclic_dependency.py.


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

Branch: refs/heads/trace
Commit: 0898a77beac5ccb9c97675148bcf853a5490e279
Parents: 7fb7a77
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Mon Oct 2 20:47:44 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Tue Oct 3 10:48:45 2017 -0500

----------------------------------------------------------------------
 cyclic_dependency.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0898a77b/cyclic_dependency.py
----------------------------------------------------------------------
diff --git a/cyclic_dependency.py b/cyclic_dependency.py
index adb5fc7..4914e0b 100755
--- a/cyclic_dependency.py
+++ b/cyclic_dependency.py
@@ -1,4 +1,4 @@
-#!/usr/bin/env python
+#!/usr/bin/env python2
 
 # Script to do analyze the dependencies in Quickstep particularly cycles in the
 # dependency graph. This script can be used to find:
@@ -169,7 +169,7 @@ def find_path(G, nodes_list, nodes_map, source, target):
         print('No path.')
 
 def main():
-    if not os.getcwd().endswith("quickstep"):
+    if not os.path.isfile("cyclic_dependency.py"):
         print("WARNING: you don't appear to be running in the root quickstep "
               "source directory. Don't blame me if something goes wrong.")
     qs_module_dirs = []


[19/27] incubator-quickstep git commit: Fixed gcc compiler warning.

Posted by ji...@apache.org.
Fixed gcc compiler warning.


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

Branch: refs/heads/trace
Commit: b5130feabc571010924cacd1fa6287f4518be8d6
Parents: e79b520
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Thu Oct 12 23:01:22 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Thu Oct 12 23:01:22 2017 -0500

----------------------------------------------------------------------
 storage/Flags.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b5130fea/storage/Flags.cpp
----------------------------------------------------------------------
diff --git a/storage/Flags.cpp b/storage/Flags.cpp
index 4672f81..7312cd3 100644
--- a/storage/Flags.cpp
+++ b/storage/Flags.cpp
@@ -40,7 +40,7 @@ static bool ValidateHdfsNameNodePort(const char *flagname,
   }
 }
 DEFINE_int32(hdfs_namenode_port, 9000, "Port of HDFS namenode.");
-static const bool hdfs_namenode_port_dummy
+static volatile const bool hdfs_namenode_port_dummy
     = gflags::RegisterFlagValidator(&FLAGS_hdfs_namenode_port, &ValidateHdfsNameNodePort);
 
 static bool ValidateHdfsNumReplications(const char *flagname,
@@ -55,7 +55,7 @@ static bool ValidateHdfsNumReplications(const char *flagname,
   }
 }
 DEFINE_int32(hdfs_num_replications, 1, "Number of HDFS file replications.");
-static const bool hdfs_num_replications_dummy
+static volatile const bool hdfs_num_replications_dummy
     = gflags::RegisterFlagValidator(&FLAGS_hdfs_num_replications, &ValidateHdfsNumReplications);
 #endif
 


[23/27] incubator-quickstep git commit: Support Multiple Tuple Inserts

Posted by ji...@apache.org.
Support Multiple Tuple Inserts

Update Fetch


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

Branch: refs/heads/trace
Commit: 0fe838dfeac901ff03b8334da46b7b9f364447e3
Parents: 79bfcf9
Author: Robert Claus <ro...@gmail.com>
Authored: Tue Oct 24 18:08:57 2017 -0500
Committer: Robert Claus <ro...@gmail.com>
Committed: Wed Oct 25 13:24:02 2017 -0500

----------------------------------------------------------------------
 parser/ParseStatement.hpp                |   22 +-
 parser/SqlParser.ypp                     |   18 +-
 parser/preprocessed/SqlParser_gen.cpp    | 2646 +++++++++++++------------
 parser/preprocessed/SqlParser_gen.hpp    |    3 +-
 query_optimizer/ExecutionGenerator.cpp   |  110 +-
 query_optimizer/logical/InsertTuple.cpp  |    6 +-
 query_optimizer/logical/InsertTuple.hpp  |   10 +-
 query_optimizer/physical/InsertTuple.cpp |    6 +-
 query_optimizer/physical/InsertTuple.hpp |    8 +-
 query_optimizer/resolver/Resolver.cpp    |  122 +-
 10 files changed, 1503 insertions(+), 1448 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/parser/ParseStatement.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseStatement.hpp b/parser/ParseStatement.hpp
index cee7221..456bdc2 100644
--- a/parser/ParseStatement.hpp
+++ b/parser/ParseStatement.hpp
@@ -653,9 +653,9 @@ class ParseStatementInsertTuple : public ParseStatementInsert {
   ParseStatementInsertTuple(const int line_number,
                             const int column_number,
                             const ParseString *relation_name,
-                            PtrList<ParseScalarLiteral> *literal_values)
+                            PtrList<PtrList<ParseScalarLiteral>> *literal_values_list)
       : ParseStatementInsert(line_number, column_number, relation_name),
-        literal_values_(literal_values) {
+        literal_values_(literal_values_list) {
   }
 
   ~ParseStatementInsertTuple() override {
@@ -666,11 +666,11 @@ class ParseStatementInsertTuple : public ParseStatementInsert {
   }
 
   /**
-   * @brief Get the parsed literal attribute values to insert.
+   * @brief Get the list of list of parsed literal attribute values to insert.
    *
-   * @return The list of literal values to insert.
+   * @return The list of lists of literal values to insert.
    **/
-  const PtrList<ParseScalarLiteral>& getLiteralValues() const {
+  const PtrList<PtrList<ParseScalarLiteral>>& getLiteralValues() const {
     return *literal_values_;
   }
 
@@ -685,15 +685,17 @@ class ParseStatementInsertTuple : public ParseStatementInsert {
     inline_field_names->push_back("relation_name");
     inline_field_values->push_back(relation_name()->value());
 
-    container_child_field_names->push_back("tuple");
-    container_child_fields->emplace_back();
-    for (const ParseScalarLiteral& literal_value : *literal_values_) {
-      container_child_fields->back().push_back(&literal_value);
+    for (const PtrList<ParseScalarLiteral>& literal_values_single_tuple : *literal_values_) {
+      container_child_field_names->push_back("tuple");
+      container_child_fields->emplace_back();
+      for (const ParseScalarLiteral& literal_value : literal_values_single_tuple) {
+        container_child_fields->back().push_back(&literal_value);
+      }
     }
   }
 
  private:
-  std::unique_ptr<PtrList<ParseScalarLiteral> > literal_values_;
+  std::unique_ptr<PtrList<PtrList<ParseScalarLiteral>>> literal_values_;
 
   DISALLOW_COPY_AND_ASSIGN(ParseStatementInsertTuple);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index 8fbcdd7..ba69b3d 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -128,6 +128,7 @@ typedef void* yyscan_t;
   quickstep::NumericParseLiteralValue *numeric_literal_value_;
   quickstep::ParseLiteralValue *literal_value_;
   quickstep::PtrList<quickstep::ParseScalarLiteral> *literal_value_list_;
+  quickstep::PtrList<quickstep::PtrList<quickstep::ParseScalarLiteral>> *literal_value_list_multiple_;
 
   quickstep::ParseExpression *expression_;
 
@@ -387,6 +388,9 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %type <literal_value_list_>
   literal_value_commalist
 
+%type <literal_value_list_multiple_>
+  literal_value_commalist_multiple
+
 %type <expression_>
   expression_base
   unary_expression
@@ -1101,8 +1105,8 @@ insert_statement:
     NotSupported(&@4, yyscanner, "list of column names in INSERT statement");
     YYERROR;
   }
-  | TOKEN_INSERT TOKEN_INTO any_name TOKEN_VALUES '(' literal_value_commalist ')' {
-    $$ = new quickstep::ParseStatementInsertTuple(@1.first_line, @1.first_column, $3, $6);
+  | TOKEN_INSERT TOKEN_INTO any_name TOKEN_VALUES literal_value_commalist_multiple {
+    $$ = new quickstep::ParseStatementInsertTuple(@1.first_line, @1.first_column, $3, $5);
   }
   | TOKEN_INSERT TOKEN_INTO any_name select_query {
     $$ = new quickstep::ParseStatementInsertSelection(@1.first_line, @2.first_column, $3, $4, nullptr);
@@ -1921,6 +1925,16 @@ literal_value_commalist:
     $$->push_back(new quickstep::ParseScalarLiteral($3));
   };
 
+literal_value_commalist_multiple:
+  '(' literal_value_commalist ')' {
+    $$ = new quickstep::PtrList<quickstep::PtrList<quickstep::ParseScalarLiteral>>();
+    $$->push_back($2);
+  }
+  | literal_value_commalist_multiple ',' '(' literal_value_commalist ')' {
+    $$ = $1;
+    $$->push_back($4);
+  };
+
 attribute_ref:
   any_name {
     $$ = new quickstep::ParseAttribute(@1.first_line, @1.first_column, $1);


[13/27] incubator-quickstep git commit: Moved InsertDestination::getTouchedBlocks as a private method.

Posted by ji...@apache.org.
Moved InsertDestination::getTouchedBlocks as a private method.


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

Branch: refs/heads/trace
Commit: a61b03dcc2446a5bd276a0117f493ba56d8a3ffe
Parents: 79710ca
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Thu Oct 5 16:41:38 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Mon Oct 9 12:00:08 2017 -0500

----------------------------------------------------------------------
 .../tests/HashJoinOperator_unittest.cpp         | 24 +++----
 storage/InsertDestination.cpp                   | 27 +++-----
 storage/InsertDestination.hpp                   | 70 ++++++++++++--------
 3 files changed, 65 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b03dc/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 1fc84fc..cfd4314 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -485,7 +485,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   InsertDestination *insert_destination = query_context_->getInsertDestination(prober->getInsertDestinationID());
   DCHECK(insert_destination);
 
-  const std::vector<block_id> &result_blocks = insert_destination->getTouchedBlocks();
+  const std::vector<block_id> result_blocks = insert_destination->getTouchedBlocks();
   for (std::size_t bid = 0; bid < result_blocks.size(); ++bid) {
     BlockReference result_block = storage_manager_->getBlock(result_blocks[bid],
                                                              insert_destination->getRelation());
@@ -640,7 +640,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   InsertDestination *insert_destination = query_context_->getInsertDestination(prober->getInsertDestinationID());
   DCHECK(insert_destination);
 
-  const std::vector<block_id> &result_blocks = insert_destination->getTouchedBlocks();
+  const std::vector<block_id> result_blocks = insert_destination->getTouchedBlocks();
   for (std::size_t bid = 0; bid < result_blocks.size(); ++bid) {
     BlockReference result_block = storage_manager_->getBlock(result_blocks[bid],
                                                              insert_destination->getRelation());
@@ -795,7 +795,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   InsertDestination *insert_destination = query_context_->getInsertDestination(prober->getInsertDestinationID());
   DCHECK(insert_destination);
 
-  const std::vector<block_id> &result_blocks = insert_destination->getTouchedBlocks();
+  const std::vector<block_id> result_blocks = insert_destination->getTouchedBlocks();
   for (std::size_t bid = 0; bid < result_blocks.size(); ++bid) {
     BlockReference result_block = storage_manager_->getBlock(result_blocks[bid],
                                                              insert_destination->getRelation());
@@ -944,7 +944,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   InsertDestination *insert_destination = query_context_->getInsertDestination(prober->getInsertDestinationID());
   DCHECK(insert_destination);
 
-  const std::vector<block_id> &result_blocks = insert_destination->getTouchedBlocks();
+  const std::vector<block_id> result_blocks = insert_destination->getTouchedBlocks();
   for (std::size_t bid = 0; bid < result_blocks.size(); ++bid) {
     BlockReference result_block = storage_manager_->getBlock(result_blocks[bid],
                                                              insert_destination->getRelation());
@@ -1123,7 +1123,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   InsertDestination *insert_destination = query_context_->getInsertDestination(prober->getInsertDestinationID());
   DCHECK(insert_destination);
 
-  const std::vector<block_id> &result_blocks = insert_destination->getTouchedBlocks();
+  const std::vector<block_id> result_blocks = insert_destination->getTouchedBlocks();
   for (std::size_t bid = 0; bid < result_blocks.size(); ++bid) {
     BlockReference result_block = storage_manager_->getBlock(result_blocks[bid],
                                                              insert_destination->getRelation());
@@ -1313,7 +1313,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   InsertDestination *insert_destination = query_context_->getInsertDestination(prober->getInsertDestinationID());
   DCHECK(insert_destination);
 
-  const std::vector<block_id> &result_blocks = insert_destination->getTouchedBlocks();
+  const std::vector<block_id> result_blocks = insert_destination->getTouchedBlocks();
   for (std::size_t bid = 0; bid < result_blocks.size(); ++bid) {
     BlockReference result_block = storage_manager_->getBlock(result_blocks[bid],
                                                              insert_destination->getRelation());
@@ -1370,7 +1370,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
 }
 
 // Hash join tests with single attribute partitions.
-TEST_P(HashJoinOperatorTest, SinlgeAttributePartitionedLongKeyHashJoinTest) {
+TEST_P(HashJoinOperatorTest, SingleAttributePartitionedLongKeyHashJoinTest) {
   insertTuplesWithSingleAttributePartitions();
 
   // Setup the hash table proto in the query context proto.
@@ -1480,7 +1480,7 @@ TEST_P(HashJoinOperatorTest, SinlgeAttributePartitionedLongKeyHashJoinTest) {
   InsertDestination *insert_destination = query_context_->getInsertDestination(prober->getInsertDestinationID());
   DCHECK(insert_destination);
 
-  const std::vector<block_id> &result_blocks = insert_destination->getTouchedBlocks();
+  const std::vector<block_id> result_blocks = insert_destination->getTouchedBlocks();
   for (std::size_t bid = 0; bid < result_blocks.size(); ++bid) {
     BlockReference result_block = storage_manager_->getBlock(result_blocks[bid],
                                                              insert_destination->getRelation());
@@ -1514,7 +1514,7 @@ TEST_P(HashJoinOperatorTest, SinlgeAttributePartitionedLongKeyHashJoinTest) {
   db_->dropRelationById(output_relation_id);
 }
 
-TEST_P(HashJoinOperatorTest, SinlgeAttributePartitionedCompositeKeyHashJoinTest) {
+TEST_P(HashJoinOperatorTest, SingleAttributePartitionedCompositeKeyHashJoinTest) {
   insertTuplesWithSingleAttributePartitions();
 
   // Setup the hash table proto in the query context proto.
@@ -1629,7 +1629,7 @@ TEST_P(HashJoinOperatorTest, SinlgeAttributePartitionedCompositeKeyHashJoinTest)
   InsertDestination *insert_destination = query_context_->getInsertDestination(prober->getInsertDestinationID());
   DCHECK(insert_destination);
 
-  const std::vector<block_id> &result_blocks = insert_destination->getTouchedBlocks();
+  const std::vector<block_id> result_blocks = insert_destination->getTouchedBlocks();
   for (std::size_t bid = 0; bid < result_blocks.size(); ++bid) {
     BlockReference result_block = storage_manager_->getBlock(result_blocks[bid],
                                                              insert_destination->getRelation());
@@ -1685,7 +1685,7 @@ TEST_P(HashJoinOperatorTest, SinlgeAttributePartitionedCompositeKeyHashJoinTest)
   db_->dropRelationById(output_relation_id);
 }
 
-TEST_P(HashJoinOperatorTest, SinlgeAttributePartitionedCompositeKeyHashJoinWithResidualPredicateTest) {
+TEST_P(HashJoinOperatorTest, SingleAttributePartitionedCompositeKeyHashJoinWithResidualPredicateTest) {
   insertTuplesWithSingleAttributePartitions();
 
   // Setup the hash table proto in the query context proto.
@@ -1810,7 +1810,7 @@ TEST_P(HashJoinOperatorTest, SinlgeAttributePartitionedCompositeKeyHashJoinWithR
   InsertDestination *insert_destination = query_context_->getInsertDestination(prober->getInsertDestinationID());
   DCHECK(insert_destination);
 
-  const std::vector<block_id> &result_blocks = insert_destination->getTouchedBlocks();
+  const std::vector<block_id> result_blocks = insert_destination->getTouchedBlocks();
   for (std::size_t bid = 0; bid < result_blocks.size(); ++bid) {
     BlockReference result_block = storage_manager_->getBlock(result_blocks[bid],
                                                              insert_destination->getRelation());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b03dc/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index ff39c55..8821019 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -475,7 +475,7 @@ void BlockPoolInsertDestination::returnBlock(MutableBlockReference &&block, cons
   sendBlockFilledMessage(block->getID());
 }
 
-const std::vector<block_id>& BlockPoolInsertDestination::getTouchedBlocksInternal() {
+std::vector<block_id> BlockPoolInsertDestination::getTouchedBlocksInternal() {
   for (std::vector<MutableBlockReference>::size_type i = 0; i < available_block_refs_.size(); ++i) {
     done_block_ids_.push_back(available_block_refs_[i]->getID());
   }
@@ -546,27 +546,22 @@ MutableBlockReference PartitionAwareInsertDestination::createNewBlockInPartition
   return storage_manager_->getBlockMutable(new_id, relation_);
 }
 
-const std::vector<block_id>& PartitionAwareInsertDestination::getTouchedBlocksInternal() {
+std::vector<block_id> PartitionAwareInsertDestination::getTouchedBlocksInternal() {
+  std::vector<block_id> all_partitions_done_block_ids;
   // Iterate through each partition and get all the touched blocks.
   for (std::size_t part_id = 0;
        part_id < partition_scheme_header_->getNumPartitions();
        ++part_id) {
-    done_block_ids_[part_id] = getTouchedBlocksInternalInPartition(part_id);
-    all_partitions_done_block_ids_.insert(
-        all_partitions_done_block_ids_.end(), done_block_ids_[part_id].begin(), done_block_ids_[part_id].end());
-    done_block_ids_[part_id].clear();
-  }
-  return all_partitions_done_block_ids_;
-}
+    for (std::size_t i = 0; i < available_block_refs_[part_id].size(); ++i) {
+      done_block_ids_[part_id].push_back(available_block_refs_[part_id][i]->getID());
+    }
+    available_block_refs_[part_id].clear();
 
-const std::vector<block_id>& PartitionAwareInsertDestination::getTouchedBlocksInternalInPartition(
-    partition_id part_id) {
-  for (std::vector<MutableBlockReference>::size_type i = 0; i < available_block_refs_[part_id].size(); ++i) {
-    done_block_ids_[part_id].push_back(available_block_refs_[part_id][i]->getID());
+    all_partitions_done_block_ids.insert(
+        all_partitions_done_block_ids.end(), done_block_ids_[part_id].begin(), done_block_ids_[part_id].end());
+    done_block_ids_[part_id].clear();
   }
-  available_block_refs_[part_id].clear();
-
-  return done_block_ids_[part_id];
+  return all_partitions_done_block_ids;
 }
 
 PartitionSchemeHeader::PartitionAttributeIds PartitionAwareInsertDestination::getPartitioningAttributes() const {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b03dc/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index c3d5641..a0a7bc2 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -58,6 +58,8 @@ class StorageManager;
 
 namespace merge_run_operator {
 class RunCreator;
+class RunMergerTest;
+class RunTest;
 }  // namespace merge_run_operator
 
 namespace serialization { class InsertDestination; }
@@ -173,21 +175,6 @@ class InsertDestination : public InsertDestinationInterface {
   }
 
   /**
-   * @brief Get the set of blocks that were used by clients of this
-   *        InsertDestination for insertion.
-   * @warning Should only be called AFTER this InsertDestination will no longer
-   *          be used, and all blocks have been returned to it via
-   *          returnBlock().
-   *
-   * @return A reference to a vector of block_ids of blocks that were used for
-   *         insertion.
-   **/
-  const std::vector<block_id>& getTouchedBlocks() {
-    SpinMutexLock lock(mutex_);
-    return getTouchedBlocksInternal();
-  }
-
-  /**
    * @brief Get the set of blocks that were partially filled by clients of this
    *        InsertDestination for insertion.
    * @warning Should only be called AFTER this InsertDestination will no longer
@@ -233,8 +220,6 @@ class InsertDestination : public InsertDestinationInterface {
   // this without holding the mutex.
   virtual MutableBlockReference createNewBlock() = 0;
 
-  virtual const std::vector<block_id>& getTouchedBlocksInternal() = 0;
-
   /**
    * @brief When a StorageBlock becomes full, pipeline the block id to the
    *        scheduler.
@@ -311,12 +296,44 @@ class InsertDestination : public InsertDestinationInterface {
   SpinMutex mutex_;
 
  private:
+  /**
+   * @brief Get the set of blocks that were used by clients of this
+   *        InsertDestination for insertion.
+   * @warning Should only be called AFTER this InsertDestination will no longer
+   *          be used, and all blocks have been returned to it via
+   *          returnBlock().
+   *
+   * @return A vector of block_ids of blocks that were used for insertion.
+   **/
+  std::vector<block_id> getTouchedBlocks() {
+    SpinMutexLock lock(mutex_);
+    return getTouchedBlocksInternal();
+  }
+
+  virtual std::vector<block_id> getTouchedBlocksInternal() = 0;
+
   // TODO(shoban): Workaround to support sort. Sort needs finegrained control of
   // blocks being used to insert, since inserting in an arbitrary block could
   // lead to unsorted results. InsertDestination API changed while sort was
   // being implemented.
   friend class merge_run_operator::RunCreator;
 
+  // NOTE(zuyu): Access getTouchedBlocks.
+  friend class AggregationOperatorTest;
+  friend class merge_run_operator::RunTest;
+  friend class merge_run_operator::RunMergerTest;
+
+  FRIEND_TEST(HashJoinOperatorTest, LongKeyHashJoinTest);
+  FRIEND_TEST(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest);
+  FRIEND_TEST(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest);
+  FRIEND_TEST(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest);
+  FRIEND_TEST(HashJoinOperatorTest, CompositeKeyHashJoinTest);
+  FRIEND_TEST(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest);
+  FRIEND_TEST(HashJoinOperatorTest, SingleAttributePartitionedLongKeyHashJoinTest);
+  FRIEND_TEST(HashJoinOperatorTest, SingleAttributePartitionedCompositeKeyHashJoinTest);
+  FRIEND_TEST(HashJoinOperatorTest,
+              SingleAttributePartitionedCompositeKeyHashJoinWithResidualPredicateTest);
+
   DISALLOW_COPY_AND_ASSIGN(InsertDestination);
 };
 
@@ -358,15 +375,15 @@ class AlwaysCreateBlockInsertDestination : public InsertDestination {
 
   MutableBlockReference createNewBlock() override;
 
-  const std::vector<block_id>& getTouchedBlocksInternal() override {
-    return returned_block_ids_;
-  }
-
   void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks,
                                 std::vector<partition_id> *part_ids) override {
   }
 
  private:
+  std::vector<block_id> getTouchedBlocksInternal() override {
+    return returned_block_ids_;
+  }
+
   std::vector<block_id> returned_block_ids_;
 
   DISALLOW_COPY_AND_ASSIGN(AlwaysCreateBlockInsertDestination);
@@ -454,11 +471,11 @@ class BlockPoolInsertDestination : public InsertDestination {
   void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks,
                                 std::vector<partition_id> *part_ids) override;
 
-  const std::vector<block_id>& getTouchedBlocksInternal() override;
-
   MutableBlockReference createNewBlock() override;
 
  private:
+  std::vector<block_id> getTouchedBlocksInternal() override;
+
   FRIEND_TEST(QueryManagerTest, TwoNodesDAGPartiallyFilledBlocksTest);
 
   // A vector of references to blocks which are loaded in memory.
@@ -585,10 +602,9 @@ class PartitionAwareInsertDestination : public InsertDestination {
   MutableBlockReference createNewBlock() override;
   MutableBlockReference createNewBlockInPartition(const partition_id part_id);
 
-  const std::vector<block_id>& getTouchedBlocksInternal() override;
-  const std::vector<block_id>& getTouchedBlocksInternalInPartition(partition_id part_id);
-
  private:
+  std::vector<block_id> getTouchedBlocksInternal() override;
+
   /**
    * @brief Get the set of blocks that were partially filled by clients of this
    *        InsertDestination for insertion.
@@ -652,8 +668,6 @@ class PartitionAwareInsertDestination : public InsertDestination {
   std::vector< std::vector<block_id> > available_block_ids_;
   // A vector of done block ids for each partition.
   std::vector< std::vector<block_id> > done_block_ids_;
-  // Done block ids across all partitions.
-  std::vector<block_id> all_partitions_done_block_ids_;
   // Mutex for locking each partition separately.
   SpinMutex *mutexes_for_partition_;
 


[22/27] incubator-quickstep git commit: Support Multiple Tuple Inserts

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0fe838df/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 72c61dd..9b77875 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -341,6 +341,7 @@ union YYSTYPE
   quickstep::NumericParseLiteralValue *numeric_literal_value_;
   quickstep::ParseLiteralValue *literal_value_;
   quickstep::PtrList<quickstep::ParseScalarLiteral> *literal_value_list_;
+  quickstep::PtrList<quickstep::PtrList<quickstep::ParseScalarLiteral>> *literal_value_list_multiple_;
 
   quickstep::ParseExpression *expression_;
 
@@ -431,7 +432,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 435 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 436 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -460,13 +461,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 222 "../SqlParser.ypp" /* yacc.c:358  */
+#line 223 "../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 470 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 471 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -710,16 +711,16 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  50
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1391
+#define YYLAST   1327
 
 /* YYNTOKENS -- Number of terminals.  */
 #define YYNTOKENS  148
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  110
+#define YYNNTS  111
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  298
+#define YYNRULES  300
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  550
+#define YYNSTATES  555
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
@@ -779,36 +780,37 @@ static const yytype_uint8 yytranslate[] =
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   640,   640,   644,   648,   652,   656,   659,   666,   669,
-     672,   675,   678,   681,   684,   687,   690,   693,   699,   705,
-     712,   718,   725,   734,   739,   748,   753,   758,   762,   768,
-     773,   776,   779,   784,   787,   790,   793,   796,   799,   802,
-     805,   808,   811,   823,   826,   829,   847,   867,   870,   873,
-     878,   883,   889,   895,   904,   908,   914,   917,   922,   927,
-     932,   939,   946,   950,   956,   959,   964,   967,   972,   975,
-     980,   983,  1002,  1005,  1010,  1014,  1020,  1023,  1026,  1029,
-    1034,  1037,  1040,  1047,  1052,  1063,  1068,  1073,  1077,  1081,
-    1087,  1090,  1096,  1104,  1107,  1110,  1116,  1121,  1126,  1130,
-    1136,  1140,  1143,  1148,  1151,  1156,  1161,  1166,  1170,  1176,
-    1185,  1188,  1193,  1196,  1215,  1220,  1224,  1230,  1236,  1245,
-    1250,  1258,  1264,  1270,  1273,  1276,  1281,  1284,  1289,  1293,
-    1299,  1302,  1305,  1310,  1315,  1320,  1323,  1326,  1331,  1334,
-    1337,  1340,  1343,  1346,  1349,  1352,  1357,  1360,  1365,  1369,
-    1373,  1376,  1380,  1383,  1388,  1391,  1396,  1399,  1404,  1408,
-    1414,  1417,  1422,  1425,  1430,  1433,  1438,  1441,  1460,  1463,
-    1468,  1472,  1478,  1484,  1489,  1492,  1497,  1500,  1505,  1508,
-    1513,  1516,  1521,  1522,  1525,  1530,  1531,  1534,  1539,  1543,
-    1549,  1556,  1559,  1562,  1567,  1570,  1573,  1579,  1582,  1587,
-    1592,  1601,  1606,  1615,  1620,  1623,  1628,  1631,  1636,  1642,
-    1648,  1651,  1654,  1657,  1660,  1663,  1669,  1678,  1681,  1686,
-    1689,  1694,  1697,  1702,  1705,  1708,  1711,  1715,  1719,  1722,
-    1725,  1728,  1731,  1736,  1740,  1744,  1747,  1752,  1757,  1761,
-    1767,  1770,  1775,  1779,  1785,  1790,  1794,  1800,  1805,  1808,
-    1813,  1817,  1823,  1826,  1829,  1832,  1844,  1848,  1867,  1880,
-    1895,  1898,  1901,  1904,  1907,  1910,  1915,  1919,  1925,  1928,
-    1933,  1937,  1944,  1947,  1950,  1953,  1956,  1959,  1962,  1965,
-    1968,  1971,  1976,  1987,  1990,  1995,  1998,  2001,  2007,  2011,
-    2017,  2020,  2028,  2031,  2034,  2037,  2043,  2048,  2053
+       0,   644,   644,   648,   652,   656,   660,   663,   670,   673,
+     676,   679,   682,   685,   688,   691,   694,   697,   703,   709,
+     716,   722,   729,   738,   743,   752,   757,   762,   766,   772,
+     777,   780,   783,   788,   791,   794,   797,   800,   803,   806,
+     809,   812,   815,   827,   830,   833,   851,   871,   874,   877,
+     882,   887,   893,   899,   908,   912,   918,   921,   926,   931,
+     936,   943,   950,   954,   960,   963,   968,   971,   976,   979,
+     984,   987,  1006,  1009,  1014,  1018,  1024,  1027,  1030,  1033,
+    1038,  1041,  1044,  1051,  1056,  1067,  1072,  1077,  1081,  1085,
+    1091,  1094,  1100,  1108,  1111,  1114,  1120,  1125,  1130,  1134,
+    1140,  1144,  1147,  1152,  1155,  1160,  1165,  1170,  1174,  1180,
+    1189,  1192,  1197,  1200,  1219,  1224,  1228,  1234,  1240,  1249,
+    1254,  1262,  1268,  1274,  1277,  1280,  1285,  1288,  1293,  1297,
+    1303,  1306,  1309,  1314,  1319,  1324,  1327,  1330,  1335,  1338,
+    1341,  1344,  1347,  1350,  1353,  1356,  1361,  1364,  1369,  1373,
+    1377,  1380,  1384,  1387,  1392,  1395,  1400,  1403,  1408,  1412,
+    1418,  1421,  1426,  1429,  1434,  1437,  1442,  1445,  1464,  1467,
+    1472,  1476,  1482,  1488,  1493,  1496,  1501,  1504,  1509,  1512,
+    1517,  1520,  1525,  1526,  1529,  1534,  1535,  1538,  1543,  1547,
+    1553,  1560,  1563,  1566,  1571,  1574,  1577,  1583,  1586,  1591,
+    1596,  1605,  1610,  1619,  1624,  1627,  1632,  1635,  1640,  1646,
+    1652,  1655,  1658,  1661,  1664,  1667,  1673,  1682,  1685,  1690,
+    1693,  1698,  1701,  1706,  1709,  1712,  1715,  1719,  1723,  1726,
+    1729,  1732,  1735,  1740,  1744,  1748,  1751,  1756,  1761,  1765,
+    1771,  1774,  1779,  1783,  1789,  1794,  1798,  1804,  1809,  1812,
+    1817,  1821,  1827,  1830,  1833,  1836,  1848,  1852,  1871,  1884,
+    1899,  1902,  1905,  1908,  1911,  1914,  1919,  1923,  1929,  1933,
+    1939,  1942,  1947,  1951,  1958,  1961,  1964,  1967,  1970,  1973,
+    1976,  1979,  1982,  1985,  1990,  2001,  2004,  2009,  2012,  2015,
+    2021,  2025,  2031,  2034,  2042,  2045,  2048,  2051,  2057,  2062,
+    2067
 };
 #endif
 
@@ -883,10 +885,11 @@ static const char *const yytname[] =
   "simple_when_clause_list", "simple_when_clause",
   "searched_when_clause_list", "searched_when_clause", "opt_else_clause",
   "expression_list", "literal_value", "datetime_unit",
-  "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
+  "literal_value_commalist", "literal_value_commalist_multiple",
+  "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
 };
 #endif
 
@@ -913,10 +916,10 @@ static const yytype_uint16 yytoknum[] =
 };
 # endif
 
-#define YYPACT_NINF -395
+#define YYPACT_NINF -410
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-395)))
+  (!!((Yystate) == (-410)))
 
 #define YYTABLE_NINF -139
 
@@ -927,61 +930,62 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-     174,  -395,  -395,   -64,    85,   -26,    14,   -31,   -16,  -395,
-      40,   196,   196,  -395,   109,   102,  -395,  -395,  -395,  -395,
-    -395,  -395,  -395,  -395,  -395,  -395,   148,    -3,    87,  -395,
-     -40,   121,   196,  -395,  -395,     1,    -5,   196,   196,   196,
-     196,   196,  -395,  -395,   716,    82,     2,  -395,   153,    63,
-    -395,  -395,  -395,    98,   152,    -3,    40,   141,  -395,    98,
-    -395,  -395,  -395,    12,    97,   116,   261,   116,   169,   126,
-     138,  -395,   176,  -395,  -395,   270,   274,  -395,  -395,  -395,
-     807,   139,  -395,   210,  -395,  -395,   154,  -395,  -395,   297,
-    -395,  -395,  -395,  -395,   172,  -395,  -395,   177,   231,   901,
-     313,   265,   192,  -395,  -395,   338,    23,  -395,  -395,   243,
-    -395,  -395,  -395,  -395,  -395,  1083,    -7,   196,   196,   214,
-     196,     1,   196,  -395,    98,   363,  -395,   205,   263,  -395,
-    -395,  -395,   255,  -395,   116,  -395,   196,   196,   625,  -395,
-    -395,   262,   196,  -395,  -395,  -395,   625,    33,   -29,  -395,
-     409,  -395,   165,   165,  1174,   411,  -395,   -14,    28,  -395,
-      13,   138,  1174,  -395,  -395,   196,  1174,  -395,  -395,  -395,
-    -395,  1174,    18,   274,  -395,   196,   398,    59,  -395,   417,
-    -395,    98,  -395,   202,  -395,   116,    98,    87,  -395,   196,
-      80,   196,   196,   196,  -395,   285,  -395,   211,  1241,   992,
-     214,   534,   422,   423,  -395,  -395,   312,   415,  1252,   219,
-      43,  1174,    61,  -395,  1174,  -395,   369,   292,  -395,  -395,
-    -395,  -395,  -395,  -395,   367,  -395,   216,   294,  -395,  -395,
-       7,   186,   267,  -395,   298,   186,     3,   372,  -395,  -395,
-      23,  -395,   347,  -395,  -395,   295,  1174,  -395,   351,   229,
-     196,  -395,  1174,  -395,   196,  -395,  -395,  -395,   303,   366,
-     368,   304,  -395,  -395,  -395,   232,  -395,  -395,  -395,  -395,
-    -395,    34,   196,   323,    80,   196,  -395,   188,  -395,  -395,
-       4,    65,   625,   625,   276,  -395,  -395,  -395,  -395,  -395,
-    -395,  -395,  -395,  1174,   311,  1174,    51,  -395,   234,   326,
-    1174,    71,  -395,   399,   351,  -395,  -395,  1174,   453,  -395,
-     160,   196,  -395,  -395,   370,  -395,   373,   374,   379,    13,
-    -395,   457,   462,   186,   430,   400,   431,   329,   380,  -395,
-     236,  -395,  1174,  -395,   351,  -395,   625,   333,   334,   196,
-    -395,   196,  -395,  -395,  -395,  -395,  -395,  -395,  -395,   196,
-    -395,  -395,  -395,   238,   454,   184,  -395,   336,   348,  -395,
-     391,   342,  1252,  -395,   403,   196,  -395,  -395,   188,  -395,
-    -395,   423,  -395,  -395,  -395,  1174,   345,   341,   901,  -395,
-     351,   401,  -395,  -395,  1252,   350,   351,  1174,  -395,    37,
-      35,  -395,  -395,  -395,  -395,  -395,    13,   267,   390,   395,
-    -395,  1174,   625,   396,  1174,  -395,   455,   108,  -395,   351,
-       8,   196,   196,   240,  -395,   242,  -395,   196,  -395,  -395,
-    -395,  -395,   354,    80,   461,   402,  -395,   625,  -395,  -395,
-     356,  -395,   346,   901,  -395,  1174,   245,  -395,  -395,  1252,
-     351,  -395,   495,  -395,   408,  -395,  -395,   358,   422,   464,
-     420,   358,  1174,  -395,  -395,  -395,   490,  -395,   249,   251,
-    -395,  -395,  -395,   196,  -395,  -395,   375,   468,  -395,    19,
-     196,  1174,   264,   351,  -395,   266,   371,   625,  1174,   504,
-     376,   377,  -395,   227,    46,   405,  -395,   269,   196,    -9,
-    -395,   381,   351,  -395,  -395,  -395,   422,   377,  -395,   196,
-    -395,   376,  -395,  1174,  -395,  -395,   421,   418,   407,   425,
-     515,   196,  -395,   277,  -395,  -395,   384,  -395,   496,  -395,
-    -395,    49,  -395,  -395,  -395,  -395,    56,   386,  -395,   196,
-     388,  -395,  -395,   466,   426,   467,  -395,   196,   279,   347,
-    -395,  -395,  -395,   281,   445,   404,  -395,   539,  -395,  -395
+     405,  -410,  -410,   -52,    43,   -13,    34,   -38,   116,  -410,
+      48,   231,   231,  -410,   115,   119,  -410,  -410,  -410,  -410,
+    -410,  -410,  -410,  -410,  -410,  -410,   166,    46,    49,  -410,
+     -36,   205,   231,  -410,  -410,    38,   132,   231,   231,   231,
+     231,   231,  -410,  -410,   652,    97,    72,  -410,   203,    95,
+    -410,  -410,  -410,   130,   175,    46,    48,   158,  -410,   130,
+    -410,  -410,  -410,    25,    61,   128,   270,   128,   184,   142,
+     149,  -410,   106,  -410,  -410,   309,   313,  -410,  -410,  -410,
+     743,   153,  -410,   218,  -410,  -410,   189,  -410,  -410,   331,
+    -410,  -410,  -410,  -410,   199,  -410,  -410,   207,   272,   837,
+     347,   305,   211,  -410,  -410,   328,    19,  -410,  -410,   262,
+    -410,  -410,  -410,  -410,  -410,  1019,     5,   231,   231,   220,
+     231,    38,   231,  -410,   130,   373,  -410,   148,   243,  -410,
+    -410,  -410,   244,  -410,   128,  -410,   231,   231,   561,  -410,
+    -410,   245,   231,  -410,  -410,  -410,   561,    52,    17,  -410,
+     400,  -410,   140,   140,  1110,   402,  -410,     1,    29,  -410,
+      20,   149,  1110,  -410,  -410,   231,  1110,  -410,  -410,  -410,
+    -410,  1110,    21,   313,  -410,   231,   323,   -64,  -410,   416,
+    -410,   130,  -410,   167,  -410,   128,   130,    49,  -410,   231,
+     152,   231,   231,   231,  -410,   269,  -410,   169,  1177,   928,
+     220,   470,   426,   433,  -410,  -410,   407,   428,  1188,   303,
+     173,    23,  1110,    64,  -410,  1110,  -410,   388,   306,  -410,
+    -410,  -410,  -410,  -410,  -410,   381,  -410,    66,   310,  -410,
+    -410,     8,   209,   215,  -410,   311,   209,     9,   385,  -410,
+    -410,    19,  -410,   360,  -410,  -410,   315,  1110,  -410,   301,
+     180,   231,  -410,  1110,  -410,   231,  -410,  -410,  -410,   320,
+     384,   386,   329,  -410,  -410,  -410,   196,  -410,  -410,  -410,
+    -410,  -410,    16,   231,   341,   152,   231,  -410,   174,  -410,
+    -410,     2,    81,   561,   561,   206,  -410,  -410,  -410,  -410,
+    -410,  -410,  -410,  -410,  1110,   334,  1110,    58,  -410,   210,
+     335,   349,  1110,    68,  -410,   420,   301,  -410,  -410,  1110,
+     476,  -410,   162,   231,  -410,  -410,   389,  -410,   390,   391,
+     406,    20,  -410,   486,   487,   209,   453,   421,   455,   353,
+     403,  -410,   223,  -410,  1110,  -410,   301,  -410,   561,   357,
+     358,   231,  -410,   231,  -410,  -410,  -410,  -410,  -410,  -410,
+    -410,   231,  -410,  -410,  -410,   225,   477,    86,  -410,   361,
+     366,  -410,   417,   364,  1188,  -410,   430,   231,  -410,  -410,
+     174,  -410,  -410,   433,  -410,  -410,  -410,  1110,   367,   284,
+     837,  -410,   301,   424,  -410,  -410,  1188,  1188,   374,   301,
+    1110,  -410,    33,   -16,  -410,  -410,  -410,  -410,  -410,    20,
+     215,   419,   422,  -410,  1110,   561,   429,  1110,  -410,   475,
+      18,  -410,   301,    31,   231,   231,   227,  -410,   233,  -410,
+     231,  -410,  -410,  -410,  -410,   382,   152,   489,   432,  -410,
+     561,  -410,  -410,   392,  -410,   289,   837,  -410,  1110,   238,
+    -410,  -410,   240,  1188,   301,  -410,   521,  -410,   437,  -410,
+    -410,   393,   426,   498,   456,   393,  1110,  -410,  -410,  -410,
+     522,  -410,   246,   252,  -410,  -410,  -410,   231,  -410,  -410,
+     399,   504,  -410,    32,   231,  1110,   254,   301,  -410,  -410,
+     257,   404,   561,  1110,   540,   412,   408,  -410,   274,    11,
+     442,  -410,   286,   231,   127,  -410,   410,   301,  -410,  -410,
+    -410,   426,   408,  -410,   231,  -410,   412,  -410,  1110,  -410,
+    -410,   460,   454,   447,   458,   549,   231,  -410,   290,  -410,
+    -410,   425,  -410,   528,  -410,  -410,    -6,  -410,  -410,  -410,
+    -410,    12,   431,  -410,   231,   434,  -410,  -410,   495,   462,
+     496,  -410,   231,   293,   360,  -410,  -410,  -410,   299,   473,
+     435,  -410,   563,  -410,  -410
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -989,93 +993,96 @@ static const yytype_int16 yypact[] =
      means the default is an error.  */
 static const yytype_uint16 yydefact[] =
 {
-       0,     6,   298,     0,     0,     0,     0,     0,     0,    18,
+       0,     6,   300,     0,     0,     0,     0,     0,     0,    18,
      123,     0,     0,     7,     0,     0,    15,     8,    10,    11,
       13,    14,     9,    17,    12,    16,     0,   112,   119,   121,
-       0,   296,     0,   290,   291,     0,     0,     0,     0,     0,
+       0,   298,     0,   292,   293,     0,     0,     0,     0,     0,
        0,     0,   124,   125,     0,     0,   114,   115,     0,   156,
        1,     3,     2,     0,     0,   112,   123,     0,   110,     0,
-       5,     4,   297,     0,     0,   103,     0,   103,     0,     0,
-     197,    25,     0,   256,   253,     0,   282,   126,    40,    29,
+       5,     4,   299,     0,     0,   103,     0,   103,     0,     0,
+     197,    25,     0,   256,   253,     0,   284,   126,    40,    29,
        0,     0,    30,    31,    34,    36,     0,    37,    39,     0,
       41,   252,    35,    38,     0,    32,    33,     0,     0,     0,
        0,     0,   127,   128,   232,   132,   218,   220,   222,   225,
-     228,   229,   230,   224,   223,     0,   268,     0,     0,     0,
+     228,   229,   230,   224,   223,     0,   270,     0,     0,     0,
        0,     0,     0,   111,     0,     0,   120,     0,     0,   100,
      102,   101,     0,    98,   103,    97,     0,     0,     0,   106,
      198,     0,     0,    94,   254,   255,     0,     0,   248,   245,
        0,    43,     0,   257,     0,     0,    44,     0,     0,   259,
-       0,   197,     0,   283,   284,     0,     0,   131,   286,   287,
-     285,     0,     0,     0,   221,     0,     0,   197,   108,     0,
-     116,     0,   117,     0,   288,   103,     0,   118,   113,     0,
+       0,   197,     0,   285,   286,     0,     0,   131,   288,   289,
+     287,     0,     0,     0,   221,     0,     0,   197,   108,     0,
+     116,     0,   117,     0,   290,   103,     0,   118,   113,     0,
        0,     0,     0,     0,    96,    66,    27,     0,     0,     0,
-       0,     0,   199,   201,   203,   205,     0,   223,     0,     0,
-       0,     0,   248,   242,     0,   246,     0,     0,   262,   263,
-     264,   261,   265,   260,     0,   258,     0,     0,   134,   231,
-       0,     0,   158,   147,   133,   152,   135,   160,   129,   130,
-     217,   219,   174,   226,   269,     0,     0,   233,   250,     0,
-       0,   105,     0,   157,     0,    99,    95,    19,     0,     0,
-       0,     0,    20,    21,    22,     0,    74,    76,    77,    78,
-      79,     0,     0,     0,    64,     0,    42,    56,   204,   212,
-       0,     0,     0,     0,     0,   272,   274,   275,   276,   277,
-     273,   278,   280,     0,     0,     0,     0,   266,     0,     0,
-       0,     0,   243,     0,   249,   241,    45,     0,     0,    46,
-     138,     0,   148,   154,   144,   139,   140,   142,     0,     0,
-     151,     0,     0,   150,     0,   162,     0,     0,   176,   234,
-       0,   235,     0,   107,   109,   289,     0,     0,     0,     0,
-     104,     0,    81,    84,    82,   294,   295,   293,   292,     0,
-      80,    85,   270,     0,   268,     0,    63,    65,    68,    28,
-       0,     0,     0,    47,     0,     0,    49,    55,    57,    26,
-     211,   200,   202,   279,   281,     0,     0,     0,     0,   213,
-     210,     0,   209,    93,     0,     0,   247,     0,   240,     0,
-       0,   153,   155,   145,   141,   143,     0,   159,     0,     0,
-     149,     0,     0,   164,     0,   227,     0,   178,   236,   251,
-       0,     0,     0,     0,    75,     0,    67,     0,    86,    87,
-      88,    89,    90,     0,     0,    70,    48,     0,    51,    50,
-       0,    54,     0,     0,   215,     0,     0,   208,   267,     0,
-     244,   237,     0,   238,     0,   136,   137,   161,   163,     0,
-     166,   175,     0,   181,   180,   173,     0,    61,     0,     0,
-      58,    83,   271,     0,    24,    62,     0,     0,    23,     0,
-       0,     0,     0,   206,   214,     0,     0,     0,     0,     0,
-     168,   177,   188,   191,     0,     0,    59,     0,     0,     0,
-      52,     0,   207,   216,    92,   239,   146,   165,   167,     0,
-     122,   169,   170,     0,   192,   193,   194,     0,     0,     0,
-       0,     0,    91,     0,    72,    73,     0,    53,     0,   171,
-     189,     0,   190,   182,   184,   183,     0,     0,    69,     0,
-       0,   195,   196,     0,     0,     0,   179,     0,     0,   174,
-     185,   187,   186,     0,     0,     0,    60,     0,   172,    71
+       0,     0,   199,   201,   203,   205,     0,   223,     0,    93,
+       0,     0,     0,   248,   242,     0,   246,     0,     0,   262,
+     263,   264,   261,   265,   260,     0,   258,     0,     0,   134,
+     231,     0,     0,   158,   147,   133,   152,   135,   160,   129,
+     130,   217,   219,   174,   226,   271,     0,     0,   233,   250,
+       0,     0,   105,     0,   157,     0,    99,    95,    19,     0,
+       0,     0,     0,    20,    21,    22,     0,    74,    76,    77,
+      78,    79,     0,     0,     0,    64,     0,    42,    56,   204,
+     212,     0,     0,     0,     0,     0,   274,   276,   277,   278,
+     279,   275,   280,   282,     0,     0,     0,     0,   266,     0,
+       0,     0,     0,     0,   243,     0,   249,   241,    45,     0,
+       0,    46,   138,     0,   148,   154,   144,   139,   140,   142,
+       0,     0,   151,     0,     0,   150,     0,   162,     0,     0,
+     176,   234,     0,   235,     0,   107,   109,   291,     0,     0,
+       0,     0,   104,     0,    81,    84,    82,   296,   297,   295,
+     294,     0,    80,    85,   272,     0,   270,     0,    63,    65,
+      68,    28,     0,     0,     0,    47,     0,     0,    49,    55,
+      57,    26,   211,   200,   202,   281,   283,     0,     0,     0,
+       0,   213,   210,     0,   209,   268,     0,     0,     0,   247,
+       0,   240,     0,     0,   153,   155,   145,   141,   143,     0,
+     159,     0,     0,   149,     0,     0,   164,     0,   227,     0,
+     178,   236,   251,     0,     0,     0,     0,    75,     0,    67,
+       0,    86,    87,    88,    89,    90,     0,     0,    70,    48,
+       0,    51,    50,     0,    54,     0,     0,   215,     0,     0,
+     208,   267,     0,     0,   244,   237,     0,   238,     0,   136,
+     137,   161,   163,     0,   166,   175,     0,   181,   180,   173,
+       0,    61,     0,     0,    58,    83,   273,     0,    24,    62,
+       0,     0,    23,     0,     0,     0,     0,   206,   214,   269,
+       0,     0,     0,     0,     0,   168,   177,   188,   191,     0,
+       0,    59,     0,     0,     0,    52,     0,   207,   216,    92,
+     239,   146,   165,   167,     0,   122,   169,   170,     0,   192,
+     193,   194,     0,     0,     0,     0,     0,    91,     0,    72,
+      73,     0,    53,     0,   171,   189,     0,   190,   182,   184,
+     183,     0,     0,    69,     0,     0,   195,   196,     0,     0,
+       0,   179,     0,     0,   174,   185,   187,   186,     0,     0,
+       0,    60,     0,   172,    71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -395,  -395,  -395,  -395,  -395,  -395,  -395,  -395,  -164,  -395,
-     349,   180,  -395,  -395,  -271,  -395,  -395,  -395,  -395,  -395,
-    -395,  -394,   209,  -395,  -395,  -395,  -395,  -395,  -395,  -395,
-    -395,    24,   -46,  -395,  -395,  -395,   301,  -395,   497,  -395,
-    -395,   435,   259,   433,   -28,   498,  -395,  -395,   397,  -395,
-     -90,  -395,  -395,  -207,   162,  -187,   -10,  -395,  -395,  -395,
-    -395,  -395,  -395,  -395,    60,    21,  -395,  -395,  -395,  -395,
-    -395,  -395,    84,    62,  -395,  -395,   -54,  -395,  -145,   282,
-     280,   382,   -35,   406,   412,   451,  -156,  -395,  -395,  -395,
-    -395,   355,  -395,   427,   359,  -232,  -203,   429,   129,  -128,
-    -395,  -395,  -395,  -395,  -395,  -136,    -4,  -395,  -395,  -395
+    -410,  -410,  -410,  -410,  -410,  -410,  -410,  -410,  -148,  -410,
+     376,   216,  -410,  -410,  -264,  -410,  -410,  -410,  -410,  -410,
+    -410,  -409,   236,  -410,  -410,  -410,  -410,  -410,  -410,  -410,
+    -410,    85,    -3,  -410,  -410,  -410,   325,  -410,   526,  -410,
+    -410,   467,   217,   464,   -43,   533,  -410,  -410,   436,  -410,
+    -114,  -410,  -410,  -186,   192,  -184,    -9,  -410,  -410,  -410,
+    -410,  -410,  -410,  -410,    89,    55,  -410,  -410,  -410,  -410,
+    -410,  -410,   109,    92,  -410,  -410,    79,  -410,  -142,   318,
+     322,   409,   -42,   441,   439,   501,  -154,  -410,  -410,  -410,
+    -410,   398,  -410,   469,   413,  -232,  -195,   449,  -324,  -410,
+    -129,  -410,  -410,  -410,  -410,  -410,  -141,    -4,  -410,  -410,
+    -410
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
       -1,    14,    15,    16,    17,    18,    19,    20,   196,   197,
-     100,   367,   368,   369,   262,   357,   358,   273,   425,   468,
-     516,   265,   266,   267,   268,   269,   270,   422,   464,    21,
+     100,   369,   370,   371,   263,   359,   360,   274,   428,   472,
+     521,   266,   267,   268,   269,   270,   271,   425,   468,    21,
       22,    65,   133,    23,    24,   177,   178,    25,    58,    26,
       46,    47,   157,    28,    29,    44,   101,   102,   103,   161,
-     104,   323,   318,   232,   233,   312,   313,   234,   325,   403,
-     450,   480,   500,   501,   502,   327,   328,   407,   455,   456,
-     510,   536,   481,   482,   506,   522,   139,   140,   202,   203,
+     104,   325,   320,   233,   234,   314,   315,   235,   327,   406,
+     454,   485,   505,   506,   507,   329,   330,   410,   459,   460,
+     515,   541,   486,   487,   511,   527,   139,   140,   202,   203,
      204,   205,   206,   106,   107,   108,   109,   110,   111,   112,
-     212,   213,   148,   149,   216,   249,   113,   224,   298,   114,
-     353,   295,   115,   166,   171,   183,   116,   351,    30,    31
+     213,   214,   148,   149,   217,   250,   113,   225,   299,   209,
+     114,   355,   296,   115,   166,   171,   183,   116,   353,    30,
+      31
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
@@ -1083,256 +1090,215 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      36,   210,    48,   356,   235,   297,   209,    45,    49,   105,
-     207,    33,   282,    34,   330,    56,   282,    33,   207,    34,
-     175,   135,    33,   310,    34,   257,    56,   282,    63,   182,
-      56,   126,   214,    68,    69,    70,    71,    72,    33,   342,
-      34,   343,   321,   127,   143,   147,   168,   169,   320,   163,
-     164,   282,    37,   507,   163,   164,   280,    32,   163,   164,
-      67,   381,   344,   533,   158,   514,    66,    41,    42,   487,
-     231,   207,   128,   207,   235,   284,   285,   286,   287,   288,
-     289,   290,   291,   292,   293,    39,   163,   164,   194,    33,
-      40,    34,   163,   164,   513,    38,   508,   515,    43,   345,
-      60,   167,   129,    61,   442,   146,   534,   237,    48,    50,
-     279,   359,   397,   179,    49,   531,   184,    59,   186,   226,
-      64,    10,   214,   251,    62,   258,    64,   105,   346,   347,
-     322,   228,   195,   198,    57,   532,   400,   176,   184,   255,
-     231,   248,   294,   382,   352,   185,   436,   176,   118,   370,
-     259,   230,   465,   457,   207,   207,   236,   230,   256,   428,
-     348,   239,   242,   235,   490,   300,   281,   211,   243,   447,
-     170,   244,   451,   229,   509,     1,   301,     2,   349,   304,
-     443,   438,   441,   260,   535,   198,   119,   263,   264,   271,
-      33,   410,    34,   387,   138,   211,    53,   117,   360,    10,
-      33,   472,    34,   413,   379,   250,     3,   120,   207,   261,
-     229,   248,    10,   415,   453,   130,   131,   334,   218,   311,
-     454,   418,     4,     5,   419,   420,   236,    49,    54,   231,
-       6,    49,   314,   361,     7,   122,   297,   163,   164,   315,
-     235,   219,    51,   362,    52,   125,   179,   316,   163,   164,
-     335,   189,   190,   132,     8,   220,   221,   448,   377,    27,
-     380,   504,    10,    35,   136,   386,   134,   350,   354,   317,
-     137,   198,   389,   138,   207,   458,   459,   144,   222,     9,
-     363,   145,   469,   150,   505,    55,   434,   308,    10,   462,
-      10,   364,   151,   373,   374,   375,   365,   409,   152,   207,
-     421,   392,   153,   223,    11,   391,   231,    49,   141,   191,
-     192,    12,   121,   156,    13,   236,   154,   366,   159,    49,
-     142,   155,   284,   285,   286,   287,   288,   289,   290,   291,
-     292,   293,   496,   163,   164,   184,   160,   271,   162,   314,
-     432,   172,    33,   248,    34,   184,   315,   253,   254,   207,
-     435,  -138,   440,   376,   316,   471,   274,   275,   181,   163,
-     164,   430,   163,   164,   299,   254,   248,   163,   164,   248,
-     188,   165,   163,   164,   331,   332,   317,   340,   341,   383,
-     384,   408,   332,   416,   417,   460,   254,   461,   254,   294,
-     474,   332,   236,   538,   485,   254,   486,   254,   248,   193,
-     473,   543,    33,    73,    34,    74,   208,   184,   184,   493,
-     332,   494,   384,   354,   512,   341,   217,   483,   227,    75,
-      76,   245,   528,   341,   544,   254,   546,   254,   252,   272,
-     282,   305,   283,    78,    79,   296,   492,   306,   307,   309,
-     329,    80,    81,   483,   319,   324,   326,   336,   339,    82,
-      83,   337,    84,   338,   355,   378,   246,    85,   385,   271,
-     390,   388,    86,   396,   398,    87,   491,   393,   483,   399,
-     394,   395,   401,   404,   405,   402,   406,   411,   412,    88,
-      89,   175,   423,   426,   271,   424,   427,    90,   429,   433,
-      91,   445,   449,   437,   439,   518,   446,   452,   463,   466,
-     470,   467,   476,   477,   332,    92,   478,   527,   479,   484,
-     489,   498,   499,   511,   521,    93,   495,   524,    94,   488,
-     523,    95,    96,   503,   526,   184,   517,   525,   529,   530,
-     537,    97,   539,   184,   540,   542,   541,    98,    33,    73,
-      34,    74,    99,   247,   199,   547,   549,   277,   431,   548,
-     414,   333,   123,   180,   124,    75,    76,   187,   444,   238,
-     545,   519,   497,   372,   371,   520,   174,   302,   475,    78,
-      79,   303,   240,     0,     0,   215,     0,    80,    81,     0,
-       0,   278,   225,   241,     0,    82,    83,     0,    84,     0,
-       0,     0,     0,    85,     0,     0,     0,   200,    86,     0,
-       0,    87,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,    88,    89,     0,     0,     0,
-       0,     0,     0,    90,     0,     0,    91,     0,     0,    33,
-      73,    34,    74,     0,     0,   199,     0,     0,     0,     0,
-       0,    92,     0,     0,     0,     0,    75,    76,    10,     0,
-       0,    93,     0,     0,    94,     0,     0,    95,    96,     0,
-      78,    79,     0,     0,     0,     0,     0,    97,    80,    81,
-       0,     0,     0,    98,     0,     0,    82,    83,   201,    84,
-       0,     0,     0,     0,    85,     0,     0,     0,   200,    86,
-       0,     0,    87,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,    88,    89,     0,     0,
-       0,     0,     0,     0,    90,     0,     0,    91,     0,     0,
-      33,    73,    34,    74,     0,     0,     0,     0,     0,     0,
-       0,     0,    92,     0,     0,     0,     0,    75,    76,    77,
-       0,     0,    93,     0,     0,    94,     0,     0,    95,    96,
-       0,    78,    79,     0,     0,     0,     0,     0,    97,    80,
-      81,     0,     0,     0,    98,     0,     0,    82,    83,   201,
-      84,     0,     0,     0,     0,    85,     0,     0,     0,     0,
-      86,     0,     0,    87,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    88,    89,     0,
-       0,     0,     0,     0,     0,    90,     0,     0,    91,     0,
-       0,    33,    73,    34,    74,     0,     0,     0,     0,     0,
-       0,     0,     0,    92,     0,     0,     0,     0,    75,    76,
-       0,     0,     0,    93,     0,     0,    94,     0,     0,    95,
-      96,     0,    78,    79,     0,     0,     0,     0,     0,    97,
-      80,    81,     0,     0,     0,    98,     0,     0,    82,    83,
-      99,    84,     0,     0,     0,     0,    85,     0,     0,     0,
-       0,    86,     0,     0,    87,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    88,    89,
-       0,     0,     0,     0,     0,     0,    90,     0,     0,    91,
-       0,     0,     0,     0,     0,    33,    73,    34,    74,     0,
-       0,     0,     0,     0,    92,     0,     0,     0,     0,     0,
-       0,     0,    75,    76,    93,     0,     0,    94,     0,     0,
-      95,    96,     0,     0,     0,     0,    78,    79,     0,     0,
-      97,   146,     0,     0,    80,    81,    98,     0,     0,     0,
-       0,    99,    82,    83,     0,    84,     0,     0,     0,     0,
-      85,     0,     0,     0,     0,    86,     0,     0,    87,     0,
+      36,   210,   105,    48,   211,   182,   236,    45,    49,   207,
+     283,   358,    33,   298,    34,   332,   126,   207,   512,   538,
+      33,   344,    34,   345,    33,    33,    34,    34,    63,   143,
+      56,   283,   175,    68,    69,    70,    71,    72,   147,   283,
+     283,   258,   168,   169,   346,   312,   232,    33,   323,    34,
+     163,   164,   322,   446,   163,   164,   127,   158,   492,   281,
+     536,   513,   539,   442,   135,    37,   129,    56,   383,    32,
+     207,   138,   207,   163,   164,    56,    42,   236,   215,    59,
+     537,   347,   251,    40,   518,   128,   280,   163,   164,   163,
+     164,   285,   286,   287,   288,   289,   290,   291,   292,   293,
+     294,   167,   163,   164,    60,    39,    43,    61,    38,    48,
+     348,   349,   227,   179,    49,    50,   184,   232,   186,   480,
+     105,    67,    10,   421,   457,   215,   422,   423,   361,   447,
+     458,   194,   195,   198,   249,   400,   324,   310,   184,   514,
+     540,   403,   350,   257,   354,   302,   229,   372,   439,   176,
+     384,   146,   231,   176,   207,   207,   237,    10,   295,   282,
+     351,   240,   469,    64,   231,   243,   170,   236,   244,   431,
+     303,   245,   451,   306,   230,   455,   461,   495,   445,   130,
+     131,   381,   256,    57,   362,   198,   212,   264,   265,   272,
+     390,   441,   298,   219,   189,   190,   413,   259,   212,    41,
+     416,   519,   424,    66,   476,   249,   185,   232,    62,   207,
+     418,   336,   117,    33,    53,    34,   220,    27,   118,   363,
+      10,    35,   260,   375,   376,   377,   230,   237,    49,   364,
+     221,   222,    49,   520,   316,    33,   119,    34,   141,   120,
+     238,   317,   313,    55,    10,   236,    54,   179,   298,   318,
+     142,   337,   379,   223,   382,   261,   252,    64,   122,    51,
+     389,    52,   125,   452,   437,   132,   365,   392,   352,   356,
+     121,   319,   198,   462,   463,   134,   207,   366,   224,   136,
+      10,   262,   367,   378,   138,   232,   137,   316,   473,   191,
+     192,   466,   412,   438,   317,   163,   164,   150,   475,  -138,
+     151,   207,   318,   368,   395,   163,   164,   394,   509,    49,
+     163,   164,   254,   255,   275,   276,   144,   237,   301,   255,
+     145,    49,   163,   164,   319,   333,   334,    33,    73,    34,
+      74,   510,    33,   152,    34,   435,   153,   184,   249,   272,
+     501,   342,   343,   154,    75,    76,   246,   184,   444,   163,
+     164,   155,   159,   207,   156,   385,   386,   162,    78,    79,
+     172,   165,   249,   433,   181,   249,    80,    81,   411,   334,
+     419,   420,   464,   255,    82,    83,   160,    84,   465,   255,
+     188,   247,    85,   478,   334,   479,   386,    86,   193,   208,
+      87,   490,   255,   543,   249,   237,   477,   491,   255,   498,
+     334,   548,   499,   386,    88,    89,     1,   218,     2,   228,
+     184,   184,    90,   273,   488,    91,   356,   285,   286,   287,
+     288,   289,   290,   291,   292,   293,   294,   253,   163,   164,
+      92,   517,   343,   497,   283,   533,   343,     3,   549,   255,
+      93,   488,   284,    94,   551,   255,    95,    96,   297,   300,
+     307,   308,   309,     4,     5,   311,    97,   321,   326,   328,
+     331,     6,    98,   272,   338,     7,   488,    99,   248,   339,
+     496,   340,   357,   341,    33,    73,    34,    74,   380,   387,
+     199,   388,   391,   393,   295,     8,   396,   397,   398,   272,
+     399,    75,    76,   401,   402,   404,   405,   407,   408,   409,
+     523,   414,   415,   427,   175,    78,    79,   426,   430,   429,
+       9,   436,   532,    80,    81,   432,   440,   456,   443,    10,
+     449,    82,    83,   450,    84,   453,   467,   470,   481,    85,
+     184,   471,   482,   200,    86,    11,   474,    87,   184,   334,
+     483,   489,    12,   493,   484,    13,   494,   503,   504,   500,
+     516,    88,    89,   526,   508,   522,   528,   529,   531,    90,
+     530,   535,    91,   545,   547,    33,    73,    34,    74,   534,
+     554,   199,   546,   552,   278,   542,   335,    92,   544,   417,
+     553,   123,    75,    76,    10,   180,   434,    93,   187,   124,
+      94,   448,   502,    95,    96,   524,    78,    79,   239,   550,
+     525,   373,   226,    97,    80,    81,   374,   241,   279,    98,
+     242,   304,    82,    83,   201,    84,   174,   216,     0,     0,
+      85,     0,     0,     0,   200,    86,   305,     0,    87,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,     0,    88,    89,     0,     0,     0,     0,     0,     0,
       90,     0,     0,    91,     0,     0,    33,    73,    34,    74,
        0,     0,     0,     0,     0,     0,     0,     0,    92,     0,
-       0,     0,     0,    75,    76,    10,     0,     0,    93,     0,
+       0,     0,     0,    75,    76,    77,     0,     0,    93,     0,
        0,    94,     0,     0,    95,    96,     0,    78,    79,     0,
        0,     0,     0,     0,    97,    80,    81,     0,     0,     0,
-      98,     0,     0,    82,    83,    99,    84,     0,     0,     0,
-       0,    85,     0,     0,     0,   200,    86,     0,     0,    87,
+      98,     0,     0,    82,    83,   201,    84,     0,     0,     0,
+       0,    85,     0,     0,     0,     0,    86,     0,     0,    87,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,     0,     0,    88,    89,     0,     0,     0,     0,     0,
        0,    90,     0,     0,    91,     0,     0,    33,    73,    34,
       74,     0,     0,     0,     0,     0,     0,     0,     0,    92,
-       0,     0,     0,     0,    75,   173,     0,     0,     0,    93,
+       0,     0,     0,     0,    75,    76,     0,     0,     0,    93,
        0,     0,    94,     0,     0,    95,    96,     0,    78,    79,
        0,     0,     0,     0,     0,    97,    80,    81,     0,     0,
-       0,    98,     0,     0,    82,    83,   201,    84,     0,     0,
+       0,    98,     0,     0,    82,    83,    99,    84,     0,     0,
        0,     0,    85,     0,     0,     0,     0,    86,     0,     0,
       87,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,     0,     0,     0,    88,    89,     0,     0,     0,     0,
-       0,     0,    90,     0,     0,    91,     0,     0,    33,    73,
-      34,    74,     0,     0,     0,     0,     0,     0,     0,     0,
-      92,     0,     0,     0,     0,    75,    76,     0,     0,     0,
-      93,     0,     0,    94,     0,     0,    95,    96,     0,    78,
-      79,     0,     0,     0,     0,     0,    97,    80,    81,     0,
-       0,     0,    98,     0,     0,    82,    83,    99,    84,     0,
-       0,     0,     0,    85,     0,     0,     0,     0,    86,     0,
-       0,    87,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,    88,    89,    73,     0,    74,
-       0,     0,     0,    90,     0,     0,    91,     0,     0,     0,
-       0,     0,     0,    75,   173,     0,    78,    79,     0,     0,
-       0,    92,     0,     0,     0,    81,     0,    78,    79,     0,
-       0,    93,    82,    83,    94,    84,    81,    95,    96,     0,
-      85,     0,     0,    82,    83,     0,    84,    97,    87,     0,
-       0,    85,     0,    98,     0,     0,     0,     0,    99,    87,
-       0,     0,    88,   276,     0,     0,     0,     0,     0,     0,
-      90,     0,     0,    88,    89,     0,     0,     0,     0,     0,
-       0,    90,     0,     0,    91,     0,     0,     0,    92,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    93,    92,
-       0,     0,     0,     0,    95,    96,     0,     0,     0,    93,
-       0,     0,     0,     0,    97,    95,    96,     0,     0,     0,
-      98,     0,     0,     0,     0,    97,     0,     0,     0,     0,
-       0,    98
+       0,     0,    90,     0,     0,    91,     0,     0,     0,     0,
+       0,    33,    73,    34,    74,     0,     0,     0,     0,     0,
+      92,     0,     0,     0,     0,     0,     0,     0,    75,    76,
+      93,     0,     0,    94,     0,     0,    95,    96,     0,     0,
+       0,     0,    78,    79,     0,     0,    97,   146,     0,     0,
+      80,    81,    98,     0,     0,     0,     0,    99,    82,    83,
+       0,    84,     0,     0,     0,     0,    85,     0,     0,     0,
+       0,    86,     0,     0,    87,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,    88,    89,
+       0,     0,     0,     0,     0,     0,    90,     0,     0,    91,
+       0,     0,    33,    73,    34,    74,     0,     0,     0,     0,
+       0,     0,     0,     0,    92,     0,     0,     0,     0,    75,
+      76,    10,     0,     0,    93,     0,     0,    94,     0,     0,
+      95,    96,     0,    78,    79,     0,     0,     0,     0,     0,
+      97,    80,    81,     0,     0,     0,    98,     0,     0,    82,
+      83,    99,    84,     0,     0,     0,     0,    85,     0,     0,
+       0,   200,    86,     0,     0,    87,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,    88,
+      89,     0,     0,     0,     0,     0,     0,    90,     0,     0,
+      91,     0,     0,    33,    73,    34,    74,     0,     0,     0,
+       0,     0,     0,     0,     0,    92,     0,     0,     0,     0,
+      75,   173,     0,     0,     0,    93,     0,     0,    94,     0,
+       0,    95,    96,     0,    78,    79,     0,     0,     0,     0,
+       0,    97,    80,    81,     0,     0,     0,    98,     0,     0,
+      82,    83,   201,    84,     0,     0,     0,     0,    85,     0,
+       0,     0,     0,    86,     0,     0,    87,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+      88,    89,     0,     0,     0,     0,     0,     0,    90,     0,
+       0,    91,     0,     0,    33,    73,    34,    74,     0,     0,
+       0,     0,     0,     0,     0,     0,    92,     0,     0,     0,
+       0,    75,    76,     0,     0,     0,    93,     0,     0,    94,
+       0,     0,    95,    96,     0,    78,    79,     0,     0,     0,
+       0,     0,    97,    80,    81,     0,     0,     0,    98,     0,
+       0,    82,    83,    99,    84,     0,     0,     0,     0,    85,
+       0,     0,     0,     0,    86,     0,     0,    87,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,    88,    89,    73,     0,    74,     0,     0,     0,    90,
+       0,     0,    91,     0,     0,     0,     0,     0,     0,    75,
+     173,     0,    78,    79,     0,     0,     0,    92,     0,     0,
+       0,    81,     0,    78,    79,     0,     0,    93,    82,    83,
+      94,    84,    81,    95,    96,     0,    85,     0,     0,    82,
+      83,     0,    84,    97,    87,     0,     0,    85,     0,    98,
+       0,     0,     0,     0,    99,    87,     0,     0,    88,   277,
+       0,     0,     0,     0,     0,     0,    90,     0,     0,    88,
+      89,     0,     0,     0,     0,     0,     0,    90,     0,     0,
+      91,     0,     0,     0,    92,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,    93,    92,     0,     0,     0,     0,
+      95,    96,     0,     0,     0,    93,     0,     0,     0,     0,
+      97,    95,    96,     0,     0,     0,    98,     0,     0,     0,
+       0,    97,     0,     0,     0,     0,     0,    98
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   146,    12,   274,   160,   208,   142,    11,    12,    44,
-     138,     4,     8,     6,   246,    29,     8,     4,   146,     6,
-      27,    67,     4,   230,     6,   189,    29,     8,    32,   119,
-      29,    59,    61,    37,    38,    39,    40,    41,     4,     5,
-       6,     7,    39,    31,    72,    80,    23,    24,   235,    21,
-      22,     8,    78,     7,    21,    22,   201,   121,    21,    22,
-      36,    10,    28,     7,    99,    74,    71,    83,    28,   463,
-     160,   199,    60,   201,   230,    10,    11,    12,    13,    14,
-      15,    16,    17,    18,    19,    71,    21,    22,   134,     4,
-     121,     6,    21,    22,   488,   121,    50,   106,    58,    65,
-     140,   105,     5,   143,    69,   134,    50,   161,   118,     0,
-     200,   275,   319,   117,   118,    66,   120,    30,   122,   154,
-     125,   114,    61,   177,     3,    45,   125,   162,    94,    95,
-     127,   145,   136,   137,   137,    86,   323,   144,   142,   185,
-     230,   176,    77,    92,   272,   121,   378,   144,   146,   145,
-      70,   144,   423,   145,   282,   283,   160,   144,   186,   362,
-     126,   165,   144,   319,   145,   122,   201,   134,   172,   401,
-     147,   175,   404,   145,   128,     1,   211,     3,   144,   214,
-     145,   384,   145,   103,   128,   189,    33,   191,   192,   193,
-       4,   336,     6,   122,   135,   134,    48,   115,    10,   114,
-       4,   433,     6,   339,   294,   146,    32,   144,   336,   129,
-     145,   246,   114,   349,   106,   118,   119,   252,    53,    33,
-     112,    37,    48,    49,    40,    41,   230,   231,    80,   319,
-      56,   235,    72,    45,    60,    83,   439,    21,    22,    79,
-     396,    76,   140,    55,   142,   104,   250,    87,    21,    22,
-     254,    46,    47,   137,    80,    90,    91,   402,   293,     0,
-     295,    34,   114,     4,    95,   300,     5,   271,   272,   109,
-     144,   275,   307,   135,   402,   411,   412,     7,   113,   105,
-      92,     7,   427,   144,    57,    26,   376,    71,   114,   417,
-     114,   103,    82,    17,    18,    19,   108,   332,   144,   427,
-     116,   311,     5,   138,   130,   145,   396,   311,   132,    46,
-      47,   137,    53,    82,   140,   319,   144,   129,     5,   323,
-     144,   144,    10,    11,    12,    13,    14,    15,    16,    17,
-      18,    19,   477,    21,    22,   339,    71,   341,   146,    72,
-     375,    98,     4,   378,     6,   349,    79,   145,   146,   477,
-       9,    84,   387,    77,    87,     9,   145,   146,   144,    21,
-      22,   365,    21,    22,   145,   146,   401,    21,    22,   404,
-       7,    33,    21,    22,   145,   146,   109,   145,   146,   145,
-     146,   145,   146,   145,   146,   145,   146,   145,   146,    77,
-     145,   146,   396,   529,   145,   146,   145,   146,   433,   144,
-     435,   537,     4,     5,     6,     7,   144,   411,   412,   145,
-     146,   145,   146,   417,   145,   146,     7,   452,     7,    21,
-      22,    23,   145,   146,   145,   146,   145,   146,    11,   144,
-       8,    62,     9,    35,    36,    20,   471,   145,    71,   145,
-     145,    43,    44,   478,   146,    73,    99,   144,   144,    51,
-      52,    85,    54,    85,   131,   144,    58,    59,   132,   463,
-       7,    62,    64,    84,     7,    67,   470,    97,   503,     7,
-      97,    97,    42,    42,   145,    75,    96,   144,   144,    81,
-      82,    27,   146,    92,   488,   137,   144,    89,    85,   144,
-      92,   101,    96,    92,   144,   499,   101,    42,   144,    38,
-     144,    99,     7,    95,   146,   107,    42,   511,    88,    19,
-      42,     7,   136,   108,    93,   117,   145,   110,   120,   144,
-     102,   123,   124,   146,     9,   529,   145,   102,   144,    33,
-     144,   133,   144,   537,    68,    68,   110,   139,     4,     5,
-       6,     7,   144,   145,    10,   100,     7,   198,   368,   145,
-     341,   250,    55,   118,    56,    21,    22,   124,   396,   162,
-     539,   501,   478,   283,   282,   503,   115,   212,   439,    35,
-      36,   212,   166,    -1,    -1,   148,    -1,    43,    44,    -1,
-      -1,   199,   153,   171,    -1,    51,    52,    -1,    54,    -1,
-      -1,    -1,    -1,    59,    -1,    -1,    -1,    63,    64,    -1,
-      -1,    67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    81,    82,    -1,    -1,    -1,
-      -1,    -1,    -1,    89,    -1,    -1,    92,    -1,    -1,     4,
-       5,     6,     7,    -1,    -1,    10,    -1,    -1,    -1,    -1,
-      -1,   107,    -1,    -1,    -1,    -1,    21,    22,   114,    -1,
-      -1,   117,    -1,    -1,   120,    -1,    -1,   123,   124,    -1,
-      35,    36,    -1,    -1,    -1,    -1,    -1,   133,    43,    44,
-      -1,    -1,    -1,   139,    -1,    -1,    51,    52,   144,    54,
-      -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,    63,    64,
-      -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    81,    82,    -1,    -1,
-      -1,    -1,    -1,    -1,    89,    -1,    -1,    92,    -1,    -1,
-       4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,   107,    -1,    -1,    -1,    -1,    21,    22,    23,
-      -1,    -1,   117,    -1,    -1,   120,    -1,    -1,   123,   124,
-      -1,    35,    36,    -1,    -1,    -1,    -1,    -1,   133,    43,
-      44,    -1,    -1,    -1,   139,    -1,    -1,    51,    52,   144,
-      54,    -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,    -1,
-      64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,    -1,
-      -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,    -1,
-      -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    21,    22,
-      -1,    -1,    -1,   117,    -1,    -1,   120,    -1,    -1,   123,
-     124,    -1,    35,    36,    -1,    -1,    -1,    -1,    -1,   133,
-      43,    44,    -1,    -1,    -1,   139,    -1,    -1,    51,    52,
-     144,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,
-      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,    -1,    -1,    -1,    -1,     4,     5,     6,     7,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    21,    22,   117,    -1,    -1,   120,    -1,    -1,
-     123,   124,    -1,    -1,    -1,    -1,    35,    36,    -1,    -1,
-     133,   134,    -1,    -1,    43,    44,   139,    -1,    -1,    -1,
-      -1,   144,    51,    52,    -1,    54,    -1,    -1,    -1,    -1,
-      59,    -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,
+       4,   142,    44,    12,   146,   119,   160,    11,    12,   138,
+       8,   275,     4,   208,     6,   247,    59,   146,     7,     7,
+       4,     5,     6,     7,     4,     4,     6,     6,    32,    72,
+      29,     8,    27,    37,    38,    39,    40,    41,    80,     8,
+       8,   189,    23,    24,    28,   231,   160,     4,    39,     6,
+      21,    22,   236,    69,    21,    22,    31,    99,   467,   201,
+      66,    50,    50,   387,    67,    78,     5,    29,    10,   121,
+     199,   135,   201,    21,    22,    29,    28,   231,    61,    30,
+      86,    65,   146,   121,   493,    60,   200,    21,    22,    21,
+      22,    10,    11,    12,    13,    14,    15,    16,    17,    18,
+      19,   105,    21,    22,   140,    71,    58,   143,   121,   118,
+      94,    95,   154,   117,   118,     0,   120,   231,   122,   443,
+     162,    36,   114,    37,   106,    61,    40,    41,   276,   145,
+     112,   134,   136,   137,   176,   321,   127,    71,   142,   128,
+     128,   325,   126,   186,   273,   122,   145,   145,   380,   144,
+      92,   134,   144,   144,   283,   284,   160,   114,    77,   201,
+     144,   165,   426,   125,   144,   144,   147,   321,   172,   364,
+     212,   175,   404,   215,   145,   407,   145,   145,   145,   118,
+     119,   295,   185,   137,    10,   189,   134,   191,   192,   193,
+     122,   386,   387,    53,    46,    47,   338,    45,   134,    83,
+     341,    74,   116,    71,   436,   247,   121,   321,     3,   338,
+     351,   253,   115,     4,    48,     6,    76,     0,   146,    45,
+     114,     4,    70,    17,    18,    19,   145,   231,   232,    55,
+      90,    91,   236,   106,    72,     4,    33,     6,   132,   144,
+     161,    79,    33,    26,   114,   399,    80,   251,   443,    87,
+     144,   255,   294,   113,   296,   103,   177,   125,    83,   140,
+     302,   142,   104,   405,   378,   137,    92,   309,   272,   273,
+      53,   109,   276,   414,   415,     5,   405,   103,   138,    95,
+     114,   129,   108,    77,   135,   399,   144,    72,   430,    46,
+      47,   420,   334,     9,    79,    21,    22,   144,     9,    84,
+      82,   430,    87,   129,   313,    21,    22,   145,    34,   313,
+      21,    22,   145,   146,   145,   146,     7,   321,   145,   146,
+       7,   325,    21,    22,   109,   145,   146,     4,     5,     6,
+       7,    57,     4,   144,     6,   377,     5,   341,   380,   343,
+     482,   145,   146,   144,    21,    22,    23,   351,   390,    21,
+      22,   144,     5,   482,    82,   145,   146,   146,    35,    36,
+      98,    33,   404,   367,   144,   407,    43,    44,   145,   146,
+     145,   146,   145,   146,    51,    52,    71,    54,   145,   146,
+       7,    58,    59,   145,   146,   145,   146,    64,   144,   144,
+      67,   145,   146,   534,   436,   399,   438,   145,   146,   145,
+     146,   542,   145,   146,    81,    82,     1,     7,     3,     7,
+     414,   415,    89,   144,   456,    92,   420,    10,    11,    12,
+      13,    14,    15,    16,    17,    18,    19,    11,    21,    22,
+     107,   145,   146,   475,     8,   145,   146,    32,   145,   146,
+     117,   483,     9,   120,   145,   146,   123,   124,    20,   146,
+      62,   145,    71,    48,    49,   145,   133,   146,    73,    99,
+     145,    56,   139,   467,   144,    60,   508,   144,   145,    85,
+     474,    85,   131,   144,     4,     5,     6,     7,   144,   144,
+      10,   132,    62,     7,    77,    80,    97,    97,    97,   493,
+      84,    21,    22,     7,     7,    42,    75,    42,   145,    96,
+     504,   144,   144,   137,    27,    35,    36,   146,   144,    92,
+     105,   144,   516,    43,    44,    85,    92,    42,   144,   114,
+     101,    51,    52,   101,    54,    96,   144,    38,     7,    59,
+     534,    99,    95,    63,    64,   130,   144,    67,   542,   146,
+      42,    19,   137,   144,    88,   140,    42,     7,   136,   145,
+     108,    81,    82,    93,   146,   145,   102,   110,     9,    89,
+     102,    33,    92,    68,    68,     4,     5,     6,     7,   144,
+       7,    10,   110,   100,   198,   144,   251,   107,   144,   343,
+     145,    55,    21,    22,   114,   118,   370,   117,   124,    56,
+     120,   399,   483,   123,   124,   506,    35,    36,   162,   544,
+     508,   283,   153,   133,    43,    44,   284,   166,   199,   139,
+     171,   213,    51,    52,   144,    54,   115,   148,    -1,    -1,
+      59,    -1,    -1,    -1,    63,    64,   213,    -1,    67,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,
       89,    -1,    -1,    92,    -1,    -1,     4,     5,     6,     7,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,
-      -1,    -1,    -1,    21,    22,   114,    -1,    -1,   117,    -1,
+      -1,    -1,    -1,    21,    22,    23,    -1,    -1,   117,    -1,
       -1,   120,    -1,    -1,   123,   124,    -1,    35,    36,    -1,
       -1,    -1,    -1,    -1,   133,    43,    44,    -1,    -1,    -1,
      139,    -1,    -1,    51,    52,   144,    54,    -1,    -1,    -1,
-      -1,    59,    -1,    -1,    -1,    63,    64,    -1,    -1,    67,
+      -1,    59,    -1,    -1,    -1,    -1,    64,    -1,    -1,    67,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,
       -1,    89,    -1,    -1,    92,    -1,    -1,     4,     5,     6,
@@ -1344,29 +1310,56 @@ static const yytype_int16 yycheck[] =
       -1,    -1,    59,    -1,    -1,    -1,    -1,    64,    -1,    -1,
       67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,
-      -1,    -1,    89,    -1,    -1,    92,    -1,    -1,     4,     5,
-       6,     7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-     107,    -1,    -1,    -1,    -1,    21,    22,    -1,    -1,    -1,
-     117,    -1,    -1,   120,    -1,    -1,   123,   124,    -1,    35,
-      36,    -1,    -1,    -1,    -1,    -1,   133,    43,    44,    -1,
-      -1,    -1,   139,    -1,    -1,    51,    52,   144,    54,    -1,
-      -1,    -1,    -1,    59,    -1,    -1,    -1,    -1,    64,    -1,
-      -1,    67,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    81,    82,     5,    -1,     7,
-      -1,    -1,    -1,    89,    -1,    -1,    92,    -1,    -1,    -1,
-      -1,    -1,    -1,    21,    22,    -1,    35,    36,    -1,    -1,
-      -1,   107,    -1,    -1,    -1,    44,    -1,    35,    36,    -1,
-      -1,   117,    51,    52,   120,    54,    44,   123,   124,    -1,
-      59,    -1,    -1,    51,    52,    -1,    54,   133,    67,    -1,
-      -1,    59,    -1,   139,    -1,    -1,    -1,    -1,   144,    67,
-      -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,
-      89,    -1,    -1,    81,    82,    -1,    -1,    -1,    -1,    -1,
-      -1,    89,    -1,    -1,    92,    -1,    -1,    -1,   107,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   117,   107,
-      -1,    -1,    -1,    -1,   123,   124,    -1,    -1,    -1,   117,
-      -1,    -1,    -1,    -1,   133,   123,   124,    -1,    -1,    -1,
-     139,    -1,    -1,    -1,    -1,   133,    -1,    -1,    -1,    -1,
-      -1,   139
+      -1,    -1,    89,    -1,    -1,    92,    -1,    -1,    -1,    -1,
+      -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
+     107,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    21,    22,
+     117,    -1,    -1,   120,    -1,    -1,   123,   124,    -1,    -1,
+      -1,    -1,    35,    36,    -1,    -1,   133,   134,    -1,    -1,
+      43,    44,   139,    -1,    -1,    -1,    -1,   144,    51,    52,
+      -1,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,
+      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
+      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
+      -1,    -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    21,
+      22,   114,    -1,    -1,   117,    -1,    -1,   120,    -1,    -1,
+     123,   124,    -1,    35,    36,    -1,    -1,    -1,    -1,    -1,
+     133,    43,    44,    -1,    -1,    -1,   139,    -1,    -1,    51,
+      52,   144,    54,    -1,    -1,    -1,    -1,    59,    -1,    -1,
+      -1,    63,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,
+      82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,
+      92,    -1,    -1,     4,     5,     6,     7,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,
+      21,    22,    -1,    -1,    -1,   117,    -1,    -1,   120,    -1,
+      -1,   123,   124,    -1,    35,    36,    -1,    -1,    -1,    -1,
+      -1,   133,    43,    44,    -1,    -1,    -1,   139,    -1,    -1,
+      51,    52,   144,    54,    -1,    -1,    -1,    -1,    59,    -1,
+      -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,
+      -1,    92,    -1,    -1,     4,     5,     6,     7,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,
+      -1,    21,    22,    -1,    -1,    -1,   117,    -1,    -1,   120,
+      -1,    -1,   123,   124,    -1,    35,    36,    -1,    -1,    -1,
+      -1,    -1,   133,    43,    44,    -1,    -1,    -1,   139,    -1,
+      -1,    51,    52,   144,    54,    -1,    -1,    -1,    -1,    59,
+      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    81,    82,     5,    -1,     7,    -1,    -1,    -1,    89,
+      -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,    21,
+      22,    -1,    35,    36,    -1,    -1,    -1,   107,    -1,    -1,
+      -1,    44,    -1,    35,    36,    -1,    -1,   117,    51,    52,
+     120,    54,    44,   123,   124,    -1,    59,    -1,    -1,    51,
+      52,    -1,    54,   133,    67,    -1,    -1,    59,    -1,   139,
+      -1,    -1,    -1,    -1,   144,    67,    -1,    -1,    81,    82,
+      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    81,
+      82,    -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,
+      92,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,   117,   107,    -1,    -1,    -1,    -1,
+     123,   124,    -1,    -1,    -1,   117,    -1,    -1,    -1,    -1,
+     133,   123,   124,    -1,    -1,    -1,   139,    -1,    -1,    -1,
+      -1,   133,    -1,    -1,    -1,    -1,    -1,   139
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
@@ -1376,58 +1369,59 @@ static const yytype_uint16 yystos[] =
        0,     1,     3,    32,    48,    49,    56,    60,    80,   105,
      114,   130,   137,   140,   149,   150,   151,   152,   153,   154,
      155,   177,   178,   181,   182,   185,   187,   190,   191,   192,
-     256,   257,   121,     4,     6,   190,   254,    78,   121,    71,
-     121,    83,    28,    58,   193,   254,   188,   189,   204,   254,
+     257,   258,   121,     4,     6,   190,   255,    78,   121,    71,
+     121,    83,    28,    58,   193,   255,   188,   189,   204,   255,
        0,   140,   142,    48,    80,   190,    29,   137,   186,    30,
-     140,   143,     3,   254,   125,   179,    71,   179,   254,   254,
-     254,   254,   254,     5,     7,    21,    22,    23,    35,    36,
+     140,   143,     3,   255,   125,   179,    71,   179,   255,   255,
+     255,   255,   255,     5,     7,    21,    22,    23,    35,    36,
       43,    44,    51,    52,    54,    59,    64,    67,    81,    82,
       89,    92,   107,   117,   120,   123,   124,   133,   139,   144,
      158,   194,   195,   196,   198,   230,   231,   232,   233,   234,
-     235,   236,   237,   244,   247,   250,   254,   115,   146,    33,
+     235,   236,   237,   244,   248,   251,   255,   115,   146,    33,
      144,   190,    83,   186,   193,   104,   192,    31,    60,     5,
      118,   119,   137,   180,     5,   180,    95,   144,   135,   224,
      225,   132,   144,   192,     7,     7,   134,   230,   240,   241,
      144,    82,   144,     5,   144,   144,    82,   190,   230,     5,
-      71,   197,   146,    21,    22,    33,   251,   254,    23,    24,
-     147,   252,    98,    22,   233,    27,   144,   183,   184,   254,
-     189,   144,   198,   253,   254,   179,   254,   191,     7,    46,
-      47,    46,    47,   144,   180,   254,   156,   157,   254,    10,
-      63,   144,   226,   227,   228,   229,   230,   247,   144,   253,
-     226,   134,   238,   239,    61,   241,   242,     7,    53,    76,
-      90,    91,   113,   138,   245,   245,   230,     7,   145,   145,
-     144,   198,   201,   202,   205,   234,   254,   224,   196,   254,
-     231,   232,   144,   254,   254,    23,    58,   145,   230,   243,
-     146,   224,    11,   145,   146,   180,   192,   156,    45,    70,
-     103,   129,   162,   254,   254,   169,   170,   171,   172,   173,
-     174,   254,   144,   165,   145,   146,    82,   158,   229,   198,
-     226,   230,     8,     9,    10,    11,    12,    13,    14,    15,
-      16,    17,    18,    19,    77,   249,    20,   244,   246,   145,
-     122,   230,   239,   242,   230,    62,   145,    71,    71,   145,
-     201,    33,   203,   204,    72,    79,    87,   109,   200,   146,
-     203,    39,   127,   199,    73,   206,    99,   213,   214,   145,
-     243,   145,   146,   184,   230,   254,   144,    85,    85,   144,
-     145,   146,     5,     7,    28,    65,    94,    95,   126,   144,
-     254,   255,   247,   248,   254,   131,   162,   163,   164,   156,
-      10,    45,    55,    92,   103,   108,   129,   159,   160,   161,
-     145,   227,   228,    17,    18,    19,    77,   230,   144,   198,
-     230,    10,    92,   145,   146,   132,   230,   122,    62,   230,
-       7,   145,   204,    97,    97,    97,    84,   201,     7,     7,
-     203,    42,    75,   207,    42,   145,    96,   215,   145,   230,
-     226,   144,   144,   253,   170,   253,   145,   146,    37,    40,
-      41,   116,   175,   146,   137,   166,    92,   144,   244,    85,
-     254,   159,   230,   144,   198,     9,   243,    92,   244,   144,
-     230,   145,    69,   145,   202,   101,   101,   243,   226,    96,
-     208,   243,    42,   106,   112,   216,   217,   145,   253,   253,
-     145,   145,   247,   144,   176,   162,    38,    99,   167,   226,
-     144,     9,   243,   230,   145,   246,     7,    95,    42,    88,
-     209,   220,   221,   230,    19,   145,   145,   169,   144,    42,
-     145,   254,   230,   145,   145,   145,   226,   220,     7,   136,
-     210,   211,   212,   146,    34,    57,   222,     7,    50,   128,
-     218,   108,   145,   169,    74,   106,   168,   145,   254,   212,
-     221,    93,   223,   102,   110,   102,     9,   254,   145,   144,
-      33,    66,    86,     7,    50,   128,   219,   144,   253,   144,
-      68,   110,    68,   253,   145,   213,   145,   100,   145,     7
+      71,   197,   146,    21,    22,    33,   252,   255,    23,    24,
+     147,   253,    98,    22,   233,    27,   144,   183,   184,   255,
+     189,   144,   198,   254,   255,   179,   255,   191,     7,    46,
+      47,    46,    47,   144,   180,   255,   156,   157,   255,    10,
+      63,   144,   226,   227,   228,   229,   230,   248,   144,   247,
+     254,   226,   134,   238,   239,    61,   241,   242,     7,    53,
+      76,    90,    91,   113,   138,   245,   245,   230,     7,   145,
+     145,   144,   198,   201,   202,   205,   234,   255,   224,   196,
+     255,   231,   232,   144,   255,   255,    23,    58,   145,   230,
+     243,   146,   224,    11,   145,   146,   180,   192,   156,    45,
+      70,   103,   129,   162,   255,   255,   169,   170,   171,   172,
+     173,   174,   255,   144,   165,   145,   146,    82,   158,   229,
+     198,   226,   230,     8,     9,    10,    11,    12,    13,    14,
+      15,    16,    17,    18,    19,    77,   250,    20,   244,   246,
+     146,   145,   122,   230,   239,   242,   230,    62,   145,    71,
+      71,   145,   201,    33,   203,   204,    72,    79,    87,   109,
+     200,   146,   203,    39,   127,   199,    73,   206,    99,   213,
+     214,   145,   243,   145,   146,   184,   230,   255,   144,    85,
+      85,   144,   145,   146,     5,     7,    28,    65,    94,    95,
+     126,   144,   255,   256,   248,   249,   255,   131,   162,   163,
+     164,   156,    10,    45,    55,    92,   103,   108,   129,   159,
+     160,   161,   145,   227,   228,    17,    18,    19,    77,   230,
+     144,   198,   230,    10,    92,   145,   146,   144,   132,   230,
+     122,    62,   230,     7,   145,   204,    97,    97,    97,    84,
+     201,     7,     7,   203,    42,    75,   207,    42,   145,    96,
+     215,   145,   230,   226,   144,   144,   254,   170,   254,   145,
+     146,    37,    40,    41,   116,   175,   146,   137,   166,    92,
+     144,   244,    85,   255,   159,   230,   144,   198,     9,   243,
+      92,   244,   246,   144,   230,   145,    69,   145,   202,   101,
+     101,   243,   226,    96,   208,   243,    42,   106,   112,   216,
+     217,   145,   254,   254,   145,   145,   248,   144,   176,   162,
+      38,    99,   167,   226,   144,     9,   243,   230,   145,   145,
+     246,     7,    95,    42,    88,   209,   220,   221,   230,    19,
+     145,   145,   169,   144,    42,   145,   255,   230,   145,   145,
+     145,   226,   220,     7,   136,   210,   211,   212,   146,    34,
+      57,   222,     7,    50,   128,   218,   108,   145,   169,    74,
+     106,   168,   145,   255,   212,   221,    93,   223,   102,   110,
+     102,     9,   255,   145,   144,    33,    66,    86,     7,    50,
+     128,   219,   144,   254,   144,    68,   110,    68,   254,   145,
+     213,   145,   100,   145,     7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
@@ -1460,9 +1454,10 @@ static const yytype_uint16 yyr1[] =
      237,   237,   238,   238,   239,   240,   240,   241,   242,   242,
      243,   243,   244,   244,   244,   244,   244,   244,   244,   244,
      245,   245,   245,   245,   245,   245,   246,   246,   247,   247,
-     248,   248,   249,   249,   249,   249,   249,   249,   249,   249,
-     249,   249,   250,   251,   251,   252,   252,   252,   253,   253,
-     254,   254,   255,   255,   255,   255,   256,   257,   257
+     248,   248,   249,   249,   250,   250,   250,   250,   250,   250,
+     250,   250,   250,   250,   251,   252,   252,   253,   253,   253,
+     254,   254,   255,   255,   256,   256,   256,   256,   257,   258,
+     258
 };
 
   /* YYR2[YYN] -- Number of symbols on the right hand side of rule YYN.  */
@@ -1477,7 +1472,7 @@ static const yytype_uint8 yyr2[] =
       10,     4,     3,     1,     0,     1,     0,     3,     0,     5,
        0,     8,     1,     1,     1,     3,     1,     1,     1,     1,
        2,     2,     2,     4,     2,     2,     1,     1,     1,     1,
-       0,     3,    10,     7,     4,     5,     5,     4,     4,     5,
+       0,     3,    10,     5,     4,     5,     5,     4,     4,     5,
        2,     2,     2,     0,     4,     5,     4,     3,     1,     3,
        2,     3,     0,     3,     2,     1,     3,     3,     4,     1,
        3,     1,    10,     0,     1,     1,     1,     1,     1,     3,
@@ -1494,10 +1489,11 @@ static const yytype_uint8 yyr2[] =
        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
+       1,     1,     1,     1,     1,     1,     1,     3,     3,     5,
+       1,     3,     1,     3,     1,     1,     1,     1,     1,     1,
+       1,     2,     1,     2,     1,     1,     1,     1,     1,     1,
+       1,     3,     1,     1,     1,     1,     1,     1,     2,     2,
+       0
 };
 
 
@@ -1994,1071 +1990,1081 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
   switch (yytype)
     {
           case 3: /* TOKEN_COMMAND  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2004 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2000 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 4: /* TOKEN_NAME  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2014 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2010 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 5: /* TOKEN_STRING_SINGLE_QUOTED  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2024 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2020 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 6: /* TOKEN_STRING_DOUBLE_QUOTED  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2034 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2030 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 7: /* TOKEN_UNSIGNED_NUMVAL  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).numeric_literal_value_) != nullptr) {
     delete ((*yyvaluep).numeric_literal_value_);
   }
 }
-#line 2044 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2040 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 150: /* sql_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 2054 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2050 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 151: /* quit_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).quit_statement_) != nullptr) {
     delete ((*yyvaluep).quit_statement_);
   }
 }
-#line 2064 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2060 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 152: /* alter_table_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 2074 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2070 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 153: /* create_table_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).create_table_statement_) != nullptr) {
     delete ((*yyvaluep).create_table_statement_);
   }
 }
-#line 2084 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2080 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 154: /* create_index_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 2094 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2090 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 155: /* drop_table_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).drop_table_statement_) != nullptr) {
     delete ((*yyvaluep).drop_table_statement_);
   }
 }
-#line 2104 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2100 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 156: /* column_def  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_definition_) != nullptr) {
     delete ((*yyvaluep).attribute_definition_);
   }
 }
-#line 2114 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2110 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 157: /* column_def_commalist  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_definition_list_) != nullptr) {
     delete ((*yyvaluep).attribute_definition_list_);
   }
 }
-#line 2124 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2120 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 158: /* data_type  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).data_type_) != nullptr) {
     delete ((*yyvaluep).data_type_);
   }
 }
-#line 2134 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2130 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 159: /* column_constraint_def  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_) != nullptr) {
     delete ((*yyvaluep).column_constraint_);
   }
 }
-#line 2144 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2140 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 160: /* column_constraint_def_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_list_) != nullptr) {
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 2154 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2150 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 161: /* opt_column_constraint_def_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_list_) != nullptr) {
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 2164 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2160 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 165: /* opt_column_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_list_) != nullptr) {
     delete ((*yyvaluep).attribute_list_);
   }
 }
-#line 2174 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2170 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 166: /* opt_block_properties  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).block_properties_) != nullptr) {
     delete ((*yyvaluep).block_properties_);
   }
 }
-#line 2184 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2180 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 167: /* opt_partition_clause  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).partition_clause_) != nullptr) {
     delete ((*yyvaluep).partition_clause_);
   }
 }
-#line 2194 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2190 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 168: /* partition_type  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2204 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2200 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 169: /* key_value_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2214 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2210 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 170: /* key_value  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_) != nullptr) {
     delete ((*yyvaluep).key_value_);
   }
 }
-#line 2224 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2220 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 171: /* key_string_value  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_string_value_) != nullptr) {
     delete ((*yyvaluep).key_string_value_);
   }
 }
-#line 2234 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2230 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 172: /* key_string_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_string_list_) != nullptr) {
     delete ((*yyvaluep).key_string_list_);
   }
 }
-#line 2244 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2240 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 173: /* key_integer_value  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_integer_value_) != nullptr) {
     delete ((*yyvaluep).key_integer_value_);
   }
 }
-#line 2254 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2250 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 174: /* key_bool_value  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_bool_value_) != nullptr) {
     delete ((*yyvaluep).key_bool_value_);
   }
 }
-#line 2264 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2260 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 175: /* index_type  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2274 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2270 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 176: /* opt_index_properties  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2284 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2280 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 177: /* insert_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).insert_statement_) != nullptr) {
     delete ((*yyvaluep).insert_statement_);
   }
 }
-#line 2294 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2290 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 178: /* copy_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).copy_statement_) != nullptr) {
     delete ((*yyvaluep).copy_statement_);
   }
 }
-#line 2304 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2300 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 179: /* copy_to_target  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2314 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2310 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 180: /* opt_copy_params  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2324 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2320 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 181: /* update_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).update_statement_) != nullptr) {
     delete ((*yyvaluep).update_statement_);
   }
 }
-#line 2334 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2330 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 182: /* delete_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).delete_statement_) != nullptr) {
     delete ((*yyvaluep).delete_statement_);
   }
 }
-#line 2344 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2340 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 183: /* assignment_list  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).assignment_list_) != nullptr) {
     delete ((*yyvaluep).assignment_list_);
   }
 }
-#line 2354 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2350 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 184: /* assignment_item  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).assignment_) != nullptr) {
     delete ((*yyvaluep).assignment_);
   }
 }
-#line 2364 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2360 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 185: /* set_operation_statement  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).set_operation_statement_) != nullptr) {
     delete ((*yyvaluep).set_operation_statement_);
   }
 }
-#line 2374 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2370 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 186: /* opt_priority_clause  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).opt_priority_clause_) != nullptr) {
     delete ((*yyvaluep).opt_priority_clause_);
   }
 }
-#line 2384 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2380 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 187: /* with_clause  */
-#line 631 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 635 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).with_list_) != nullptr) {
     delete ((*yyvalu

<TRUNCATED>


[03/27] incubator-quickstep git commit: Fix a bug in LineReader for recognizing command

Posted by ji...@apache.org.
Fix a bug in LineReader for recognizing command


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

Branch: refs/heads/trace
Commit: 77960a42dcfb3d27de5601548a04d81a6be79375
Parents: 71aa8d2
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Tue Sep 19 22:02:02 2017 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Tue Sep 19 23:00:20 2017 -0500

----------------------------------------------------------------------
 cli/CMakeLists.txt                 |  2 ++
 cli/LineReader.cpp                 | 11 +++++++++--
 cli/tests/command_executor/D.test  | 23 +++++++++++++++++++++++
 cli/tests/command_executor/Dt.test |  4 ++++
 4 files changed, 38 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/77960a42/cli/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/CMakeLists.txt b/cli/CMakeLists.txt
index 33d10e3..03c5408 100644
--- a/cli/CMakeLists.txt
+++ b/cli/CMakeLists.txt
@@ -189,10 +189,12 @@ if(QUICKSTEP_HAVE_LIBNUMA)
 endif()
 if(USE_LINENOISE)
   target_link_libraries(quickstep_cli_LineReader
+                        glog
                         linenoise
                         quickstep_utility_Macros)
 else()
   target_link_libraries(quickstep_cli_LineReader
+                        glog
                         quickstep_utility_Macros)
 endif()
 target_link_libraries(quickstep_cli_LineReaderBuffered

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/77960a42/cli/LineReader.cpp
----------------------------------------------------------------------
diff --git a/cli/LineReader.cpp b/cli/LineReader.cpp
index 002727d..1a23dd3 100644
--- a/cli/LineReader.cpp
+++ b/cli/LineReader.cpp
@@ -23,6 +23,8 @@
 #include <cctype>
 #include <string>
 
+#include "glog/logging.h"
+
 using std::ispunct;
 using std::size_t;
 using std::string;
@@ -171,7 +173,7 @@ std::string LineReader::getNextCommand() {
             case '.':
             case '\\':  //  Fall Through.
               // If the dot or forward slash begins the line, begin a command search.
-              if (scan_position == 0) {
+              if (special_char_location == multiline_buffer.find_first_not_of(" \t\r\n")) {
                 line_state = kCommand;
               } else {
                 // This is a regular character, so skip over it.
@@ -217,7 +219,12 @@ std::string LineReader::getNextCommand() {
             if (std::all_of(leftover_.begin(), leftover_.end(), ::isspace)) {
               leftover_.clear();
             }
-            return multiline_buffer.substr(0, special_char_location + 1);
+            // Skip all the whitespaces before the command.
+            const std::size_t start_position =
+                multiline_buffer.find_first_not_of(" \t\r\n");
+            DCHECK_LT(start_position, special_char_location + 1);
+            return multiline_buffer.substr(start_position,
+                                           special_char_location + 1 - start_position);
           }
           break;
       }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/77960a42/cli/tests/command_executor/D.test
----------------------------------------------------------------------
diff --git a/cli/tests/command_executor/D.test b/cli/tests/command_executor/D.test
index 36e9a92..c3564a6 100644
--- a/cli/tests/command_executor/D.test
+++ b/cli/tests/command_executor/D.test
@@ -58,6 +58,8 @@ INSERT INTO foo2 values(5, 1, 1.0, 1.0, 'XYZ');
 INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
 --
 ==
+
+
 \d foo
 --
  Table "foo"
@@ -69,6 +71,7 @@ INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
  col4   | Float  
  col5   | Char(5)
 ==
+
 \d foo2
 --
  Table "foo2"
@@ -80,6 +83,7 @@ INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
  col4                           | Float  
  averyverylongcolumnnamefortest | Char(5)
 ==
+
 \d foo3
 --
  Table "foo3"
@@ -93,6 +97,7 @@ INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
  Indexes
   "foo3_index_1" CSB_TREE (col1)
 ==
+
 \d foo4
 --
  Table "foo4"
@@ -107,6 +112,7 @@ INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
   "foo4_index_2" CSB_TREE (col3, col4)
   "foo4_index_1" CSB_TREE (col1, col2)
 ==
+
 \d foo_hash_part
 --
  Table "foo_hash_part"
@@ -118,6 +124,7 @@ INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
   PARTITION BY HASH ( col1 ) PARTITIONS 4
   | 1 | 1 | 1 | 1 |
 ==
+
 \d
 --
        List of relations
@@ -132,6 +139,22 @@ INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
  averylongtablenamethatseemstoneverend | table | 1      
 
 ==
+
+\d
+--
+       List of relations
+
+ Name                                  | Type  | Blocks 
++--------------------------------------+-------+---------+
+ foo                                   | table | 1      
+ foo2                                  | table | 1      
+ foo3                                  | table | 1      
+ foo4                                  | table | 0      
+ foo_hash_part                         | table | 4      
+ averylongtablenamethatseemstoneverend | table | 1      
+
+==
+
 \d invalidtable
 --
 ERROR:  Unrecognized relation invalidtable (1 : 4)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/77960a42/cli/tests/command_executor/Dt.test
----------------------------------------------------------------------
diff --git a/cli/tests/command_executor/Dt.test b/cli/tests/command_executor/Dt.test
index 8d81029..022cae6 100644
--- a/cli/tests/command_executor/Dt.test
+++ b/cli/tests/command_executor/Dt.test
@@ -50,6 +50,8 @@ INSERT INTO foo2 values(5, 1, 1.0, 1.0, 'XYZ');
 INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
 --
 ==
+
+
 \dt
 --
        List of relations
@@ -63,6 +65,7 @@ INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
  averylongtablenamethatseemstoneverend | table | 1      
 
 ==
+
 \dt foo
 --
        List of relations
@@ -72,6 +75,7 @@ INSERT INTO foo3 values(5, 1, 1.0, 1.0, 'XYZZ');
  foo    | table | 1      
 
 ==
+
 \dt invalidtable
 --
 ERROR:  Unrecognized relation invalidtable (1 : 5)


[02/27] incubator-quickstep git commit: Prune columns after partition rule.

Posted by ji...@apache.org.
Prune columns after partition rule.


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

Branch: refs/heads/trace
Commit: 71aa8d265cc2240da0ebf8275a70884002a1ea45
Parents: 475704e
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Tue Sep 19 19:17:29 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Tue Sep 19 19:17:29 2017 -0500

----------------------------------------------------------------------
 query_optimizer/PhysicalGenerator.cpp | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/71aa8d26/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 6932b30..865cd11 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -173,6 +173,7 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   // set output PartitionSchemeHeader in a Physical Plan node, when needed.
   if (FLAGS_use_partition_rule) {
     rules.push_back(std::make_unique<Partition>(optimizer_context_));
+    rules.push_back(std::make_unique<PruneColumns>());
   }
 
   // NOTE(jianqiao): Adding rules after InjectJoinFilters (or AttachLIPFilters)


[15/27] incubator-quickstep git commit: Relax the sort requirement in columnstore.

Posted by ji...@apache.org.
Relax the sort requirement in columnstore.


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

Branch: refs/heads/trace
Commit: ffb8e055a890a9002235a8516e5f2dece2d6228a
Parents: 69fd94b
Author: Zuyu Zhang <zu...@cs.wisc.edu>
Authored: Mon Oct 9 11:23:08 2017 -0500
Committer: Zuyu Zhang <zu...@cs.wisc.edu>
Committed: Tue Oct 10 14:10:03 2017 -0500

----------------------------------------------------------------------
 parser/CMakeLists.txt                      |  1 +
 parser/ParseBlockProperties.hpp            | 10 ++++--
 parser/tests/Create.test                   | 45 +++++++++++++++++++++++++
 query_optimizer/OptimizerTree.hpp          |  9 +++--
 query_optimizer/resolver/Resolver.cpp      | 15 +++++----
 query_optimizer/tests/resolver/Create.test | 38 +++++++++++++++++----
 6 files changed, 99 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ffb8e055/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index b3ddf30..d4aaab4 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -150,6 +150,7 @@ target_link_libraries(quickstep_parser_ParseBlockProperties
                       quickstep_parser_ParseTreeNode
                       quickstep_utility_Macros
                       quickstep_utility_PtrList
+                      quickstep_utility_SqlError
                       quickstep_utility_StringUtil)
 target_link_libraries(quickstep_parser_ParseCaseExpressions
                       quickstep_parser_ParseExpression

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ffb8e055/parser/ParseBlockProperties.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseBlockProperties.hpp b/parser/ParseBlockProperties.hpp
index ce0cd92..fa176b1 100644
--- a/parser/ParseBlockProperties.hpp
+++ b/parser/ParseBlockProperties.hpp
@@ -31,6 +31,7 @@
 #include "parser/ParseTreeNode.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrList.hpp"
+#include "utility/SqlError.hpp"
 #include "utility/StringUtil.hpp"
 
 #include "glog/logging.h"
@@ -143,10 +144,13 @@ class ParseBlockProperties : public ParseTreeNode {
     if (sort_key_value == nullptr) {
       return nullptr;
     }
-    if (sort_key_value->getKeyValueType() !=
-        ParseKeyValue::KeyValueType::kStringString) {
-      return nullptr;
+    if (sort_key_value->getKeyValueType() ==
+        ParseKeyValue::KeyValueType::kStringStringList) {
+      THROW_SQL_ERROR_AT(sort_key_value)
+          << "The SORT property must be a string, not a string list.";
     }
+
+    DCHECK(sort_key_value->getKeyValueType() == ParseKeyValue::KeyValueType::kStringString);
     return static_cast<const ParseKeyStringValue*>(sort_key_value)->value();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ffb8e055/parser/tests/Create.test
----------------------------------------------------------------------
diff --git a/parser/tests/Create.test b/parser/tests/Create.test
index 8c11054..3923c13 100644
--- a/parser/tests/Create.test
+++ b/parser/tests/Create.test
@@ -259,6 +259,51 @@ CreateTableStatement[relation_name=test]
       +-value=String[value=rowstore]
 ==
 
+CREATE TABLE test (attr INT, attr2 INT) WITH BLOCKPROPERTIES
+(TYPE columnstore)
+--
+CreateTableStatement[relation_name=test]
++-attribute_list=
+| +-AttributeDefinition[name=attr,type=Int]
+| +-AttributeDefinition[name=attr2,type=Int]
++-block_properties=
+  +-BlockProperties
+    +-block_property=KeyStringValue[key=TYPE]
+      +-value=String[value=columnstore]
+==
+
+CREATE TABLE test (attr INT, attr2 INT) WITH BLOCKPROPERTIES
+(TYPE columnstore, SORT attr)
+--
+CreateTableStatement[relation_name=test]
++-attribute_list=
+| +-AttributeDefinition[name=attr,type=Int]
+| +-AttributeDefinition[name=attr2,type=Int]
++-block_properties=
+  +-BlockProperties
+    +-block_property=KeyStringValue[key=TYPE]
+    | +-value=String[value=columnstore]
+    +-block_property=KeyStringValue[key=SORT]
+      +-value=String[value=attr]
+==
+
+CREATE TABLE test (attr INT, attr2 INT) WITH BLOCKPROPERTIES
+(TYPE columnstore, SORT (attr, attr2))
+--
+CreateTableStatement[relation_name=test]
++-attribute_list=
+| +-AttributeDefinition[name=attr,type=Int]
+| +-AttributeDefinition[name=attr2,type=Int]
++-block_properties=
+  +-BlockProperties
+    +-block_property=KeyStringValue[key=TYPE]
+    | +-value=String[value=columnstore]
+    +-block_property=KeyStringList[key=SORT]
+      +-value_list=
+        +-String[value=attr]
+        +-String[value=attr2]
+==
+
 CREATE TABLE test (attr INT) WITH BLOCKPROPERTIES
 (TYPE compressed_columnstore, SORT attr, COMPRESS ALL)
 --

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ffb8e055/query_optimizer/OptimizerTree.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/OptimizerTree.hpp b/query_optimizer/OptimizerTree.hpp
index c54ce20..0f4713e 100644
--- a/query_optimizer/OptimizerTree.hpp
+++ b/query_optimizer/OptimizerTree.hpp
@@ -240,9 +240,12 @@ OptimizerProtoRepresentation<TreeNodeType>* getOptimizerRepresentationForProto(
     }
     case TupleStorageSubBlockDescription::BASIC_COLUMN_STORE: {
       node->addProperty("blocktype", "columnstore");
-      node->addProperty("sort",
-          storage_block_description.GetExtension(
-              quickstep::BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id));
+      if (storage_block_description.HasExtension(
+              quickstep::BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id)) {
+        node->addProperty("sort",
+            storage_block_description.GetExtension(
+                quickstep::BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id));
+      }
       break;
     }
     case TupleStorageSubBlockDescription::COMPRESSED_COLUMN_STORE: {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ffb8e055/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 935e235..2991568 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -687,7 +687,7 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
 
   // Resolve TYPE property.
   // The type of the block will determine these:
-  bool block_requires_sort = false;
+  bool block_allows_sort = false;
   bool block_requires_compress = false;
 
   const ParseString *type_parse_string = block_properties->getType();
@@ -702,7 +702,8 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
   } else if (type_string.compare("columnstore") == 0) {
     description->set_sub_block_type(
         quickstep::TupleStorageSubBlockDescription::BASIC_COLUMN_STORE);
-    block_requires_sort = true;
+    // NOTE(zuyu): sort is optional.
+    block_allows_sort = true;
   } else if (type_string.compare("compressed_rowstore") == 0) {
     description->set_sub_block_type(
         quickstep::TupleStorageSubBlockDescription::COMPRESSED_PACKED_ROW_STORE);
@@ -710,7 +711,7 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
   } else if (type_string.compare("compressed_columnstore") == 0) {
     description->set_sub_block_type(
         quickstep::TupleStorageSubBlockDescription::COMPRESSED_COLUMN_STORE);
-    block_requires_sort = true;
+    block_allows_sort = true;
     block_requires_compress = true;
   } else {
     THROW_SQL_ERROR_AT(type_parse_string) << "Unrecognized storage type.";
@@ -718,10 +719,12 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
 
   // Resolve the SORT property.
   const ParseString *sort_parse_string = block_properties->getSort();
-  if (block_requires_sort) {
+  if (block_allows_sort) {
     if (sort_parse_string == nullptr) {
-      THROW_SQL_ERROR_AT(type_parse_string)
-          << "The SORT property must be specified as an attribute name.";
+      if (description->sub_block_type() != TupleStorageSubBlockDescription::BASIC_COLUMN_STORE) {
+        THROW_SQL_ERROR_AT(type_parse_string)
+            << "The SORT property must be specified as an attribute name.";
+      }
     } else {
       // Lookup the name and map to a column id.
       const attribute_id sort_id = GetAttributeIdFromName(create_table_statement.attribute_definition_list(),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ffb8e055/query_optimizer/tests/resolver/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Create.test b/query_optimizer/tests/resolver/Create.test
index c216c85..ed9158a 100644
--- a/query_optimizer/tests/resolver/Create.test
+++ b/query_optimizer/tests/resolver/Create.test
@@ -133,13 +133,37 @@ ERROR: The SORT property does not apply to this block type. (2 : 28)
                            ^
 ==
 
-# Columnstores require a sort attribute.
+# Columnstores do not require a sort attribute.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
 (TYPE columnstore);
 --
-ERROR: The SORT property must be specified as an attribute name. (2 : 7)
-(TYPE columnstore);
-      ^
+TopLevelPlan
++-plan=CreateTable[relation=foo]
+| +-block_properties=ProtoDescription
+| | +-Property=ProtoProperty[Property=blocktype,Value=columnstore]
+| | +-Property=ProtoProperty[Property=slots,Value=1]
+| +-attributes=
+|   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
+==
+
+# Columnstores have a optional sort attribute.
+CREATE TABLE foo (attr INT, attr2 INT) WITH BLOCKPROPERTIES
+(TYPE columnstore, SORT attr2);
+--
+TopLevelPlan
++-plan=CreateTable[relation=foo]
+| +-block_properties=ProtoDescription
+| | +-Property=ProtoProperty[Property=blocktype,Value=columnstore]
+| | +-Property=ProtoProperty[Property=sort,Value=1]
+| | +-Property=ProtoProperty[Property=slots,Value=1]
+| +-attributes=
+|   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
+|   +-AttributeReference[id=1,name=attr2,relation=foo,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
+  +-AttributeReference[id=1,name=attr2,relation=foo,type=Int]
 ==
 
 # Non-existant columns should be caught.
@@ -155,9 +179,9 @@ ERROR: The SORT property did not match any attribute name. (2 : 25)
 CREATE TABLE foo (attr INT, attr2 INT) WITH BLOCKPROPERTIES
 (TYPE columnstore, SORT (attr, attr2));
 --
-ERROR: The SORT property must be specified as an attribute name. (2 : 7)
-(TYPE columnstore, SORT (attr, attr2...
-      ^
+ERROR: The SORT property must be a string, not a string list. (2 : 20)
+(TYPE columnstore, SORT (attr, attr2));
+                   ^
 ==
 # Compress should only be specified on compressed blocks.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES