You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by zu...@apache.org on 2016/05/05 06:16:37 UTC

[01/30] incubator-quickstep git commit: Removed optional bitweaving submodule (#182)

Repository: incubator-quickstep
Updated Branches:
  refs/heads/master cd93fd743 -> c0a02d007


Removed optional bitweaving submodule (#182)

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

Branch: refs/heads/master
Commit: 49e8b339ac59eb890593e9bd62a29595c631418c
Parents: 3ae0d1c
Author: Marc S <cr...@users.noreply.github.com>
Authored: Thu Apr 21 10:16:26 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:33 2016 -0700

----------------------------------------------------------------------
 .gitmodules | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/49e8b339/.gitmodules
----------------------------------------------------------------------
diff --git a/.gitmodules b/.gitmodules
index cf380b5..7671b11 100644
--- a/.gitmodules
+++ b/.gitmodules
@@ -7,6 +7,3 @@
 [submodule "third_party/googletest"]
 	path = third_party/googletest
 	url = https://github.com/google/googletest
-[submodule "storage/bitweaving"]
-	path = storage/bitweaving
-	url = https://github.com/UWQuickstep/bitweaving.git


[19/30] incubator-quickstep git commit: Refactored makeRoomForBlock. (#192)

Posted by zu...@apache.org.
Refactored makeRoomForBlock. (#192)

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

Branch: refs/heads/master
Commit: 85edc76cf2829383ec3bfe2a3f0b2ef3a7c11111
Parents: 30a1577
Author: Zuyu ZHANG <zu...@users.noreply.github.com>
Authored: Wed Apr 27 23:00:23 2016 -0700
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 storage/StorageManager.cpp | 89 ++++++++++++++++++++---------------------
 storage/StorageManager.hpp | 10 +++--
 2 files changed, 50 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/85edc76c/storage/StorageManager.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.cpp b/storage/StorageManager.cpp
index b98a28c..b990a22 100644
--- a/storage/StorageManager.cpp
+++ b/storage/StorageManager.cpp
@@ -397,7 +397,7 @@ void* StorageManager::allocateSlots(const std::size_t num_slots,
       = MAP_PRIVATE | MAP_ANONYMOUS | MAP_ALIGNED_SUPER;
 #endif
 
-  makeRoomForBlock(num_slots);
+  makeRoomForBlockOrBlob(num_slots);
   void *slots = nullptr;
 
 #if defined(QUICKSTEP_HAVE_MMAP_LINUX_HUGETLB) || defined(QUICKSTEP_HAVE_MMAP_BSD_SUPERPAGE)
@@ -568,65 +568,62 @@ MutableBlobReference StorageManager::getBlobInternal(const block_id blob,
   return ret;
 }
 
-void StorageManager::makeRoomForBlock(const size_t slots) {
+void StorageManager::makeRoomForBlockOrBlob(const size_t slots) {
+  block_id block_to_evict;
   while (total_memory_usage_ + slots > max_memory_usage_) {
-    block_id block_index;
-    EvictionPolicy::Status status = eviction_policy_->chooseBlockToEvict(&block_index);
-
-    if (status == EvictionPolicy::Status::kOk) {
-      bool has_collision = false;
-      SpinSharedMutexExclusiveLock<false> eviction_lock(*lock_manager_.get(block_index, &has_collision));
-      if (has_collision) {
-        // We have a collision in the shared lock manager, where some callers
-        // of this function (i.e., getBlockInternal or getBlobInternal) has
-        // acquired an exclusive lock, and we are trying to evict a block that
-        // hashes to the same location. This will cause a deadlock.
-
-        // For now simply treat this situation as the case where there is not
-        // enough memory and we temporarily go over the memory limit.
-        break;
-      }
+    const EvictionPolicy::Status status = eviction_policy_->chooseBlockToEvict(&block_to_evict);
+    if (status != EvictionPolicy::Status::kOk) {
+      // If status was not ok, then we must not have been able to evict enough
+      // blocks; therefore, we return anyway, temporarily going over the memory
+      // limit.
+      break;
+    }
 
-      StorageBlockBase* block;
-      {
-        SpinSharedMutexSharedLock<false> read_lock(blocks_shared_mutex_);
-        if (blocks_.find(block_index) == blocks_.end()) {
-          // another thread must have jumped in and evicted it before us
-
-          // NOTE(zuyu): It is ok to release the shard for a block or blob,
-          // before 'eviction_lock' destructs, because we will never encounter a
-          // self-deadlock in a single thread, and in multiple-thread case some
-          // thread will block but not deadlock if there is a shard collision.
-          lock_manager_.release(block_index);
-          continue;
-        }
-        block = blocks_[block_index].block;
-      }
-      if (eviction_policy_->getRefCount(block->getID()) > 0) {
-        // Someone sneaked in and referenced the block before we could evict it.
+    bool has_collision = false;
+    SpinSharedMutexExclusiveLock<false> eviction_lock(*lock_manager_.get(block_to_evict, &has_collision));
+    if (has_collision) {
+      // We have a collision in the shared lock manager, where some callers
+      // of this function (i.e., getBlockInternal or getBlobInternal) has
+      // acquired an exclusive lock, and we are trying to evict a block that
+      // hashes to the same location. This will cause a deadlock.
+
+      // For now simply treat this situation as the case where there is not
+      // enough memory and we temporarily go over the memory limit.
+      break;
+    }
 
-        // NOTE(zuyu): It is ok to release the shard for a block or blob, before
+    {
+      SpinSharedMutexSharedLock<false> read_lock(blocks_shared_mutex_);
+      if (blocks_.find(block_to_evict) == blocks_.end()) {
+        // another thread must have jumped in and evicted it before us
+
+        // NOTE(zuyu): It is ok to release the shard for a block or blob,
         // before 'eviction_lock' destructs, because we will never encounter a
         // self-deadlock in a single thread, and in multiple-thread case some
         // thread will block but not deadlock if there is a shard collision.
-        lock_manager_.release(block_index);
+        lock_manager_.release(block_to_evict);
         continue;
       }
-      if (saveBlockOrBlob(block->getID())) {
-        evictBlockOrBlob(block->getID());
-      }  // else : Someone sneaked in and evicted the block before we could.
+    }
+    if (eviction_policy_->getRefCount(block_to_evict) > 0) {
+      // Someone sneaked in and referenced the block before we could evict it.
 
       // NOTE(zuyu): It is ok to release the shard for a block or blob, before
       // before 'eviction_lock' destructs, because we will never encounter a
       // self-deadlock in a single thread, and in multiple-thread case some
       // thread will block but not deadlock if there is a shard collision.
-      lock_manager_.release(block_index);
-    } else {
-      // If status was not ok, then we must not have been able to evict enough
-      // blocks; therefore, we return anyway, temporarily going over the memory
-      // limit.
-      break;
+      lock_manager_.release(block_to_evict);
+      continue;
     }
+    if (saveBlockOrBlob(block_to_evict)) {
+      evictBlockOrBlob(block_to_evict);
+    }  // else : Someone sneaked in and evicted the block before we could.
+
+    // NOTE(zuyu): It is ok to release the shard for a block or blob, before
+    // before 'eviction_lock' destructs, because we will never encounter a
+    // self-deadlock in a single thread, and in multiple-thread case some
+    // thread will block but not deadlock if there is a shard collision.
+    lock_manager_.release(block_to_evict);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/85edc76c/storage/StorageManager.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.hpp b/storage/StorageManager.hpp
index dab33f6..0b68b76 100644
--- a/storage/StorageManager.hpp
+++ b/storage/StorageManager.hpp
@@ -402,12 +402,16 @@ class StorageManager {
                                        const int numa_node);
 
   /**
-   * @brief Evict blocks until there is enough space for a new block of the
-   *        requested size.
+   * @brief Evict blocks or blobs until there is enough space for a new block
+   *        or blob of the requested size.
+   *
+   * @note This non-blocking method gives up evictions if there is a shard
+   *       collision, and thus the buffer pool size may temporarily go beyond
+   *       the memory limit.
    *
    * @param slots Number of slots to make room for.
    */
-  void makeRoomForBlock(const std::size_t slots);
+  void makeRoomForBlockOrBlob(const std::size_t slots);
 
   /**
    * @brief Load a block from the persistent storage into memory.


[30/30] incubator-quickstep git commit: Support for NUMA aware preloading (#206)

Posted by zu...@apache.org.
Support for NUMA aware preloading (#206)

* Support for preloading of NUMA-partitioned relations.

- Stored relations which have been NUMA-partitioned, can now be
  preloaded. The preloading can maintain the same NUMA placement of
  storage blocks.

* Added missing glog library in CMakeLists


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

Branch: refs/heads/master
Commit: d9687b99a65514b6c14be86f574ffcbae3730ee6
Parents: db47654
Author: Harshad Deshmukh <d....@gmail.com>
Authored: Wed May 4 10:32:47 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:36 2016 -0700

----------------------------------------------------------------------
 storage/CMakeLists.txt      |  5 +++
 storage/PreloaderThread.cpp | 86 +++++++++++++++++++++++++++++++++++-----
 storage/PreloaderThread.hpp | 23 +++++++++++
 3 files changed, 105 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9687b99/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index ed23802..dacacfa 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -741,6 +741,7 @@ target_link_libraries(quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_utility_BitVector
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_PreloaderThread
+                      glog
                       quickstep_catalog_CatalogDatabase
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
@@ -750,6 +751,10 @@ target_link_libraries(quickstep_storage_PreloaderThread
                       quickstep_threading_Thread
                       quickstep_threading_ThreadUtil
                       quickstep_utility_Macros)
+if (QUICKSTEP_HAVE_LIBNUMA)
+  target_link_libraries(quickstep_storage_PreloaderThread
+                        quickstep_catalog_NUMAPlacementScheme)
+endif()
 target_link_libraries(quickstep_storage_SMAIndexSubBlock
                       glog
                       quickstep_catalog_CatalogAttribute

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9687b99/storage/PreloaderThread.cpp
----------------------------------------------------------------------
diff --git a/storage/PreloaderThread.cpp b/storage/PreloaderThread.cpp
index d5dc55b..8f600b8 100644
--- a/storage/PreloaderThread.cpp
+++ b/storage/PreloaderThread.cpp
@@ -17,8 +17,13 @@
 
 #include "storage/PreloaderThread.hpp"
 
+#include <cstddef>
 #include <vector>
 
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+#include <unordered_map>
+#endif
+
 #include "catalog/CatalogDatabase.hpp"
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogTypedefs.hpp"
@@ -27,6 +32,12 @@
 #include "storage/StorageManager.hpp"
 #include "threading/ThreadUtil.hpp"
 
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+#include "catalog/NUMAPlacementScheme.hpp"
+#endif
+
+#include "glog/logging.h"
+
 namespace quickstep {
 
 void PreloaderThread::run() {
@@ -38,24 +49,81 @@ void PreloaderThread::run() {
   std::size_t blocks_loaded = 0;
 
   for (const CatalogRelation &relation : database_) {
-    std::vector<block_id> blocks = relation.getBlocksSnapshot();
-    for (block_id current_block_id : blocks) {
+    if (relation.hasPartitionScheme()) {
+      blocks_loaded += preloadNUMAAware(relation, blocks_loaded, num_slots);
+    } else {
+      std::vector<block_id> blocks = relation.getBlocksSnapshot();
+      for (block_id current_block_id : blocks) {
+        try {
+          BlockReference current_block = storage_manager_->getBlock(current_block_id, relation);
+        } catch (...) {
+          LOG(ERROR) << "Error after loading " << blocks_loaded << "blocks\n";
+          throw;
+        }
+        ++blocks_loaded;
+        if (blocks_loaded == num_slots) {
+          // The buffer pool has filled up. But, some database blocks are not loaded.
+          printf(" The database is larger than the buffer pool. Only %lu blocks were loaded ",
+                 blocks_loaded);
+          return;
+        }
+      }
+    }
+  }
+  printf(" Loaded %lu blocks ", blocks_loaded);
+}
+
+std::size_t PreloaderThread::preloadNUMAAware(
+    const CatalogRelation &relation,
+    const std::size_t num_previously_loaded_blocks,
+    const std::size_t num_slots) {
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+  std::size_t blocks_loaded = 0;
+  const NUMAPlacementScheme *placement_scheme = relation.getNUMAPlacementSchemePtr();
+  DCHECK(placement_scheme != nullptr);
+  DCHECK(relation.hasPartitionScheme());
+  const PartitionScheme &part_scheme = relation.getPartitionScheme();
+  const PartitionSchemeHeader &part_scheme_header =
+      part_scheme.getPartitionSchemeHeader();
+  const std::size_t num_partitions = part_scheme_header.getNumPartitions();
+  // Key = NUMA node ID, value = number of blocks loaded from that NUMA node.
+  std::unordered_map<numa_node_id, std::size_t> num_blocks_loaded;
+  for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
+    const numa_node_id partition_numa_node_id =
+        placement_scheme->getNUMANodeForPartition(part_id);
+    for (block_id curr_block_id : part_scheme.getBlocksInPartition(part_id)) {
       try {
-        BlockReference current_block = storage_manager_->getBlock(current_block_id, relation);
+        BlockReference current_block = storage_manager_->getBlock(
+            curr_block_id, relation, partition_numa_node_id);
       } catch (...) {
-        LOG(ERROR) << "Error after loading " << blocks_loaded << "blocks\n";
+        LOG(ERROR) << "Error after loading "
+                   << blocks_loaded + num_previously_loaded_blocks
+                   << " blocks\n";
         throw;
       }
       ++blocks_loaded;
-      if (blocks_loaded == num_slots) {
+      num_blocks_loaded[partition_numa_node_id]++;
+      if ((blocks_loaded + num_previously_loaded_blocks) == num_slots) {
         // The buffer pool has filled up. But, some database blocks are not loaded.
-        printf(" The database is larger than the buffer pool. Only %lu blocks were loaded ",
-               blocks_loaded);
-        return;
+        printf(
+            " The database is larger than the buffer pool. Only %lu blocks "
+            "were loaded ",
+            blocks_loaded + num_previously_loaded_blocks);
+        return blocks_loaded;
       }
     }
   }
-  printf(" Loaded %lu blocks ", blocks_loaded);
+  LOG(INFO) << "For relation: " << relation.getName();
+  for (auto numa_block_loaded_info : num_blocks_loaded) {
+    LOG(INFO) << "NUMA node: " << numa_block_loaded_info.first
+              << " Number of loaded blocks: " << numa_block_loaded_info.second;
+  }
+  return blocks_loaded;
+#else
+  LOG(INFO) << "Relation: " << relation.getName()
+            << " has partition scheme but the system doesn't support NUMA";
+  return 0;
+#endif
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9687b99/storage/PreloaderThread.hpp
----------------------------------------------------------------------
diff --git a/storage/PreloaderThread.hpp b/storage/PreloaderThread.hpp
index ed866b4..f16fd50 100644
--- a/storage/PreloaderThread.hpp
+++ b/storage/PreloaderThread.hpp
@@ -18,12 +18,15 @@
 #ifndef QUICKSTEP_STORAGE_PRELOADER_THREAD_HPP_
 #define QUICKSTEP_STORAGE_PRELOADER_THREAD_HPP_
 
+#include <cstddef>
+
 #include "threading/Thread.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
 class CatalogDatabase;
+class CatalogRelation;
 class StorageManager;
 
 /** \addtogroup Storage
@@ -65,6 +68,26 @@ class PreloaderThread : public Thread {
   void run() override;
 
  private:
+  /**
+   * @brief Preload a relation which has a partition and a NUMA placement scheme.
+   *
+   * @param relation The relation to be preloaded.
+   * @param num_previously_loaded_blocks The number of blocks already preloaded.
+   * @param num_slots The maximum number of slots in the StorageManager.
+   *
+   * @warning This function may not detect skew on NUMA sockets, i.e. if a given
+   *          NUMA socket has large number of blocks, preloading may cause the
+   *          memory on that NUMA socket to be full. It is recommended to use
+   *          this preloading when we are sure that each NUMA socket has been
+   *          allocated sufficient amount of memory so as not to exceed that
+   *          socket's memory limit.
+   *
+   * @return The number of blocks loaded during this function call.
+   **/
+  std::size_t preloadNUMAAware(const CatalogRelation &relation,
+                               const std::size_t num_previously_loaded_blocks,
+                               const std::size_t num_slots);
+
   const CatalogDatabase &database_;
   StorageManager *storage_manager_;
 


[11/30] incubator-quickstep git commit: Merge pull request #170 from pivotalsoftware/quickstep_date_support

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 32149a1..f2cb8ca 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -227,78 +227,84 @@ extern int quickstep_yydebug;
     TOKEN_CREATE = 297,
     TOKEN_DATE = 298,
     TOKEN_DATETIME = 299,
-    TOKEN_DECIMAL = 300,
-    TOKEN_DEFAULT = 301,
-    TOKEN_DELETE = 302,
-    TOKEN_DELIMITER = 303,
-    TOKEN_DESC = 304,
-    TOKEN_DISTINCT = 305,
-    TOKEN_DOUBLE = 306,
-    TOKEN_DROP = 307,
-    TOKEN_ELSE = 308,
-    TOKEN_END = 309,
-    TOKEN_ESCAPE_STRINGS = 310,
-    TOKEN_EXISTS = 311,
-    TOKEN_EXTRACT = 312,
-    TOKEN_FALSE = 313,
-    TOKEN_FIRST = 314,
-    TOKEN_FLOAT = 315,
-    TOKEN_FOREIGN = 316,
-    TOKEN_FROM = 317,
-    TOKEN_FULL = 318,
-    TOKEN_GROUP = 319,
-    TOKEN_HASH = 320,
-    TOKEN_HAVING = 321,
-    TOKEN_IN = 322,
-    TOKEN_INDEX = 323,
-    TOKEN_INNER = 324,
-    TOKEN_INSERT = 325,
-    TOKEN_INTEGER = 326,
-    TOKEN_INTERVAL = 327,
-    TOKEN_INTO = 328,
-    TOKEN_JOIN = 329,
-    TOKEN_KEY = 330,
-    TOKEN_LAST = 331,
-    TOKEN_LEFT = 332,
-    TOKEN_LIMIT = 333,
-    TOKEN_LONG = 334,
-    TOKEN_NULL = 335,
-    TOKEN_NULLS = 336,
-    TOKEN_OFF = 337,
-    TOKEN_ON = 338,
-    TOKEN_ORDER = 339,
-    TOKEN_OUTER = 340,
-    TOKEN_PARTITION = 341,
-    TOKEN_PARTITIONS = 342,
-    TOKEN_PERCENT = 343,
-    TOKEN_PRIMARY = 344,
-    TOKEN_QUIT = 345,
-    TOKEN_RANGE = 346,
-    TOKEN_REAL = 347,
-    TOKEN_REFERENCES = 348,
-    TOKEN_RIGHT = 349,
-    TOKEN_ROW_DELIMITER = 350,
-    TOKEN_SELECT = 351,
-    TOKEN_SET = 352,
-    TOKEN_SMA = 353,
-    TOKEN_SMALLINT = 354,
-    TOKEN_TABLE = 355,
-    TOKEN_THEN = 356,
-    TOKEN_TIME = 357,
-    TOKEN_TIMESTAMP = 358,
-    TOKEN_TRUE = 359,
-    TOKEN_TUPLESAMPLE = 360,
-    TOKEN_UNIQUE = 361,
-    TOKEN_UPDATE = 362,
-    TOKEN_USING = 363,
-    TOKEN_VALUES = 364,
-    TOKEN_VARCHAR = 365,
-    TOKEN_WHEN = 366,
-    TOKEN_WHERE = 367,
-    TOKEN_WITH = 368,
-    TOKEN_YEARMONTH = 369,
-    TOKEN_EOF = 370,
-    TOKEN_LEX_ERROR = 371
+    TOKEN_DAY = 300,
+    TOKEN_DECIMAL = 301,
+    TOKEN_DEFAULT = 302,
+    TOKEN_DELETE = 303,
+    TOKEN_DELIMITER = 304,
+    TOKEN_DESC = 305,
+    TOKEN_DISTINCT = 306,
+    TOKEN_DOUBLE = 307,
+    TOKEN_DROP = 308,
+    TOKEN_ELSE = 309,
+    TOKEN_END = 310,
+    TOKEN_ESCAPE_STRINGS = 311,
+    TOKEN_EXISTS = 312,
+    TOKEN_EXTRACT = 313,
+    TOKEN_FALSE = 314,
+    TOKEN_FIRST = 315,
+    TOKEN_FLOAT = 316,
+    TOKEN_FOREIGN = 317,
+    TOKEN_FROM = 318,
+    TOKEN_FULL = 319,
+    TOKEN_GROUP = 320,
+    TOKEN_HASH = 321,
+    TOKEN_HAVING = 322,
+    TOKEN_HOUR = 323,
+    TOKEN_IN = 324,
+    TOKEN_INDEX = 325,
+    TOKEN_INNER = 326,
+    TOKEN_INSERT = 327,
+    TOKEN_INTEGER = 328,
+    TOKEN_INTERVAL = 329,
+    TOKEN_INTO = 330,
+    TOKEN_JOIN = 331,
+    TOKEN_KEY = 332,
+    TOKEN_LAST = 333,
+    TOKEN_LEFT = 334,
+    TOKEN_LIMIT = 335,
+    TOKEN_LONG = 336,
+    TOKEN_MINUTE = 337,
+    TOKEN_MONTH = 338,
+    TOKEN_NULL = 339,
+    TOKEN_NULLS = 340,
+    TOKEN_OFF = 341,
+    TOKEN_ON = 342,
+    TOKEN_ORDER = 343,
+    TOKEN_OUTER = 344,
+    TOKEN_PARTITION = 345,
+    TOKEN_PARTITIONS = 346,
+    TOKEN_PERCENT = 347,
+    TOKEN_PRIMARY = 348,
+    TOKEN_QUIT = 349,
+    TOKEN_RANGE = 350,
+    TOKEN_REAL = 351,
+    TOKEN_REFERENCES = 352,
+    TOKEN_RIGHT = 353,
+    TOKEN_ROW_DELIMITER = 354,
+    TOKEN_SECOND = 355,
+    TOKEN_SELECT = 356,
+    TOKEN_SET = 357,
+    TOKEN_SMA = 358,
+    TOKEN_SMALLINT = 359,
+    TOKEN_TABLE = 360,
+    TOKEN_THEN = 361,
+    TOKEN_TIME = 362,
+    TOKEN_TIMESTAMP = 363,
+    TOKEN_TRUE = 364,
+    TOKEN_TUPLESAMPLE = 365,
+    TOKEN_UNIQUE = 366,
+    TOKEN_UPDATE = 367,
+    TOKEN_USING = 368,
+    TOKEN_VALUES = 369,
+    TOKEN_VARCHAR = 370,
+    TOKEN_WHEN = 371,
+    TOKEN_WHERE = 372,
+    TOKEN_WITH = 373,
+    TOKEN_YEAR = 374,
+    TOKEN_YEARMONTH = 375,
+    TOKEN_EOF = 376,
+    TOKEN_LEX_ERROR = 377
   };
 #endif
 
@@ -390,7 +396,7 @@ union YYSTYPE
   quickstep::ParseOrderBy *opt_order_by_clause_;
   bool *order_direction_;
   quickstep::ParseLimit *opt_limit_clause_;
-  
+
   quickstep::ParseSample *opt_sample_clause_;
 
   quickstep::PtrList<quickstep::ParseOrderByItem> *order_commalist_;
@@ -399,7 +405,7 @@ union YYSTYPE
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
 
-#line 403 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 409 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -434,7 +440,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 438 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 444 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -678,21 +684,21 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  47
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1072
+#define YYLAST   1191
 
 /* YYNTOKENS -- Number of terminals.  */
-#define YYNTOKENS  128
+#define YYNTOKENS  134
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  94
+#define YYNNTS  95
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  255
+#define YYNRULES  262
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  471
+#define YYNSTATES  478
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
 #define YYUNDEFTOK  2
-#define YYMAXUTOK   371
+#define YYMAXUTOK   377
 
 #define YYTRANSLATE(YYX)                                                \
   ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK)
@@ -702,11 +708,11 @@ union yyalloc
 static const yytype_uint8 yytranslate[] =
 {
        0,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-     123,     2,     2,     2,     2,     2,     2,     2,     2,     2,
+     129,     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,   127,     2,     2,
-     124,   125,    23,    21,   126,    22,    27,    24,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,     2,     2,   122,
+       2,     2,     2,     2,     2,     2,     2,   133,     2,     2,
+     130,   131,    23,    21,   132,    22,    27,    24,     2,     2,
+       2,     2,     2,     2,     2,     2,     2,     2,     2,   128,
        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,
@@ -738,39 +744,40 @@ static const yytype_uint8 yytranslate[] =
       90,    91,    92,    93,    94,    95,    96,    97,    98,    99,
      100,   101,   102,   103,   104,   105,   106,   107,   108,   109,
      110,   111,   112,   113,   114,   115,   116,   117,   118,   119,
-     120,   121
+     120,   121,   122,   123,   124,   125,   126,   127
 };
 
 #if YYDEBUG
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   575,   575,   579,   583,   587,   591,   594,   601,   604,
-     607,   610,   613,   616,   619,   622,   625,   628,   634,   640,
-     647,   653,   660,   669,   674,   683,   688,   693,   697,   703,
-     708,   711,   714,   719,   722,   725,   728,   731,   734,   737,
-     740,   743,   746,   758,   761,   764,   782,   802,   805,   808,
-     813,   818,   824,   830,   839,   843,   849,   852,   857,   862,
-     867,   874,   881,   885,   891,   894,   899,   902,   907,   910,
-     915,   918,   937,   941,   947,   951,   957,   960,   963,   968,
-     971,   978,   983,   994,   999,  1003,  1007,  1013,  1016,  1022,
-    1030,  1033,  1036,  1042,  1047,  1050,  1055,  1059,  1063,  1067,
-    1073,  1078,  1083,  1087,  1093,  1099,  1102,  1107,  1112,  1116,
-    1122,  1128,  1134,  1137,  1141,  1147,  1150,  1155,  1159,  1165,
-    1168,  1171,  1176,  1181,  1186,  1189,  1192,  1197,  1200,  1203,
-    1206,  1209,  1212,  1215,  1218,  1223,  1226,  1231,  1235,  1239,
-    1242,  1246,  1249,  1254,  1257,  1262,  1265,  1270,  1274,  1280,
-    1283,  1288,  1291,  1296,  1299,  1304,  1307,  1326,  1330,  1336,
-    1343,  1346,  1349,  1354,  1357,  1360,  1366,  1369,  1374,  1379,
-    1388,  1393,  1402,  1407,  1410,  1415,  1418,  1423,  1429,  1435,
-    1438,  1441,  1444,  1447,  1450,  1456,  1465,  1468,  1473,  1476,
-    1481,  1484,  1489,  1492,  1495,  1498,  1501,  1504,  1507,  1512,
-    1516,  1520,  1523,  1528,  1533,  1536,  1541,  1545,  1551,  1556,
-    1560,  1566,  1571,  1574,  1579,  1583,  1589,  1592,  1595,  1598,
-    1610,  1614,  1633,  1648,  1652,  1658,  1661,  1666,  1670,  1677,
-    1680,  1683,  1686,  1689,  1692,  1695,  1698,  1701,  1704,  1709,
-    1720,  1723,  1728,  1731,  1734,  1740,  1744,  1750,  1753,  1761,
-    1764,  1767,  1770,  1776,  1781,  1786
+       0,   582,   582,   586,   590,   594,   598,   601,   608,   611,
+     614,   617,   620,   623,   626,   629,   632,   635,   641,   647,
+     654,   660,   667,   676,   681,   690,   695,   700,   704,   710,
+     715,   718,   721,   726,   729,   732,   735,   738,   741,   744,
+     747,   750,   753,   765,   768,   771,   789,   809,   812,   815,
+     820,   825,   831,   837,   846,   850,   856,   859,   864,   869,
+     874,   881,   888,   892,   898,   901,   906,   909,   914,   917,
+     922,   925,   944,   948,   954,   958,   964,   967,   970,   975,
+     978,   985,   990,  1001,  1006,  1010,  1014,  1020,  1023,  1029,
+    1037,  1040,  1043,  1049,  1054,  1057,  1062,  1066,  1070,  1074,
+    1080,  1085,  1090,  1094,  1100,  1106,  1109,  1114,  1119,  1123,
+    1129,  1135,  1141,  1144,  1148,  1154,  1157,  1162,  1166,  1172,
+    1175,  1178,  1183,  1188,  1193,  1196,  1199,  1204,  1207,  1210,
+    1213,  1216,  1219,  1222,  1225,  1230,  1233,  1238,  1242,  1246,
+    1249,  1253,  1256,  1261,  1264,  1269,  1272,  1277,  1281,  1287,
+    1290,  1295,  1298,  1303,  1306,  1311,  1314,  1333,  1337,  1343,
+    1350,  1353,  1356,  1361,  1364,  1367,  1373,  1376,  1381,  1386,
+    1395,  1400,  1409,  1414,  1417,  1422,  1425,  1430,  1436,  1442,
+    1445,  1448,  1451,  1454,  1457,  1463,  1472,  1475,  1480,  1483,
+    1488,  1491,  1496,  1499,  1502,  1505,  1508,  1511,  1514,  1519,
+    1523,  1527,  1530,  1535,  1540,  1543,  1548,  1552,  1558,  1563,
+    1567,  1573,  1578,  1581,  1586,  1590,  1596,  1599,  1602,  1605,
+    1617,  1621,  1640,  1653,  1668,  1671,  1674,  1677,  1680,  1683,
+    1688,  1692,  1698,  1701,  1706,  1710,  1717,  1720,  1723,  1726,
+    1729,  1732,  1735,  1738,  1741,  1744,  1749,  1760,  1763,  1768,
+    1771,  1774,  1780,  1784,  1790,  1793,  1801,  1804,  1807,  1810,
+    1816,  1821,  1826
 };
 #endif
 
@@ -790,26 +797,27 @@ static const char *const yytname[] =
   "TOKEN_BLOCKSAMPLE", "TOKEN_BLOOM_FILTER", "TOKEN_CSB_TREE", "TOKEN_BY",
   "TOKEN_CASE", "TOKEN_CHARACTER", "TOKEN_CHECK", "TOKEN_COLUMN",
   "TOKEN_CONSTRAINT", "TOKEN_COPY", "TOKEN_CREATE", "TOKEN_DATE",
-  "TOKEN_DATETIME", "TOKEN_DECIMAL", "TOKEN_DEFAULT", "TOKEN_DELETE",
-  "TOKEN_DELIMITER", "TOKEN_DESC", "TOKEN_DISTINCT", "TOKEN_DOUBLE",
-  "TOKEN_DROP", "TOKEN_ELSE", "TOKEN_END", "TOKEN_ESCAPE_STRINGS",
-  "TOKEN_EXISTS", "TOKEN_EXTRACT", "TOKEN_FALSE", "TOKEN_FIRST",
-  "TOKEN_FLOAT", "TOKEN_FOREIGN", "TOKEN_FROM", "TOKEN_FULL",
-  "TOKEN_GROUP", "TOKEN_HASH", "TOKEN_HAVING", "TOKEN_IN", "TOKEN_INDEX",
-  "TOKEN_INNER", "TOKEN_INSERT", "TOKEN_INTEGER", "TOKEN_INTERVAL",
-  "TOKEN_INTO", "TOKEN_JOIN", "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT",
-  "TOKEN_LIMIT", "TOKEN_LONG", "TOKEN_NULL", "TOKEN_NULLS", "TOKEN_OFF",
+  "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL", "TOKEN_DEFAULT",
+  "TOKEN_DELETE", "TOKEN_DELIMITER", "TOKEN_DESC", "TOKEN_DISTINCT",
+  "TOKEN_DOUBLE", "TOKEN_DROP", "TOKEN_ELSE", "TOKEN_END",
+  "TOKEN_ESCAPE_STRINGS", "TOKEN_EXISTS", "TOKEN_EXTRACT", "TOKEN_FALSE",
+  "TOKEN_FIRST", "TOKEN_FLOAT", "TOKEN_FOREIGN", "TOKEN_FROM",
+  "TOKEN_FULL", "TOKEN_GROUP", "TOKEN_HASH", "TOKEN_HAVING", "TOKEN_HOUR",
+  "TOKEN_IN", "TOKEN_INDEX", "TOKEN_INNER", "TOKEN_INSERT",
+  "TOKEN_INTEGER", "TOKEN_INTERVAL", "TOKEN_INTO", "TOKEN_JOIN",
+  "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT", "TOKEN_LIMIT", "TOKEN_LONG",
+  "TOKEN_MINUTE", "TOKEN_MONTH", "TOKEN_NULL", "TOKEN_NULLS", "TOKEN_OFF",
   "TOKEN_ON", "TOKEN_ORDER", "TOKEN_OUTER", "TOKEN_PARTITION",
   "TOKEN_PARTITIONS", "TOKEN_PERCENT", "TOKEN_PRIMARY", "TOKEN_QUIT",
   "TOKEN_RANGE", "TOKEN_REAL", "TOKEN_REFERENCES", "TOKEN_RIGHT",
-  "TOKEN_ROW_DELIMITER", "TOKEN_SELECT", "TOKEN_SET", "TOKEN_SMA",
-  "TOKEN_SMALLINT", "TOKEN_TABLE", "TOKEN_THEN", "TOKEN_TIME",
+  "TOKEN_ROW_DELIMITER", "TOKEN_SECOND", "TOKEN_SELECT", "TOKEN_SET",
+  "TOKEN_SMA", "TOKEN_SMALLINT", "TOKEN_TABLE", "TOKEN_THEN", "TOKEN_TIME",
   "TOKEN_TIMESTAMP", "TOKEN_TRUE", "TOKEN_TUPLESAMPLE", "TOKEN_UNIQUE",
   "TOKEN_UPDATE", "TOKEN_USING", "TOKEN_VALUES", "TOKEN_VARCHAR",
-  "TOKEN_WHEN", "TOKEN_WHERE", "TOKEN_WITH", "TOKEN_YEARMONTH",
-  "TOKEN_EOF", "TOKEN_LEX_ERROR", "';'", "'\\n'", "'('", "')'", "','",
-  "'%'", "$accept", "start", "sql_statement", "quit_statement",
-  "alter_table_statement", "create_table_statement",
+  "TOKEN_WHEN", "TOKEN_WHERE", "TOKEN_WITH", "TOKEN_YEAR",
+  "TOKEN_YEARMONTH", "TOKEN_EOF", "TOKEN_LEX_ERROR", "';'", "'\\n'", "'('",
+  "')'", "','", "'%'", "$accept", "start", "sql_statement",
+  "quit_statement", "alter_table_statement", "create_table_statement",
   "create_index_statement", "drop_table_statement", "column_def",
   "column_def_commalist", "data_type", "column_constraint_def",
   "column_constraint_def_list", "opt_column_constraint_def_list",
@@ -835,10 +843,11 @@ static const char *const yytname[] =
   "extract_function", "case_expression", "simple_when_clause_list",
   "simple_when_clause", "searched_when_clause_list",
   "searched_when_clause", "opt_else_clause", "expression_list",
-  "literal_value", "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", "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
 };
 #endif
 
@@ -859,14 +868,15 @@ static const yytype_uint16 yytoknum[] =
      340,   341,   342,   343,   344,   345,   346,   347,   348,   349,
      350,   351,   352,   353,   354,   355,   356,   357,   358,   359,
      360,   361,   362,   363,   364,   365,   366,   367,   368,   369,
-     370,   371,    59,    10,    40,    41,    44,    37
+     370,   371,   372,   373,   374,   375,   376,   377,    59,    10,
+      40,    41,    44,    37
 };
 # endif
 
-#define YYPACT_NINF -216
+#define YYPACT_NINF -223
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-216)))
+  (!!((Yystate) == (-223)))
 
 #define YYTABLE_NINF -128
 
@@ -877,139 +887,139 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-     198,  -216,  -216,   -46,   175,   -19,    46,   -27,    20,  -216,
-      32,   175,   175,  -216,    88,   127,  -216,  -216,  -216,  -216,
-    -216,  -216,  -216,  -216,  -216,  -216,    -8,  -216,   115,   114,
-     175,  -216,  -216,    54,   175,   175,   175,   175,   175,  -216,
-    -216,   527,    52,    34,  -216,   135,    67,  -216,  -216,  -216,
-     191,  -216,  -216,  -216,  -216,    37,   197,   117,   152,   180,
-    -216,    86,  -216,  -216,   293,   294,  -216,  -216,  -216,   560,
-     188,  -216,   236,  -216,  -216,   202,  -216,  -216,   312,  -216,
-    -216,  -216,  -216,  -216,  -216,   212,   260,   648,   334,   274,
-     214,  -216,  -216,   164,    -1,  -216,  -216,  -216,  -216,  -216,
-    -216,  -216,   769,    23,   175,   175,   224,   175,   175,    28,
-     171,   233,   175,   175,   439,  -216,  -216,   228,   175,  -216,
-    -216,  -216,   439,    27,   -17,  -216,   346,  -216,   175,  -216,
-     347,  -216,   231,     7,  -216,    15,   180,   802,  -216,  -216,
-     175,   802,  -216,  -216,  -216,  -216,   802,   294,  -216,   175,
-     273,    56,  -216,   348,  -216,   272,  -216,   101,  -216,   272,
-     175,   118,   175,   175,   250,  -216,   251,  -216,   111,   885,
-     681,   224,   406,   370,   374,  -216,  -216,  1000,   359,   890,
-     131,    17,   802,   -11,  -216,   802,  -216,   325,   261,   318,
-     262,  -216,  -216,    40,    75,   124,  -216,   263,    75,   -13,
-     321,  -216,  -216,    -1,  -216,  -216,   266,   802,  -216,   219,
-     133,   175,  -216,   802,  -216,   175,  -216,  -216,   269,   315,
-     316,   275,  -216,  -216,  -216,   144,   175,   287,   118,   175,
-    -216,    82,  -216,  -216,    -3,   350,   439,   439,    58,  -216,
-    -216,  -216,  -216,  -216,  -216,  -216,  -216,   802,   278,   802,
-      25,  -216,   137,   289,   802,    47,  -216,   345,   219,  -216,
-    -216,   802,  -216,    89,   175,  -216,  -216,   317,  -216,   319,
-     324,   326,    15,  -216,   399,   401,    75,   375,   352,  -216,
-     139,  -216,   802,  -216,   219,  -216,   439,   300,   301,   175,
-     412,     3,   141,  -216,   145,   391,    45,  -216,   303,   308,
-    -216,   349,   306,   890,  -216,   351,   175,  -216,  -216,    82,
-    -216,  -216,   374,  -216,  -216,  -216,   802,   313,   187,   648,
-    -216,   219,   356,  -216,  -216,   890,   327,   219,   802,  -216,
-      21,  -216,  -216,  -216,  -216,  -216,    15,   124,   339,   343,
-    -216,   802,   439,   353,  -216,   219,    12,   175,   175,   149,
-    -216,  -216,  -216,  -216,  -216,  -216,  -216,   193,  -216,   175,
-    -216,  -216,  -216,  -216,   328,   118,   414,   367,  -216,   439,
-    -216,  -216,   335,  -216,   221,   648,  -216,   802,   156,  -216,
-    -216,   890,   219,  -216,   376,  -216,  -216,   337,   370,   425,
-     383,  -216,   158,   162,  -216,   464,     3,  -216,   175,  -216,
-    -216,   354,   430,  -216,    19,   175,   802,   166,   219,  -216,
-     179,   439,   802,   467,  -216,   378,  -216,  -216,  -216,   183,
-    -216,  -216,  -216,  -216,    11,   175,   -14,  -216,   360,   219,
-    -216,  -216,   370,   358,  -216,    31,  -216,   175,  -216,   175,
-    -216,  -216,   175,  -216,   194,  -216,  -216,   355,  -216,   802,
-    -216,  -216,   400,   368,  -216,   199,  -216,   175,  -216,    38,
-    -216,   175,  -216,   205,  -216,  -216,   208,   385,  -216,   486,
-    -216
+     921,  -223,  -223,   -82,   231,   -14,    56,    22,    74,  -223,
+      70,   231,   231,  -223,   135,   120,  -223,  -223,  -223,  -223,
+    -223,  -223,  -223,  -223,  -223,  -223,   -35,  -223,   -73,   177,
+     231,  -223,  -223,   121,   231,   231,   231,   231,   231,  -223,
+    -223,   576,    85,    63,  -223,   173,    77,  -223,  -223,  -223,
+     140,  -223,  -223,  -223,  -223,    18,   218,   144,    97,   119,
+    -223,     4,  -223,  -223,   240,   245,  -223,  -223,  -223,   642,
+     134,  -223,   187,  -223,  -223,   146,  -223,  -223,   265,  -223,
+    -223,  -223,  -223,  -223,  -223,   164,   203,   708,   290,   230,
+     176,  -223,  -223,   256,    20,  -223,  -223,  -223,  -223,  -223,
+    -223,  -223,   840,   -11,   231,   231,   182,   231,   231,   167,
+     206,   204,   231,   231,   483,  -223,  -223,   205,   231,  -223,
+    -223,  -223,   483,    47,   -10,  -223,   329,  -223,   129,   129,
+     330,  -223,   207,    26,  -223,    32,   119,   906,  -223,  -223,
+     231,   906,  -223,  -223,  -223,  -223,   906,   245,  -223,   231,
+     324,   -70,  -223,   331,  -223,   234,  -223,   125,  -223,   234,
+     231,    49,   231,   231,   211,  -223,   213,  -223,   137,   991,
+     774,   182,   390,   336,   339,  -223,  -223,  1117,   335,  1001,
+     142,    10,   906,    -9,  -223,   906,  -223,   296,   228,  -223,
+    -223,  -223,  -223,  -223,  -223,   292,  -223,   232,  -223,  -223,
+      21,   163,   122,  -223,   229,   163,   -13,   294,  -223,  -223,
+      20,  -223,  -223,   236,   906,  -223,   259,   152,   231,  -223,
+     906,  -223,   231,  -223,  -223,   238,   287,   288,   241,  -223,
+    -223,  -223,    61,   231,   258,    49,   231,  -223,   156,  -223,
+    -223,    -6,    69,   483,   483,    55,  -223,  -223,  -223,  -223,
+    -223,  -223,  -223,  -223,   906,   244,   906,     5,  -223,   154,
+     260,   906,    37,  -223,   317,   259,  -223,  -223,   906,  -223,
+     102,   231,  -223,  -223,   284,  -223,   291,   297,   301,    32,
+    -223,   377,   379,   163,   348,   320,  -223,   157,  -223,   906,
+    -223,   259,  -223,   483,   263,   268,   231,   394,   151,   159,
+    -223,   168,   378,    25,  -223,   272,   283,  -223,   318,   279,
+    1001,  -223,   332,   231,  -223,  -223,   156,  -223,  -223,   339,
+    -223,  -223,  -223,   906,   285,   161,   708,  -223,   259,   327,
+    -223,  -223,  1001,   289,   259,   906,  -223,    33,  -223,  -223,
+    -223,  -223,  -223,    32,   122,   321,   323,  -223,   906,   483,
+     328,  -223,   259,    13,   231,   231,   170,  -223,  -223,  -223,
+    -223,  -223,  -223,  -223,    76,  -223,   231,  -223,  -223,  -223,
+    -223,   299,    49,   381,   340,  -223,   483,  -223,  -223,   300,
+    -223,   200,   708,  -223,   906,   172,  -223,  -223,  1001,   259,
+    -223,   342,  -223,  -223,   311,   336,   382,   361,  -223,   175,
+     179,  -223,   443,   151,  -223,   231,  -223,  -223,   333,   410,
+    -223,    29,   231,   906,   183,   259,  -223,   185,   483,   906,
+     444,  -223,   355,  -223,  -223,  -223,   188,  -223,  -223,  -223,
+    -223,    16,   231,    -5,  -223,   334,   259,  -223,  -223,   336,
+     326,  -223,   155,  -223,   231,  -223,   231,  -223,  -223,   231,
+    -223,   190,  -223,  -223,   337,  -223,   906,  -223,  -223,   369,
+     341,  -223,   192,  -223,   231,  -223,   110,  -223,   231,  -223,
+     194,  -223,  -223,   201,   365,  -223,   455,  -223
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
      Performed when YYTABLE does not specify something else to do.  Zero
      means the default is an error.  */
-static const yytype_uint8 yydefact[] =
+static const yytype_uint16 yydefact[] =
 {
-       0,     6,   255,     0,     0,     0,     0,     0,     0,    18,
+       0,     6,   262,     0,     0,     0,     0,     0,     0,    18,
      112,     0,     0,     7,     0,     0,    15,     8,    10,    11,
-      13,    14,     9,    17,    12,    16,     0,   105,     0,   253,
-       0,   247,   248,     0,     0,     0,     0,     0,     0,   113,
+      13,    14,     9,    17,    12,    16,     0,   105,     0,   260,
+       0,   254,   255,     0,     0,     0,     0,     0,     0,   113,
      114,     0,     0,   107,   108,     0,   145,     1,     3,     2,
-       0,   106,     5,     4,   254,     0,     0,     0,     0,   166,
-      25,     0,   220,   217,     0,   239,   115,    40,    29,     0,
+       0,   106,     5,     4,   261,     0,     0,     0,     0,   166,
+      25,     0,   220,   217,     0,   246,   115,    40,    29,     0,
        0,    30,    31,    34,    36,     0,    37,    39,     0,    41,
      216,    35,    38,    32,    33,     0,     0,     0,     0,     0,
      116,   117,   198,   121,   187,   189,   191,   194,   195,   196,
-     193,   192,     0,   225,     0,     0,     0,     0,     0,     0,
+     193,   192,     0,   232,     0,     0,     0,     0,     0,     0,
        0,    94,     0,     0,     0,   101,   167,     0,     0,    91,
      218,   219,     0,     0,   212,   209,     0,    43,     0,   221,
-       0,    44,     0,     0,   222,     0,   166,     0,   240,   241,
-       0,     0,   120,   243,   244,   242,     0,     0,   190,     0,
-       0,   166,   103,     0,   109,     0,   110,     0,   245,     0,
+       0,    44,     0,     0,   223,     0,   166,     0,   247,   248,
+       0,     0,   120,   250,   251,   249,     0,     0,   190,     0,
+       0,   166,   103,     0,   109,     0,   110,     0,   252,     0,
        0,     0,     0,     0,     0,    93,    66,    27,     0,     0,
        0,     0,     0,   168,   170,   172,   174,     0,   192,     0,
-       0,     0,     0,   212,   206,     0,   210,     0,     0,     0,
-       0,   123,   197,     0,     0,   147,   136,   122,   141,   124,
-     149,   118,   119,   186,   188,   226,     0,     0,   199,   214,
-       0,     0,   100,     0,   146,     0,    92,    19,     0,     0,
-       0,     0,    20,    21,    22,     0,     0,     0,    64,     0,
-      42,    56,   173,   181,     0,     0,     0,     0,     0,   229,
-     231,   232,   233,   234,   230,   235,   237,     0,     0,     0,
-       0,   223,     0,     0,     0,     0,   207,     0,   213,   205,
-      45,     0,    46,   127,     0,   137,   143,   133,   128,   129,
-     131,     0,     0,   140,     0,     0,   139,     0,   151,   200,
-       0,   201,     0,   102,   104,   246,     0,     0,     0,     0,
-       0,     0,     0,   227,     0,   225,     0,    63,    65,    68,
-      28,     0,     0,     0,    47,     0,     0,    49,    55,    57,
-      26,   180,   169,   171,   236,   238,     0,     0,     0,     0,
-     182,   179,     0,   178,    90,     0,     0,   211,     0,   204,
-       0,   142,   144,   134,   130,   132,     0,   148,     0,     0,
-     138,     0,     0,   153,   202,   215,     0,     0,     0,     0,
-      96,   251,   252,   250,   249,    97,    95,     0,    67,     0,
-      83,    84,    85,    86,    87,     0,     0,    70,    48,     0,
-      51,    50,     0,    54,     0,     0,   184,     0,     0,   177,
-     224,     0,   208,   203,     0,   125,   126,   150,   152,     0,
-     155,    61,     0,     0,    58,     0,     0,   228,     0,    24,
-      62,     0,     0,    23,     0,     0,     0,     0,   175,   183,
-       0,     0,     0,     0,   111,     0,    59,    98,    99,     0,
-      74,    76,    77,    78,     0,     0,     0,    52,     0,   176,
-     185,    89,   135,   154,   157,   160,   156,     0,    88,     0,
-      82,    80,     0,    79,     0,    72,    73,     0,    53,     0,
-     161,   162,   163,     0,    75,     0,    69,     0,   158,     0,
-     159,     0,    81,     0,   164,   165,     0,     0,    60,     0,
-      71
+       0,     0,     0,   212,   206,     0,   210,     0,     0,   226,
+     227,   228,   225,   229,   224,     0,   222,     0,   123,   197,
+       0,     0,   147,   136,   122,   141,   124,   149,   118,   119,
+     186,   188,   233,     0,     0,   199,   214,     0,     0,   100,
+       0,   146,     0,    92,    19,     0,     0,     0,     0,    20,
+      21,    22,     0,     0,     0,    64,     0,    42,    56,   173,
+     181,     0,     0,     0,     0,     0,   236,   238,   239,   240,
+     241,   237,   242,   244,     0,     0,     0,     0,   230,     0,
+       0,     0,     0,   207,     0,   213,   205,    45,     0,    46,
+     127,     0,   137,   143,   133,   128,   129,   131,     0,     0,
+     140,     0,     0,   139,     0,   151,   200,     0,   201,     0,
+     102,   104,   253,     0,     0,     0,     0,     0,     0,     0,
+     234,     0,   232,     0,    63,    65,    68,    28,     0,     0,
+       0,    47,     0,     0,    49,    55,    57,    26,   180,   169,
+     171,   243,   245,     0,     0,     0,     0,   182,   179,     0,
+     178,    90,     0,     0,   211,     0,   204,     0,   142,   144,
+     134,   130,   132,     0,   148,     0,     0,   138,     0,     0,
+     153,   202,   215,     0,     0,     0,     0,    96,   258,   259,
+     257,   256,    97,    95,     0,    67,     0,    83,    84,    85,
+      86,    87,     0,     0,    70,    48,     0,    51,    50,     0,
+      54,     0,     0,   184,     0,     0,   177,   231,     0,   208,
+     203,     0,   125,   126,   150,   152,     0,   155,    61,     0,
+       0,    58,     0,     0,   235,     0,    24,    62,     0,     0,
+      23,     0,     0,     0,     0,   175,   183,     0,     0,     0,
+       0,   111,     0,    59,    98,    99,     0,    74,    76,    77,
+      78,     0,     0,     0,    52,     0,   176,   185,    89,   135,
+     154,   157,   160,   156,     0,    88,     0,    82,    80,     0,
+      79,     0,    72,    73,     0,    53,     0,   161,   162,   163,
+       0,    75,     0,    69,     0,   158,     0,   159,     0,    81,
+       0,   164,   165,     0,     0,    60,     0,    71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -216,  -216,  -216,  -216,  -216,  -216,  -216,  -216,  -102,  -216,
-     329,   185,  -216,  -216,  -215,  -216,  -216,  -216,  -216,  -216,
-    -216,    71,    60,  -216,  -216,  -216,  -216,  -216,  -216,  -216,
-    -216,  -216,  -216,  -216,  -216,   286,  -216,  -216,  -216,   397,
-      10,  -216,  -216,  -216,   369,  -216,   -97,  -216,  -216,  -138,
-     169,  -161,   -10,  -216,  -216,  -216,  -216,  -216,  -216,    59,
-    -216,  -216,   -91,  -216,  -121,   276,   280,   341,   -30,   377,
-     363,   417,  -123,  -216,  -216,  -216,   344,  -216,   396,   357,
-    -203,  -163,   147,  -108,  -216,  -216,  -216,  -216,  -216,  -115,
-      -4,   126,  -216,  -216
+    -223,  -223,  -223,  -223,  -223,  -223,  -223,  -223,  -131,  -223,
+     303,   150,  -223,  -223,  -222,  -223,  -223,  -223,  -223,  -223,
+    -223,    38,    27,  -223,  -223,  -223,  -223,  -223,  -223,  -223,
+    -223,  -223,  -223,  -223,  -223,   257,  -223,  -223,  -223,   372,
+      14,  -223,  -223,  -223,   343,  -223,   -94,  -223,  -223,  -181,
+     131,  -170,    -8,  -223,  -223,  -223,  -223,  -223,  -223,    28,
+    -223,  -223,   -58,  -223,  -121,   235,   237,   312,   -30,   344,
+     346,   384,  -130,  -223,  -223,  -223,   314,  -223,   359,   315,
+    -208,  -169,   366,   106,  -105,  -223,  -223,  -223,  -223,  -223,
+    -115,    -4,    98,  -223,  -223
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
       -1,    14,    15,    16,    17,    18,    19,    20,   167,   168,
-      88,   308,   309,   310,   222,   298,   299,   227,   367,   403,
-     447,   419,   420,   421,   422,   423,   364,   399,    21,    22,
-     165,   292,    23,    24,   151,   152,    25,    26,    43,    44,
-     132,    41,    89,    90,    91,   136,    92,   276,   271,   195,
-     196,   265,   266,   197,   278,   343,   390,   414,   433,   434,
-     452,   460,   115,   116,   173,   174,   175,   176,   177,    94,
+      88,   315,   316,   317,   229,   305,   306,   234,   374,   410,
+     454,   426,   427,   428,   429,   430,   371,   406,    21,    22,
+     165,   299,    23,    24,   151,   152,    25,    26,    43,    44,
+     132,    41,    89,    90,    91,   136,    92,   283,   278,   202,
+     203,   272,   273,   204,   285,   350,   397,   421,   440,   441,
+     459,   467,   115,   116,   173,   174,   175,   176,   177,    94,
       95,    96,    97,    98,    99,   183,   184,   124,   125,   187,
-     210,   100,   252,   101,   294,   249,   102,   141,   146,   157,
-     103,   355,    28,    29
+     217,   100,   195,   259,   101,   301,   256,   102,   141,   146,
+     157,   103,   362,    28,    29
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
@@ -1017,311 +1027,336 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      33,   181,    45,   180,   280,   236,   178,    42,    46,   156,
-      27,    93,   198,   297,   178,    31,   251,    32,   440,    31,
-     236,    32,   143,   144,   274,   236,    55,   236,   138,   139,
-      57,    58,    59,    60,    61,   322,    51,   273,   194,   123,
-     441,   185,   138,   139,    31,   200,    32,   185,   138,   139,
-     149,   234,   138,   139,    34,   263,   445,   133,   217,    30,
-     212,    39,   178,   450,   178,   109,   351,    50,   138,   139,
-     198,   119,   160,   161,   233,   314,   315,   316,    37,    31,
-     360,    32,   446,   361,   362,   451,    35,    40,    47,   142,
-     352,   353,   301,    10,   110,    45,   194,   275,    38,   122,
-     153,    46,   464,   158,   159,   182,   264,    93,   166,   169,
-     323,   150,   354,    36,   158,   340,   378,    54,   293,   465,
-     209,    56,   311,   254,   189,   302,   145,   300,   178,   178,
-     317,   199,   192,   303,   337,   442,   202,   391,   387,   193,
-     370,    10,   235,   182,   427,   205,   383,   150,   363,   198,
-     400,   320,   255,   328,   104,   258,   169,   267,   223,   224,
-     105,   218,   380,   268,   193,   346,   106,   304,    31,   216,
-      32,   269,   407,   114,   349,   194,   305,   209,   178,    31,
-     306,    32,   211,   284,   219,   138,   139,    10,   270,   199,
-      46,   107,   267,   307,    46,   140,   377,   290,   268,     1,
-     117,     2,   111,  -127,   291,   112,   269,   153,   138,   139,
-     118,   285,   220,   198,   331,   162,   163,   318,   251,   321,
-     376,   388,   295,   270,   327,   169,   214,   215,     3,   221,
-     406,   330,   392,   393,   178,    52,   228,   229,    53,   194,
-     138,   139,   138,   139,     4,     5,   395,    48,   404,    49,
-       6,   397,   345,   396,   332,     7,   253,   215,   281,   282,
-      46,   178,   324,   325,   344,   282,   356,   357,   199,   108,
-     358,   359,    46,     8,   394,   215,   113,    31,    62,    32,
-      63,   409,   282,   415,   215,   158,   374,   416,   215,   209,
-     432,   430,   282,     9,    64,    65,   206,   114,   382,    10,
-     120,   121,   372,   178,   431,   325,    67,    68,   438,   439,
-      11,   209,   126,   127,    69,    70,    12,   129,    13,   456,
-     439,    71,    72,    73,   462,   215,   128,   455,   207,    74,
-     467,   215,   199,   468,   215,    75,   130,   131,    76,   134,
-     137,   135,   463,   158,   158,   209,   466,   408,   155,    77,
-      78,   164,   179,   188,   190,   295,   191,    79,    80,   213,
-     238,   239,   240,   241,   242,   243,   244,   245,   246,   247,
-      81,   138,   139,    10,   225,   226,   429,    82,   236,   250,
-      83,    84,   435,   237,   259,   261,   260,   262,    85,   272,
-     277,   279,    86,   286,   424,   287,   288,    87,   208,   289,
-     296,   428,   319,   326,   329,   336,   338,   333,   339,   334,
-      31,    62,    32,    63,   335,   341,   170,   350,   149,   435,
-     443,   424,   248,   342,   347,   348,   366,    64,    65,   365,
-     369,   371,   385,   453,   368,   424,   386,   375,   158,    67,
-      68,   379,   389,    31,    62,    32,    63,    69,    70,   170,
-     401,   381,   398,   158,    71,    72,    73,   158,   402,   405,
-      64,    65,    74,   282,   411,   412,   413,   171,    75,   417,
-     426,    76,    67,    68,   436,   192,   437,   469,   425,   457,
-      69,    70,    77,    78,   449,   448,   459,    71,    72,    73,
-      79,    80,   461,   470,   373,    74,   444,   283,   231,   454,
-     171,    75,   154,    81,    76,   384,   201,    10,   458,   204,
-      82,   232,   312,    83,    84,    77,    78,   313,   203,   148,
-     186,    85,   418,    79,    80,    86,     0,   256,   410,     0,
-     172,    31,    62,    32,    63,     0,    81,     0,     0,     0,
-     257,     0,     0,    82,     0,     0,    83,    84,    64,    65,
-      66,     0,     0,     0,    85,     0,     0,     0,    86,     0,
-      67,    68,     0,   172,    31,    62,    32,    63,    69,    70,
-       0,     0,     0,     0,     0,    71,    72,    73,     0,     0,
-       0,    64,    65,    74,     0,     0,     0,     0,     0,    75,
-       0,     0,    76,    67,    68,     0,     0,     0,     0,     0,
-       0,    69,    70,    77,    78,     0,     0,     0,    71,    72,
-      73,    79,    80,     0,     0,     0,    74,     0,     0,     0,
-       0,     0,    75,     0,    81,    76,     0,     0,     0,     0,
-       0,    82,     0,     0,    83,    84,    77,    78,     0,     0,
-       0,     0,    85,     0,    79,    80,    86,     0,     0,     0,
-       0,    87,    31,    62,    32,    63,     0,    81,     0,     0,
-       0,     0,     0,     0,    82,     0,     0,    83,    84,    64,
-      65,     0,     0,     0,     0,    85,   122,     0,     0,    86,
-       0,    67,    68,     0,    87,    31,    62,    32,    63,    69,
-      70,     0,     0,     0,     0,     0,    71,    72,    73,     0,
-       0,     0,    64,    65,    74,     0,     0,     0,     0,     0,
-      75,     0,     0,    76,    67,    68,     0,     0,     0,     0,
-       0,     0,    69,    70,    77,    78,     0,     0,     0,    71,
-      72,    73,    79,    80,     0,     0,     0,    74,     0,     0,
-       0,     0,   171,    75,     0,    81,    76,     0,     0,    10,
-       0,     0,    82,     0,     0,    83,    84,    77,    78,     0,
-       0,     0,     0,    85,     0,    79,    80,    86,     0,     0,
-       0,     0,    87,    31,    62,    32,    63,     0,    81,     0,
+      33,   181,   243,   180,    45,   205,   287,    42,    46,   178,
+     258,    93,   156,   304,    27,   329,   149,   178,   243,   270,
+      31,   243,    32,   447,   281,    31,    55,    32,    30,   224,
+      57,    58,    59,    60,    61,   280,    31,   243,    32,   123,
+      51,   201,    50,   143,   144,   448,   109,   138,   139,   185,
+     185,   241,   114,    52,   138,   139,    53,   133,   138,   139,
+     367,    34,   218,   368,   369,   178,   452,   178,   138,   139,
+     205,    10,   321,   322,   323,   119,   110,   240,   207,   245,
+     246,   247,   248,   249,   250,   251,   252,   253,   254,   142,
+     138,   139,   225,   219,   330,   453,    35,    45,   344,    39,
+     153,    46,   282,   158,   159,   307,   201,    93,   166,   169,
+      10,   122,   182,   347,   158,   297,   226,   150,   385,   150,
+     216,   261,   298,   117,    36,   318,    40,    10,   300,   324,
+     402,   206,    37,   370,   118,    47,   209,   403,   178,   178,
+     394,   377,   242,   255,   398,   212,   449,   227,   335,   205,
+     407,   200,   262,   145,    38,   265,   169,   199,   230,   231,
+     434,   327,   200,   387,   390,   228,   308,    31,   182,    32,
+     384,   274,   353,   223,   414,   471,   138,   139,   275,   189,
+      54,   356,   138,   139,   216,   201,   276,   457,   178,    56,
+     291,   274,   104,   472,   271,   105,   206,    46,   275,   309,
+     199,    46,   190,  -127,   106,   277,   276,   107,   310,   413,
+     458,   160,   161,   205,   153,   358,   191,   192,   292,   258,
+     108,   138,   139,   111,   325,   277,   328,   113,   395,   302,
+     383,   334,   169,   338,   193,    31,   112,    32,   337,   399,
+     400,   114,   359,   360,   178,   311,    48,   120,    49,   201,
+     162,   163,   121,   194,   312,   411,   221,   222,   313,   352,
+      31,   404,    32,   339,   126,   361,   127,    46,   235,   236,
+     129,   178,   314,   260,   222,   206,   128,   138,   139,    46,
+     138,   139,   131,   288,   289,   331,   332,   140,   351,   289,
+     363,   364,   158,   381,   130,   134,   216,   439,   135,   365,
+     366,   401,   222,   416,   289,   389,   422,   222,   137,   379,
+     423,   222,   155,   178,   437,   289,   438,   332,   216,   445,
+     446,   463,   446,   469,   222,   474,   222,   164,    31,    62,
+      32,    63,   475,   222,   462,   179,   188,   197,   198,   206,
+      10,   232,   220,   233,   243,    64,    65,   213,   244,   470,
+     158,   158,   216,   473,   415,   257,   266,    67,    68,   267,
+     268,   279,   302,   269,   284,    69,    70,   286,   293,   294,
+     295,   296,    71,    72,   326,    73,   303,   336,   340,   333,
+     214,    74,   343,   436,   345,   341,   346,    75,   348,   442,
+      76,   342,   349,   354,    31,    62,    32,    63,   355,   357,
+     170,   431,    77,    78,   372,   149,   373,   375,   435,   376,
+      79,    64,    65,    80,   378,   382,   386,   408,   392,   388,
+     393,   396,   419,    67,    68,    81,   442,   450,   431,   405,
+     412,    69,    70,    82,   418,   409,    83,    84,    71,    72,
+     460,    73,   431,   289,    85,   158,   420,    74,   424,    86,
+     433,   443,   171,    75,    87,   215,    76,   444,   456,   466,
+     158,   476,   477,   432,   158,   455,   380,   464,    77,    78,
+     451,   468,   238,   461,   391,   290,    79,   154,   319,    80,
+     208,   320,   239,   186,   465,   210,   148,    31,    62,    32,
+      63,    81,   211,   170,   417,   196,    10,   263,   264,    82,
+       0,   425,    83,    84,    64,    65,     0,     0,     0,     0,
+      85,     0,     0,     0,     0,    86,    67,    68,     0,     0,
+     172,     0,     0,     0,    69,    70,     0,     0,     0,     0,
+       0,    71,    72,     0,    73,     0,     0,     0,     0,     0,
+      74,     0,     0,     0,     0,   171,    75,     0,     0,    76,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,    77,    78,     0,     0,     0,     0,     0,     0,    79,
+       0,     0,    80,     0,     0,     0,     0,     0,     0,     0,
+      31,    62,    32,    63,    81,     0,     0,     0,     0,     0,
+       0,     0,    82,     0,     0,    83,    84,    64,    65,    66,
+       0,     0,     0,    85,     0,     0,     0,     0,    86,    67,
+      68,     0,     0,   172,     0,     0,     0,    69,    70,     0,
+       0,     0,     0,     0,    71,    72,     0,    73,     0,     0,
+       0,     0,     0,    74,     0,     0,     0,     0,     0,    75,
+       0,     0,    76,     0,     0,     0,    31,    62,    32,    63,
+       0,     0,     0,     0,    77,    78,     0,     0,     0,     0,
+       0,     0,    79,    64,    65,    80,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,    67,    68,    81,     0,     0,
+       0,     0,     0,    69,    70,    82,     0,     0,    83,    84,
+      71,    72,     0,    73,     0,     0,    85,     0,     0,    74,
+       0,    86,     0,     0,     0,    75,    87,     0,    76,     0,
+       0,     0,    31,    62,    32,    63,     0,     0,     0,     0,
+      77,    78,     0,     0,     0,     0,     0,     0,    79,    64,
+      65,    80,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,    67,    68,    81,     0,     0,     0,     0,     0,    69,
+      70,    82,     0,     0,    83,    84,    71,    72,     0,    73,
+       0,     0,    85,   122,     0,    74,     0,    86,     0,     0,
+       0,    75,    87,     0,    76,     0,     0,     0,    31,    62,
+      32,    63,     0,     0,     0,     0,    77,    78,     0,     0,
+       0,     0,     0,     0,    79,    64,    65,    80,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    67,    68,    81,
+       0,     0,     0,     0,    10,    69,    70,    82,     0,     0,
+      83,    84,    71,    72,     0,    73,     0,     0,    85,     0,
+       0,    74,     0,    86,     0,     0,   171,    75,    87,     0,
+      76,     0,     0,     0,    31,    62,    32,    63,     0,     0,
+       0,     0,    77,    78,     0,     0,     0,     0,     0,     0,
+      79,    64,   147,    80,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    67,    68,    81,     0,     0,     0,     0,
+       0,    69,    70,    82,     0,     0,    83,    84,    71,    72,
+       0,    73,     0,     0,    85,     0,     0,    74,     0,    86,
+       0,     0,     0,    75,   172,     0,    76,     0,     0,     0,
+      31,    62,    32,    63,     0,     0,     0,     0,    77,    78,
+       0,     0,     1,     0,     2,     0,    79,    64,    65,    80,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,    67,
+      68,    81,     0,     0,     0,     0,     0,    69,    70,    82,
+       0,     3,    83,    84,    71,    72,     0,    73,     0,     0,
+      85,     0,     0,    74,     0,    86,     0,     4,     5,    75,
+      87,     0,    76,     0,     6,     0,     0,     0,     0,     7,
+       0,     0,     0,     0,    77,    78,     0,     0,     0,     0,
+       0,     0,    79,     0,     0,    80,     0,     0,     8,     0,
+       0,     0,     0,     0,     0,     0,    62,    81,    63,     0,
        0,     0,     0,     0,     0,    82,     0,     0,    83,    84,
-      64,   147,     0,     0,     0,     0,    85,     0,     0,     0,
-      86,     0,    67,    68,     0,   172,    31,    62,    32,    63,
-      69,    70,     0,     0,     0,     0,     0,    71,    72,    73,
-       0,     0,     0,    64,    65,    74,     0,     0,     0,     0,
-       0,    75,     0,     0,    76,    67,    68,     0,     0,     0,
-       0,     0,     0,    69,    70,    77,    78,     0,     0,     0,
-      71,    72,    73,    79,    80,     0,     0,     0,    74,     0,
-       0,     0,     0,     0,    75,     0,    81,    76,     0,     0,
-       0,     0,     0,    82,     0,     0,    83,    84,    77,    78,
-       0,     0,     0,     0,    85,     0,    79,    80,    86,     0,
-       0,     0,     0,    87,     0,    62,     0,    63,     0,    81,
-       0,     0,     0,     0,     0,     0,    82,     0,     0,    83,
-      84,    64,   147,     0,     0,     0,     0,    85,    67,    68,
-       0,    86,     0,    67,    68,     0,    87,    70,     0,     0,
-       0,     0,    70,    71,    72,    73,     0,     0,    71,    72,
-      73,    74,     0,     0,     0,     0,    74,     0,     0,     0,
-      76,     0,     0,     0,     0,    76,     0,     0,     0,     0,
-       0,    77,   230,     0,     0,     0,    77,    78,     0,    79,
-       0,     0,     0,     0,    79,    80,     0,     0,     0,     0,
-       0,     0,    81,     0,     0,     0,     0,    81,     0,    82,
-       0,     0,    83,    84,    82,     0,     0,    83,    84,     0,
-      85,     0,     0,     0,    86,    85,     0,     0,     0,    86,
-     238,   239,   240,   241,   242,   243,   244,   245,   246,   247,
-       0,   138,   139,     0,     0,     0,     0,     0,     0,     0,
+       9,     0,    64,   147,    67,    68,    85,    10,     0,     0,
+       0,    86,     0,    70,    67,    68,    87,     0,    11,    71,
+      72,     0,    73,    70,    12,     0,     0,    13,    74,    71,
+      72,     0,    73,     0,     0,     0,     0,    76,    74,     0,
+       0,     0,     0,     0,     0,     0,     0,    76,     0,    77,
+     237,     0,     0,     0,     0,     0,     0,    79,     0,    77,
+      78,     0,     0,     0,     0,     0,     0,    79,     0,     0,
+      80,     0,    81,     0,     0,     0,     0,     0,     0,     0,
+      82,     0,    81,    83,    84,     0,     0,     0,     0,     0,
+      82,    85,     0,    83,    84,     0,    86,     0,     0,     0,
+       0,    85,     0,     0,     0,     0,    86,   245,   246,   247,
+     248,   249,   250,   251,   252,   253,   254,     0,   138,   139,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,   248
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,   255
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   122,    12,   118,   207,     8,   114,    11,    12,   106,
-       0,    41,   135,   228,   122,     4,   179,     6,     7,     4,
-       8,     6,    23,    24,    37,     8,    30,     8,    21,    22,
-      34,    35,    36,    37,    38,    10,    26,   198,   135,    69,
-      29,    58,    21,    22,     4,   136,     6,    58,    21,    22,
-      27,   172,    21,    22,    73,   193,    70,    87,   160,   105,
-     151,    29,   170,    32,   172,    28,    63,    75,    21,    22,
-     193,    61,    44,    45,   171,    17,    18,    19,   105,     4,
-      35,     6,    96,    38,    39,    54,   105,    55,     0,    93,
-      87,    88,    10,   101,    57,   105,   193,   110,    78,   116,
-     104,   105,    64,   107,   108,   116,    31,   137,   112,   113,
-      85,   124,   109,    67,   118,   276,   319,     3,   226,    81,
-     150,    67,   125,   106,   128,    43,   127,   229,   236,   237,
-      72,   135,   125,    51,   272,   124,   140,   125,   341,   124,
-     303,   101,   172,   116,   125,   149,   125,   124,   103,   272,
-     365,   248,   182,   106,   102,   185,   160,    68,   162,   163,
-     126,    43,   325,    74,   124,   286,    31,    85,     4,   159,
-       6,    82,   375,   117,   289,   272,    94,   207,   286,     4,
-      98,     6,   126,   213,    66,    21,    22,   101,    99,   193,
-     194,   124,    68,   111,   198,    31,     9,    53,    74,     1,
-     114,     3,     5,    79,    60,    88,    82,   211,    21,    22,
-     124,   215,    94,   336,   125,    44,    45,   247,   381,   249,
-     317,   342,   226,    99,   254,   229,   125,   126,    30,   111,
-       9,   261,   347,   348,   342,   120,   125,   126,   123,   336,
-      21,    22,    21,    22,    46,    47,    53,   120,   369,   122,
-      52,   359,   282,    60,   264,    57,   125,   126,   125,   126,
-     264,   369,   125,   126,   125,   126,   125,   126,   272,    78,
-     125,   126,   276,    75,   125,   126,   124,     4,     5,     6,
-       7,   125,   126,   125,   126,   289,   316,   125,   126,   319,
-     411,   125,   126,    95,    21,    22,    23,   117,   328,   101,
-       7,     7,   306,   411,   125,   126,    33,    34,   125,   126,
-     112,   341,   124,    77,    41,    42,   118,     5,   120,   125,
-     126,    48,    49,    50,   125,   126,   124,   442,    55,    56,
-     125,   126,   336,   125,   126,    62,   124,    77,    65,     5,
-     126,    67,   457,   347,   348,   375,   461,   377,   124,    76,
-      77,   118,   124,     7,     7,   359,   125,    84,    85,    11,
-      10,    11,    12,    13,    14,    15,    16,    17,    18,    19,
-      97,    21,    22,   101,   124,   124,   406,   104,     8,    20,
-     107,   108,   412,     9,    59,    67,   125,   125,   115,   126,
-      69,   125,   119,   124,   398,    80,    80,   124,   125,   124,
-     113,   405,   124,   114,    59,    79,     7,    90,     7,    90,
-       4,     5,     6,     7,    90,    40,    10,     5,    27,   449,
-     424,   425,    72,    71,   124,   124,   118,    21,    22,   126,
-     124,    80,    93,   437,    85,   439,    93,   124,   442,    33,
-      34,    85,    89,     4,     5,     6,     7,    41,    42,    10,
-      36,   124,   124,   457,    48,    49,    50,   461,    91,   124,
-      21,    22,    56,   126,    88,    40,    83,    61,    62,     5,
-      40,    65,    33,    34,     7,   125,    98,    92,   124,   124,
-      41,    42,    76,    77,   126,   125,    86,    48,    49,    50,
-      84,    85,   124,     7,   309,    56,   425,   211,   169,   439,
-      61,    62,   105,    97,    65,   336,   137,   101,   449,   146,
-     104,   170,   236,   107,   108,    76,    77,   237,   141,   102,
-     124,   115,   396,    84,    85,   119,    -1,   183,   381,    -1,
-     124,     4,     5,     6,     7,    -1,    97,    -1,    -1,    -1,
-     183,    -1,    -1,   104,    -1,    -1,   107,   108,    21,    22,
-      23,    -1,    -1,    -1,   115,    -1,    -1,    -1,   119,    -1,
-      33,    34,    -1,   124,     4,     5,     6,     7,    41,    42,
-      -1,    -1,    -1,    -1,    -1,    48,    49,    50,    -1,    -1,
-      -1,    21,    22,    56,    -1,    -1,    -1,    -1,    -1,    62,
-      -1,    -1,    65,    33,    34,    -1,    -1,    -1,    -1,    -1,
-      -1,    41,    42,    76,    77,    -1,    -1,    -1,    48,    49,
-      50,    84,    85,    -1,    -1,    -1,    56,    -1,    -1,    -1,
-      -1,    -1,    62,    -1,    97,    65,    -1,    -1,    -1,    -1,
-      -1,   104,    -1,    -1,   107,   108,    76,    77,    -1,    -1,
-      -1,    -1,   115,    -1,    84,    85,   119,    -1,    -1,    -1,
-      -1,   124,     4,     5,     6,     7,    -1,    97,    -1,    -1,
-      -1,    -1,    -1,    -1,   104,    -1,    -1,   107,   108,    21,
-      22,    -1,    -1,    -1,    -1,   115,   116,    -1,    -1,   119,
-      -1,    33,    34,    -1,   124,     4,     5,     6,     7,    41,
-      42,    -1,    -1,    -1,    -1,    -1,    48,    49,    50,    -1,
-      -1,    -1,    21,    22,    56,    -1,    -1,    -1,    -1,    -1,
-      62,    -1,    -1,    65,    33,    34,    -1,    -1,    -1,    -1,
-      -1,    -1,    41,    42,    76,    77,    -1,    -1,    -1,    48,
-      49,    50,    84,    85,    -1,    -1,    -1,    56,    -1,    -1,
-      -1,    -1,    61,    62,    -1,    97,    65,    -1,    -1,   101,
-      -1,    -1,   104,    -1,    -1,   107,   108,    76,    77,    -1,
-      -1,    -1,    -1,   115,    -1,    84,    85,   119,    -1,    -1,
-      -1,    -1,   124,     4,     5,     6,     7,    -1,    97,    -1,
-      -1,    -1,    -1,    -1,    -1,   104,    -1,    -1,   107,   108,
-      21,    22,    -1,    -1,    -1,    -1,   115,    -1,    -1,    -1,
-     119,    -1,    33,    34,    -1,   124,     4,     5,     6,     7,
-      41,    42,    -1,    -1,    -1,    -1,    -1,    48,    49,    50,
-      -1,    -1,    -1,    21,    22,    56,    -1,    -1,    -1,    -1,
-      -1,    62,    -1,    -1,    65,    33,    34,    -1,    -1,    -1,
-      -1,    -1,    -1,    41,    42,    76,    77,    -1,    -1,    -1,
-      48,    49,    50,    84,    85,    -1,    -1,    -1,    56,    -1,
-      -1,    -1,    -1,    -1,    62,    -1,    97,    65,    -1,    -1,
-      -1,    -1,    -1,   104,    -1,    -1,   107,   108,    76,    77,
-      -1,    -1,    -1,    -1,   115,    -1,    84,    85,   119,    -1,
-      -1,    -1,    -1,   124,    -1,     5,    -1,     7,    -1,    97,
-      -1,    -1,    -1,    -1,    -1,    -1,   104,    -1,    -1,   107,
-     108,    21,    22,    -1,    -1,    -1,    -1,   115,    33,    34,
-      -1,   119,    -1,    33,    34,    -1,   124,    42,    -1,    -1,
-      -1,    -1,    42,    48,    49,    50,    -1,    -1,    48,    49,
-      50,    56,    -1,    -1,    -1,    -1,    56,    -1,    -1,    -1,
-      65,    -1,    -1,    -1,    -1,    65,    -1,    -1,    -1,    -1,
-      -1,    76,    77,    -1,    -1,    -1,    76,    77,    -1,    84,
-      -1,    -1,    -1,    -1,    84,    85,    -1,    -1,    -1,    -1,
-      -1,    -1,    97,    -1,    -1,    -1,    -1,    97,    -1,   104,
-      -1,    -1,   107,   108,   104,    -1,    -1,   107,   108,    -1,
-     115,    -1,    -1,    -1,   119,   115,    -1,    -1,    -1,   119,
-      10,    11,    12,    13,    14,    15,    16,    17,    18,    19,
-      -1,    21,    22,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+       4,   122,     8,   118,    12,   135,   214,    11,    12,   114,
+     179,    41,   106,   235,     0,    10,    27,   122,     8,   200,
+       4,     8,     6,     7,    37,     4,    30,     6,   110,   160,
+      34,    35,    36,    37,    38,   205,     4,     8,     6,    69,
+      26,   135,    77,    23,    24,    29,    28,    21,    22,    59,
+      59,   172,   122,   126,    21,    22,   129,    87,    21,    22,
+      35,    75,   132,    38,    39,   170,    71,   172,    21,    22,
+     200,   106,    17,    18,    19,    61,    58,   171,   136,    10,
+      11,    12,    13,    14,    15,    16,    17,    18,    19,    93,
+      21,    22,    43,   151,    89,   100,   110,   105,   279,    29,
+     104,   105,   115,   107,   108,   236,   200,   137,   112,   113,
+     106,   121,   121,   283,   118,    54,    67,   130,   326,   130,
+     150,   111,    61,   119,    68,   131,    56,   106,   233,    74,
+      54,   135,   110,   108,   130,     0,   140,    61,   243,   244,
+     348,   310,   172,    74,   131,   149,   130,    98,   111,   279,
+     372,   130,   182,   133,    80,   185,   160,   131,   162,   163,
+     131,   255,   130,   332,   131,   116,    10,     4,   121,     6,
+       9,    69,   293,   159,   382,    65,    21,    22,    76,    50,
+       3,   296,    21,    22,   214,   279,    84,    32,   293,    68,
+     220,    69,   107,    83,    31,   132,   200,   201,    76,    43,
+     131,   205,    73,    81,    31,   103,    84,   130,    52,     9,
+      55,    44,    45,   343,   218,    64,    87,    88,   222,   388,
+      80,    21,    22,     5,   254,   103,   256,   130,   349,   233,
+     324,   261,   236,   131,   105,     4,    92,     6,   268,   354,
+     355,   122,    91,    92,   349,    89,   126,     7,   128,   343,
+      44,    45,     7,   124,    98,   376,   131,   132,   102,   289,
+       4,   366,     6,   271,   130,   114,    79,   271,   131,   132,
+       5,   376,   116,   131,   132,   279,   130,    21,    22,   283,
+      21,    22,    79,   131,   132,   131,   132,    31,   131,   132,
+     131,   132,   296,   323,   130,     5,   326,   418,    68,   131,
+     132,   131,   132,   131,   132,   335,   131,   132,   132,   313,
+     131,   132,   130,   418,   131,   132,   131,   132,   348,   131,
+     132,   131,   132,   131,   132,   131,   132,   123,     4,     5,
+       6,     7,   131,   132,   449,   130,     7,     7,   131,   343,
+     106,   130,    11,   130,     8,    21,    22,    23,     9,   464,
+     354,   355,   382,   468,   384,    20,    60,    33,    34,   131,
+      68,   132,   366,   131,    70,    41,    42,   131,   130,    82,
+      82,   130,    48,    49,   130,    51,   118,    60,    94,   119,
+      56,    57,    81,   413,     7,    94,     7,    63,    40,   419,
+      66,    94,    72,   130,     4,     5,     6,     7,   130,     5,
+      10,   405,    78,    79,   132,    27,   123,    89,   412,   130,
+      86,    21,    22,    89,    82,   130,    89,    36,    97,   130,
+      97,    93,    40,    33,    34,   101,   456,   431,   432,   130,
+     130,    41,    42,   109,    92,    95,   112,   113,    48,    49,
+     444,    51,   446,   132,   120,   449,    85,    57,     5,   125,
+      40,     7,    62,    63,   130,   131,    66,   102,   132,    90,
+     464,    96,     7,   130,   468,   131,   316,   130,    78,    79,
+     432,   130,   169,   446,   343,   218,    86,   105,   243,    89,
+     137,   244,   170,   124,   456,   141,   102,     4,     5,     6,
+       7,   101,   146,    10,   388,   129,   106,   183,   183,   109,
+      -1,   403,   112,   113,    21,    22,    -1,    -1,    -1,    -1,
+     120,    -1,    -1,    -1,    -1,   125,    33,    34,    -1,    -1,
+     130,    -1,    -1,    -1,    41,    42,    -1,    -1,    -1,    -1,
+      -1,    48,    49,    -1,    51,    -1,    -1,    -1,    -1,    -1,
+      57,    -1,    -1,    -1,    -1,    62,    63,    -1,    -1,    66,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    78,    79,    -1,    -1,    -1,    -1,    -1,    -1,    86,
+      -1,    -1,    89,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+       4,     5,     6,     7,   101,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,   109,    -1,    -1,   112,   113,    21,    22,    23,
+      -1,    -1,    -1,   120,    -1,    -1,    -1,    -1,   125,    33,
+      34,    -1,    -1,   130,    -1,    -1,    -1,    41,    42,    -1,
+      -1,    -1,    -1,    -1,    48,    49,    -1,    51,    -1,    -1,
+      -1,    -1,    -1,    57,    -1,    -1,    -1,    -1,    -1,    63,
+      -1,    -1,    66,    -1,    -1,    -1,     4,     5,     6,     7,
+      -1,    -1,    -1,    -1,    78,    79,    -1,    -1,    -1,    -1,
+      -1,    -1,    86,    21,    22,    89,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    33,    34,   101,    -1,    -1,
+      -1,    -1,    -1,    41,    42,   109,    -1,    -1,   112,   113,
+      48,    49,    -1,    51,    -1,    -1,   120,    -1,    -1,    57,
+      -1,   125,    -1,    -1,    -1,    63,   130,    -1,    66,    -1,
+      -1,    -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,
+      78,    79,    -1,    -1,    -1,    -1,    -1,    -1,    86,    21,
+      22,    89,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    33,    34,   101,    -1,    -1,    -1,    -1,    -1,    41,
+      42,   109,    -1,    -1,   112,   113,    48,    49,    -1,    51,
+      -1,    -1,   120,   121,    -1,    57,    -1,   125,    -1,    -1,
+      -1,    63,   130,    -1,    66,    -1,    -1,    -1,     4,     5,
+       6,     7,    -1,    -1,    -1,    -1,    78,    79,    -1,    -1,
+      -1,    -1,    -1,    -1,    86,    21,    22,    89,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    33,    34,   101,
+      -1,    -1,    -1,    -1,   106,    41,    42,   109,    -1,    -1,
+     112,   113,    48,    49,    -1,    51,    -1,    -1,   120,    -1,
+      -1,    57,    -1,   125,    -1,    -1,    62,    63,   130,    -1,
+      66,    -1,    -1,    -1,     4,     5,     6,     7,    -1,    -1,
+      -1,    -1,    78,    79,    -1,    -1,    -1,    -1,    -1,    -1,
+      86,    21,    22,    89,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    33,    34,   101,    -1,    -1,    -1,    -1,
+      -1,    41,    42,   109,    -1,    -1,   112,   113,    48,    49,
+      -1,    51,    -1,    -1,   120,    -1,    -1,    57,    -1,   125,
+      -1,    -1,    -1,    63,   130,    -1,    66,    -1,    -1,    -1,
+       4,     5,     6,     7,    -1,    -1,    -1,    -1,    78,    79,
+      -1,    -1,     1,    -1,     3,    -1,    86,    21,    22,    89,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    33,
+      34,   101,    -1,    -1,    -1,    -1,    -1,    41,    42,   109,
+      -1,    30,   112,   113,    48,    49,    -1,    51,    -1,    -1,
+     120,    -1,    -1,    57,    -1,   125,    -1,    46,    47,    63,
+     130,    -1,    66,    -1,    53,    -1,    -1,    -1,    -1,    58,
+      -1,    -1,    -1,    -1,    78,    79,    -1,    -1,    -1,    -1,
+      -1,    -1,    86,    -1,    -1,    89,    -1,    -1,    77,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,     5,   101,     7,    -1,
+      -1,    -1,    -1,    -1,    -1,   109,    -1,    -1,   112,   113,
+      99,    -1,    21,    22,    33,    34,   120,   106,    -1,    -1,
+      -1,   125,    -1,    42,    33,    34,   130,    -1,   117,    48,
+      49,    -1,    51,    42,   123,    -1,    -1,   126,    57,    48,
+      49,    -1,    51,    -1,    -1,    -1,    -1,    66,    57,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    66,    -1,    78,
+      79,    -1,    -1,    -1,    -1,    -1,    -1,    86,    -1,    78,
+      79,    -1,    -1,    -1,    -1,    -1,    -1,    86,    -1,    -1,
+      89,    -1,   101,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+     109,    -1,   101,   112,   113,    -1,    -1,    -1,    -1,    -1,
+     109,   120,    -1,   112,   113,    -1,   125,    -1,    -1,    -1,
+      -1,   120,    -1,    -1,    -1,    -1,   125,    10,    11,    12,
+      13,    14,    15,    16,    17,    18,    19,    -1,    21,    22,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    72
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    74
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
      symbol of state STATE-NUM.  */
 static const yytype_uint8 yystos[] =
 {
-       0,     1,     3,    30,    46,    47,    52,    57,    75,    95,
-     101,   112,   118,   120,   129,   130,   131,   132,   133,   134,
-     135,   156,   157,   160,   161,   164,   165,   168,   220,   221,
-     105,     4,     6,   218,    73,   105,    67,   105,    78,    29,
-      55,   169,   218,   166,   167,   180,   218,     0,   120,   122,
-      75,   168,   120,   123,     3,   218,    67,   218,   218,   218,
-     218,   218,     5,     7,    21,    22,    23,    33,    34,    41,
-      42,    48,    49,    50,    56,    62,    65,    76,    77,    84,
-      85,    97,   104,   107,   108,   115,   119,   124,   138,   170,
-     171,   172,   174,   196,   197,   198,   199,   200,   201,   202,
-     209,   211,   214,   218,   102,   126,    31,   124,    78,    28,
-      57,     5,    88,   124,   117,   190,   191,   114,   124,   168,
-       7,     7,   116,   196,   205,   206,   124,    77,   124,     5,
-     124,    77,   168,   196,     5,    67,   173,   126,    21,    22,
-      31,   215,   218,    23,    24,   127,   216,    22,   199,    27,
-     124,   162,   163,   218,   167,   124,   174,   217,   218,   218,
-      44,    45,    44,    45,   118,   158,   218,   136,   137,   218,
-      10,    61,   124,   192,   193,   194,   195,   196,   211,   124,
-     217,   192,   116,   203,   204,    58,   206,   207,     7,   218,
-       7,   125,   125,   124,   174,   177,   178,   181,   200,   218,
-     190,   172,   218,   197,   198,   218,    23,    55,   125,   196,
-     208,   126,   190,    11,   125,   126,   168,   136,    43,    66,
-      94,   111,   142,   218,   218,   124,   124,   145,   125,   126,
-      77,   138,   195,   174,   192,   196,     8,     9,    10,    11,
-      12,    13,    14,    15,    16,    17,    18,    19,    72,   213,
-      20,   209,   210,   125,   106,   196,   204,   207,   196,    59,
-     125,    67,   125,   177,    31,   179,   180,    68,    74,    82,
-      99,   176,   126,   179,    37,   110,   175,    69,   182,   125,
-     208,   125,   126,   163,   196,   218,   124,    80,    80,   124,
-      53,    60,   159,   211,   212,   218,   113,   142,   143,   144,
-     136,    10,    43,    51,    85,    94,    98,   111,   139,   140,
-     141,   125,   193,   194,    17,    18,    19,    72,   196,   124,
-     174,   196,    10,    85,   125,   126,   114,   196,   106,    59,
-     196,   125,   180,    90,    90,    90,    79,   177,     7,     7,
-     179,    40,    71,   183,   125,   196,   192,   124,   124,   217,
-       5,    63,    87,    88,   109,   219,   125,   126,   125,   126,
-      35,    38,    39,   103,   154,   126,   118,   146,    85,   124,
-     209,    80,   218,   139,   196,   124,   174,     9,   208,    85,
-     209,   124,   196,   125,   178,    93,    93,   208,   192,    89,
-     184,   125,   217,   217,   125,    53,    60,   211,   124,   155,
-     142,    36,    91,   147,   192,   124,     9,   208,   196,   125,
-     210,    88,    40,    83,   185,   125,   125,     5,   219,   149,
-     150,   151,   152,   153,   218,   124,    40,   125,   218,   196,
-     125,   125,   192,   186,   187,   196,     7,    98,   125,   126,
-       7,    29,   124,   218,   149,    70,    96,   148,   125,   126,
-      32,    54,   188,   218,   150,   217,   125,   124,   187,    86,
-     189,   124,   125,   217,    64,    81,   217,   125,   125,    92,
-       7
+       0,     1,     3,    30,    46,    47,    53,    58,    77,    99,
+     106,   117,   123,   126,   135,   136,   137,   138,   139,   140,
+     141,   162,   163,   166,   167,   170,   171,   174,   227,   228,
+     110,     4,     6,   225,    75,   110,    68,   110,    80,    29,
+      56,   175,   225,   172,   173,   186,   225,     0,   126,   128,
+      77,   174,   126,   129,     3,   225,    68,   225,   225,   225,
+     225,   225,     5,     7,    21,    22,    23,    33,    34,    41,
+      42,    48,    49,    51,    57,    63,    66,    78,    79,    86,
+      89,   101,   109,   112,   113,   120,   125,   130,   144,   176,
+     177,   178,   180,   202,   203,   204,   205,   206,   207,   208,
+     215,   218,   221,   225,   107,   132,    31,   130,    80,    28,
+      58,     5,    92,   130,   122,   196,   197,   119,   130,   174,
+       7,     7,   121,   202,   211,   212,   130,    79,   130,     5,
+     130,    79,   174,   202,     5,    68,   179,   132,    21,    22,
+      31,   222,   225,    23,    24,   133,   223,    22,   205,    27,
+     130,   168,   169,   225,   173,   130,   180,   224,   225,   225,
+      44,    45,    44,    45,   123,   164,   225,   142,   143,   225,
+      10,    62,   130,   198,   199,   200,   201,   202,   218,   130,
+     224,   198,   121,   209,   210,    59,   212,   213,     7,    50,
+      73,    87,    88,   105,   124,   216,   216,     7,   131,   131,
+     130,   180,   183,   184,   187,   206,   225,   196,   178,   225,
+     203,   204,   225,    23,    56,   131,   202,   214,   132,   196,
+      11,   131,   132,   174,   142,    43,    67,    98,   116,   148,
+     225,   225,   130,   130,   151,   131,   132,    79,   144,   201,
+     180,   198,   202,     8,     9,    10,    11,    12,    13,    14,
+      15,    16,    17,    18,    19,    74,   220,    20,   215,   217,
+     131,   111,   202,   210,   213,   202,    60,   131,    68,   131,
+     183,    31,   185,   186,    69,    76,    84,   103,   182,   132,
+     185,    37,   115,   181,    70,   188,   131,   214,   131,   132,
+     169,   202,   225,   130,    82,    82,   130,    54,    61,   165,
+     218,   219,   225,   118,   148,   149,   150,   142,    10,    43,
+      52,    89,    98,   102,   116,   145,   146,   147,   131,   199,
+     200,    17,    18,    19,    74,   202,   130,   180,   202,    10,
+      89,   131,   132,   119,   202,   111,    60,   202,   131,   186,
+      94,    94,    94,    81,   183,     7,     7,   185,    40,    72,
+     189,   131,   202,   198,   130,   130,   224,     5,    64,    91,
+      92,   114,   226,   131,   132,   131,   132,    35,    38,    39,
+     108,   160,   132,   123,   152,    89,   130,   215,    82,   225,
+     145,   202,   130,   180,     9,   214,    89,   215,   130,   202,
+     131,   184,    97,    97,   214,   198,    93,   190,   131,   224,
+     224,   131,    54,    61,   218,   130,   161,   148,    36,    95,
+     153,   198,   130,     9,   214,   202,   131,   217,    92,    40,
+      85,   191,   131,   131,     5,   226,   155,   156,   157,   158,
+     159,   225,   130,    40,   131,   225,   202,   131,   131,   198,
+     192,   193,   202,     7,   102,   131,   132,     7,    29,   130,
+     225,   155,    71,   100,   154,   131,   132,    32,    55,   194,
+     225,   156,   224,   131,   130,   193,    90,   195,   130,   131,
+     224,    65,    83,   224,   131,   131,    96,     7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
 static const yytype_uint8 yyr1[] =
 {
-       0,   128,   129,   129,   129,   129,   129,   129,   130,   130,
-     130,   130,   130,   130,   130,   130,   130,   130,   131,   132,
-     132,   132,   132,   133,   134,   135,   136,   137,   137,   138,
-     138,   138,   138,   138,   138,   138,   138,   138,   138,   138,
-     138,   138,   138,   138,   138,   138,   138,   139,   139,   139,
-     139,   139,   139,   139,   140,   140,   141,   141,   142,   142,
-     142,   142,   143,   143,   144,   144,   145,   145,   146,   146,
-     147,   147,   148,   148,   149,   149,   150,   150,   150,   151,
-     151,   152,   153,   154,   154,   154,   154,   155,   155,   156,
-     156,   156,   156,   157,   158,   158,   159,   159,   159,   159,
-     160,   161,   162,   162,   163,   164,   164,   165,   166,   166,
-     167,   168,   169,   169,   169,   170,   170,   171,   171,   172,
-     172,   172,   173,   174,   175,   175,   175,   176,   176,   176,
-     176,   176,   176,   176,   176,   177,   177,   178,   178,   178,
-     178,   178,   178,   179,   179,   180,   180,   181,   181,   182,
-     182,   183,   183,   184,   184,   185,   185,   186,   186,   187,
-     188,   188,   188,   189,   189,   189,   190,   190,   191,   192,
-     192,   193,   193,   194,   194,   195,   195,   195,   195,   195,
-     195,   195,   195,   195,   195,   195,   196,   196,   197,   197,
-     198,   198,   199,   199,   199,   199,   199,   199,   199,   200,
-     200,   200,   200,   201,   202,   202,   203,   203,   204,   205,
-     205,   206,   207,   207,   208,   208,   209,   209,   209,   209,
-     209,   209,   209,   210,   210,   211,   211,   212,   212,   213,
-     213,   213,   213,   213,   213,   213,   213,   213,   213,   214,
-     215,   215,   216,   216,   216,   217,   217,   218,   218,   219,
-     219,   219,   219,   220,   221,   221
+       0,   134,   135,   135,   135,   135,   135,   135,   136,   136,
+     136,   136,   136,   136,   136,   136,   136,   136,   137,   138,
+     138,   138,   138,   139,   140,   141,   142,   143,   143,   144,
+     144,   144,   144,   144,   144,   144,   144,   144,   144,   144,
+     144,   144,   144,   144,   144,   144,   144,   145,   145,   145,
+     145,   145,   145,   145,   146,   146,   147,   147,   148,   148,
+     148,   148,   149,   149,   150,   150,   151,   151,   152,   152,
+     153,   153,   154,   154,   155,   155,   156,   156,   156,   157,
+     157,   158,   159,   160,   160,   160,   160,   161,   161,   162,
+     162,   162,   162,   163,   164,   164,   165,   165,   165,   165,
+     166,   167,   168,   168,   169,   170,   170,   171,   172,   172,
+     173,   174,   175,   175,   175,   176,   176,   177,   177,   178,
+     178,   178,   179,   180,   181,   181,   181,   182,   182,   182,
+     182,   182,   182,   182,   182,   183,   183,   184,   184,   184,
+     184,   184,   184,   185,   185,   186,   186,   187,   187,   188,
+     188,   189,   189,   190,   190,   191,   191,   192,   192,   193,
+     194,   194,   194,   195,   195,   195,   196,   196,   197,   198,
+     198,   199,   199,   200,   200,   201,   201,   201,   201,   201,
+     201,   201,   201,   201,   201,   201,   202,   202,   203,   203,
+     204,   204,   205,   205,   205,   205,   205,   205,   205,   206,
+     206,   206,   206,   207,   208,   208,   209,   209,   210,   211,
+     211,   212,   213,   213,   214,   214,   215,   215,   215,   215,
+     215,   215,   215,   215,   216,   216,   216,   216,   216,   216,
+     217,   217,   218,   218,   219,   219,   220,   220,   220,   220,
+     220,   220,   220,   220,   220,   220,   221,   222,   222,   223,
+     223,   223,   224,   224,   225,   225,   226,   226,   226,   226,
+     227,   228,   228
 };
 
   /* YYR2[YYN] -- Number of symbols on the right hand side of rule YYN.  */
@@ -1349,10 +1384,11 @@ static const yytype_uint8 yyr2[] =
        2,     1,     1,     1,     1,     1,     1,     3,     1,     3,
        4,     4,     5,     6,     5,     4,     1,     2,     4,     1,
        2,     4,     0,     2,     1,     3,     1,     1,     2,     2,
-       1,     2,     2,     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,     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
 };
 
 
@@ -1849,909 +1885,919 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
   switch (yytype)
     {
           case 3: /* TOKEN_COMMAND  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1859 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1895 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 4: /* TOKEN_NAME  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1869 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1905 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 5: /* TOKEN_STRING_SINGLE_QUOTED  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1879 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1915 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 6: /* TOKEN_STRING_DOUBLE_QUOTED  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1889 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1925 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 7: /* TOKEN_UNSIGNED_NUMVAL  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).numeric_literal_value_) != nullptr) {
     delete ((*yyvaluep).numeric_literal_value_);
   }
 }
-#line 1899 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1935 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 130: /* sql_statement  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 136: /* sql_statement  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 1909 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1945 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 131: /* quit_statement  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 137: /* quit_statement  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).quit_statement_) != nullptr) {
     delete ((*yyvaluep).quit_statement_);
   }
 }
-#line 1919 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1955 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 132: /* alter_table_statement  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 138: /* alter_table_statement  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 1929 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1965 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 133: /* create_table_statement  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 139: /* create_table_statement  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).create_table_statement_) != nullptr) {
     delete ((*yyvaluep).create_table_statement_);
   }
 }
-#line 1939 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1975 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 134: /* create_index_statement  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 140: /* create_index_statement  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 1949 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1985 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 135: /* drop_table_statement  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 141: /* drop_table_statement  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).drop_table_statement_) != nullptr) {
     delete ((*yyvaluep).drop_table_statement_);
   }
 }
-#line 1959 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1995 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 136: /* column_def  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 142: /* column_def  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_definition_) != nullptr) {
     delete ((*yyvaluep).attribute_definition_);
   }
 }
-#line 1969 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2005 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 137: /* column_def_commalist  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 143: /* column_def_commalist  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_definition_list_) != nullptr) {
     delete ((*yyvaluep).attribute_definition_list_);
   }
 }
-#line 1979 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2015 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 138: /* data_type  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 144: /* data_type  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).data_type_) != nullptr) {
     delete ((*yyvaluep).data_type_);
   }
 }
-#line 1989 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2025 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 139: /* column_constraint_def  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 145: /* column_constraint_def  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_) != nullptr) {
     delete ((*yyvaluep).column_constraint_);
   }
 }
-#line 1999 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2035 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 140: /* column_constraint_def_list  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 146: /* column_constraint_def_list  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_list_) != nullptr) {
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 2009 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2045 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 141: /* opt_column_constraint_def_list  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 147: /* opt_column_constraint_def_list  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_list_) != nullptr) {
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 2019 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2055 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 145: /* opt_column_list  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 151: /* opt_column_list  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_list_) != nullptr) {
     delete ((*yyvaluep).attribute_list_);
   }
 }
-#line 2029 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2065 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 146: /* opt_block_properties  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 152: /* opt_block_properties  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).block_properties_) != nullptr) {
     delete ((*yyvaluep).block_properties_);
   }
 }
-#line 2039 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2075 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 147: /* opt_partition_clause  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 153: /* opt_partition_clause  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).partition_clause_) != nullptr) {
     delete ((*yyvaluep).partition_clause_);
   }
 }
-#line 2049 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2085 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 148: /* partition_type  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 154: /* partition_type  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2059 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2095 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 149: /* key_value_list  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 155: /* key_value_list  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2069 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2105 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 150: /* key_value  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 156: /* key_value  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_) != nullptr) {
     delete ((*yyvaluep).key_value_);
   }
 }
-#line 2079 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2115 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 151: /* key_string_value  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 157: /* key_string_value  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_string_value_) != nullptr) {
     delete ((*yyvaluep).key_string_value_);
   }
 }
-#line 2089 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2125 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 152: /* key_string_list  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 158: /* key_string_list  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_string_list_) != nullptr) {
     delete ((*yyvaluep).key_string_list_);
   }
 }
-#line 2099 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2135 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 153: /* key_integer_value  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 159: /* key_integer_value  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_integer_value_) != nullptr) {
     delete ((*yyvaluep).key_integer_value_);
   }
 }
-#line 2109 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2145 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 154: /* index_type  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 160: /* index_type  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2119 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2155 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 155: /* opt_index_properties  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 161: /* opt_index_properties  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2129 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2165 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 156: /* insert_statement  */
-#line 566 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 162: /* insert_statement  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).insert_statement_) != nullptr) {
     delete ((*yyvaluep).insert_statement_);
   }
 }
-#lin

<TRUNCATED>


[27/30] incubator-quickstep git commit: Revert "Change default aggregate_hashtable_type from LinearOpenAddressing to SeparateChaining" (#208)

Posted by zu...@apache.org.
Revert "Change default aggregate_hashtable_type from LinearOpenAddressing to SeparateChaining" (#208)

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

Branch: refs/heads/master
Commit: 8e38e86551214c79dad77597c97b1798639c3e72
Parents: 8532c23
Author: Jignesh Patel <pa...@users.noreply.github.com>
Authored: Wed May 4 14:21:58 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:36 2016 -0700

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8e38e865/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 3698701..c34f084 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -135,7 +135,7 @@ static const volatile bool join_hashtable_type_dummy
     = gflags::RegisterFlagValidator(&FLAGS_join_hashtable_type,
                                     &ValidateHashTableImplTypeString);
 
-DEFINE_string(aggregate_hashtable_type, "SeparateChaining",
+DEFINE_string(aggregate_hashtable_type, "LinearOpenAddressing",
               "HashTable implementation to use for aggregates with GROUP BY "
               "(valid options are SeparateChaining or LinearOpenAddressing)");
 static const volatile bool aggregate_hashtable_type_dummy


[26/30] incubator-quickstep git commit: Change default aggregate_hashtable_type from LinearOpenAddressing to SeparateChaining (#207)

Posted by zu...@apache.org.
Change default aggregate_hashtable_type from LinearOpenAddressing to SeparateChaining (#207)

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

Branch: refs/heads/master
Commit: 8532c231c1eb42bff926765428d69481afe7b5d0
Parents: d9687b9
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Wed May 4 13:54:55 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:36 2016 -0700

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8532c231/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index c34f084..3698701 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -135,7 +135,7 @@ static const volatile bool join_hashtable_type_dummy
     = gflags::RegisterFlagValidator(&FLAGS_join_hashtable_type,
                                     &ValidateHashTableImplTypeString);
 
-DEFINE_string(aggregate_hashtable_type, "LinearOpenAddressing",
+DEFINE_string(aggregate_hashtable_type, "SeparateChaining",
               "HashTable implementation to use for aggregates with GROUP BY "
               "(valid options are SeparateChaining or LinearOpenAddressing)");
 static const volatile bool aggregate_hashtable_type_dummy


[09/30] incubator-quickstep git commit: Adds support for scalar subqueries (#185)

Posted by zu...@apache.org.
Adds support for scalar subqueries (#185)

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

Branch: refs/heads/master
Commit: 9b3d91562feb6b9f561d751f8a0fb30be02e3064
Parents: 49e8b33
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Apr 21 16:08:39 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:34 2016 -0700

----------------------------------------------------------------------
 parser/ParseSubqueryExpression.cpp              |    5 +-
 parser/SqlParser.ypp                            |    3 +
 parser/preprocessed/SqlParser_gen.cpp           | 1675 +++++++++---------
 parser/tests/Select.test                        |   36 +-
 parser/tests/TPCH.test                          |  463 ++++-
 .../expressions/SubqueryExpression.cpp          |   11 +-
 query_optimizer/resolver/Resolver.cpp           |    8 +-
 query_optimizer/rules/CMakeLists.txt            |    1 +
 query_optimizer/rules/UnnestSubqueries.cpp      |   49 +-
 .../tests/execution_generator/Select.test       |   55 +-
 .../tests/logical_generator/Select.test         |  370 ++++
 .../tests/physical_generator/Select.test        |  614 +++++++
 query_optimizer/tests/resolver/Select.test      |  144 ++
 13 files changed, 2563 insertions(+), 871 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/parser/ParseSubqueryExpression.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseSubqueryExpression.cpp b/parser/ParseSubqueryExpression.cpp
index 1673a74..556b19d 100644
--- a/parser/ParseSubqueryExpression.cpp
+++ b/parser/ParseSubqueryExpression.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -27,7 +29,8 @@ namespace quickstep {
 class ParseTreeNode;
 
 std::string ParseSubqueryExpression::generateName() const {
-  LOG(FATAL) << "ParseSubqueryExpression::generateName() is not implemented yet";
+  // TODO(jianqiao): generate a more informative name for ParseSubqueryExpression.
+  return getName();
 }
 
 void ParseSubqueryExpression::getFieldStringItems(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index bb19d8b..a8ec821 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -1503,6 +1503,9 @@ expression_base:
   }
   | '(' add_expression ')' {
     $$ = $2;
+  }
+  | subquery_expression {
+    $$ = $1;
   };
 
 function_call:


[04/30] incubator-quickstep git commit: Better comments in eviction policy. (#193)

Posted by zu...@apache.org.
Better comments in eviction policy. (#193)

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

Branch: refs/heads/master
Commit: a64bc4356f7c716f36160d4376185d0be0751759
Parents: 4ec8d4d
Author: Marc S <cr...@users.noreply.github.com>
Authored: Tue Apr 26 16:11:08 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:34 2016 -0700

----------------------------------------------------------------------
 storage/EvictionPolicy.hpp | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a64bc435/storage/EvictionPolicy.hpp
----------------------------------------------------------------------
diff --git a/storage/EvictionPolicy.hpp b/storage/EvictionPolicy.hpp
index 0552916..ae2a258 100644
--- a/storage/EvictionPolicy.hpp
+++ b/storage/EvictionPolicy.hpp
@@ -32,6 +32,12 @@
 
 namespace quickstep {
 
+/**
+ * @brief Base class for classes with the responsibility of selecting blocks
+ *        to evict from the buffer pool.
+ * @note  In this class, and its subclasses, block refers to both blocks and
+ *        blobs. From the eviction policy's perspective, they are the same.
+ */
 class EvictionPolicy {
  public:
   enum class Status {


[05/30] incubator-quickstep git commit: Operator Unit Tests Fixes (#186)

Posted by zu...@apache.org.
Operator Unit Tests Fixes (#186)

* Set the unique storage path for each operator tests.

* Delete blocks after use.


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

Branch: refs/heads/master
Commit: ff3293bdb551383d27f00a3fffb8fc99e97b822b
Parents: 0f79a9a
Author: Zuyu ZHANG <zu...@users.noreply.github.com>
Authored: Sat Apr 23 13:39:09 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:34 2016 -0700

----------------------------------------------------------------------
 relational_operators/TextScanOperator.cpp       |  2 +-
 .../tests/AggregationOperator_unittest.cpp      | 21 +++++++++-
 .../tests/HashJoinOperator_unittest.cpp         | 37 +++++++++++++++-
 .../tests/SortMergeRunOperator_unittest.cpp     | 44 ++++++++++++++++++--
 .../SortRunGenerationOperator_unittest.cpp      | 13 +++++-
 .../tests/TextScanOperator_unittest.cpp         |  2 +-
 6 files changed, 111 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ff3293bd/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index e6d5200..5ede6f7 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -288,7 +288,7 @@ void TextScanWorkOrder::execute() {
 
     std::fclose(file);
   } else {
-    MutableBlobReference blob = storage_manager_->getBlobMutable(text_blob_);
+    BlobReference blob = storage_manager_->getBlob(text_blob_);
     const char *blob_pos = static_cast<const char*>(blob->getMemory());
     const char *blob_end = blob_pos + text_size_;
     bool have_row = false;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ff3293bd/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index da9f267..f2207c2 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -170,6 +170,17 @@ class AggregationOperatorTest : public ::testing::Test {
 
   virtual void TearDown() {
     thread_id_map_->removeValue();
+
+    // Drop blocks from relations.
+    const std::vector<block_id> table_blocks = table_->getBlocksSnapshot();
+    for (const block_id block : table_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
+
+    const std::vector<block_id> result_table_blocks = result_table_->getBlocksSnapshot();
+    for (const block_id block : result_table_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
   }
 
   Tuple* createTuple(const CatalogRelation &relation, const std::int64_t val) {
@@ -420,6 +431,10 @@ class AggregationOperatorTest : public ::testing::Test {
         sub_block.getAttributeValueTyped(0, result_table_->getAttributeByName("result-1")->getID());
     test(expected0, actual0);
     test(expected1, actual1);
+
+    // Drop the block.
+    block.release();
+    storage_manager_->deleteBlockOrBlobFile(result[0]);
   }
 
   template <class FinalDataType>
@@ -463,6 +478,10 @@ class AggregationOperatorTest : public ::testing::Test {
         check_fn(group_by_id, actual0, actual1);
       }
       total_tuples += sub_block.numTuples();
+
+      // Drop the block.
+      block.release();
+      storage_manager_->deleteBlockOrBlobFile(result[bid]);
     }
     EXPECT_EQ(num_tuples, total_tuples);
   }
@@ -500,7 +519,7 @@ class AggregationOperatorTest : public ::testing::Test {
 
 const char AggregationOperatorTest::kDatabaseName[] = "database";
 const char AggregationOperatorTest::kTableName[] = "table";
-const char AggregationOperatorTest::kStoragePath[] = "./test_data";
+const char AggregationOperatorTest::kStoragePath[] = "./aggregation_operator_test_data";
 
 namespace {
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ff3293bd/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 0e6d96b..333c3f0 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -114,7 +114,7 @@ class HashJoinOperatorTest : public ::testing::TestWithParam<HashTableImplType>
     bus_.RegisterClientAsSender(foreman_client_id_, kCatalogRelationNewBlockMessage);
     bus_.RegisterClientAsReceiver(foreman_client_id_, kCatalogRelationNewBlockMessage);
 
-    storage_manager_.reset(new StorageManager("./test_data/"));
+    storage_manager_.reset(new StorageManager("./hash_join_operator_test_data/"));
 
     // Create a database.
     db_.reset(new CatalogDatabase(nullptr, "database"));
@@ -193,6 +193,17 @@ class HashJoinOperatorTest : public ::testing::TestWithParam<HashTableImplType>
 
   virtual void TearDown() {
     thread_id_map_->removeValue();
+
+    // Drop blocks from relations.
+    const std::vector<block_id> dim_blocks = dim_table_->getBlocksSnapshot();
+    for (const block_id block : dim_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
+
+    const std::vector<block_id> fact_blocks = fact_table_->getBlocksSnapshot();
+    for (const block_id block : fact_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
   }
 
   StorageBlockLayout* createStorageLayout(const CatalogRelation &relation) {
@@ -398,6 +409,10 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
         ++counts[value];
       }
     }
+
+    // Drop the block.
+    result_block.release();
+    storage_manager_->deleteBlockOrBlobFile(result_blocks[bid]);
   }
   EXPECT_EQ(static_cast<std::size_t>(kNumDimTuples), num_result_tuples);
 
@@ -550,6 +565,10 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
         ++fact_counts[value];
       }
     }
+
+    // Drop the block.
+    result_block.release();
+    storage_manager_->deleteBlockOrBlobFile(result_blocks[bid]);
   }
   EXPECT_EQ(static_cast<std::size_t>(kNumDimTuples), num_result_tuples);
 
@@ -689,6 +708,10 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
         ++counts[value];
       }
     }
+
+    // Drop the block.
+    result_block.release();
+    storage_manager_->deleteBlockOrBlobFile(result_blocks[bid]);
   }
   EXPECT_EQ(static_cast<std::size_t>(kNumDimTuples * kNumFactTuples),
             num_result_tuples);
@@ -835,6 +858,10 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
         ++fact_counts[value];
       }
     }
+
+    // Drop the block.
+    result_block.release();
+    storage_manager_->deleteBlockOrBlobFile(result_blocks[bid]);
   }
   EXPECT_EQ(static_cast<std::size_t>(kNumDimTuples), num_result_tuples);
 
@@ -1004,6 +1031,10 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
         ++fact_counts[value];
       }
     }
+
+    // Drop the block.
+    result_block.release();
+    storage_manager_->deleteBlockOrBlobFile(result_blocks[bid]);
   }
   EXPECT_EQ(static_cast<std::size_t>(kNumDimTuples) / 2, num_result_tuples);
 
@@ -1184,6 +1215,10 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
         ++fact_counts[value];
       }
     }
+
+    // Drop the block.
+    result_block.release();
+    storage_manager_->deleteBlockOrBlobFile(result_blocks[bid]);
   }
   EXPECT_EQ(8u, num_result_tuples);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ff3293bd/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index 5068391..50c508d 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -196,6 +196,17 @@ class RunTest : public ::testing::Test {
     // Usually the worker thread makes the following call. In this test setup,
     // we don't have a worker thread hence we have to explicitly make the call.
     thread_id_map_->removeValue();
+
+    // Drop blocks from relations and InsertDestination.
+    const vector<block_id> tmp_blocks = insert_destination_->getTouchedBlocks();
+    for (const block_id block : tmp_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
+
+    const vector<block_id> blocks = table_->getBlocksSnapshot();
+    for (const block_id block : blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
   }
 
   // Helper method to insert test tuples.
@@ -225,7 +236,7 @@ class RunTest : public ::testing::Test {
 };
 
 const char RunTest::kTableName[] = "table";
-const char RunTest::kStoragePath[] = "./test_data";
+const char RunTest::kStoragePath[] = "./sort_merge_run_operator_test_data";
 const tuple_id RunTest::kNumTuples = 100;
 const tuple_id RunTest::kNumTuplesPerBlock = 10;
 
@@ -429,6 +440,17 @@ class RunMergerTest : public ::testing::Test {
     // Usually the worker thread makes the following call. In this test setup,
     // we don't have a worker thread hence we have to explicitly make the call.
     thread_id_map_->removeValue();
+
+    // Drop blocks from relations and InsertDestination.
+    const vector<block_id> tmp_blocks = insert_destination_->getTouchedBlocks();
+    for (const block_id block : tmp_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
+
+    const vector<block_id> blocks = table_->getBlocksSnapshot();
+    for (const block_id block : blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
   }
 
   // Helper method to create test tuples.
@@ -642,7 +664,7 @@ class RunMergerTest : public ::testing::Test {
 };
 
 const char RunMergerTest::kTableName[] = "table";
-const char RunMergerTest::kStoragePath[] = "./test_data";
+const char RunMergerTest::kStoragePath[] = "./sort_merge_run_operator_test_data";
 const std::size_t RunMergerTest::kNumTuplesPerBlock = 10;
 const tuple_id RunMergerTest::kNumBlocksPerRun = 10;
 const std::size_t RunMergerTest::kNumRuns = 10;
@@ -1285,6 +1307,22 @@ class SortMergeRunOperatorTest : public ::testing::Test {
     // Usually the worker thread makes the following call. In this test setup,
     // we don't have a worker thread hence we have to explicitly make the call.
     thread_id_map_->removeValue();
+
+    // Drop blocks from relations.
+    const vector<block_id> input_blocks = input_table_->getBlocksSnapshot();
+    for (const block_id block : input_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
+
+    const vector<block_id> result_blocks = result_table_->getBlocksSnapshot();
+    for (const block_id block : result_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
+
+    const vector<block_id> run_blocks = run_table_->getBlocksSnapshot();
+    for (const block_id block : run_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
   }
 
   CatalogRelation *createTable(const char *name, const relation_id rel_id) {
@@ -1681,7 +1719,7 @@ class SortMergeRunOperatorTest : public ::testing::Test {
 const char SortMergeRunOperatorTest::kTableName[] = "table";
 const char SortMergeRunOperatorTest::kResultTableName[] = "result-table";
 const char SortMergeRunOperatorTest::kRunTableName[] = "run-table";
-const char SortMergeRunOperatorTest::kStoragePath[] = "./test_data";
+const char SortMergeRunOperatorTest::kStoragePath[] = "./sort_merge_run_operator_test_data";
 const char SortMergeRunOperatorTest::kDatabaseName[] = "database";
 
 namespace {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ff3293bd/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index e11676c..7491778 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -193,6 +193,17 @@ class SortRunGenerationOperatorTest : public ::testing::Test {
     // Usually the worker thread makes the following call. In this test setup,
     // we don't have a worker thread hence we have to explicitly make the call.
     thread_id_map_->removeValue();
+
+    // Drop blocks from relations.
+    const vector<block_id> input_blocks = input_table_->getBlocksSnapshot();
+    for (const block_id block : input_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
+
+    const vector<block_id> result_blocks = result_table_->getBlocksSnapshot();
+    for (const block_id block : result_blocks) {
+      storage_manager_->deleteBlockOrBlobFile(block);
+    }
   }
 
   // Helper method to create catalog relation.
@@ -414,7 +425,7 @@ class SortRunGenerationOperatorTest : public ::testing::Test {
 
 const char SortRunGenerationOperatorTest::kTableName[] = "table";
 const char SortRunGenerationOperatorTest::kResultTableName[] = "result-table";
-const char SortRunGenerationOperatorTest::kStoragePath[] = "./test_data";
+const char SortRunGenerationOperatorTest::kStoragePath[] = "./sort_run_generation_operator_test_data";
 
 namespace {
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ff3293bd/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index 355c4e2..1dfad7b 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -94,7 +94,7 @@ class TextScanOperatorTest : public ::testing::Test {
     relation_->addAttribute(
         new CatalogAttribute(relation_, "varchar_attr", TypeFactory::GetType(kVarChar, 20, true)));
 
-    storage_manager_.reset(new StorageManager("./test_data/"));
+    storage_manager_.reset(new StorageManager("./text_scan_operator_test_data/"));
   }
 
   virtual void TearDown() {


[06/30] incubator-quickstep git commit: Honor the buffer pool slots flag. (#184)

Posted by zu...@apache.org.
Honor the buffer pool slots flag. (#184)

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

Branch: refs/heads/master
Commit: 7187c864fd97cf622206c36e25d03e34475807d6
Parents: 9b3d915
Author: Zuyu ZHANG <zu...@users.noreply.github.com>
Authored: Thu Apr 21 16:10:07 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:34 2016 -0700

----------------------------------------------------------------------
 query_optimizer/tests/CMakeLists.txt                    |  2 +-
 query_optimizer/tests/ExecutionGeneratorTest.cpp        |  5 ++++-
 relational_operators/CMakeLists.txt                     |  9 +++++----
 .../tests/AggregationOperator_unittest.cpp              | 12 ++++++++++--
 .../tests/HashJoinOperator_unittest.cpp                 | 11 ++++++++++-
 .../tests/SortMergeRunOperator_unittest.cpp             | 11 ++++++++++-
 .../tests/SortRunGenerationOperator_unittest.cpp        | 11 ++++++++++-
 .../tests/TextScanOperator_unittest.cpp                 |  2 +-
 8 files changed, 51 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7187c864/query_optimizer/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/CMakeLists.txt b/query_optimizer/tests/CMakeLists.txt
index eb64070..1d2fa10 100644
--- a/query_optimizer/tests/CMakeLists.txt
+++ b/query_optimizer/tests/CMakeLists.txt
@@ -86,9 +86,9 @@ add_executable(quickstep_queryoptimizer_tests_OptimizerTextTest
                "${PROJECT_SOURCE_DIR}/utility/textbased_test/TextBasedTest.hpp")
 
 target_link_libraries(quickstep_queryoptimizer_tests_ExecutionGeneratorTest
+                      gflags_nothreads-static
                       glog
                       gtest
-                      gtest_main
                       quickstep_catalog_CatalogDatabase
                       quickstep_cli_DropRelation
                       quickstep_cli_PrintToScreen

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7187c864/query_optimizer/tests/ExecutionGeneratorTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTest.cpp b/query_optimizer/tests/ExecutionGeneratorTest.cpp
index 92e9e7c..42b246b 100644
--- a/query_optimizer/tests/ExecutionGeneratorTest.cpp
+++ b/query_optimizer/tests/ExecutionGeneratorTest.cpp
@@ -1,6 +1,6 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2015-2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -22,6 +22,7 @@
 #include "query_optimizer/tests/ExecutionGeneratorTestRunner.hpp"
 #include "utility/textbased_test/TextBasedTestDriver.hpp"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
 
 using quickstep::TextBasedTest;
@@ -30,6 +31,8 @@ QUICKSTEP_GENERATE_TEXT_TEST(EXECUTION_GENERATOR_TEST);
 
 int main(int argc, char** argv) {
   google::InitGoogleLogging(argv[0]);
+  // Honor FLAGS_buffer_pool_slots in StorageManager.
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
 
   if (argc < 4) {
     LOG(ERROR) << "Must have at least 3 arguments, but " << argc - 1

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7187c864/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index f204d67..759a233 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -462,9 +462,9 @@ target_link_libraries(quickstep_relationaloperators
 add_executable(AggregationOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationOperator_unittest.cpp")
 target_link_libraries(AggregationOperator_unittest
+                      gflags_nothreads-static
                       glog
                       gtest
-                      gtest_main
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogDatabase
                       quickstep_catalog_CatalogRelation
@@ -515,9 +515,9 @@ add_test(AggregationOperator_unittest AggregationOperator_unittest)
 add_executable(HashJoinOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/HashJoinOperator_unittest.cpp")
 target_link_libraries(HashJoinOperator_unittest
+                      gflags_nothreads-static
                       glog
                       gtest
-                      gtest_main
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogDatabase
                       quickstep_catalog_CatalogRelation
@@ -565,9 +565,9 @@ add_test(HashJoinOperator_unittest HashJoinOperator_unittest)
 add_executable(SortMergeRunOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/SortMergeRunOperator_unittest.cpp")
 target_link_libraries(SortMergeRunOperator_unittest
+                      gflags_nothreads-static
                       glog
                       gtest
-                      gtest_main
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogDatabase
                       quickstep_catalog_CatalogRelation
@@ -616,9 +616,9 @@ add_test(SortMergeRunOperator_unittest SortMergeRunOperator_unittest)
 add_executable(SortRunGenerationOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/SortRunGenerationOperator_unittest.cpp")
 target_link_libraries(SortRunGenerationOperator_unittest
+                      gflags_nothreads-static
                       glog
                       gtest
-                      gtest_main
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogDatabase
                       quickstep_catalog_CatalogRelation
@@ -664,6 +664,7 @@ add_test(SortRunGenerationOperator_unittest SortRunGenerationOperator_unittest)
 add_executable(TextScanOperator_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/TextScanOperator_unittest.cpp")
 target_link_libraries(TextScanOperator_unittest
+                      gflags_nothreads-static
                       glog
                       gtest
                       quickstep_catalog_CatalogAttribute

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7187c864/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index 5b4ae6e..da9f267 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -66,8 +66,8 @@
 #include "types/operations/comparisons/ComparisonID.hpp"
 #include "utility/PtrList.hpp"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
-
 #include "gtest/gtest.h"
 
 #include "tmb/id_typedefs.h"
@@ -1773,5 +1773,13 @@ TEST_F(AggregationOperatorTest, GroupBy_Count_zeroRows) {
 }
 
 }  // namespace
-
 }  // namespace quickstep
+
+int main(int argc, char* argv[]) {
+  google::InitGoogleLogging(argv[0]);
+  // Honor FLAGS_buffer_pool_slots in StorageManager.
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  testing::InitGoogleTest(&argc, argv);
+
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7187c864/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 3118b73..0e6d96b 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -69,8 +69,8 @@
 #include "types/operations/comparisons/ComparisonID.hpp"
 #include "utility/Macros.hpp"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
-
 #include "gtest/gtest.h"
 
 #include "tmb/id_typedefs.h"
@@ -1227,3 +1227,12 @@ INSTANTIATE_TEST_CASE_P(
         HashTableImplType::kSimpleScalarSeparateChaining),);  // NOLINT(whitespace/comma)
 
 }  // namespace quickstep
+
+int main(int argc, char* argv[]) {
+  google::InitGoogleLogging(argv[0]);
+  // Honor FLAGS_buffer_pool_slots in StorageManager.
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  testing::InitGoogleTest(&argc, argv);
+
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7187c864/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index ef896d3..5068391 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -69,8 +69,8 @@
 #include "utility/SortConfiguration.hpp"
 #include "utility/SortConfiguration.pb.h"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
-
 #include "gtest/gtest.h"
 
 #include "tmb/id_typedefs.h"
@@ -2260,3 +2260,12 @@ TEST_F(SortMergeRunOperatorTest, Pipleined_ManyPasses_MergeFactor17_SlowFeed_Top
 }
 
 }  // namespace quickstep
+
+int main(int argc, char* argv[]) {
+  google::InitGoogleLogging(argv[0]);
+  // Honor FLAGS_buffer_pool_slots in StorageManager.
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  testing::InitGoogleTest(&argc, argv);
+
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7187c864/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index 76ea40a..e11676c 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -64,8 +64,8 @@
 #include "utility/SortConfiguration.hpp"
 #include "utility/SortConfiguration.pb.h"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
-
 #include "gtest/gtest.h"
 
 #include "tmb/id_typedefs.h"
@@ -770,3 +770,12 @@ TEST_F(SortRunGenerationOperatorTest, 3Column_MixedNullOrdering_MixedOrdering) {
 }
 
 }  // namespace quickstep
+
+int main(int argc, char* argv[]) {
+  google::InitGoogleLogging(argv[0]);
+  // Honor FLAGS_buffer_pool_slots in StorageManager.
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  testing::InitGoogleTest(&argc, argv);
+
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7187c864/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index c43a127..355c4e2 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -39,8 +39,8 @@
 #include "types/TypeID.hpp"
 #include "utility/MemStream.hpp"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
-
 #include "gtest/gtest.h"
 
 #include "tmb/id_typedefs.h"


[29/30] incubator-quickstep git commit: Transaction Part 4: LockManager, CycleDetector and DeadLockDetector. (#187)

Posted by zu...@apache.org.
Transaction Part 4: LockManager, CycleDetector and DeadLockDetector. (#187)

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

Branch: refs/heads/master
Commit: c0a02d007c1f01acf74cf856ad220cdb56d028e3
Parents: 8e38e86
Author: Hakan Memisoglu <ha...@gmail.com>
Authored: Wed May 4 23:30:51 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:36 2016 -0700

----------------------------------------------------------------------
 transaction/AccessMode.hpp                      |  76 +++++-
 transaction/CMakeLists.txt                      |  59 ++++-
 transaction/CycleDetector.cpp                   | 120 ++++++++++
 transaction/CycleDetector.hpp                   |  83 +++++++
 transaction/DeadLockDetector.cpp                | 177 ++++++++++++++
 transaction/DeadLockDetector.hpp                | 156 ++++++++++++
 transaction/DirectedGraph.hpp                   |  56 ++---
 transaction/LockManager.cpp                     | 237 +++++++++++++++++++
 transaction/LockManager.hpp                     | 128 ++++++++++
 transaction/LockTable.cpp                       |  22 +-
 transaction/LockTable.hpp                       |  36 +--
 transaction/ResourceId.hpp                      |   9 +-
 transaction/StronglyConnectedComponents.cpp     |   1 -
 transaction/Transaction.cpp                     |  48 ----
 transaction/TransactionTable.cpp                |   8 +-
 transaction/TransactionTable.hpp                |  11 +-
 transaction/tests/AccessMode_unittest.cpp       |  12 +-
 transaction/tests/CycleDetector_unittest.cpp    | 157 ++++++++++++
 transaction/tests/DeadLockDetector_unittest.cpp |  96 ++++++++
 transaction/tests/DirectedGraph_unittest.cpp    | 113 +++++----
 transaction/tests/LockRequest_unittest.cpp      |   4 +-
 transaction/tests/LockTable_unittest.cpp        |  63 ++---
 transaction/tests/Lock_unittest.cpp             |  13 +-
 .../StronglyConnectedComponents_unittest.cpp    |  15 +-
 transaction/tests/TransactionTable_unittest.cpp | 102 ++++----
 25 files changed, 1512 insertions(+), 290 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/AccessMode.hpp
----------------------------------------------------------------------
diff --git a/transaction/AccessMode.hpp b/transaction/AccessMode.hpp
index 34ace36..bb06689 100644
--- a/transaction/AccessMode.hpp
+++ b/transaction/AccessMode.hpp
@@ -32,12 +32,12 @@ namespace transaction {
  * @brief Represents mode type. Possible options are NL, IS, IX, S, SIX, X.
  **/
 enum class AccessModeType : std::uint8_t {
-  kNoLock = 0,
-  kIsLock,
-  kIxLock,
-  kSLock,
-  kSixLock,
-  kXLock,
+  kNoLockMode = 0,
+  kIsLockMode,
+  kIxLockMode,
+  kSLockMode,
+  kSixLockMode,
+  kXLockMode,
   kNumAccessModeTypes,
 };
 
@@ -55,6 +55,60 @@ class AccessMode {
       : access_mode_(access_mode) {}
 
   /**
+   * @brief Factory method for NoLockMode.
+   *
+   * @return NoLockMode instance.
+   **/
+  static AccessMode NoLockMode() {
+    return AccessMode(AccessModeType::kNoLockMode);
+  }
+
+  /**
+   * @brief Factory method for IsLockMode.
+   *
+   * @return IsLockMode instance.
+   **/
+  static AccessMode IsLockMode() {
+    return AccessMode(AccessModeType::kIsLockMode);
+  }
+
+  /**
+   * @brief Factory method for IxLockMode.
+   *
+   * @return IxLockMode instance.
+   **/
+  static AccessMode IxLockMode() {
+    return AccessMode(AccessModeType::kIxLockMode);
+  }
+
+  /**
+   * @brief Factory method for SixLockMode.
+   *
+   * @return SixLockMode instance.
+   **/
+  static AccessMode SixLockMode() {
+    return AccessMode(AccessModeType::kSixLockMode);
+  }
+
+  /**
+   * @brief Factory method for SLockMode.
+   *
+   * @return SLockMode instance.
+   **/
+  static AccessMode SLockMode() {
+    return AccessMode(AccessModeType::kSLockMode);
+  }
+
+  /**
+   * @brief Factory method for XLockMode.
+   *
+   * @return XLockMode instance.
+   **/
+  static AccessMode XLockMode() {
+    return AccessMode(AccessModeType::kXLockMode);
+  }
+
+  /**
    * @brief Checks whether this access mode is compatible with the other.
    *
    * @param other Other access mode that will be checked against to this one.
@@ -74,7 +128,7 @@ class AccessMode {
    * @return True if it is IS mode, false otherwise.
    **/
   inline bool isIntentionShareLock() const {
-    return access_mode_ == AccessModeType::kIsLock;
+    return access_mode_ == AccessModeType::kIsLockMode;
   }
 
   /**
@@ -83,7 +137,7 @@ class AccessMode {
    * @return True if it is IX mode, false otherwise.
    **/
   inline bool isIntentionExclusiveLock() const {
-    return access_mode_ == AccessModeType::kIxLock;
+    return access_mode_ == AccessModeType::kIxLockMode;
   }
 
   /**
@@ -92,7 +146,7 @@ class AccessMode {
    * @return True if it is SIX mode, false otherwise.
    **/
   inline bool isShareAndIntentionExclusiveLock() const {
-    return access_mode_ == AccessModeType::kSixLock;
+    return access_mode_ == AccessModeType::kSixLockMode;
   }
 
   /**
@@ -101,7 +155,7 @@ class AccessMode {
    * @return True if it is S mode, false otherwise.
    **/
   inline bool isShareLock() const {
-    return access_mode_ == AccessModeType::kSLock;
+    return access_mode_ == AccessModeType::kSLockMode;
   }
 
   /**
@@ -110,7 +164,7 @@ class AccessMode {
    * @return True if it is X mode, false otherwise.
    **/
   inline bool isExclusiveLock() const {
-    return access_mode_ == AccessModeType::kXLock;
+    return access_mode_ == AccessModeType::kXLockMode;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/transaction/CMakeLists.txt b/transaction/CMakeLists.txt
index 05fc96a..c6c87b6 100644
--- a/transaction/CMakeLists.txt
+++ b/transaction/CMakeLists.txt
@@ -16,12 +16,21 @@
 add_library(quickstep_transaction_AccessMode
             AccessMode.cpp
             AccessMode.hpp)
+add_library(quickstep_transaction_CycleDetector
+            CycleDetector.cpp
+            CycleDetector.hpp)
+add_library(quickstep_transaction_DeadLockDetector
+            DeadLockDetector.cpp
+            DeadLockDetector.cpp)
 add_library(quickstep_transaction_DirectedGraph
             ../empty_src.cpp
             DirectedGraph.hpp)
 add_library(quickstep_transaction_Lock
             ../empty_src.cpp
             Lock.hpp)
+add_library(quickstep_transaction_LockManager
+            LockManager.hpp
+            LockManager.cpp)
 add_library(quickstep_transaction_LockRequest
             ../empty_src.cpp
             LockRequest.hpp)
@@ -40,7 +49,17 @@ add_library(quickstep_transaction_Transaction
 add_library(quickstep_transaction_TransactionTable
             TransactionTable.cpp
             TransactionTable.hpp)
-          
+
+target_link_libraries(quickstep_transaction_CycleDetector
+                      quickstep_transaction_DirectedGraph
+                      quickstep_transaction_StronglyConnectedComponents
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_transaction_DeadLockDetector
+                      glog
+                      quickstep_transaction_CycleDetector
+                      quickstep_transaction_DirectedGraph
+                      quickstep_transaction_LockTable
+                      quickstep_transaction_Transaction)
 target_link_libraries(quickstep_transaction_DirectedGraph
                       glog
                       quickstep_transaction_Transaction
@@ -48,6 +67,18 @@ target_link_libraries(quickstep_transaction_DirectedGraph
 target_link_libraries(quickstep_transaction_Lock
                       quickstep_transaction_AccessMode
                       quickstep_transaction_ResourceId)
+target_link_libraries(quickstep_transaction_LockManager
+                      gflags_nothreads-static
+                      glog
+                      quickstep_utility_ThreadSafeQueue
+                      quickstep_threading_Thread
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_DeadLockDetector
+                      quickstep_transaction_LockRequest
+                      quickstep_transaction_LockTable
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction
+                      quickstep_transaction_TransactionTable)
 target_link_libraries(quickstep_transaction_LockRequest
                       quickstep_transaction_AccessMode
                       quickstep_transaction_ResourceId
@@ -80,8 +111,11 @@ add_library(quickstep_transaction
             TransactionModule.hpp)
 target_link_libraries(quickstep_transaction
                       quickstep_transaction_AccessMode
+                      quickstep_transaction_CycleDetector
+                      quickstep_transaction_DeadLockDetector
                       quickstep_transaction_DirectedGraph
                       quickstep_transaction_Lock
+                      quickstep_transaction_LockManager
                       quickstep_transaction_LockRequest
                       quickstep_transaction_LockTable
                       quickstep_transaction_ResourceId
@@ -97,6 +131,29 @@ target_link_libraries(AccessMode_unittest
                       quickstep_transaction_AccessMode)
 add_test(AccessMode_unittest AccessMode_unittest)
 
+add_executable(CycleDetector_unittest
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/CycleDetector_unittest.cpp")
+target_link_libraries(CycleDetector_unittest
+                      gtest
+                      gtest_main
+                      quickstep_transaction_CycleDetector
+                      quickstep_transaction_DirectedGraph
+                      quickstep_transaction_StronglyConnectedComponents)
+add_test(CycleDetector_unittest CycleDetector_unittest)
+
+add_executable(DeadLockDetector_unittest
+  "${CMAKE_CURRENT_SOURCE_DIR}/tests/DeadLockDetector_unittest.cpp")
+target_link_libraries(DeadLockDetector_unittest
+                      gtest
+                      gtest_main
+                      quickstep_threading_Thread
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_DeadLockDetector
+                      quickstep_transaction_LockTable
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction)
+add_test(DeadLockDetector_unittest DeadLockDetector_unittest)
+
 add_executable(DirectedGraph_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/DirectedGraph_unittest.cpp")
 target_link_libraries(DirectedGraph_unittest

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/CycleDetector.cpp
----------------------------------------------------------------------
diff --git a/transaction/CycleDetector.cpp b/transaction/CycleDetector.cpp
new file mode 100644
index 0000000..b12897f
--- /dev/null
+++ b/transaction/CycleDetector.cpp
@@ -0,0 +1,120 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "transaction/CycleDetector.hpp"
+
+#include <cstdint>
+#include <memory>
+#include <stack>
+#include <unordered_map>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/StronglyConnectedComponents.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+CycleDetector::CycleDetector(DirectedGraph *wait_for_graph)
+    : wait_for_graph_(wait_for_graph),
+      strongly_connected_components_(
+          std::make_unique<StronglyConnectedComponents>(*wait_for_graph)) {
+}
+
+std::vector<DirectedGraph::node_id>
+CycleDetector::chooseVictimsToBreakCycle() const {
+  std::vector<DirectedGraph::node_id> nodes_to_kill;
+  const std::unordered_map<std::uint64_t, std::vector<DirectedGraph::node_id>>
+      component_mapping = strongly_connected_components_->getComponentMapping();
+  for (const auto &entry : component_mapping) {
+    // One node means no cycle.
+    if (entry.second.size() == 1) {
+      continue;
+    }
+    const std::vector<DirectedGraph::node_id> nodes =
+        chooseVictimsInComponent(entry.second);
+    nodes_to_kill.insert(nodes_to_kill.end(), nodes.begin(), nodes.end());
+  }
+  return nodes_to_kill;
+}
+
+std::vector<DirectedGraph::node_id> CycleDetector::chooseVictimsInComponent(
+    const std::vector<DirectedGraph::node_id> &nodes) const {
+  std::vector<DirectedGraph::node_id> targets;
+  // Convert it to set to ensure defensively that the elements are unique.
+  std::unordered_set<DirectedGraph::node_id> nodes_set(nodes.begin(),
+                                                       nodes.end());
+
+  while (true) {
+    if (!hasCycle(nodes_set)) {
+      break;
+    }
+    // Connected component still has a cycle, therefore choose a
+    // victim and keep trying to remove nodes until there is no cycle.
+    const DirectedGraph::node_id victim = chooseVictim(nodes_set);
+    // Remove the victim node from the connected component.
+    nodes_set.erase(victim);
+    // Removed node is a victim now.
+    targets.push_back(victim);
+  }
+  return targets;
+}
+
+bool CycleDetector::hasCycle(
+    const std::unordered_set<DirectedGraph::node_id> &nodes) const {
+  // Keeps track of the nodes the algorithms visited.
+  std::unordered_set<DirectedGraph::node_id> visited;
+  for (const DirectedGraph::node_id node_id : nodes) {
+    // If it is visited, then pass to the next one.
+    if (visited.count(node_id) == 1) {
+      continue;
+    }
+    // Save the backtracking information.
+    std::stack<DirectedGraph::node_id> to_visit;
+    // Mark this id as "to be visited".
+    to_visit.push(node_id);
+    // Start to visit nodes until it is done.
+    while (!to_visit.empty()) {
+      const DirectedGraph::node_id current_node = to_visit.top();
+      to_visit.pop();
+      // Mark the node coming from stack as "visited".
+      visited.insert(current_node);
+      // For all adjacent nodes of this "visited" node,
+      const std::vector<DirectedGraph::node_id> adjacents
+          = wait_for_graph_->getAdjacentNodes(current_node);
+      for (const DirectedGraph::node_id adjacent : adjacents) {
+        if (visited.count(adjacent) == 1) {
+          // If this adjacent node is a node we already visited, then
+          // there is a cycle.
+          return true;
+        } else if (nodes.count(adjacent) == 1 && visited.count(adjacent) == 0) {
+          // Otherwise, if it is a node that we did not visit before
+          // mark this nodes as "to be visited".
+          to_visit.push(adjacent);
+        }
+      }
+    }
+  }
+  // If we have already visited all nodes and could not find a cycle,
+  // then we should return "no cycle" result.
+  return false;
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/CycleDetector.hpp
----------------------------------------------------------------------
diff --git a/transaction/CycleDetector.hpp b/transaction/CycleDetector.hpp
new file mode 100644
index 0000000..6865e2d
--- /dev/null
+++ b/transaction/CycleDetector.hpp
@@ -0,0 +1,83 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_TRANSACTION_CYCLE_DETECTOR_HPP_
+#define QUICKSTEP_TRANSACTION_CYCLE_DETECTOR_HPP_
+
+#include <memory>
+#include <unordered_set>
+#include <vector>
+
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/StronglyConnectedComponents.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+/** \addtogroup Transaction
+ *  @{
+ */
+
+/**
+ * @brief Class for running cycle detection algorithm on directed graph.
+ */
+class CycleDetector {
+ public:
+  /**
+   * @brief Constructor for DirectedGraph.
+   *
+   * @param wait_for_graph Pointer to a directed wait-for graph.
+   */
+  explicit CycleDetector(DirectedGraph *wait_for_graph);
+
+  /**
+   * @brief Calculate which nodes should be killed to eliminate all cycles
+   *        in the graph.
+   *
+   * @return Vector of node ids that should be killed to break all cycles.
+   */
+  std::vector<DirectedGraph::node_id> chooseVictimsToBreakCycle() const;
+
+ private:
+  std::vector<DirectedGraph::node_id> chooseVictimsInComponent(
+      const std::vector<DirectedGraph::node_id> &nodes) const;
+
+  inline DirectedGraph::node_id chooseVictim(
+      const std::unordered_set<DirectedGraph::node_id> &nodes_set) const {
+    // TODO(Hakan): This is very inefficient scheme, however in the
+    //              future, we can use the transaction's priority
+    //              as the victim selection parameter.
+    return *(nodes_set.begin());
+  }
+
+  // Checks whether the nodes in the set make a cycle.
+  bool hasCycle(const std::unordered_set<DirectedGraph::node_id> &within) const;
+
+  DirectedGraph *wait_for_graph_;
+
+  std::unique_ptr<StronglyConnectedComponents> strongly_connected_components_;
+
+  DISALLOW_COPY_AND_ASSIGN(CycleDetector);
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TRANSACTION_CYCLE_DETECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/DeadLockDetector.cpp
----------------------------------------------------------------------
diff --git a/transaction/DeadLockDetector.cpp b/transaction/DeadLockDetector.cpp
new file mode 100644
index 0000000..26ab115
--- /dev/null
+++ b/transaction/DeadLockDetector.cpp
@@ -0,0 +1,177 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "transaction/DeadLockDetector.hpp"
+
+#include <algorithm>
+#include <atomic>
+#include <chrono>
+#include <memory>
+#include <thread>  // NOLINT(build/c++11)
+#include <utility>
+#include <vector>
+
+#include "transaction/CycleDetector.hpp"
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/LockTable.hpp"
+#include "transaction/Transaction.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace transaction {
+
+constexpr std::int64_t DeadLockDetector::kSleepDurationInSeconds;
+
+DeadLockDetector::DeadLockDetector(LockTable *lock_table,
+                                   std::atomic<DeadLockDetectorStatus> *status,
+                                   std::vector<DirectedGraph::node_id> *victims)
+    : tid_node_mapping_(std::make_unique<transaction_id_node_map>()),
+      lock_table_(lock_table),
+      status_(status),
+      victims_(victims) {
+}
+
+void DeadLockDetector::run() {
+  while (true) {
+    if (status_->load() == DeadLockDetectorStatus::kQuit) {
+      // DeadLockDetector should stop.
+      return;
+    }
+    while (status_->load() == DeadLockDetectorStatus::kDone) {
+      // LockTable has not process the previous batch yet.
+    }
+
+    // TODO(Hakan): Implement logging mechanism for deadlock detection
+    //              start and end times.
+    std::vector<DirectedGraph::node_id> victim_new_batch = getAllVictims();
+
+    // Swap new batch with old batch to make LockTable to see new victims.
+    std::swap(victim_new_batch, *victims_);
+
+    // Signal LockTable that new batch is ready.
+    status_->store(DeadLockDetectorStatus::kDone);
+
+    // DeadLockDetector should run once in a predefined interval.
+    std::this_thread::sleep_for(
+        std::chrono::seconds(kSleepDurationInSeconds));
+  }
+}
+
+void DeadLockDetector::addPendingInfo(const transaction_id pending,
+                                      const transaction_id owner) {
+  const DirectedGraph::node_id pending_node_id = getNodeId(pending);
+  const DirectedGraph::node_id owner_node_id = getNodeId(owner);
+
+  // TODO(Hakan): Check first whether link is already created. Use checked
+  //              version for adding an edge.
+  wait_for_graph_->addEdgeUnchecked(pending_node_id, owner_node_id);
+}
+
+void DeadLockDetector::deletePendingInfo(const transaction_id pending,
+                                         const transaction_id owner) {
+  LOG(FATAL) << "Not implemented";
+}
+
+bool DeadLockDetector::isDependent(const transaction_id pending,
+                                   const transaction_id owner) const {
+  LOG(FATAL) << "Not implemented";
+}
+
+std::vector<transaction_id>
+DeadLockDetector::getAllDependents(const transaction_id owner) const {
+  LOG(FATAL) << "Not implemented";
+}
+
+std::vector<transaction_id>
+DeadLockDetector::getAllDependees(transaction_id pending) {
+  const DirectedGraph::node_id pending_node_id = getNodeId(pending);
+  const std::vector<DirectedGraph::node_id> nodes
+      = wait_for_graph_->getAdjacentNodes(pending_node_id);
+  std::vector<transaction_id> transactions;
+  transactions.reserve(nodes.size());
+  for (const DirectedGraph::node_id node_id : nodes) {
+    const transaction_id tid = wait_for_graph_->getDataFromNode(node_id);
+    transactions.push_back(tid);
+  }
+  return transactions;
+}
+
+DirectedGraph::node_id DeadLockDetector::getNodeId(const transaction_id tid) {
+  DirectedGraph::node_id node_id;
+  if (tid_node_mapping_->count(tid) == 0) {
+    // If it is not created, create it.
+    node_id = addNode(tid);
+  } else {
+    // Otherwise find it in the map.
+    node_id = (*tid_node_mapping_)[tid];
+  }
+  return node_id;
+}
+
+
+DirectedGraph::node_id DeadLockDetector::addNode(const transaction_id tid) {
+  const DirectedGraph::node_id node_id =
+      wait_for_graph_->addNodeUnchecked(tid);
+  tid_node_mapping_->emplace(tid, node_id);
+  return node_id;
+}
+
+std::vector<transaction_id> DeadLockDetector::getAllVictims()  {
+  std::vector<transaction_id> result_victims;
+  wait_for_graph_.reset(new DirectedGraph());
+
+  // Critical region on LockTable starts here.
+  lock_table_->latchShared();
+  for (const auto &lock_control_block : *lock_table_) {
+    const LockTable::lock_own_list &own_list = lock_control_block.second.first;
+    const LockTable::lock_pending_list &pending_list =
+        lock_control_block.second.second;
+
+    for (const auto &owned_lock_info : own_list) {
+      const transaction_id owned_transaction = owned_lock_info.first;
+      const DirectedGraph::node_id owned_node = getNodeId(owned_transaction);
+
+      for (const auto &pending_lock_info : pending_list) {
+        const transaction_id pending_transaction = pending_lock_info.first;
+        const DirectedGraph::node_id pending_node = getNodeId(pending_transaction);
+
+        wait_for_graph_->addEdgeUnchecked(pending_node, owned_node);
+      }
+    }
+  }
+
+  lock_table_->unlatchShared();
+  // Critical region on LockTable ends here.
+
+  const CycleDetector cycle_detector(wait_for_graph_.get());
+  const std::vector<DirectedGraph::node_id> victim_nodes =
+      cycle_detector.chooseVictimsToBreakCycle();
+  for (const DirectedGraph::node_id node_id : victim_nodes) {
+    const transaction_id victim_tid = wait_for_graph_->getDataFromNode(node_id);
+    result_victims.push_back(victim_tid);
+  }
+
+  // Destroy the wait graph. It will be reconstructed kSleepDurationSeconds
+  // seconds later.
+  wait_for_graph_.reset();
+
+  return result_victims;
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/DeadLockDetector.hpp
----------------------------------------------------------------------
diff --git a/transaction/DeadLockDetector.hpp b/transaction/DeadLockDetector.hpp
new file mode 100644
index 0000000..6897afb
--- /dev/null
+++ b/transaction/DeadLockDetector.hpp
@@ -0,0 +1,156 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_TRANSACTION_DEAD_LOCK_DETECTOR_HPP_
+#define QUICKSTEP_TRANSACTION_DEAD_LOCK_DETECTOR_HPP_
+
+#include <atomic>
+#include <cstdint>
+#include <memory>
+#include <unordered_map>
+#include <vector>
+
+#include "threading/Thread.hpp"
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/Transaction.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+class LockTable;
+
+/** \addtogroup Transaction
+ *  @{
+ */
+
+/**
+ * @brief Notification mechanism between LockManager and DeadLockDetector.
+ **/
+enum class DeadLockDetectorStatus {
+  kNotReady = 0,
+  kDone,
+  kQuit,
+};
+
+/**
+ * @brief Class for deadlock detection on wait-for graph.
+ **/
+class DeadLockDetector : public Thread {
+ public:
+  typedef std::unordered_map<transaction_id, DirectedGraph::node_id>
+      transaction_id_node_map;
+
+  /**
+   * @brief Constructor for DeadLockDetector.
+   *
+   * @param lock_table Pointer to lock table, which this class gets the
+   *        necessary information.
+   * @param status Pointer to status object which will act as a message
+   *        passing algorithm between LockManager.
+   * @param victims Message passing buffer betwen DeadLockDetector and
+   *        LockManager.
+   **/
+  DeadLockDetector(LockTable *lock_table,
+                   std::atomic<DeadLockDetectorStatus> *status,
+                   std::vector<DirectedGraph::node_id> *victims);
+
+  void run() override;
+
+  /**
+   * @brief Adds pending information based on a resource conflict.
+   *
+   * @param pending Id of the transaction that waits for the resource lock.
+   * @param owner Id of the transaction that owns the resource lock.
+   */
+  void addPendingInfo(const transaction_id pending,
+                      const transaction_id owner);
+
+  /**
+   * @brief Deletes pending information on a resource.
+   *
+   * @param pending Id of the transaction that waits for the resource lock.
+   * @param owner Id of the transaction that owns the resource lock.
+   *
+   * @warning This method is not implemented yet.
+   */
+  void deletePendingInfo(const transaction_id pending,
+                         const transaction_id owner);
+
+  /**
+   * @brief Check whether first transaction waits for the latter.
+   *
+   * @param pending Id of the transaction which will be checked whether
+   *        it waits for the other.
+   * @param owner Id of the transaction which will be checked whether
+   *        it is waited by the first.
+   *
+   * @warning This method is not implemented yet.
+   */
+  bool isDependent(const transaction_id pending,
+                   const transaction_id owner) const;
+
+  /**
+   * @brief Gives the ids of transactions that wait for the owner transaction.
+   *
+   * @param owner Id of the transaction whose the penders will be returned.
+   * @return Vector of transaction ids that wait for owner.
+   */
+  std::vector<transaction_id> getAllDependents(const transaction_id owner) const;
+
+  /**
+   * @brief Gives the ids of transaction that the pending transaction waits for.
+   * @warning This method is not implemented yet.
+
+   * @param pending Id of the transaction that is pending.
+   * @return Vector of transaction ids which the pending transaction waits for.
+   */
+  std::vector<transaction_id> getAllDependees(transaction_id pending);
+
+  /**
+   * @brief Gives the list of victims whose the terminations will end the cycle.
+   *
+   * @return Vector of victim transaction ids.
+   */
+  std::vector<transaction_id> getAllVictims();
+
+ private:
+  static constexpr std::int64_t kSleepDurationInSeconds = 5;
+
+  DirectedGraph::node_id getNodeId(const transaction_id tid);
+
+  DirectedGraph::node_id addNode(const transaction_id tid);
+
+  // Owned pointer to wait-for graph.
+  std::unique_ptr<DirectedGraph> wait_for_graph_;
+
+  // Mapping from TransactioId to NodeId in graph.
+  std::unique_ptr<transaction_id_node_map> tid_node_mapping_;
+
+  // Pointer to lock table. Dependence edges will be created
+  // by the information got from lock table.
+  LockTable *lock_table_;
+
+  std::atomic<DeadLockDetectorStatus> *status_;
+  std::vector<DirectedGraph::node_id> *victims_;
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TRANSACTION_DEAD_LOCK_DETECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/DirectedGraph.hpp
----------------------------------------------------------------------
diff --git a/transaction/DirectedGraph.hpp b/transaction/DirectedGraph.hpp
index 89ce9c6..16b551a 100644
--- a/transaction/DirectedGraph.hpp
+++ b/transaction/DirectedGraph.hpp
@@ -21,8 +21,8 @@
 #include <algorithm>
 #include <cstddef>
 #include <cstdint>
+#include <iterator>
 #include <memory>
-#include <stack>
 #include <unordered_set>
 #include <vector>
 
@@ -39,7 +39,7 @@ namespace transaction {
  */
 
 /**
- * @brief Class for representing a directed graph. Vertices are transaction 
+ * @brief Class for representing a directed graph. Vertices are transaction
  *        ids, edges are wait-for relations.
  **/
 class DirectedGraph {
@@ -54,34 +54,31 @@ class DirectedGraph {
   /**
    * @brief Adds a new node to the graph with the given transaction id.
    *        It does not check whether the transaction id is valid or not.
-   * @warning Pointer ownership will pass to the graph, therefore it
-   *          should not be deleted.
    *
-   * @param data Pointer to the transaction id that will be contained
+   * @param transaction_id_payload Transaction id that will be contained
    *        in the node.
    * @return Id of the newly created node.
    **/
-  inline node_id addNodeUnchecked(transaction_id *data) {
-    nodes_.emplace_back(data);
+  inline
+  node_id addNodeUnchecked(const transaction_id transaction_id_payload) {
+    nodes_.emplace_back(transaction_id_payload);
     return nodes_.size() - 1;
   }
 
   /**
    * @brief Adds a new node to the graph with the given transaction id.
    *        It checks whether the transaction id is valid or not.
-   * @warning Pointer ownership will pass to the graph, therefore it
-   *          should not be deleted.
    *
-   * @param data Pointer to the transaction id that will be contained
+   * @param transaction_id_payload Transaction id that will be contained
    *        in the node.
    * @return Id of the newly created node.
    **/
-  inline node_id addNodeCheckExists(transaction_id *data) {
-    for (std::vector<DirectedGraphNode>::const_iterator
-           it = nodes_.cbegin(); it != nodes_.cend(); ++it) {
-      CHECK(*data != it->getData());
+  node_id addNodeCheckExists(const transaction_id transaction_id_payload) {
+    for (const auto &node : nodes_) {
+      CHECK(transaction_id_payload != node.getData());
     }
-    nodes_.emplace_back(data);
+
+    nodes_.emplace_back(transaction_id_payload);
     return nodes_.size() - 1;
   }
 
@@ -91,10 +88,10 @@ class DirectedGraph {
    * @warning Does not check arguments are legit. It may cause
    *          out of range errors.
    *
-   * @param fromNode The node that edge is orginated.
+   * @param fromNode The node that edge is originated.
    * @param toNode The node that edge is ended.
    **/
-  inline void addEdgeUnchecked(node_id from_node, node_id to_node) {
+  void addEdgeUnchecked(node_id from_node, node_id to_node) {
     nodes_[from_node].addOutgoingEdge(to_node);
   }
 
@@ -105,7 +102,7 @@ class DirectedGraph {
    * @param fromNode The node that edge is orginated.
    * @param toNode The node that edge is ended.
    **/
-  inline void addEdgeCheckExists(node_id from_node, node_id to_node) {
+  void addEdgeCheckExists(node_id from_node, node_id to_node) {
     CHECK(from_node < getNumNodes() && to_node < getNumNodes());
     nodes_[from_node].addOutgoingEdge(to_node);
   }
@@ -119,7 +116,7 @@ class DirectedGraph {
    * @param toNode Id of the node that edge is ended.
    * @return True if there is an edge, false otherwise.
    **/
-  inline bool hasEdge(node_id from_node, node_id to_node) const {
+  bool hasEdge(node_id from_node, node_id to_node) const {
     DCHECK(from_node < getNumNodes() && to_node < getNumNodes());
     return nodes_[from_node].hasOutgoingEdge(to_node);
   }
@@ -130,7 +127,7 @@ class DirectedGraph {
    * @param node Id of the node that the data is got from.
    * @return Id of the transaction that this node contains.
    **/
-  inline transaction_id getDataFromNode(node_id node) const {
+  transaction_id getDataFromNode(node_id node) const {
     DCHECK(node < getNumNodes());
     return nodes_[node].getData();
   }
@@ -140,7 +137,7 @@ class DirectedGraph {
    *
    * @return The number of nodes the graph has.
    **/
-  inline std::size_t getNumNodes() const {
+  std::size_t getNumNodes() const {
     return nodes_.size();
   }
 
@@ -158,18 +155,18 @@ class DirectedGraph {
   // Class for representing a graph node.
   class DirectedGraphNode {
    public:
-    explicit DirectedGraphNode(transaction_id *data)
-      : data_(data) {}
+    explicit DirectedGraphNode(const transaction_id payload)
+      : transaction_id_payload_(payload) {}
 
-    inline void addOutgoingEdge(node_id to_node) {
+    void addOutgoingEdge(node_id to_node) {
       outgoing_edges_.insert(to_node);
     }
 
-    inline bool hasOutgoingEdge(node_id to_node) const {
+    bool hasOutgoingEdge(node_id to_node) const {
       return outgoing_edges_.count(to_node) == 1;
     }
 
-    inline std::vector<node_id> getOutgoingEdges() const {
+    std::vector<node_id> getOutgoingEdges() const {
       // TODO(hakan): Benchmark this version and the alternative which the
       //              function returns const reference and the uniqueness
       //              is imposed in the outgoing_edges_ as a vector.
@@ -179,13 +176,12 @@ class DirectedGraph {
       return result;
     }
 
-    inline transaction_id getData() const {
-      return *(data_.get());
+    transaction_id getData() const {
+      return transaction_id_payload_;
     }
 
    private:
-    // Owner pointer to transaction id.
-    std::unique_ptr<transaction_id> data_;
+    const transaction_id transaction_id_payload_;
 
     // Endpoint nodes of outgoing edges originated from this node.
     std::unordered_set<node_id> outgoing_edges_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/LockManager.cpp
----------------------------------------------------------------------
diff --git a/transaction/LockManager.cpp b/transaction/LockManager.cpp
new file mode 100644
index 0000000..da6181a
--- /dev/null
+++ b/transaction/LockManager.cpp
@@ -0,0 +1,237 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "transaction/LockManager.hpp"
+
+#include <cstdint>
+#include <stack>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/DeadLockDetector.hpp"
+#include "transaction/LockRequest.hpp"
+#include "transaction/LockTable.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+#include "transaction/TransactionTable.hpp"
+#include "utility/ThreadSafeQueue.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace transaction {
+
+DEFINE_uint64(max_try_incoming, 10000,
+              "The maximum number of tries that lock manager checks incoming "
+              "request buffer until the buffer is empty.");
+
+DEFINE_uint64(max_try_inner, 6000,
+              "The maximum number of tries that lock manager checks inner "
+              "request buffer until the buffer is empty.");
+
+LockManager::LockManager(ThreadSafeQueue<LockRequest> *incoming_requests,
+                         ThreadSafeQueue<LockRequest> *permitted_requests)
+    : lock_table_(std::make_unique<LockTable>()),
+      transaction_table_(std::make_unique<TransactionTable>()),
+      detector_status_(DeadLockDetectorStatus::kNotReady),
+      deadlock_detector_(std::make_unique<DeadLockDetector>(lock_table_.get(),
+                                                            &detector_status_,
+                                                            &victim_result_)),
+      incoming_requests_(*incoming_requests),
+      permitted_requests_(*permitted_requests),
+      inner_pending_requests_() {
+}
+
+LockManager::~LockManager() {
+  deadlock_detector_->join();
+}
+
+void LockManager::run() {
+  deadlock_detector_->start();
+
+  const std::uint64_t kMaxTryIncoming =
+      static_cast<std::uint64_t>(FLAGS_max_try_incoming);
+  const std::uint64_t kMaxTryInner =
+      static_cast<std::uint64_t>(FLAGS_max_try_incoming);
+
+  while (true) {
+    for (std::uint64_t tries = 0; tries < kMaxTryIncoming; ++tries) {
+      if (!incoming_requests_.empty()) {
+        const LockRequest request = incoming_requests_.popOne();
+        if (request.getRequestType() == RequestType::kReleaseLocks) {
+          CHECK(releaseAllLocks(request.getTransactionId()))
+              << "Unexpected condition occured.";
+
+        } else if (acquireLock(request.getTransactionId(),
+                               request.getResourceId(),
+                               request.getAccessMode())) {
+          LOG(INFO) << "Transaction "
+                    << std::to_string(request.getTransactionId())
+                    << " is waiting " + request.getResourceId().toString();
+
+            inner_pending_requests_.push(request);
+        } else {
+            LOG(INFO) << "Transaction "
+                      << std::to_string(request.getTransactionId())
+                      << " acquired " + request.getResourceId().toString();
+
+            permitted_requests_.push(request);
+        }
+      }
+    }
+
+    for (std::uint64_t tries = 0; tries < kMaxTryInner; ++tries) {
+      if (!inner_pending_requests_.empty()) {
+        const LockRequest request = inner_pending_requests_.front();
+
+        if (acquireLock(request.getTransactionId(), request.getResourceId(),
+                        request.getAccessMode())) {
+          inner_pending_requests_.pop();
+          permitted_requests_.push(request);
+        }
+      }
+    }
+
+    // Resolve deadlocks.
+    killVictims();
+  }
+}
+
+bool LockManager::acquireLock(const transaction_id tid,
+                              const ResourceId &rid,
+                              const AccessMode &access_mode) {
+  std::stack<std::pair<ResourceId, AccessMode>> stack;
+  ResourceId current_rid = rid;
+  AccessMode current_access_mode = access_mode;
+  stack.push(std::make_pair(current_rid, current_access_mode));
+
+  while (current_rid.hasParent()) {
+    current_rid = current_rid.getParentResourceId();
+    current_access_mode = (current_access_mode.isShareLock() ||
+                           current_access_mode.isIntentionShareLock())
+                              ? AccessMode(AccessMode::IsLockMode())
+                              : AccessMode(AccessMode::IxLockMode());
+
+    stack.push(std::make_pair(current_rid, current_access_mode));
+  }
+
+  lock_table_->latchExclusive();
+
+  while (!stack.empty()) {
+    const std::pair<ResourceId, AccessMode> pair_to_pick = stack.top();
+    const ResourceId rid_to_pick = pair_to_pick.first;
+    const AccessMode access_mode_to_pick = pair_to_pick.second;
+
+    if (!acquireLockInternal(tid, rid_to_pick, access_mode_to_pick)) {
+      lock_table_->unlatchExclusive();
+      return false;
+    }
+    stack.pop();
+  }
+  lock_table_->unlatchExclusive();
+  return true;
+}
+
+bool LockManager::releaseAllLocks(const transaction_id tid,
+                                  const bool latch_table) {
+  const std::vector<ResourceId> resource_ids
+      = transaction_table_->getResourceIdList(tid);
+  const TransactionTableResult transaction_deleted
+      = transaction_table_->deleteTransaction(tid);
+
+  CHECK(transaction_deleted != TransactionTableResult::kTransactionDeleteError)
+      << "In LockManager.releaseAllLocks: Transaction could not be deleted!";
+
+  if (latch_table) {
+    lock_table_->latchExclusive();
+  }
+  for (const auto &resource_id : resource_ids) {
+    const LockTableResult lock_deleted = lock_table_->deleteLock(tid, resource_id);
+
+    LOG(INFO) << "Transaction "
+              << std::to_string(tid)
+              << " released lock:"
+              << resource_id.toString();
+    CHECK(lock_deleted != LockTableResult::kDeleteError)
+        << "In LockManager.releaseAllLock lock could not be deleted from "
+           "LockTable";
+  }
+  if (latch_table) {
+    lock_table_->unlatchExclusive();
+  }
+  return true;
+}
+
+bool LockManager::acquireLockInternal(const transaction_id tid,
+                                      const ResourceId &rid,
+                                      const AccessMode &access_mode) {
+  const LockTableResult lock_result = lock_table_->putLock(tid, rid, access_mode);
+  CHECK(lock_result != LockTableResult::kPutError)
+      << "Unexpected result in LockManager.acquireLockInternal";
+
+  switch (lock_result) {
+  case LockTableResult::kAlreadyInOwned: {
+    return true;
+  }
+  case LockTableResult::kPlacedInOwned: {
+    const TransactionTableResult transaction_result
+        = transaction_table_->putOwnEntry(tid, rid, access_mode);
+    CHECK(transaction_result == TransactionTableResult::kPlacedInOwned)
+        << "Unexpected result in LockManager.acquireLockInternal: "
+           "Mismatch of table results: LockTable entry is owned, "
+           "whereas TransactionTable entry is not owned.";
+    return true;
+  }
+  case LockTableResult::kAlreadyInPending: {
+    return false;
+  }
+  case LockTableResult::kPlacedInPending: {
+    const TransactionTableResult transaction_result =
+      transaction_table_->putPendingEntry(tid, rid, access_mode);
+    CHECK(transaction_result == TransactionTableResult::kPlacedInPending)
+        << "Unexpected result in LockManager.acquireLockInternal: "
+           "Mismatch of table results: LockTable entry is pending, "
+           "whereas TransactionTable entry is not pending";
+    return false;
+  }
+  default: {
+    return false;
+  }
+  }
+}
+
+void LockManager::killVictims() {
+  if (detector_status_.load() == DeadLockDetectorStatus::kDone) {
+    lock_table_->latchExclusive();
+    for (const auto victim_transaction_id : victim_result_) {
+      releaseAllLocks(victim_transaction_id, false);
+      // TODO(Hakan): Find a way to kill transaction, so that requests with this
+      //              tid should be ignored.
+      LOG(INFO) << "Killed transaction "
+                << std::to_string(victim_transaction_id);
+    }
+    lock_table_->unlatchExclusive();
+  }
+  victim_result_.clear();
+  detector_status_.store(DeadLockDetectorStatus::kNotReady);
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/LockManager.hpp
----------------------------------------------------------------------
diff --git a/transaction/LockManager.hpp b/transaction/LockManager.hpp
new file mode 100644
index 0000000..40ee6c8
--- /dev/null
+++ b/transaction/LockManager.hpp
@@ -0,0 +1,128 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_TRANSACTION_LOCK_MANAGER_HPP_
+#define QUICKSTEP_TRANSACTION_LOCK_MANAGER_HPP_
+
+#include <atomic>
+#include <memory>
+#include <queue>
+#include <vector>
+
+#include "threading/Thread.hpp"
+#include "transaction/DeadLockDetector.hpp"
+#include "transaction/LockRequest.hpp"
+#include "transaction/Transaction.hpp"
+
+namespace quickstep {
+
+template <typename T> class ThreadSafeQueue;
+
+namespace transaction {
+
+class AccessMode;
+class LockTable;
+class ResourceId;
+class TransactionTable;
+
+/** \addtogroup Transaction
+ *  @{
+ **/
+
+/**
+ * @brief Class for centralized location of acquisition and releasing
+ *        of resource locks.
+ **/
+class LockManager : public Thread {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param incoming_requests Queue for the lock requests that are waiting
+   *        for the permission.
+   * @param permitted_requests Queue for the lock requests that are granted
+   *        permission.
+   **/
+  LockManager(ThreadSafeQueue<LockRequest> *incoming_requests,
+              ThreadSafeQueue<LockRequest> *permitted_requests);
+
+  /**
+   * @brief Destructor for LockManager. It handles the thread
+   *        joins that it owns.
+   **/
+  ~LockManager();
+
+  /**
+   * @brief Method for defining the LockManager's thread main logic.
+   *
+   * @warning Users must not use this function directly. Instead use
+   *          start() method inherited from the Thread class.
+   **/
+  void run() override;
+
+  /**
+   * @brief Acquires the lock on resource with specified access mode.
+   *
+   * @param tid Id of the transaction which the resource lock is acquired for.
+   * @param rid Id of the resource on which the resource lock is acquired.
+   * @param access_mode Permissible access mode on resource.
+   *
+   * @return True if it can acquire the lock from root to leaf lock hierarchy,
+   *         false otherwise.
+   **/
+  bool acquireLock(const transaction_id tid,
+                   const ResourceId &rid,
+                   const AccessMode &access_mode);
+
+  /**
+   * @brief Releases all locks hold by the transaction.
+   *
+   * @param tid Id of the transaction whose locks will be released.
+   * @param latch_table If it is true, the method latch the whole
+   *        lock table, which is default.
+   * @return True if tid releases all of its locks.
+   **/
+  bool releaseAllLocks(const transaction_id tid,
+                       const bool latch_table = true);
+
+  /**
+   * @brief Release the locks acquired by the transactions contained
+   *        in victim buffer to break the deadlock.
+   **/
+  void killVictims();
+
+ private:
+  bool acquireLockInternal(const transaction_id tid,
+                           const ResourceId &rid,
+                           const AccessMode &access_mode);
+
+  std::unique_ptr<LockTable> lock_table_;
+  std::unique_ptr<TransactionTable> transaction_table_;
+  std::atomic<DeadLockDetectorStatus> detector_status_;
+  std::vector<transaction_id> victim_result_;
+  std::unique_ptr<DeadLockDetector> deadlock_detector_;
+  ThreadSafeQueue<LockRequest> &incoming_requests_;
+  ThreadSafeQueue<LockRequest> &permitted_requests_;
+  std::queue<LockRequest> inner_pending_requests_;
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TRANSACTION_LOCK_MANAGER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/LockTable.cpp
----------------------------------------------------------------------
diff --git a/transaction/LockTable.cpp b/transaction/LockTable.cpp
index 77986f6..7b568fc 100644
--- a/transaction/LockTable.cpp
+++ b/transaction/LockTable.cpp
@@ -18,15 +18,11 @@
 #include "transaction/LockTable.hpp"
 
 #include <list>
-#include <unordered_map>
 #include <utility>
 
-#include "threading/SharedMutex.hpp"
 #include "transaction/AccessMode.hpp"
 #include "transaction/Lock.hpp"
-#include "transaction/ResourceId.hpp"
 #include "transaction/Transaction.hpp"
-#include "utility/Macros.hpp"
 
 namespace quickstep {
 namespace transaction {
@@ -34,7 +30,7 @@ namespace transaction {
 LockTableResult
 LockTable::putLock(const transaction_id tid,
                    const ResourceId &rid,
-                   const AccessMode access_mode) {
+                   const AccessMode &access_mode) {
   // TODO(hakan): Lock upgrade is not supported.
   lock_list_pair &lock_list_pair = internal_map_[rid];
 
@@ -47,7 +43,7 @@ LockTable::putLock(const transaction_id tid,
   for (lock_own_list::const_iterator it = lock_own_list.cbegin();
        it != lock_own_list.cend(); ++it) {
     if (it->first == tid && it->second.getAccessMode() == access_mode) {
-      return LockTableResult::kALREADY_IN_OWNED;
+      return LockTableResult::kAlreadyInOwned;
     }
   }
 
@@ -56,7 +52,7 @@ LockTable::putLock(const transaction_id tid,
   for (lock_pending_list::const_iterator it = lock_pending_list.cbegin();
        it != lock_pending_list.cend(); ++it) {
     if (it->first == tid && it->second.getAccessMode() == access_mode) {
-      return LockTableResult::kALREADY_IN_PENDING;
+      return LockTableResult::kAlreadyInPending;
     }
   }
 
@@ -68,18 +64,18 @@ LockTable::putLock(const transaction_id tid,
       if (!access_mode.isCompatible(it->second.getAccessMode())) {
         lock_pending_list.push_back(std::make_pair(tid,
                                                    Lock(rid, access_mode)));
-        return LockTableResult::kPLACED_IN_PENDING;
+        return LockTableResult::kPlacedInPending;
       }
     }
 
     lock_own_list.push_back(std::make_pair(tid, Lock(rid, access_mode)));
-    return LockTableResult::kPLACED_IN_OWNED;
+    return LockTableResult::kPlacedInOwned;
   } else {
     // If the pending list is not empty, even if the lock request is compatible
     // with other owned lock entries, we put the new request into the pending
     // list to eliminate starvation.
     lock_pending_list.push_back(std::make_pair(tid, Lock(rid, access_mode)));
-    return LockTableResult::kPLACED_IN_PENDING;
+    return LockTableResult::kPlacedInPending;
   }
 }
 
@@ -105,7 +101,7 @@ LockTable::deleteLock(const transaction_id tid,
       // compatible with the remaining owned entries.
       movePendingToOwned(rid);
 
-      return LockTableResult::kDEL_FROM_OWNED;
+      return LockTableResult::kDeleteFromOwned;
     }
   }
 
@@ -116,13 +112,13 @@ LockTable::deleteLock(const transaction_id tid,
     if (it->first == tid) {
       // If it exists, erase it from pending list.
       lock_pending_list.erase(it);
-      return LockTableResult::kDEL_FROM_PENDING;
+      return LockTableResult::kDeleteFromPending;
     }
   }
 
   // Execution reaches here, if we cannot find the corresponding lock entry
   // in the both list.
-  return LockTableResult::kDEL_ERROR;
+  return LockTableResult::kDeleteError;
 }
 
 void LockTable::movePendingToOwned(const ResourceId &rid) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/LockTable.hpp
----------------------------------------------------------------------
diff --git a/transaction/LockTable.hpp b/transaction/LockTable.hpp
index 5a0612e..529db12 100644
--- a/transaction/LockTable.hpp
+++ b/transaction/LockTable.hpp
@@ -23,7 +23,6 @@
 #include <utility>
 
 #include "threading/SharedMutex.hpp"
-#include "transaction/AccessMode.hpp"
 #include "transaction/Lock.hpp"
 #include "transaction/ResourceId.hpp"
 #include "transaction/Transaction.hpp"
@@ -32,6 +31,8 @@
 namespace quickstep {
 namespace transaction {
 
+class AccessMode;
+
 /** \addtogroup Transaction
  * @{
  */
@@ -40,14 +41,14 @@ namespace transaction {
  * @brief Represents different results for LockTable's methods.
  **/
 enum class LockTableResult {
-  kPLACED_IN_OWNED = 0,
-  kPLACED_IN_PENDING,
-  kALREADY_IN_OWNED,
-  kALREADY_IN_PENDING,
-  kDEL_FROM_OWNED,
-  kDEL_FROM_PENDING,
-  kDEL_ERROR,
-  kPUT_ERROR,
+  kPlacedInOwned = 0,
+  kPlacedInPending,
+  kAlreadyInOwned,
+  kAlreadyInPending,
+  kDeleteFromOwned,
+  kDeleteFromPending,
+  kDeleteError,
+  kPutError,
 };
 
 /**
@@ -79,32 +80,31 @@ class LockTable {
    * @param rid Id of the resource to be locked.
    * @param access_mode Access mode of the lock.
    *
-   * @return LockTableResult::kPLACED_IN_OWNED if lock is granted,
-   *         LockTableResult::kPLACED_IN_PENDING if lock is not granted,
-   *         LockTableResult::kALREADY_IN_OWNED if lock has been
+   * @return LockTableResult::kPlacedInOwned if lock is granted,
+   *         LockTableResult::kPlacedInPending if lock is not granted,
+   *         LockTableResult::kAlreadyInOwned if lock has been
    *         already granted,
-   *         LockTableResult::kALREADY_IN_PENDING if lock has been
+   *         LockTableResult::kAlreadyInPending if lock has been
    *         already pending.
    **/
   LockTableResult putLock(const transaction_id tid,
                           const ResourceId &rid,
-                          const AccessMode access_mode);
+                          const AccessMode &access_mode);
   /**
    * @brief Deletes the lock entry.
    *
    * @param tid Id of the transaction that owns or awaits.
    * @param rid Id of resource that the lock covers.
    *
-   * @return LockTableResult::kDEL_FROM_OWNED if the lock is deleted from
+   * @return LockTableResult::kDelFromOwned if the lock is deleted from
    *         owned list,
-   *         LockTableResult::kDEL_FROM_PENDING if the lock is deleted from
+   *         LockTableResult::kDelFromPending if the lock is deleted from
    *         pending list,
-   *         LockTableResult::kDEL_ERROR if the lock cannot be found
+   *         LockTableResult::kDelError if the lock cannot be found
    **/
   LockTableResult deleteLock(const transaction_id tid,
                              const ResourceId &rid);
 
-
   /**
    * @brief Iterator for begin position.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/ResourceId.hpp
----------------------------------------------------------------------
diff --git a/transaction/ResourceId.hpp b/transaction/ResourceId.hpp
index b9d1cdf..3a770dd 100644
--- a/transaction/ResourceId.hpp
+++ b/transaction/ResourceId.hpp
@@ -19,7 +19,6 @@
 #define QUICKSTEP_TRANSACTION_RESOURCE_ID_HPP_
 
 #include <cstddef>
-#include <limits>
 #include <string>
 
 #include "catalog/CatalogTypedefs.hpp"
@@ -202,10 +201,10 @@ class ResourceId {
     return tuple_id_ == kTupleIdPlaceholder;
   }
 
-  const database_id db_id_;
-  const relation_id rel_id_;
-  const block_id block_id_;
-  const tuple_id tuple_id_;
+  database_id db_id_;
+  relation_id rel_id_;
+  block_id block_id_;
+  tuple_id tuple_id_;
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/StronglyConnectedComponents.cpp
----------------------------------------------------------------------
diff --git a/transaction/StronglyConnectedComponents.cpp b/transaction/StronglyConnectedComponents.cpp
index 89daf46..f50ed85 100644
--- a/transaction/StronglyConnectedComponents.cpp
+++ b/transaction/StronglyConnectedComponents.cpp
@@ -17,7 +17,6 @@
 
 #include "transaction/StronglyConnectedComponents.hpp"
 
-#include <cstddef>
 #include <cstdint>
 #include <stack>
 #include <unordered_map>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/Transaction.cpp
----------------------------------------------------------------------
diff --git a/transaction/Transaction.cpp b/transaction/Transaction.cpp
deleted file mode 100644
index 3478d01..0000000
--- a/transaction/Transaction.cpp
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
- *
- *   Licensed under the Apache License, Version 2.0 (the "License");
- *   you may not use this file except in compliance with the License.
- *   You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- **/
-
-#include "transaction/Transaction.hpp"
-
-#include <functional>
-
-namespace quickstep {
-
-namespace transaction {
-
-TransactionId Transaction::getTransactionId() const {
-  return tid_;
-}
-
-void Transaction::setStatus(TransactionStatus status) {
-  status_ = status;
-}
-
-TransactionStatus Transaction::getStatus() const {
-  return status_;
-}
-
-bool Transaction::operator==(const Transaction &other) const {
-  return tid_ == other.tid_;
-}
-
-std::size_t Transaction::TransactionHasher::operator()(const Transaction &transaction) const {
-  return std::hash<TransactionId>()(transaction.tid_);
-}
-
-}  // namespace transaction
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/TransactionTable.cpp
----------------------------------------------------------------------
diff --git a/transaction/TransactionTable.cpp b/transaction/TransactionTable.cpp
index 993703a..3e37439 100644
--- a/transaction/TransactionTable.cpp
+++ b/transaction/TransactionTable.cpp
@@ -33,7 +33,7 @@ namespace transaction {
 TransactionTableResult
 TransactionTable::putOwnEntry(const transaction_id tid,
                               const ResourceId &rid,
-                              const AccessMode access_mode) {
+                              const AccessMode &access_mode) {
   transaction_list_pair &transaction_list_pair = internal_map_[tid];
   transaction_own_list &transaction_own_list = transaction_list_pair.first;
 
@@ -45,7 +45,7 @@ TransactionTable::putOwnEntry(const transaction_id tid,
 TransactionTableResult
 TransactionTable::putPendingEntry(const transaction_id tid,
                                   const ResourceId &rid,
-                                  const AccessMode access_mode) {
+                                  const AccessMode &access_mode) {
   transaction_list_pair &transaction_list_pair = internal_map_[tid];
   transaction_pending_list &transaction_pending_list
       = transaction_list_pair.second;
@@ -59,7 +59,7 @@ TransactionTable::putPendingEntry(const transaction_id tid,
 TransactionTableResult
 TransactionTable::deleteOwnEntry(const transaction_id tid,
                                  const ResourceId &rid,
-                                 const AccessMode access_mode) {
+                                 const AccessMode &access_mode) {
   transaction_list_pair &transaction_list_pair = internal_map_[tid];
   transaction_own_list &transaction_own_list = transaction_list_pair.first;
 
@@ -79,7 +79,7 @@ TransactionTable::deleteOwnEntry(const transaction_id tid,
 TransactionTableResult
 TransactionTable::deletePendingEntry(const transaction_id tid,
                                      const ResourceId &rid,
-                                     const AccessMode access_mode) {
+                                     const AccessMode &access_mode) {
   transaction_list_pair &transaction_list_pair = internal_map_[tid];
   transaction_pending_list &transaction_pending_list
       = transaction_list_pair.second;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/TransactionTable.hpp
----------------------------------------------------------------------
diff --git a/transaction/TransactionTable.hpp b/transaction/TransactionTable.hpp
index a5e1da4..29df536 100644
--- a/transaction/TransactionTable.hpp
+++ b/transaction/TransactionTable.hpp
@@ -23,7 +23,6 @@
 #include <utility>
 #include <vector>
 
-#include "transaction/AccessMode.hpp"
 #include "transaction/Lock.hpp"
 #include "transaction/ResourceId.hpp"
 #include "transaction/Transaction.hpp"
@@ -32,6 +31,8 @@
 namespace quickstep {
 namespace transaction {
 
+class AccessMode;
+
 /** \addtogroup Transaction
  *  @{
  */
@@ -81,7 +82,7 @@ class TransactionTable {
    **/
   TransactionTableResult putOwnEntry(const transaction_id tid,
                                      const ResourceId &rid,
-                                     const AccessMode access_mode);
+                                     const AccessMode &access_mode);
 
   /**
    * @brief Puts a pending entry of the given resource id in the given
@@ -95,7 +96,7 @@ class TransactionTable {
    **/
   TransactionTableResult putPendingEntry(const transaction_id tid,
                                          const ResourceId &rid,
-                                         const AccessMode access_mode);
+                                         const AccessMode &access_mode);
 
   /**
    * @brief Deletes the owned entry corresponding to the resource id
@@ -110,7 +111,7 @@ class TransactionTable {
    **/
   TransactionTableResult deleteOwnEntry(const transaction_id tid,
                                         const ResourceId &rid,
-                                        const AccessMode access_mode);
+                                        const AccessMode &access_mode);
 
   /**
    * @brief Deletes the pending entry corresponding to the resource id
@@ -124,7 +125,7 @@ class TransactionTable {
    **/
   TransactionTableResult deletePendingEntry(const transaction_id tid,
                                             const ResourceId &rid,
-                                            const AccessMode access_mode);
+                                            const AccessMode &access_mode);
 
   /**
    * @brief Returns a vector of resource ids which the corresponding transaction

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/tests/AccessMode_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/AccessMode_unittest.cpp b/transaction/tests/AccessMode_unittest.cpp
index fa51525..3287fb0 100644
--- a/transaction/tests/AccessMode_unittest.cpp
+++ b/transaction/tests/AccessMode_unittest.cpp
@@ -25,12 +25,12 @@ namespace transaction {
 class AccessModeTest : public ::testing::Test {
  protected:
   AccessModeTest()
-      : nl_mode_(AccessModeType::kNoLock),
-        is_mode_(AccessModeType::kIsLock),
-        ix_mode_(AccessModeType::kIxLock),
-        s_mode_(AccessModeType::kSLock),
-        six_mode_(AccessModeType::kSixLock),
-        x_mode_(AccessModeType::kXLock) {
+      : nl_mode_(AccessMode::NoLockMode()),
+        is_mode_(AccessMode::IsLockMode()),
+        ix_mode_(AccessMode::IxLockMode()),
+        s_mode_(AccessMode::SLockMode()),
+        six_mode_(AccessMode::SixLockMode()),
+        x_mode_(AccessMode::XLockMode()) {
   }
 
   const AccessMode nl_mode_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/tests/CycleDetector_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/CycleDetector_unittest.cpp b/transaction/tests/CycleDetector_unittest.cpp
new file mode 100644
index 0000000..6edaa63
--- /dev/null
+++ b/transaction/tests/CycleDetector_unittest.cpp
@@ -0,0 +1,157 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "transaction/CycleDetector.hpp"
+
+#include <cstdint>
+#include <memory>
+#include <stack>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/Transaction.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+namespace transaction {
+
+class CycleDetectorTest : public testing::Test {
+ protected:
+  const std::uint64_t kNumberOfTransactions = 12;
+
+  CycleDetectorTest()
+      : wait_for_graph_(std::make_unique<DirectedGraph>()) {
+  }
+
+  virtual void SetUp() {
+    std::vector<transaction_id> transactions(kNumberOfTransactions);
+    for (std::uint64_t i = 0; i < kNumberOfTransactions; ++i) {
+      transactions.push_back(transaction_id(i));
+    }
+
+    std::vector<DirectedGraph::node_id> node_ids;
+    for (std::uint64_t i = 0; i < kNumberOfTransactions; ++i) {
+      node_ids.push_back(wait_for_graph_->addNodeUnchecked(transactions[i]));
+    }
+  }
+
+  void initializeCycleDetector() {
+    for (const auto &edge : edges_) {
+      wait_for_graph_->addEdgeUnchecked(edge.first, edge.second);
+    }
+
+    cycle_detector_.reset(new CycleDetector(wait_for_graph_.get()));
+  }
+
+  void checkVictims(
+      const std::unordered_set<DirectedGraph::node_id> &expected_victims) {
+    const std::vector<DirectedGraph::node_id> victims =
+        cycle_detector_->chooseVictimsToBreakCycle();
+
+    std::unordered_set<DirectedGraph::node_id> remaining_nodes;
+
+    for (DirectedGraph::node_id node = 0; node < wait_for_graph_->getNumNodes();
+         ++node) {
+      if (std::find(victims.begin(), victims.end(), node) == victims.end()) {
+        // Node is not in victims, then insert it to remaining set.
+        remaining_nodes.insert(node);
+      }
+    }
+
+    for (const auto node : remaining_nodes) {
+      ASSERT_FALSE(isSelfReachableNode(node, remaining_nodes));
+    }
+  }
+
+  bool isSelfReachableNode(
+      const DirectedGraph::node_id start_node,
+      const std::unordered_set<DirectedGraph::node_id> &node_set) {
+    std::unordered_set<DirectedGraph::node_id> marked_nodes;
+    std::stack<DirectedGraph::node_id> to_be_visied_nodes;
+
+    const std::vector<DirectedGraph::node_id> neighbors_of_start_node =
+        wait_for_graph_->getAdjacentNodes(start_node);
+    for (const auto node : neighbors_of_start_node) {
+      marked_nodes.insert(node);
+      to_be_visied_nodes.push(node);
+    }
+
+    while (!to_be_visied_nodes.empty()) {
+      const DirectedGraph::node_id current_node = to_be_visied_nodes.top();
+      to_be_visied_nodes.pop();
+      if (current_node == start_node) {
+        return true;
+      }
+      if (node_set.count(current_node) == 1 &&
+          marked_nodes.count(current_node) == 0) {
+        // Means, we did not visited this node yet, and it is in the node set,
+        // so we should process it (mark it and push all of its neighbors
+        // into stack).
+        marked_nodes.insert(current_node);
+        const auto neighbors = wait_for_graph_->getAdjacentNodes(current_node);
+        for (const auto neighbor : neighbors) {
+          to_be_visied_nodes.push(neighbor);
+        }
+      }
+    }
+    return false;
+  }
+
+  std::vector<std::pair<DirectedGraph::node_id, DirectedGraph::node_id>> edges_;
+  std::unique_ptr<DirectedGraph> wait_for_graph_;
+  std::unique_ptr<CycleDetector> cycle_detector_;
+};
+
+TEST_F(CycleDetectorTest, Interleaving) {
+  edges_ = {{0, 1},
+            {1, 0}};
+
+  initializeCycleDetector();
+
+  std::unordered_set<DirectedGraph::node_id> expected_victims = {1};
+
+  checkVictims(expected_victims);
+}
+
+TEST_F(CycleDetectorTest, MultipleCycle) {
+  // This edge contains lots of cycles of degree 1, 2 and 3.
+  edges_ = {{0, 1},
+            {1, 2}, {1, 3}, {1, 4},
+            {2, 5},
+            {3, 4}, {3, 6},
+            {4, 1}, {4, 5}, {4, 6},
+            {5, 2}, {5, 7},
+            {6, 7}, {6, 9},
+            {7, 6},
+            {8, 6},
+            {9, 8}, {9, 10},
+            {10, 11},
+            {11, 9}};
+
+  initializeCycleDetector();
+
+  std::unordered_set<DirectedGraph::node_id> expected_victims
+      = {4, 5, 7, 8, 9, 10, 11};
+
+  checkVictims(expected_victims);
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/tests/DeadLockDetector_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/DeadLockDetector_unittest.cpp b/transaction/tests/DeadLockDetector_unittest.cpp
new file mode 100644
index 0000000..bc65ef5
--- /dev/null
+++ b/transaction/tests/DeadLockDetector_unittest.cpp
@@ -0,0 +1,96 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "transaction/DeadLockDetector.hpp"
+
+#include <atomic>
+#include <memory>
+#include <vector>
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/LockTable.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+namespace transaction {
+
+class DeadLockDetectorTest : public ::testing::Test {
+ protected:
+  DeadLockDetectorTest()
+      : lock_table_(std::make_unique<LockTable>()),
+        status_(DeadLockDetectorStatus::kDone) {
+  }
+
+  std::unique_ptr<LockTable> lock_table_;
+  std::atomic<DeadLockDetectorStatus> status_;
+  std::vector<DirectedGraph::node_id> victims_;
+};
+
+TEST_F(DeadLockDetectorTest, SimpleCycle) {
+  const transaction_id transaction_one(1), transaction_two(2);
+  const ResourceId resource_one(1, 2), resource_two(4, 5);
+
+  const AccessMode x_lock_mode(AccessMode::XLockMode());
+
+  // Produce a conflicting schedule.
+  // Transaction 1 will acquire X lock on resource 1.
+  lock_table_->putLock(transaction_one,
+                       resource_one,
+                       x_lock_mode);
+
+  // Transaction 2 will acquire X lock on resource 2.
+  lock_table_->putLock(transaction_two,
+                       resource_two,
+                       x_lock_mode);
+
+  // Transaction 1 will try to acquire X lock on resource 2,
+  // but it will fail since Transaction 2 has already acquired
+  // X lock on resource 2.
+  lock_table_->putLock(transaction_one,
+                       resource_two,
+                       x_lock_mode);
+
+  // Transaction 2 will try to acquire X lock on resource 1,
+  // but it will fail since Transaction 1 has already acquired
+  // X lock on resource 2.
+  lock_table_->putLock(transaction_two,
+                       resource_one,
+                       x_lock_mode);
+
+  // Run deadlock detector.
+  DeadLockDetector deadlock_detector(lock_table_.get(), &status_, &victims_);
+  status_.store(DeadLockDetectorStatus::kNotReady);
+
+  deadlock_detector.start();
+
+  // Signal deadlock detector.
+  while (status_.load() == DeadLockDetectorStatus::kNotReady) {
+  }
+
+  status_.store(DeadLockDetectorStatus::kQuit);
+  deadlock_detector.join();
+
+  // Victim size must be 1.
+  ASSERT_EQ(1u, victims_.size());
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/tests/DirectedGraph_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/DirectedGraph_unittest.cpp b/transaction/tests/DirectedGraph_unittest.cpp
index 43ad972..00fe276 100644
--- a/transaction/tests/DirectedGraph_unittest.cpp
+++ b/transaction/tests/DirectedGraph_unittest.cpp
@@ -26,101 +26,96 @@
 namespace quickstep {
 namespace transaction {
 
-TEST(DirectedGraphTest, AddNode) {
-  // Prepare the data, but do not include in the graph.
-  DirectedGraph wait_for_graph;
-  transaction_id *tid3 = new transaction_id(3);
-  transaction_id *tid4 = new transaction_id(4);
-  transaction_id *tid5 = new transaction_id(5);
-  transaction_id *tid6 = new transaction_id(6);
-
+class DirectedGraphTest : public ::testing::Test {
+ protected:
+  DirectedGraphTest()
+      : tid3_(3),
+        tid4_(4),
+        tid5_(5),
+        tid6_(6) {
+  }
+
+  DirectedGraph wait_for_graph_;
+  transaction_id tid3_;
+  transaction_id tid4_;
+  transaction_id tid5_;
+  transaction_id tid6_;
+};
+
+TEST_F(DirectedGraphTest, AddNode) {
   // The nodes are not added yet, total no of nodesshould be zero.
-  EXPECT_EQ(0u, wait_for_graph.getNumNodes());
+  EXPECT_EQ(0u, wait_for_graph_.getNumNodes());
 
-  wait_for_graph.addNodeUnchecked(tid3);
+  wait_for_graph_.addNodeUnchecked(tid3_);
 
   // One node is added.
-  EXPECT_EQ(1u, wait_for_graph.getNumNodes());
+  EXPECT_EQ(1u, wait_for_graph_.getNumNodes());
 
-  wait_for_graph.addNodeUnchecked(tid4);
+  wait_for_graph_.addNodeUnchecked(tid4_);
 
   // Another node is added.
-  EXPECT_EQ(2u, wait_for_graph.getNumNodes());
+  EXPECT_EQ(2u, wait_for_graph_.getNumNodes());
 
-  wait_for_graph.addNodeUnchecked(tid5);
-  wait_for_graph.addNodeUnchecked(tid6);
+  wait_for_graph_.addNodeUnchecked(tid5_);
+  wait_for_graph_.addNodeUnchecked(tid6_);
 
   // Total no of nodes should be 4 right now.
-  EXPECT_EQ(4u, wait_for_graph.getNumNodes());
+  EXPECT_EQ(4u, wait_for_graph_.getNumNodes());
 }
 
-TEST(DirectedGraphTest, AddEdge) {
-  // Prepare the graph.
-  DirectedGraph wait_for_graph;
-  transaction_id *tid3 = new transaction_id(3);
-  transaction_id *tid4 = new transaction_id(4);
-  transaction_id *tid5 = new transaction_id(5);
-  transaction_id *tid6 = new transaction_id(6);
-
-  DirectedGraph::node_id nid3 = wait_for_graph.addNodeUnchecked(tid3);
-  DirectedGraph::node_id nid6 = wait_for_graph.addNodeUnchecked(tid6);
-  DirectedGraph::node_id nid4 = wait_for_graph.addNodeUnchecked(tid4);
-  DirectedGraph::node_id nid5 = wait_for_graph.addNodeUnchecked(tid5);
+TEST_F(DirectedGraphTest, AddEdge) {
+  DirectedGraph::node_id nid3 = wait_for_graph_.addNodeUnchecked(tid3_);
+  DirectedGraph::node_id nid6 = wait_for_graph_.addNodeUnchecked(tid6_);
+  DirectedGraph::node_id nid4 = wait_for_graph_.addNodeUnchecked(tid4_);
+  DirectedGraph::node_id nid5 = wait_for_graph_.addNodeUnchecked(tid5_);
 
   // Add edges.
-  wait_for_graph.addEdgeUnchecked(nid3, nid5);
-  wait_for_graph.addEdgeUnchecked(nid6, nid4);
-  wait_for_graph.addEdgeUnchecked(nid3, nid6);
-  wait_for_graph.addEdgeUnchecked(nid4, nid6);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid5);
+  wait_for_graph_.addEdgeUnchecked(nid6, nid4);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid6);
+  wait_for_graph_.addEdgeUnchecked(nid4, nid6);
 
   // Check whether the edges are already there.
-  EXPECT_TRUE(wait_for_graph.hasEdge(nid3, nid5));
-  EXPECT_TRUE(wait_for_graph.hasEdge(nid6, nid4));
-  EXPECT_TRUE(wait_for_graph.hasEdge(nid4, nid6));
-  EXPECT_TRUE(wait_for_graph.hasEdge(nid3, nid6));
+  EXPECT_TRUE(wait_for_graph_.hasEdge(nid3, nid5));
+  EXPECT_TRUE(wait_for_graph_.hasEdge(nid6, nid4));
+  EXPECT_TRUE(wait_for_graph_.hasEdge(nid4, nid6));
+  EXPECT_TRUE(wait_for_graph_.hasEdge(nid3, nid6));
 
   // Check non-existent edges.
-  EXPECT_FALSE(wait_for_graph.hasEdge(nid5, nid3));
-  EXPECT_FALSE(wait_for_graph.hasEdge(nid6, nid3));
-  EXPECT_FALSE(wait_for_graph.hasEdge(nid4, nid5));
+  EXPECT_FALSE(wait_for_graph_.hasEdge(nid5, nid3));
+  EXPECT_FALSE(wait_for_graph_.hasEdge(nid6, nid3));
+  EXPECT_FALSE(wait_for_graph_.hasEdge(nid4, nid5));
 }
 
-TEST(DirectedGraphTest, GetAdjacentNodes) {
-  // Prepare the graph.
-  DirectedGraph wait_for_graph;
-  transaction_id *tid3 = new transaction_id(3);
-  transaction_id *tid4 = new transaction_id(4);
-  transaction_id *tid5 = new transaction_id(5);
-  transaction_id *tid6 = new transaction_id(6);
-
+TEST_F(DirectedGraphTest, GetAdjacentNodes) {
   // Add 4 disconnected nodes to the graph.
-  DirectedGraph::node_id nid3 = wait_for_graph.addNodeUnchecked(tid3);
-  DirectedGraph::node_id nid6 = wait_for_graph.addNodeUnchecked(tid6);
-  DirectedGraph::node_id nid4 = wait_for_graph.addNodeUnchecked(tid4);
-  DirectedGraph::node_id nid5 = wait_for_graph.addNodeUnchecked(tid5);
+  DirectedGraph::node_id nid3 = wait_for_graph_.addNodeUnchecked(tid3_);
+  DirectedGraph::node_id nid6 = wait_for_graph_.addNodeUnchecked(tid6_);
+  DirectedGraph::node_id nid4 = wait_for_graph_.addNodeUnchecked(tid4_);
+  DirectedGraph::node_id nid5 = wait_for_graph_.addNodeUnchecked(tid5_);
 
-  std::vector<DirectedGraph::node_id> result1 = wait_for_graph.getAdjacentNodes(nid3);
+  std::vector<DirectedGraph::node_id> result1 = wait_for_graph_.getAdjacentNodes(nid3);
   // nid3 has no edge to other nodes.
   EXPECT_EQ(0u, result1.size());
 
   // Now nid3 has outgoing edge to nid4 and nid5.
-  wait_for_graph.addEdgeUnchecked(nid3, nid4);
-  wait_for_graph.addEdgeUnchecked(nid3, nid5);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid4);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid5);
 
-  std::vector<DirectedGraph::node_id> result2 = wait_for_graph.getAdjacentNodes(nid3);
+  std::vector<DirectedGraph::node_id> result2 = wait_for_graph_.getAdjacentNodes(nid3);
   // Therefore, number of outgoing edges from nid3 is 2.
   EXPECT_EQ(2u, result2.size());
 
   // Add an edge from nid3 to nid6.
-  wait_for_graph.addEdgeUnchecked(nid3, nid6);
-  std::vector<DirectedGraph::node_id> result3 = wait_for_graph.getAdjacentNodes(nid3);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid6);
+  std::vector<DirectedGraph::node_id> result3 = wait_for_graph_.getAdjacentNodes(nid3);
 
   // Now there are 3 outgoing edges.
   EXPECT_EQ(3u, result3.size());
 
   // Again add edge from nid3 to nid6.
-  wait_for_graph.addEdgeUnchecked(nid3, nid6);
-  std::vector<DirectedGraph::node_id> result4 = wait_for_graph.getAdjacentNodes(nid3);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid6);
+  std::vector<DirectedGraph::node_id> result4 = wait_for_graph_.getAdjacentNodes(nid3);
   // Since we have already add same edge before, number of edges are still 3.
   EXPECT_EQ(3u, result4.size());
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/tests/LockRequest_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/LockRequest_unittest.cpp b/transaction/tests/LockRequest_unittest.cpp
index 0e4138a..77047d9 100644
--- a/transaction/tests/LockRequest_unittest.cpp
+++ b/transaction/tests/LockRequest_unittest.cpp
@@ -31,7 +31,7 @@ class LockRequestTest : public ::testing::Test {
   LockRequestTest()
       : lock_request_(transaction_id(3),
                       ResourceId(5),
-                      AccessMode(AccessModeType::kSLock),
+                      AccessMode::SLockMode(),
                       RequestType::kAcquireLock) {
   }
 
@@ -41,7 +41,7 @@ class LockRequestTest : public ::testing::Test {
 TEST_F(LockRequestTest, CheckGetters) {
   EXPECT_EQ(transaction_id(3), lock_request_.getTransactionId());
   EXPECT_EQ(ResourceId(5), lock_request_.getResourceId());
-  EXPECT_EQ(AccessMode(AccessModeType::kSLock), lock_request_.getAccessMode());
+  EXPECT_EQ(AccessMode::SLockMode(), lock_request_.getAccessMode());
   EXPECT_EQ(RequestType::kAcquireLock, lock_request_.getRequestType());
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/tests/LockTable_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/LockTable_unittest.cpp b/transaction/tests/LockTable_unittest.cpp
index 577cb79..1aed0b8 100644
--- a/transaction/tests/LockTable_unittest.cpp
+++ b/transaction/tests/LockTable_unittest.cpp
@@ -41,64 +41,73 @@ class LockTableTest : public ::testing::Test {
 };
 
 TEST_F(LockTableTest, CompatibleRequestsFromDifferentTransactions) {
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  const AccessMode is_lock_mode = AccessMode::IsLockMode();
+  const AccessMode s_lock_mode = AccessMode::SLockMode();
+
+  EXPECT_EQ(LockTableResult::kPlacedInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kPLACED_IN_OWNED);
+                                is_lock_mode));
 
   // Acquire the same lock mode on same resource.
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  EXPECT_EQ(LockTableResult::kAlreadyInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kALREADY_IN_OWNED);
+                                is_lock_mode));
 
   // Another transaction acquires compatible lock on the same resource.
-  EXPECT_EQ(lock_table_.putLock(tid_2_,
+  EXPECT_EQ(LockTableResult::kPlacedInOwned,
+            lock_table_.putLock(tid_2_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kSLock)),
-            LockTableResult::kPLACED_IN_OWNED);
+                                s_lock_mode));
 }
 
 TEST_F(LockTableTest, IncompatibleRequestsFromDifferentTransactions) {
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  const AccessMode is_lock_mode = AccessMode::IsLockMode();
+  const AccessMode x_lock_mode = AccessMode::XLockMode();
+
+  EXPECT_EQ(LockTableResult::kPlacedInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kPLACED_IN_OWNED);
+                                is_lock_mode));
 
   // Acquire the same lock mode on same resource.
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  EXPECT_EQ(LockTableResult::kAlreadyInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kALREADY_IN_OWNED);
+                                is_lock_mode));
 
   // Another transaction acquires incompatible lock on the same resource.
-  EXPECT_EQ(lock_table_.putLock(tid_2_,
+  EXPECT_EQ(LockTableResult::kPlacedInPending,
+            lock_table_.putLock(tid_2_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kXLock)),
-            LockTableResult::kPLACED_IN_PENDING);
+                                x_lock_mode));
 }
 
 TEST_F(LockTableTest, StarvationProtection) {
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  const AccessMode is_lock_mode = AccessMode::IsLockMode();
+  const AccessMode x_lock_mode = AccessMode::XLockMode();
+
+  EXPECT_EQ(LockTableResult::kPlacedInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kPLACED_IN_OWNED);
+                                is_lock_mode));
 
   // Another transaction requests incompatible lock on the same resource.
   // It should wait for the previous transaction.
-  EXPECT_EQ(lock_table_.putLock(tid_2_,
+  EXPECT_EQ(LockTableResult::kPlacedInPending,
+            lock_table_.putLock(tid_2_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kXLock)),
-            LockTableResult::kPLACED_IN_PENDING);
+                                x_lock_mode));
 
   // Another third transaction requests a compatible lock on the same resource.
   // Normally, it should acquire the lock, however, there is a pending
   // transaction waiting on the same resource. To prevent starvation, we should
   // put in the pending list.
-  EXPECT_EQ(lock_table_.putLock(tid_3_,
+  EXPECT_EQ(LockTableResult::kPlacedInPending,
+            lock_table_.putLock(tid_3_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kPLACED_IN_PENDING);
+                                is_lock_mode));
 }
 
 }  // namespace transaction

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/tests/Lock_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/Lock_unittest.cpp b/transaction/tests/Lock_unittest.cpp
index 2ab8b3e..59a5e7a 100644
--- a/transaction/tests/Lock_unittest.cpp
+++ b/transaction/tests/Lock_unittest.cpp
@@ -18,6 +18,7 @@
 #include "transaction/Lock.hpp"
 
 #include <cstddef>
+#include <memory>
 #include <vector>
 
 #include "transaction/AccessMode.hpp"
@@ -31,12 +32,12 @@ namespace transaction {
 class LockTest : public ::testing::Test {
  protected:
   LockTest()
-      : modes_({AccessMode(AccessModeType::kNoLock),
-                AccessMode(AccessModeType::kIsLock),
-                AccessMode(AccessModeType::kIxLock),
-                AccessMode(AccessModeType::kSLock),
-                AccessMode(AccessModeType::kSixLock),
-                AccessMode(AccessModeType::kXLock)}),
+      : modes_({AccessMode::NoLockMode(),
+                AccessMode::IsLockMode(),
+                AccessMode::IxLockMode(),
+                AccessMode::SLockMode(),
+                AccessMode::SixLockMode(),
+                AccessMode::XLockMode()}),
         resource_a_(3, 10, 2, 5),
         resource_b_(4, 5, 3, 2),
         locks_on_resource_a_({Lock(resource_a_, modes_[0]),



[24/30] incubator-quickstep git commit: Refactored HashJoinWorkOrder protos. (#203)

Posted by zu...@apache.org.
Refactored HashJoinWorkOrder protos. (#203)

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

Branch: refs/heads/master
Commit: 6f4dd8f2a349601e8dd562d31359b1478ddf5c22
Parents: 2abc988
Author: Zuyu ZHANG <zu...@users.noreply.github.com>
Authored: Fri Apr 29 21:11:23 2016 -0700
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 relational_operators/CMakeLists.txt       |   3 +
 relational_operators/HashJoinOperator.cpp |   5 +-
 relational_operators/WorkOrder.proto      |  98 +++----
 relational_operators/WorkOrderFactory.cpp | 362 +++++++++----------------
 4 files changed, 162 insertions(+), 306 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f4dd8f2/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 759a233..a4600e6 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -181,6 +181,9 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_storage_TupleReference
                       quickstep_storage_TupleStorageSubBlock
                       quickstep_storage_ValueAccessor
+                      quickstep_types_Type
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_utility_Macros
                       tmb)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f4dd8f2/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 82f6b2a..104a02d 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -42,6 +42,9 @@
 #include "storage/TupleReference.hpp"
 #include "storage/TupleStorageSubBlock.hpp"
 #include "storage/ValueAccessor.hpp"
+#include "types/Type.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 
 #include "gflags/gflags.h"
@@ -820,7 +823,7 @@ void HashOuterJoinWorkOrder::execute() {
         // where x is an attribute of the build relation.
         // In that case, this HashOuterJoinWorkOrder needs to be updated to
         // correctly handle the selections.
-        const Type& column_type = selection_[i]->getType().getNullableVersion();
+        const Type &column_type = selection_[i]->getType().getNullableVersion();
         if (NativeColumnVector::UsableForType(column_type)) {
           NativeColumnVector *result = new NativeColumnVector(
               column_type, num_tuples_without_matches);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f4dd8f2/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 8ed2080..5d0619a 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -29,21 +29,18 @@ enum WorkOrderType {
   DESTROY_HASH = 6;
   DROP_TABLE = 7;
   FINALIZE_AGGREGATION = 8;
-  HASH_ANTI_JOIN = 9;
-  HASH_INNER_JOIN = 10;
-  HASH_OUTER_JOIN = 11;
-  HASH_SEMI_JOIN = 12;
-  INSERT = 13;
-  NESTED_LOOP_JOIN = 14;
-  SAMPLE = 15;
-  SAVE_BLOCKS = 16;
-  SELECT = 17;
-  SORT_MERGE_RUN = 18;
-  SORT_RUN_GENERATION = 19;
-  TABLE_GENERATOR = 20;
-  TEXT_SCAN = 21;
-  TEXT_SPLIT = 22;
-  UPDATE = 23;
+  HASH_JOIN = 9;
+  INSERT = 10;
+  NESTED_LOOP_JOIN = 11;
+  SAMPLE = 12;
+  SAVE_BLOCKS = 13;
+  SELECT = 14;
+  SORT_MERGE_RUN = 15;
+  SORT_RUN_GENERATION = 16;
+  TABLE_GENERATOR = 17;
+  TEXT_SCAN = 18;
+  TEXT_SPLIT = 19;
+  UPDATE = 20;
 }
 
 message WorkOrder {
@@ -107,63 +104,30 @@ message FinalizeAggregationWorkOrder {
   }
 }
 
-message HashInnerJoinWorkOrder {
-  extend WorkOrder {
-    // All required.
-    optional int32 build_relation_id = 160;
-    optional int32 probe_relation_id = 161;
-    repeated int32 join_key_attributes = 162;
-    optional bool any_join_key_attributes_nullable = 163;
-    optional int32 insert_destination_index = 164;
-    optional uint32 join_hash_table_index = 165;
-    optional int32 residual_predicate_index = 166;
-    optional int32 selection_index = 167;
-    optional fixed64 block_id = 168;
-  }
-}
-
-message HashAntiJoinWorkOrder {
-  extend WorkOrder {
-    // All required.
-    optional int32 build_relation_id = 350;
-    optional int32 probe_relation_id = 351;
-    repeated int32 join_key_attributes = 352;
-    optional bool any_join_key_attributes_nullable = 353;
-    optional int32 insert_destination_index = 354;
-    optional uint32 join_hash_table_index = 355;
-    optional int32 residual_predicate_index = 356;
-    optional int32 selection_index = 357;
-    optional fixed64 block_id = 358;
+message HashJoinWorkOrder {
+  enum HashJoinWorkOrderType {
+    HASH_ANTI_JOIN = 0;
+    HASH_INNER_JOIN = 1;
+    HASH_OUTER_JOIN = 2;
+    HASH_SEMI_JOIN = 3;
   }
-}
 
-message HashSemiJoinWorkOrder {
   extend WorkOrder {
     // All required.
-    optional int32 build_relation_id = 360;
-    optional int32 probe_relation_id = 361;
-    repeated int32 join_key_attributes = 362;
-    optional bool any_join_key_attributes_nullable = 363;
-    optional int32 insert_destination_index = 364;
-    optional uint32 join_hash_table_index = 365;
-    optional int32 residual_predicate_index = 366;
-    optional int32 selection_index = 367;
-    optional fixed64 block_id = 368;
-  }
-}
+    optional HashJoinWorkOrderType hash_join_work_order_type = 160;
+    optional int32 build_relation_id = 161;
+    optional int32 probe_relation_id = 162;
+    repeated int32 join_key_attributes = 163;
+    optional bool any_join_key_attributes_nullable = 164;
+    optional int32 insert_destination_index = 165;
+    optional uint32 join_hash_table_index = 166;
+    optional int32 selection_index = 167;
+    optional fixed64 block_id = 168;
 
-message HashOuterJoinWorkOrder {
-  extend WorkOrder {
-    // All required.
-    optional int32 build_relation_id = 370;
-    optional int32 probe_relation_id = 371;
-    repeated int32 join_key_attributes = 372;
-    optional bool any_join_key_attributes_nullable = 373;
-    optional int32 insert_destination_index = 374;
-    optional uint32 join_hash_table_index = 375;
-    optional int32 selection_index = 376;
-    repeated bool is_selection_on_build = 377;
-    optional fixed64 block_id = 378;
+    // Used by all but HashOuterJoinWorkOrder.
+    optional int32 residual_predicate_index = 169;
+    // Used by HashOuterJoinWorkOrder only.
+    repeated bool is_selection_on_build = 170;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f4dd8f2/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 964c11c..4157d0f 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -16,6 +16,7 @@
 
 #include "relational_operators/WorkOrderFactory.hpp"
 
+#include <memory>
 #include <utility>
 #include <vector>
 
@@ -55,6 +56,10 @@ using std::vector;
 
 namespace quickstep {
 
+class InsertDestination;
+class Predicate;
+class Scalar;
+
 WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder &proto,
                                                   CatalogDatabaseLite *catalog_database,
                                                   QueryContext *query_context,
@@ -135,123 +140,115 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           query_context->getInsertDestination(
               proto.GetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index)));
     }
-    case serialization::HASH_ANTI_JOIN: {
-      LOG(INFO) << "Creating HashAntiJoinWorkOrder";
-      vector<attribute_id> join_key_attributes;
-      const int join_key_attributes_size =
-          proto.ExtensionSize(serialization::HashAntiJoinWorkOrder::join_key_attributes);
-      for (int i = 0; i < join_key_attributes_size; ++i) {
-        join_key_attributes.push_back(
-            proto.GetExtension(serialization::HashAntiJoinWorkOrder::join_key_attributes, i));
-      }
+    case serialization::HASH_JOIN: {
+      const auto hash_join_work_order_type =
+          proto.GetExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type);
 
-      return new HashAntiJoinWorkOrder(
+      const CatalogRelationSchema &build_relation =
           catalog_database->getRelationSchemaById(
-              proto.GetExtension(serialization::HashAntiJoinWorkOrder::build_relation_id)),
+              proto.GetExtension(serialization::HashJoinWorkOrder::build_relation_id));
+      const CatalogRelationSchema &probe_relation =
           catalog_database->getRelationSchemaById(
-              proto.GetExtension(serialization::HashAntiJoinWorkOrder::probe_relation_id)),
-          move(join_key_attributes),
-          proto.GetExtension(serialization::HashAntiJoinWorkOrder::any_join_key_attributes_nullable),
-          proto.GetExtension(serialization::HashAntiJoinWorkOrder::block_id),
-          query_context->getPredicate(
-              proto.GetExtension(serialization::HashAntiJoinWorkOrder::residual_predicate_index)),
-          query_context->getScalarGroup(
-              proto.GetExtension(serialization::HashAntiJoinWorkOrder::selection_index)),
-          *query_context->getJoinHashTable(
-              proto.GetExtension(serialization::HashAntiJoinWorkOrder::join_hash_table_index)),
-          query_context->getInsertDestination(
-              proto.GetExtension(serialization::HashAntiJoinWorkOrder::insert_destination_index)),
-          storage_manager);
-    }
-    case serialization::HASH_INNER_JOIN: {
-      LOG(INFO) << "Creating HashInnerJoinWorkOrder";
-      vector<attribute_id> join_key_attributes;
-      const int join_key_attributes_size =
-          proto.ExtensionSize(serialization::HashInnerJoinWorkOrder::join_key_attributes);
-      for (int i = 0; i < join_key_attributes_size; ++i) {
-        join_key_attributes.push_back(
-            proto.GetExtension(serialization::HashInnerJoinWorkOrder::join_key_attributes, i));
-      }
+              proto.GetExtension(serialization::HashJoinWorkOrder::probe_relation_id));
 
-      return new HashInnerJoinWorkOrder(
-          catalog_database->getRelationSchemaById(
-              proto.GetExtension(serialization::HashInnerJoinWorkOrder::build_relation_id)),
-          catalog_database->getRelationSchemaById(
-              proto.GetExtension(serialization::HashInnerJoinWorkOrder::probe_relation_id)),
-          move(join_key_attributes),
-          proto.GetExtension(serialization::HashInnerJoinWorkOrder::any_join_key_attributes_nullable),
-          proto.GetExtension(serialization::HashInnerJoinWorkOrder::block_id),
-          query_context->getPredicate(
-              proto.GetExtension(serialization::HashInnerJoinWorkOrder::residual_predicate_index)),
-          query_context->getScalarGroup(
-              proto.GetExtension(serialization::HashInnerJoinWorkOrder::selection_index)),
-          *query_context->getJoinHashTable(
-              proto.GetExtension(serialization::HashInnerJoinWorkOrder::join_hash_table_index)),
-          query_context->getInsertDestination(
-              proto.GetExtension(serialization::HashInnerJoinWorkOrder::insert_destination_index)),
-          storage_manager);
-    }
-    case serialization::HASH_OUTER_JOIN: {
-      LOG(INFO) << "Creating HashOuterJoinWorkOrder";
       vector<attribute_id> join_key_attributes;
       const int join_key_attributes_size =
-          proto.ExtensionSize(serialization::HashOuterJoinWorkOrder::join_key_attributes);
+          proto.ExtensionSize(serialization::HashJoinWorkOrder::join_key_attributes);
       for (int i = 0; i < join_key_attributes_size; ++i) {
         join_key_attributes.push_back(
-            proto.GetExtension(serialization::HashOuterJoinWorkOrder::join_key_attributes, i));
-      }
-      vector<bool> is_selection_on_build;
-      const int is_selection_on_build_size =
-          proto.ExtensionSize(serialization::HashOuterJoinWorkOrder::is_selection_on_build);
-      for (int i = 0; i < is_selection_on_build_size; ++i) {
-        is_selection_on_build.push_back(
-            proto.GetExtension(serialization::HashOuterJoinWorkOrder::is_selection_on_build, i));
+            proto.GetExtension(serialization::HashJoinWorkOrder::join_key_attributes, i));
       }
 
-      return new HashOuterJoinWorkOrder(
-          catalog_database->getRelationSchemaById(
-              proto.GetExtension(serialization::HashOuterJoinWorkOrder::build_relation_id)),
-          catalog_database->getRelationSchemaById(
-              proto.GetExtension(serialization::HashOuterJoinWorkOrder::probe_relation_id)),
-          move(join_key_attributes),
-          proto.GetExtension(serialization::HashOuterJoinWorkOrder::any_join_key_attributes_nullable),
-          proto.GetExtension(serialization::HashOuterJoinWorkOrder::block_id),
-          query_context->getScalarGroup(
-              proto.GetExtension(serialization::HashOuterJoinWorkOrder::selection_index)),
-          move(is_selection_on_build),
-          *query_context->getJoinHashTable(
-              proto.GetExtension(serialization::HashOuterJoinWorkOrder::join_hash_table_index)),
-          query_context->getInsertDestination(
-              proto.GetExtension(serialization::HashOuterJoinWorkOrder::insert_destination_index)),
-          storage_manager);
-    }
-    case serialization::HASH_SEMI_JOIN: {
-      LOG(INFO) << "Creating HashSemiJoinWorkOrder";
-      vector<attribute_id> join_key_attributes;
-      const int join_key_attributes_size =
-          proto.ExtensionSize(serialization::HashSemiJoinWorkOrder::join_key_attributes);
-      for (int i = 0; i < join_key_attributes_size; ++i) {
-        join_key_attributes.push_back(
-            proto.GetExtension(serialization::HashSemiJoinWorkOrder::join_key_attributes, i));
+      const bool any_join_key_attributes_nullable =
+          proto.GetExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable);
+      const block_id lookup_block_id =
+          proto.GetExtension(serialization::HashJoinWorkOrder::block_id);
+
+      const Predicate *residual_predicate = nullptr;
+      if (hash_join_work_order_type != serialization::HashJoinWorkOrder::HASH_OUTER_JOIN) {
+        residual_predicate =
+            query_context->getPredicate(
+                proto.GetExtension(serialization::HashJoinWorkOrder::residual_predicate_index));
       }
 
-      return new HashSemiJoinWorkOrder(
-          catalog_database->getRelationSchemaById(
-              proto.GetExtension(serialization::HashSemiJoinWorkOrder::build_relation_id)),
-          catalog_database->getRelationSchemaById(
-              proto.GetExtension(serialization::HashSemiJoinWorkOrder::probe_relation_id)),
-          move(join_key_attributes),
-          proto.GetExtension(serialization::HashSemiJoinWorkOrder::any_join_key_attributes_nullable),
-          proto.GetExtension(serialization::HashSemiJoinWorkOrder::block_id),
-          query_context->getPredicate(
-              proto.GetExtension(serialization::HashSemiJoinWorkOrder::residual_predicate_index)),
+      const std::vector<std::unique_ptr<const Scalar>> &selection =
           query_context->getScalarGroup(
-              proto.GetExtension(serialization::HashSemiJoinWorkOrder::selection_index)),
+              proto.GetExtension(serialization::HashJoinWorkOrder::selection_index));
+      const JoinHashTable &hash_table =
           *query_context->getJoinHashTable(
-              proto.GetExtension(serialization::HashSemiJoinWorkOrder::join_hash_table_index)),
+              proto.GetExtension(serialization::HashJoinWorkOrder::join_hash_table_index));
+      InsertDestination *output_destination =
           query_context->getInsertDestination(
-              proto.GetExtension(serialization::HashSemiJoinWorkOrder::insert_destination_index)),
-          storage_manager);
+              proto.GetExtension(serialization::HashJoinWorkOrder::insert_destination_index));
+
+      switch (hash_join_work_order_type) {
+        case serialization::HashJoinWorkOrder::HASH_ANTI_JOIN: {
+          LOG(INFO) << "Creating HashAntiJoinWorkOrder";
+          return new HashAntiJoinWorkOrder(
+              build_relation,
+              probe_relation,
+              move(join_key_attributes),
+              any_join_key_attributes_nullable,
+              lookup_block_id,
+              residual_predicate,
+              selection,
+              hash_table,
+              output_destination,
+              storage_manager);
+        }
+        case serialization::HashJoinWorkOrder::HASH_INNER_JOIN: {
+          LOG(INFO) << "Creating HashInnerJoinWorkOrder";
+          return new HashInnerJoinWorkOrder(
+              build_relation,
+              probe_relation,
+              move(join_key_attributes),
+              any_join_key_attributes_nullable,
+              lookup_block_id,
+              residual_predicate,
+              selection,
+              hash_table,
+              output_destination,
+              storage_manager);
+        }
+        case serialization::HashJoinWorkOrder::HASH_OUTER_JOIN: {
+          vector<bool> is_selection_on_build;
+          const int is_selection_on_build_size =
+              proto.ExtensionSize(serialization::HashJoinWorkOrder::is_selection_on_build);
+          for (int i = 0; i < is_selection_on_build_size; ++i) {
+            is_selection_on_build.push_back(
+                proto.GetExtension(serialization::HashJoinWorkOrder::is_selection_on_build, i));
+          }
+
+          LOG(INFO) << "Creating HashOuterJoinWorkOrder";
+          return new HashOuterJoinWorkOrder(
+              build_relation,
+              probe_relation,
+              move(join_key_attributes),
+              any_join_key_attributes_nullable,
+              lookup_block_id,
+              selection,
+              move(is_selection_on_build),
+              hash_table,
+              output_destination,
+              storage_manager);
+        }
+        case serialization::HashJoinWorkOrder::HASH_SEMI_JOIN: {
+          LOG(INFO) << "Creating HashSemiJoinWorkOrder";
+          return new HashSemiJoinWorkOrder(
+              build_relation,
+              probe_relation,
+              move(join_key_attributes),
+              any_join_key_attributes_nullable,
+              lookup_block_id,
+              residual_predicate,
+              selection,
+              hash_table,
+              output_destination,
+              storage_manager);
+        }
+        default:
+          LOG(FATAL) << "Unknown HashJoinWorkOrder Type in WorkOrderFactory::ReconstructFromProto";
+      }
     }
     case serialization::INSERT: {
       LOG(INFO) << "Creating InsertWorkOrder";
@@ -486,179 +483,68 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
              query_context.isValidInsertDestinationId(
                  proto.GetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index));
     }
-    case serialization::HASH_ANTI_JOIN: {
-      if (!proto.HasExtension(serialization::HashAntiJoinWorkOrder::build_relation_id) ||
-          !proto.HasExtension(serialization::HashAntiJoinWorkOrder::probe_relation_id)) {
+    case serialization::HASH_JOIN: {
+      if (!proto.HasExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type)) {
         return false;
       }
 
-      const relation_id build_relation_id =
-          proto.GetExtension(serialization::HashAntiJoinWorkOrder::build_relation_id);
-      if (!catalog_database.hasRelationWithId(build_relation_id)) {
+      const auto hash_join_work_order_type =
+          proto.GetExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type);
+      if (!serialization::HashJoinWorkOrder_HashJoinWorkOrderType_IsValid(hash_join_work_order_type)) {
         return false;
       }
 
-      const relation_id probe_relation_id =
-          proto.GetExtension(serialization::HashAntiJoinWorkOrder::probe_relation_id);
-      if (!catalog_database.hasRelationWithId(probe_relation_id)) {
-        return false;
-      }
-
-      const CatalogRelationSchema &build_relation = catalog_database.getRelationSchemaById(build_relation_id);
-      const CatalogRelationSchema &probe_relation = catalog_database.getRelationSchemaById(probe_relation_id);
-      for (int i = 0; i < proto.ExtensionSize(serialization::HashAntiJoinWorkOrder::join_key_attributes); ++i) {
-        const attribute_id attr_id =
-            proto.GetExtension(serialization::HashAntiJoinWorkOrder::join_key_attributes, i);
-        if (!build_relation.hasAttributeWithId(attr_id) ||
-            !probe_relation.hasAttributeWithId(attr_id)) {
-          return false;
-        }
-      }
-
-      return proto.HasExtension(serialization::HashAntiJoinWorkOrder::any_join_key_attributes_nullable) &&
-             proto.HasExtension(serialization::HashAntiJoinWorkOrder::insert_destination_index) &&
-             query_context.isValidInsertDestinationId(
-                 proto.GetExtension(serialization::HashAntiJoinWorkOrder::insert_destination_index)) &&
-             proto.HasExtension(serialization::HashAntiJoinWorkOrder::join_hash_table_index) &&
-             query_context.isValidJoinHashTableId(
-                 proto.GetExtension(serialization::HashAntiJoinWorkOrder::join_hash_table_index)) &&
-             proto.HasExtension(serialization::HashAntiJoinWorkOrder::residual_predicate_index) &&
-             query_context.isValidPredicate(
-                 proto.GetExtension(serialization::HashAntiJoinWorkOrder::residual_predicate_index)) &&
-             proto.HasExtension(serialization::HashAntiJoinWorkOrder::selection_index) &&
-             query_context.isValidScalarGroupId(
-                 proto.GetExtension(serialization::HashAntiJoinWorkOrder::selection_index)) &&
-             proto.HasExtension(serialization::HashAntiJoinWorkOrder::block_id);
-    }
-    case serialization::HASH_INNER_JOIN: {
-      if (!proto.HasExtension(serialization::HashInnerJoinWorkOrder::build_relation_id) ||
-          !proto.HasExtension(serialization::HashInnerJoinWorkOrder::probe_relation_id)) {
+      if (!proto.HasExtension(serialization::HashJoinWorkOrder::build_relation_id) ||
+          !proto.HasExtension(serialization::HashJoinWorkOrder::probe_relation_id)) {
         return false;
       }
 
       const relation_id build_relation_id =
-          proto.GetExtension(serialization::HashInnerJoinWorkOrder::build_relation_id);
+          proto.GetExtension(serialization::HashJoinWorkOrder::build_relation_id);
       if (!catalog_database.hasRelationWithId(build_relation_id)) {
         return false;
       }
 
       const relation_id probe_relation_id =
-          proto.GetExtension(serialization::HashInnerJoinWorkOrder::probe_relation_id);
+          proto.GetExtension(serialization::HashJoinWorkOrder::probe_relation_id);
       if (!catalog_database.hasRelationWithId(probe_relation_id)) {
         return false;
       }
 
       const CatalogRelationSchema &build_relation = catalog_database.getRelationSchemaById(build_relation_id);
       const CatalogRelationSchema &probe_relation = catalog_database.getRelationSchemaById(probe_relation_id);
-      for (int i = 0; i < proto.ExtensionSize(serialization::HashInnerJoinWorkOrder::join_key_attributes); ++i) {
+      for (int i = 0; i < proto.ExtensionSize(serialization::HashJoinWorkOrder::join_key_attributes); ++i) {
         const attribute_id attr_id =
-            proto.GetExtension(serialization::HashInnerJoinWorkOrder::join_key_attributes, i);
+            proto.GetExtension(serialization::HashJoinWorkOrder::join_key_attributes, i);
         if (!build_relation.hasAttributeWithId(attr_id) ||
             !probe_relation.hasAttributeWithId(attr_id)) {
           return false;
         }
       }
 
-      return proto.HasExtension(serialization::HashInnerJoinWorkOrder::any_join_key_attributes_nullable) &&
-             proto.HasExtension(serialization::HashInnerJoinWorkOrder::insert_destination_index) &&
-             query_context.isValidInsertDestinationId(
-                 proto.GetExtension(serialization::HashInnerJoinWorkOrder::insert_destination_index)) &&
-             proto.HasExtension(serialization::HashInnerJoinWorkOrder::join_hash_table_index) &&
-             query_context.isValidJoinHashTableId(
-                 proto.GetExtension(serialization::HashInnerJoinWorkOrder::join_hash_table_index)) &&
-             proto.HasExtension(serialization::HashInnerJoinWorkOrder::residual_predicate_index) &&
-             query_context.isValidPredicate(
-                 proto.GetExtension(serialization::HashInnerJoinWorkOrder::residual_predicate_index)) &&
-             proto.HasExtension(serialization::HashInnerJoinWorkOrder::selection_index) &&
-             query_context.isValidScalarGroupId(
-                 proto.GetExtension(serialization::HashInnerJoinWorkOrder::selection_index)) &&
-             proto.HasExtension(serialization::HashInnerJoinWorkOrder::block_id);
-    }
-    case serialization::HASH_OUTER_JOIN: {
-      if (!proto.HasExtension(serialization::HashOuterJoinWorkOrder::build_relation_id) ||
-          !proto.HasExtension(serialization::HashOuterJoinWorkOrder::probe_relation_id)) {
-        return false;
-      }
-
-      const relation_id build_relation_id =
-          proto.GetExtension(serialization::HashOuterJoinWorkOrder::build_relation_id);
-      if (!catalog_database.hasRelationWithId(build_relation_id)) {
-        return false;
-      }
-
-      const relation_id probe_relation_id =
-          proto.GetExtension(serialization::HashOuterJoinWorkOrder::probe_relation_id);
-      if (!catalog_database.hasRelationWithId(probe_relation_id)) {
-        return false;
-      }
-
-      const CatalogRelationSchema &build_relation = catalog_database.getRelationSchemaById(build_relation_id);
-      const CatalogRelationSchema &probe_relation = catalog_database.getRelationSchemaById(probe_relation_id);
-      for (int i = 0; i < proto.ExtensionSize(serialization::HashOuterJoinWorkOrder::join_key_attributes); ++i) {
-        const attribute_id attr_id =
-            proto.GetExtension(serialization::HashOuterJoinWorkOrder::join_key_attributes, i);
-        if (!build_relation.hasAttributeWithId(attr_id) ||
-            !probe_relation.hasAttributeWithId(attr_id)) {
+      if (hash_join_work_order_type == serialization::HashJoinWorkOrder::HASH_OUTER_JOIN) {
+        if (!proto.HasExtension(serialization::HashJoinWorkOrder::is_selection_on_build)) {
           return false;
         }
-      }
-
-      return proto.HasExtension(serialization::HashOuterJoinWorkOrder::any_join_key_attributes_nullable) &&
-             proto.HasExtension(serialization::HashOuterJoinWorkOrder::insert_destination_index) &&
-             query_context.isValidInsertDestinationId(
-                 proto.GetExtension(serialization::HashOuterJoinWorkOrder::insert_destination_index)) &&
-             proto.HasExtension(serialization::HashOuterJoinWorkOrder::join_hash_table_index) &&
-             query_context.isValidJoinHashTableId(
-                 proto.GetExtension(serialization::HashOuterJoinWorkOrder::join_hash_table_index)) &&
-             proto.HasExtension(serialization::HashOuterJoinWorkOrder::selection_index) &&
-             query_context.isValidScalarGroupId(
-                 proto.GetExtension(serialization::HashOuterJoinWorkOrder::selection_index)) &&
-             proto.HasExtension(serialization::HashOuterJoinWorkOrder::is_selection_on_build) &&
-             proto.HasExtension(serialization::HashOuterJoinWorkOrder::block_id);
-    }
-    case serialization::HASH_SEMI_JOIN: {
-      if (!proto.HasExtension(serialization::HashSemiJoinWorkOrder::build_relation_id) ||
-          !proto.HasExtension(serialization::HashSemiJoinWorkOrder::probe_relation_id)) {
-        return false;
-      }
-
-      const relation_id build_relation_id =
-          proto.GetExtension(serialization::HashSemiJoinWorkOrder::build_relation_id);
-      if (!catalog_database.hasRelationWithId(build_relation_id)) {
-        return false;
-      }
-
-      const relation_id probe_relation_id =
-          proto.GetExtension(serialization::HashSemiJoinWorkOrder::probe_relation_id);
-      if (!catalog_database.hasRelationWithId(probe_relation_id)) {
-        return false;
-      }
-
-      const CatalogRelationSchema &build_relation = catalog_database.getRelationSchemaById(build_relation_id);
-      const CatalogRelationSchema &probe_relation = catalog_database.getRelationSchemaById(probe_relation_id);
-      for (int i = 0; i < proto.ExtensionSize(serialization::HashSemiJoinWorkOrder::join_key_attributes); ++i) {
-        const attribute_id attr_id =
-            proto.GetExtension(serialization::HashSemiJoinWorkOrder::join_key_attributes, i);
-        if (!build_relation.hasAttributeWithId(attr_id) ||
-            !probe_relation.hasAttributeWithId(attr_id)) {
+      } else {
+        if (!proto.HasExtension(serialization::HashJoinWorkOrder::residual_predicate_index) ||
+            !query_context.isValidPredicate(
+                 proto.GetExtension(serialization::HashJoinWorkOrder::residual_predicate_index))) {
           return false;
         }
       }
 
-      return proto.HasExtension(serialization::HashSemiJoinWorkOrder::any_join_key_attributes_nullable) &&
-             proto.HasExtension(serialization::HashSemiJoinWorkOrder::insert_destination_index) &&
+      return proto.HasExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable) &&
+             proto.HasExtension(serialization::HashJoinWorkOrder::insert_destination_index) &&
              query_context.isValidInsertDestinationId(
-                 proto.GetExtension(serialization::HashSemiJoinWorkOrder::insert_destination_index)) &&
-             proto.HasExtension(serialization::HashSemiJoinWorkOrder::join_hash_table_index) &&
+                 proto.GetExtension(serialization::HashJoinWorkOrder::insert_destination_index)) &&
+             proto.HasExtension(serialization::HashJoinWorkOrder::join_hash_table_index) &&
              query_context.isValidJoinHashTableId(
-                 proto.GetExtension(serialization::HashSemiJoinWorkOrder::join_hash_table_index)) &&
-             proto.HasExtension(serialization::HashSemiJoinWorkOrder::residual_predicate_index) &&
-             query_context.isValidPredicate(
-                 proto.GetExtension(serialization::HashSemiJoinWorkOrder::residual_predicate_index)) &&
-             proto.HasExtension(serialization::HashSemiJoinWorkOrder::selection_index) &&
+                 proto.GetExtension(serialization::HashJoinWorkOrder::join_hash_table_index)) &&
+             proto.HasExtension(serialization::HashJoinWorkOrder::selection_index) &&
              query_context.isValidScalarGroupId(
-                 proto.GetExtension(serialization::HashSemiJoinWorkOrder::selection_index)) &&
-             proto.HasExtension(serialization::HashSemiJoinWorkOrder::block_id);
+                 proto.GetExtension(serialization::HashJoinWorkOrder::selection_index)) &&
+             proto.HasExtension(serialization::HashJoinWorkOrder::block_id);
     }
     case serialization::INSERT: {
       return proto.HasExtension(serialization::InsertWorkOrder::insert_destination_index) &&


[13/30] incubator-quickstep git commit: Refactored how min and max BLOCKSIZEMBs are calculated. (#191)

Posted by zu...@apache.org.
Refactored how min and max BLOCKSIZEMBs are calculated. (#191)

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

Branch: refs/heads/master
Commit: 4ec8d4d788d890314b5a29419565e75098c5d5cc
Parents: 4423550
Author: Marc S <cr...@users.noreply.github.com>
Authored: Tue Apr 26 12:35:27 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:34 2016 -0700

----------------------------------------------------------------------
 query_optimizer/resolver/Resolver.cpp             | 18 ++++++++++--------
 .../tests/logical_generator/Create.test           |  4 ++--
 query_optimizer/tests/resolver/Create.test        | 13 +++++++++++--
 storage/StorageConstants.hpp                      | 12 ++++++++++++
 4 files changed, 35 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ec8d4d7/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 8ddcf3f..1cf5c28 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -600,7 +600,7 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
     }
   }
   // Resolve the Block size (size -> # of slots).
-  std::int64_t slots = 1;  // The default.
+  std::int64_t slots = kDefaultBlockSizeInSlots;
   if (block_properties->hasBlockSizeMb()) {
     std::int64_t blocksizemb = block_properties->getBlockSizeMbValue();
     if (blocksizemb == -1) {
@@ -608,16 +608,18 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
       THROW_SQL_ERROR_AT(block_properties->getBlockSizeMb())
           << "The BLOCKSIZEMB property must be an integer.";
     }
-    slots = (blocksizemb * 1000000) / kSlotSizeBytes;
+    slots = (blocksizemb * kAMegaByte) / kSlotSizeBytes;
     DLOG(INFO) << "Resolver using BLOCKSIZEMB of " << slots << " slots"
         << " which is " << (slots * kSlotSizeBytes) << " bytes versus"
-        << " user requested " << (blocksizemb * 1000000) << " bytes.";
-    // 1Gb is the max size.
-    const std::int64_t max_size_slots = 1000000000 / kSlotSizeBytes;
-    // TODO(marc) The upper bound is arbitrary.
-    if (slots < 1 || slots > max_size_slots) {
+        << " user requested " << (blocksizemb * kAMegaByte) << " bytes.";
+    const std::uint64_t max_size_slots = kBlockSizeUpperBoundBytes / kSlotSizeBytes;
+    const std::uint64_t min_size_slots = kBlockSizeLowerBoundBytes / kSlotSizeBytes;
+    if (static_cast<std::uint64_t>(slots) < min_size_slots ||
+        static_cast<std::uint64_t>(slots) > max_size_slots) {
       THROW_SQL_ERROR_AT(block_properties->getBlockSizeMb())
-        << "The BLOCKSIZEMB property must be between 2Mb and 1000Mb.";
+          << "The BLOCKSIZEMB property must be between "
+          << std::to_string(kBlockSizeLowerBoundBytes / kAMegaByte) << "MB and "
+          << std::to_string(kBlockSizeUpperBoundBytes / kAMegaByte) << "MB.";
     }
   }
   storage_block_description->set_num_slots(slots);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ec8d4d7/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 6ceaa8f..eb99759 100644
--- a/query_optimizer/tests/logical_generator/Create.test
+++ b/query_optimizer/tests/logical_generator/Create.test
@@ -19,7 +19,7 @@ TopLevelPlan
 +-plan=CreateTable[relation=foo]
 | +-block_properties=ProtoDescription
 | | +-Property=ProtoProperty[Property=blocktype,Value=rowstore]
-| | +-Property=ProtoProperty[Property=slots,Value=4]
+| | +-Property=ProtoProperty[Property=slots,Value=5]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
 +-output_attributes=
@@ -36,7 +36,7 @@ TopLevelPlan
 | | +-Property=ProtoProperty[Property=sort,Value=0]
 | | +-Property=ProtoProperty[Property=compress,Value=0]
 | | +-Property=ProtoProperty[Property=compress,Value=1]
-| | +-Property=ProtoProperty[Property=slots,Value=4]
+| | +-Property=ProtoProperty[Property=slots,Value=5]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
 |   +-AttributeReference[id=1,name=attr2,relation=foo,type=Int]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ec8d4d7/query_optimizer/tests/resolver/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Create.test b/query_optimizer/tests/resolver/Create.test
index d4a1098..18beacd 100644
--- a/query_optimizer/tests/resolver/Create.test
+++ b/query_optimizer/tests/resolver/Create.test
@@ -202,10 +202,19 @@ ERROR: The BLOCKSIZEMB property must be an integer. (2 : 17)
                 ^
 ==
 
-# BLOCKSIZEMB must not be 0.
+# BLOCKSIZEMB must be greater than the minimum (defined in StorageConstants.hpp).
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
 (TYPE rowstore, BLOCKSIZEMB 0);
 --
-ERROR: The BLOCKSIZEMB property must be between 2Mb and 1000Mb. (2 : 17)
+ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 17)
 (TYPE rowstore, BLOCKSIZEMB 0);
                 ^
+==
+
+# BLOCKSIZEMB must be less than the maximum (defined in StorageConstants.hpp).
+CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
+(TYPE rowstore, BLOCKSIZEMB 2000);
+--
+ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 17)
+(TYPE rowstore, BLOCKSIZEMB 2000);
+                ^

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ec8d4d7/storage/StorageConstants.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageConstants.hpp b/storage/StorageConstants.hpp
index 43996c6..de54345 100644
--- a/storage/StorageConstants.hpp
+++ b/storage/StorageConstants.hpp
@@ -37,6 +37,18 @@ const std::size_t kSlotSizeBytes = 0x200000;
 
 // A GigaByte.
 const std::uint64_t kAGigaByte = (1 << 30);
+// A MegaByte.
+const std::uint64_t kAMegaByte = (1 << 20);
+
+// Constants for the minimum and maximum user-specifiable BLOCKSIZEMB in
+// the SQL clause BLOCKPROPERTIES.
+const std::uint64_t kBlockSizeUpperBoundBytes = kAGigaByte;
+
+// 2 Megabytes.
+const std::uint64_t kBlockSizeLowerBoundBytes = kAMegaByte << 1;
+
+// The default size of a new relation in terms of the number of slots.
+const std::uint64_t kDefaultBlockSizeInSlots = 1;
 
 // To determine the size of a buffer pool, we use a threshold (see below)
 // to check if the system has "large" amounts of installed memory. This


[23/30] incubator-quickstep git commit: Increased the size of the sharded lock manager to a large primary number (#198)

Posted by zu...@apache.org.
Increased the size of the sharded lock manager to a large primary number (#198)

Having small number of entries in the sharded lock manager introduces artificial conflict causing the buffer pool to grow. The size of an entry in the lock_manager_ is small, so don't be so stingy with the size of the lock manager. By default, we want to run well on large memory boxes.

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

Branch: refs/heads/master
Commit: ae3ca334f11258cc34148e402336683ea9e6aecc
Parents: 517bf5e
Author: Jignesh Patel <pa...@users.noreply.github.com>
Authored: Thu Apr 28 16:11:22 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 storage/StorageManager.hpp | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ae3ca334/storage/StorageManager.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.hpp b/storage/StorageManager.hpp
index 0b68b76..52326c2 100644
--- a/storage/StorageManager.hpp
+++ b/storage/StorageManager.hpp
@@ -479,7 +479,11 @@ class StorageManager {
   //   (2) If it is not safe to evict a block, then either that block's
   //       reference count is greater than 0 or a shared lock is held on the
   //       block's lock shard.
-  static constexpr std::size_t kLockManagerNumShards = 256;
+  // TODO(jmp): Would be good to set this more intelligently in the future
+  //            based on the hardware concurrency, the amount of main memory
+  //            and slot size. For now pick the largest prime that is less
+  //            than 8K.
+  static constexpr std::size_t kLockManagerNumShards = 0x2000-1;
   ShardedLockManager<block_id, kLockManagerNumShards, SpinSharedMutex<false>> lock_manager_;
 
   FRIEND_TEST(StorageManagerTest, DifferentNUMANodeBlobTestWithEviction);


[28/30] incubator-quickstep git commit: Transaction Part 4: LockManager, CycleDetector and DeadLockDetector. (#187)

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/tests/StronglyConnectedComponents_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/StronglyConnectedComponents_unittest.cpp b/transaction/tests/StronglyConnectedComponents_unittest.cpp
index 79d6881..35ef842 100644
--- a/transaction/tests/StronglyConnectedComponents_unittest.cpp
+++ b/transaction/tests/StronglyConnectedComponents_unittest.cpp
@@ -35,21 +35,20 @@ namespace transaction {
 class GraphConfiguration {
  public:
   GraphConfiguration(DirectedGraph *graph,
-                     std::size_t no_transaction,
+                     const std::size_t num_transactions,
                      const std::vector<std::pair<transaction_id,
                                                  transaction_id>> &mapping)
     : graph_(graph) {
-    for (std::size_t index = 0; index < no_transaction; ++index) {
-      std::unique_ptr<transaction_id> tid =
-        std::make_unique<transaction_id>(transaction_id(index));
-      transaction_list_.push_back(*tid);
-      DirectedGraph::node_id nid = graph->addNodeUnchecked(tid.release());
+    for (std::size_t index = 0; index < num_transactions; ++index) {
+      const transaction_id transaction = static_cast<transaction_id>(index);
+      transaction_list_.push_back(transaction);
+      const DirectedGraph::node_id nid = graph->addNodeUnchecked(transaction);
       node_id_list_.push_back(nid);
     }
 
     for (const std::pair<transaction_id, transaction_id> &edge : mapping) {
-      transaction_id pending = edge.first;
-      transaction_id owner = edge.second;
+      const transaction_id pending = edge.first;
+      const transaction_id owner = edge.second;
       graph_->addEdgeUnchecked(pending, owner);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0a02d00/transaction/tests/TransactionTable_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/TransactionTable_unittest.cpp b/transaction/tests/TransactionTable_unittest.cpp
index f5b5bc9..cd47a2b 100644
--- a/transaction/tests/TransactionTable_unittest.cpp
+++ b/transaction/tests/TransactionTable_unittest.cpp
@@ -41,91 +41,101 @@ class TransactionTableTest : public ::testing::Test {
 };
 
 TEST_F(TransactionTableTest, NormalOperations) {
-  EXPECT_EQ(transaction_table_.putOwnEntry(tid_1_,
+  const AccessMode is_lock_mode = AccessMode::IsLockMode();
+  const AccessMode x_lock_mode = AccessMode::XLockMode();
+
+  EXPECT_EQ(TransactionTableResult::kPlacedInOwned,
+            transaction_table_.putOwnEntry(tid_1_,
                                            ResourceId(3),
-                                           AccessMode(AccessModeType::kIsLock)),
-            TransactionTableResult::kPlacedInOwned);
+                                           is_lock_mode));
+
 
-  EXPECT_EQ(transaction_table_.putPendingEntry(tid_1_,
+  EXPECT_EQ(TransactionTableResult::kPlacedInPending,
+            transaction_table_.putPendingEntry(tid_1_,
                                                ResourceId(5),
-                                               AccessMode(AccessModeType::kXLock)),
-            TransactionTableResult::kPlacedInPending);
+                                               x_lock_mode));
 }
 
 TEST_F(TransactionTableTest, DeleteEntryOperations) {
-  EXPECT_EQ(transaction_table_.deleteOwnEntry(tid_2_,
+  const AccessMode s_lock_mode = AccessMode::SLockMode();
+  const AccessMode x_lock_mode = AccessMode::XLockMode();
+
+  EXPECT_EQ(TransactionTableResult::kDelError,
+            transaction_table_.deleteOwnEntry(tid_2_,
                                               ResourceId(5),
-                                              AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kDelError);
+                                              s_lock_mode));
 
-  EXPECT_EQ(transaction_table_.putOwnEntry(tid_2_,
+  EXPECT_EQ(TransactionTableResult::kPlacedInOwned,
+            transaction_table_.putOwnEntry(tid_2_,
                                            ResourceId(5),
-                                           AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kPlacedInOwned);
+                                           s_lock_mode));
 
   // Tring to delete a lock with different acces mode on same resource id
   // will result in an error.
-  EXPECT_EQ(transaction_table_.deleteOwnEntry(tid_2_,
+  EXPECT_EQ(TransactionTableResult::kDelError,
+            transaction_table_.deleteOwnEntry(tid_2_,
                                               ResourceId(5),
-                                              AccessMode(AccessModeType::kXLock)),
-            TransactionTableResult::kDelError);
+                                              x_lock_mode));
 
   // Transaction 3 does not have a lock on this resource id.
-  EXPECT_EQ(transaction_table_.deleteOwnEntry(tid_3_,
+  EXPECT_EQ(TransactionTableResult::kDelError,
+            transaction_table_.deleteOwnEntry(tid_3_,
                                               ResourceId(5),
-                                              AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kDelError);
+                                              s_lock_mode));
 
   // This will result in success since transaction 2 have acquired the lock on
   // this resource with the corresponding mode.
-  EXPECT_EQ(transaction_table_.deleteOwnEntry(tid_2_,
+  EXPECT_EQ(TransactionTableResult::kDelFromOwned,
+            transaction_table_.deleteOwnEntry(tid_2_,
                                               ResourceId(5),
-                                              AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kDelFromOwned);
+                                              s_lock_mode));
 
   // Repeat the previous sequence, with pending list.
-  EXPECT_EQ(transaction_table_.deletePendingEntry(tid_2_,
+  EXPECT_EQ(TransactionTableResult::kDelError,
+            transaction_table_.deletePendingEntry(tid_2_,
                                                   ResourceId(5),
-                                                  AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kDelError);
+                                                  s_lock_mode));
 
-  EXPECT_EQ(transaction_table_.putPendingEntry(tid_2_,
+  EXPECT_EQ(TransactionTableResult::kPlacedInPending,
+            transaction_table_.putPendingEntry(tid_2_,
                                                ResourceId(5),
-                                               AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kPlacedInPending);
+                                               s_lock_mode));
 
-  EXPECT_EQ(transaction_table_.deletePendingEntry(tid_2_,
+
+  EXPECT_EQ(TransactionTableResult::kDelError,
+            transaction_table_.deletePendingEntry(tid_2_,
                                                   ResourceId(5),
-                                                  AccessMode(AccessModeType::kXLock)),
-            TransactionTableResult::kDelError);
+                                                  x_lock_mode));
 
-  EXPECT_EQ(transaction_table_.deletePendingEntry(tid_3_,
+  EXPECT_EQ(TransactionTableResult::kDelError,
+            transaction_table_.deletePendingEntry(tid_3_,
                                                   ResourceId(5),
-                                                  AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kDelError);
+                                                  s_lock_mode));
 
-  EXPECT_EQ(transaction_table_.deletePendingEntry(tid_2_,
+  EXPECT_EQ(TransactionTableResult::kDelFromPending,
+            transaction_table_.deletePendingEntry(tid_2_,
                                                   ResourceId(5),
-                                                  AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kDelFromPending);
+                                                  s_lock_mode));
 }
 
 TEST_F(TransactionTableTest, TransactionEntries) {
-  EXPECT_EQ(transaction_table_.deleteTransaction(tid_1_),
-            TransactionTableResult::kTransactionDeleteError);
+  const AccessMode s_lock_mode = AccessMode::SLockMode();
+
+  EXPECT_EQ(TransactionTableResult::kTransactionDeleteError,
+            transaction_table_.deleteTransaction(tid_1_));
 
-  EXPECT_EQ(transaction_table_.putOwnEntry(tid_1_,
+  EXPECT_EQ(TransactionTableResult::kPlacedInOwned,
+            transaction_table_.putOwnEntry(tid_1_,
                                            ResourceId(4),
-                                           AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kPlacedInOwned);
+                                           s_lock_mode));
 
-  EXPECT_EQ(transaction_table_.deleteTransaction(tid_1_),
-            TransactionTableResult::kTransactionDeleteOk);
+  EXPECT_EQ(TransactionTableResult::kTransactionDeleteOk,
+            transaction_table_.deleteTransaction(tid_1_));
 
-  EXPECT_EQ(transaction_table_.deleteOwnEntry(tid_1_,
+  EXPECT_EQ(TransactionTableResult::kDelError,
+            transaction_table_.deleteOwnEntry(tid_1_,
                                               ResourceId(4),
-                                              AccessMode(AccessModeType::kSLock)),
-            TransactionTableResult::kDelError);
+                                              s_lock_mode));
 }
 
 }  // namespace transaction


[03/30] incubator-quickstep git commit: Fixed IWYU include paths. (#194)

Posted by zu...@apache.org.
Fixed IWYU include paths. (#194)

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

Branch: refs/heads/master
Commit: 9372b943fa037979542d92f1e547b7f8fd2c6818
Parents: a64bc43
Author: Zuyu ZHANG <zu...@users.noreply.github.com>
Authored: Wed Apr 27 03:23:43 2016 -0700
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:34 2016 -0700

----------------------------------------------------------------------
 third_party/iwyu/iwyu_helper.py | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9372b943/third_party/iwyu/iwyu_helper.py
----------------------------------------------------------------------
diff --git a/third_party/iwyu/iwyu_helper.py b/third_party/iwyu/iwyu_helper.py
index dff1837..a204c50 100755
--- a/third_party/iwyu/iwyu_helper.py
+++ b/third_party/iwyu/iwyu_helper.py
@@ -21,9 +21,10 @@ QUICKSTEP_INCLUDES = [ '.',
                        './build/third_party',
                        './build/third_party/protobuf/include',
                        './build/third_party/gflags/include',
-                       './third_party/gtest/include',
-                       './third_party/glog/src',
                        './third_party/benchmark/include',
+                       './third_party/glog/src',
+                       './third_party/googletest/googletest/include',
+                       './third_party/re2',
                        './third_party/tmb/include']
 QUICKSTEP_DEFINES = [ '-DQUICKSTEP_DEBUG',
                       '-DQUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION', ]


[16/30] incubator-quickstep git commit: Fixes bug which disallows the creation of BITWEAVING_H indices. (#204)

Posted by zu...@apache.org.
Fixes bug which disallows the creation of BITWEAVING_H indices. (#204)

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

Branch: refs/heads/master
Commit: 9d64761e6e9aad0c691237ae0bf66ffc91b3879d
Parents: 52b758f
Author: Marc S <cr...@users.noreply.github.com>
Authored: Sun May 1 17:14:20 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 parser/ParseIndexProperties.hpp |  2 +-
 parser/tests/Index.test         | 11 +++++++++++
 2 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9d64761e/parser/ParseIndexProperties.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseIndexProperties.hpp b/parser/ParseIndexProperties.hpp
index 9ca1419..2cb5df9 100644
--- a/parser/ParseIndexProperties.hpp
+++ b/parser/ParseIndexProperties.hpp
@@ -220,7 +220,7 @@ class BitWeavingIndexProperties : public IndexProperties {
       }
       const std::string key = ToLower(key_value.key()->value());
       const std::string value = ToLower(
-          static_cast<const ParseKeyStringValue&>(key_value).key()->value());
+          static_cast<const ParseKeyStringValue&>(key_value).value()->value());
       if (key.compare(kBitWeavingType) == 0) {
         if (value.compare("h") == 0) {
           index_sub_block_description_->set_sub_block_type(IndexSubBlockDescription::BITWEAVING_H);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9d64761e/parser/tests/Index.test
----------------------------------------------------------------------
diff --git a/parser/tests/Index.test b/parser/tests/Index.test
index 605768b..39cd523 100644
--- a/parser/tests/Index.test
+++ b/parser/tests/Index.test
@@ -66,3 +66,14 @@ CREATE INDEX bwIndex ON test(int_col) USING bitweaving;
 CreateIndexStatement[index_name=bwIndex,relation_name=test,index_type=bitweaving]
 +-attribute_list=
   +-AttributeReference[attribute_name=int_col]
+==
+CREATE INDEX bwhIndex ON test(int_col) USING bitweaving (TYPE H);
+--
+CreateIndexStatement[index_name=bwhIndex,relation_name=test,
+  index_type=bitweaving]
++-attribute_list=
+| +-AttributeReference[attribute_name=int_col]
++-index_property_list=
+  +-IndexProperties
+    +-index_property=KeyStringValue[key=TYPE]
+      +-value=String[value=H]


[20/30] incubator-quickstep git commit: Fixes a bug where numa settings were not properly set (not compiled in) (#205)

Posted by zu...@apache.org.
Fixes a bug where numa settings were not properly set (not compiled in) (#205)

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

Branch: refs/heads/master
Commit: db47654f5d4e93082c6118ef45ea54eb71972c9d
Parents: 9d64761
Author: Marc S <cr...@users.noreply.github.com>
Authored: Tue May 3 12:06:38 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 cli/DefaultsConfigurator.hpp | 16 ++++++++++++++++
 cli/InputParserUtil.cpp      |  1 +
 cli/QuickstepCli.cpp         |  6 +++---
 3 files changed, 20 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db47654f/cli/DefaultsConfigurator.hpp
----------------------------------------------------------------------
diff --git a/cli/DefaultsConfigurator.hpp b/cli/DefaultsConfigurator.hpp
index b40ef87..4da05b2 100644
--- a/cli/DefaultsConfigurator.hpp
+++ b/cli/DefaultsConfigurator.hpp
@@ -22,6 +22,7 @@
 #include <unordered_map>
 #include <vector>
 
+#include "storage/StorageConfig.h"
 #include "utility/Macros.hpp"
 
 #ifdef QUICKSTEP_HAVE_LIBNUMA
@@ -51,6 +52,21 @@ class DefaultsConfigurator {
   }
 
   /**
+   * @brief Get the number of available numa sockets.
+   *
+   * @return Number of available numa sockets. Always 1 if the system doesn't
+   *         have libnuma.
+   **/
+  static std::size_t GetNumNUMANodes() {
+  #ifdef QUICKSTEP_HAVE_LIBNUMA
+    // Id of the maximum node.
+    return numa_max_node() + 1;
+  #else
+    return 1;
+  #endif
+  }
+
+  /**
    * @brief Get the number of NUMA nodes covered by the given worker affinities
    *        to the CPU cores.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db47654f/cli/InputParserUtil.cpp
----------------------------------------------------------------------
diff --git a/cli/InputParserUtil.cpp b/cli/InputParserUtil.cpp
index 328aaeb..352883e 100644
--- a/cli/InputParserUtil.cpp
+++ b/cli/InputParserUtil.cpp
@@ -24,6 +24,7 @@
 #include <vector>
 
 #include "catalog/CatalogConfig.h"
+#include "storage/StorageConfig.h"
 #include "utility/StringUtil.hpp"
 
 #include "glog/logging.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db47654f/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 8dee1f7..ec195f7 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -258,8 +258,7 @@ int main(int argc, char* argv[]) {
       InputParserUtil::ParseWorkerAffinities(real_num_workers,
                                              quickstep::FLAGS_worker_affinities);
 
-  const std::size_t num_numa_nodes_covered =
-      DefaultsConfigurator::GetNumNUMANodesCoveredByWorkers(worker_cpu_affinities);
+  const std::size_t num_numa_nodes_system = DefaultsConfigurator::GetNumNUMANodes();
 
   if (quickstep::FLAGS_preload_buffer_pool) {
     std::chrono::time_point<std::chrono::steady_clock> preload_start, preload_end;
@@ -280,7 +279,8 @@ int main(int argc, char* argv[]) {
   Foreman foreman(&bus,
                   query_processor->getDefaultDatabase(),
                   query_processor->getStorageManager(),
-                  num_numa_nodes_covered);
+                  -1, /* CPU id to bind foreman. -1 is unbound. */
+                  num_numa_nodes_system);
 
   // Get the NUMA affinities for workers.
   vector<int> cpu_numa_nodes = InputParserUtil::GetNUMANodesForCPUs();


[14/30] incubator-quickstep git commit: Fixed deadlocks when loading a block or blob while evicting another. (#183)

Posted by zu...@apache.org.
Fixed deadlocks when loading a block or blob while evicting another. (#183)

* Revert "Storage manager concurrency bug"

* Avoid deadlocks when loading a block or blob.


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

Branch: refs/heads/master
Commit: 44235508ef89ff3bbd29a8eeec9dca586534b721
Parents: 43b9260
Author: Zuyu ZHANG <zu...@users.noreply.github.com>
Authored: Tue Apr 26 04:36:30 2016 -0700
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:34 2016 -0700

----------------------------------------------------------------------
 .../tests/execution_generator/CMakeLists.txt    |   2 +-
 storage/CMakeLists.txt                          |   1 -
 storage/FileManagerHdfs.cpp                     |   2 +-
 storage/FileManagerPosix.cpp                    |   4 +-
 storage/FileManagerWindows.cpp                  |   2 +-
 storage/StorageManager.cpp                      | 102 ++++++++++---------
 storage/StorageManager.hpp                      |  33 +-----
 storage/tests/StorageManager_unittest.cpp       |  42 ++++++++
 utility/CMakeLists.txt                          |   1 +
 utility/ShardedLockManager.hpp                  |  56 ++++++++--
 10 files changed, 154 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/query_optimizer/tests/execution_generator/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/CMakeLists.txt b/query_optimizer/tests/execution_generator/CMakeLists.txt
index 149721c..56bae16 100644
--- a/query_optimizer/tests/execution_generator/CMakeLists.txt
+++ b/query_optimizer/tests/execution_generator/CMakeLists.txt
@@ -81,4 +81,4 @@ file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Join)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Select)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/StringPatternMatching)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/TableGenerator)
-file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Update)
+file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Update)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 11c2819..ed23802 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -599,7 +599,6 @@ if (QUICKSTEP_HAVE_FILE_MANAGER_POSIX)
   target_link_libraries(quickstep_storage_FileManagerLocal
                         quickstep_storage_FileManagerPosix)
   target_link_libraries(quickstep_storage_FileManagerPosix
-                        glog
                         quickstep_storage_FileManager
                         quickstep_storage_StorageBlockInfo
                         quickstep_storage_StorageConstants

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/storage/FileManagerHdfs.cpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerHdfs.cpp b/storage/FileManagerHdfs.cpp
index 7f30c59..5f9706e 100644
--- a/storage/FileManagerHdfs.cpp
+++ b/storage/FileManagerHdfs.cpp
@@ -140,7 +140,7 @@ size_t FileManagerHdfs::numSlots(const block_id block) const {
   hdfsFreeFileInfo(file_info, 1);
 
   if ((file_size % kSlotSizeBytes) != 0) {
-    LOG(FATAL) << "The file " << filename << " was corrupted.";
+    throw CorruptPersistentStorage();
   }
 
   return file_size / kSlotSizeBytes;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/storage/FileManagerPosix.cpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerPosix.cpp b/storage/FileManagerPosix.cpp
index 27257e5..3bfb69d 100644
--- a/storage/FileManagerPosix.cpp
+++ b/storage/FileManagerPosix.cpp
@@ -38,8 +38,6 @@
 #include "utility/Macros.hpp"
 #include "utility/StringUtil.hpp"
 
-#include "glog/logging.h"
-
 using std::size_t;
 using std::sscanf;
 using std::strerror;
@@ -88,7 +86,7 @@ size_t FileManagerPosix::numSlots(const block_id block) const {
   }
 
   if ((file_stat.st_size % kSlotSizeBytes) != 0) {
-    LOG(FATAL) << "The file " << filename << " was corrupted.";
+    throw CorruptPersistentStorage();
   }
 
   return static_cast<size_t>(file_stat.st_size) / kSlotSizeBytes;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/storage/FileManagerWindows.cpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerWindows.cpp b/storage/FileManagerWindows.cpp
index cfa8819..9e3d4c8 100644
--- a/storage/FileManagerWindows.cpp
+++ b/storage/FileManagerWindows.cpp
@@ -106,7 +106,7 @@ size_t FileManagerWindows::numSlots(const block_id block) const {
   uint64_t file_size = (static_cast<uint64_t>(file_stat.nFileSizeHigh) << 32) | file_stat.nFileSizeLow;
 
   if ((file_size % kSlotSizeBytes) != 0) {
-    LOG(FATAL) << "The file " << filename << " was corrupted.";
+    throw CorruptPersistentStorage();
   }
 
   return static_cast<size_t>(file_size / kSlotSizeBytes);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/storage/StorageManager.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.cpp b/storage/StorageManager.cpp
index 1cdbcb6..a3f265d 100644
--- a/storage/StorageManager.cpp
+++ b/storage/StorageManager.cpp
@@ -214,7 +214,7 @@ block_id StorageManager::createBlock(const CatalogRelationSchema &relation,
 
     // Because block IDs are generated by atomically incrementing block_index_,
     // there should never be collisions.
-    DCHECK(blocks_.find(new_block_id) == blocks_.end());
+    DEBUG_ASSERT(blocks_.find(new_block_id) == blocks_.end());
     blocks_[new_block_id] = new_block_handle;
   }
 
@@ -241,7 +241,7 @@ block_id StorageManager::createBlob(const std::size_t num_slots,
 
     // Because block IDs are generated by atomically incrementing block_index_,
     // there should never be collisions.
-    DCHECK(blocks_.find(new_block_id) == blocks_.end());
+    DEBUG_ASSERT(blocks_.find(new_block_id) == blocks_.end());
     blocks_[new_block_id] = new_block_handle;
   }
 
@@ -287,12 +287,6 @@ StorageBlob* StorageManager::loadBlob(const block_id blob,
 }
 
 bool StorageManager::saveBlockOrBlob(const block_id block, const bool force) {
-  SpinSharedMutexSharedLock<false> read_lock(*lock_manager_.get(block));
-  return saveBlockOrBlobInternal(block, force);
-}
-
-
-bool StorageManager::saveBlockOrBlobInternal(const block_id block, const bool force) {
   // TODO(chasseur): This lock is held for the entire duration of this call
   // (including I/O), but really we only need to prevent the eviction of the
   // particular entry in 'blocks_' for the specified 'block'. If and when we
@@ -358,10 +352,10 @@ void StorageManager::deleteBlockOrBlobFile(const block_id block) {
 block_id StorageManager::allocateNewBlockOrBlob(const std::size_t num_slots,
                                                 BlockHandle *handle,
                                                 const int numa_node) {
-  DCHECK_GT(num_slots, 0u);
+  DEBUG_ASSERT(num_slots > 0);
   DEBUG_ASSERT(handle != nullptr);
 
-  handle->block_memory = allocateSlots(num_slots, numa_node, kInvalidBlockId);
+  handle->block_memory = allocateSlots(num_slots, numa_node);
   handle->block_memory_size = num_slots;
 
   return ++block_index_;
@@ -373,8 +367,8 @@ StorageManager::BlockHandle StorageManager::loadBlockOrBlob(
   // mutex in the lock manager. The caller has ensured that the block is not
   // already loaded before this function gets called.
   size_t num_slots = file_manager_->numSlots(block);
-  DCHECK_NE(num_slots, 0u);
-  void* block_buffer = allocateSlots(num_slots, numa_node, block);
+  DEBUG_ASSERT(num_slots != 0);
+  void *block_buffer = allocateSlots(num_slots, numa_node);
 
   const bool status = file_manager_->readBlockOrBlob(block, block_buffer, kSlotSizeBytes * num_slots);
   CHECK(status) << "Failed to read block from persistent storage: " << block;
@@ -389,13 +383,12 @@ StorageManager::BlockHandle StorageManager::loadBlockOrBlob(
 void StorageManager::insertBlockHandleAfterLoad(const block_id block,
                                                 const BlockHandle &handle) {
   SpinSharedMutexExclusiveLock<false> lock(blocks_shared_mutex_);
-  DCHECK(blocks_.find(block) == blocks_.end());
+  DEBUG_ASSERT(blocks_.find(block) == blocks_.end());
   blocks_[block] = handle;
 }
 
 void* StorageManager::allocateSlots(const std::size_t num_slots,
-                                    const int numa_node,
-                                    const block_id locked_block_id) {
+                                    const int numa_node) {
 #if defined(QUICKSTEP_HAVE_MMAP_LINUX_HUGETLB)
   static constexpr int kLargePageMmapFlags
       = MAP_PRIVATE | MAP_ANONYMOUS | MAP_HUGETLB;
@@ -404,6 +397,7 @@ void* StorageManager::allocateSlots(const std::size_t num_slots,
       = MAP_PRIVATE | MAP_ANONYMOUS | MAP_ALIGNED_SUPER;
 #endif
 
+  makeRoomForBlock(num_slots);
   void *slots = nullptr;
 
 #if defined(QUICKSTEP_HAVE_MMAP_LINUX_HUGETLB) || defined(QUICKSTEP_HAVE_MMAP_BSD_SUPERPAGE)
@@ -450,7 +444,7 @@ void* StorageManager::allocateSlots(const std::size_t num_slots,
 
 #if defined(QUICKSTEP_HAVE_LIBNUMA)
   if (numa_node != -1) {
-    DCHECK(numa_node < numa_num_configured_nodes());
+    DEBUG_ASSERT(numa_node < numa_num_configured_nodes());
     struct bitmask *numa_node_bitmask = numa_allocate_nodemask();
     // numa_node can be 0 through n-1, where n is the num of NUMA nodes.
     numa_bitmask_setbit(numa_node_bitmask, numa_node);
@@ -487,23 +481,18 @@ MutableBlockReference StorageManager::getBlockInternal(
     const block_id block,
     const CatalogRelationSchema &relation,
     const int numa_node) {
-  std::size_t num_slots = 0u;
   MutableBlockReference ret;
   {
-    // First, see if the block is in the buffer pool. If it is, we can return
-    // a reference to it immediately.
     SpinSharedMutexSharedLock<false> eviction_lock(*lock_manager_.get(block));
     SpinSharedMutexSharedLock<false> read_lock(blocks_shared_mutex_);
     std::unordered_map<block_id, BlockHandle>::iterator it = blocks_.find(block);
     if (it != blocks_.end()) {
-      DCHECK(!it->second.block->isBlob());
+      DEBUG_ASSERT(!it->second.block->isBlob());
       ret = MutableBlockReference(static_cast<StorageBlock*>(it->second.block), eviction_policy_.get());
-    } else {
-      // The block was not loaded. Taking advantage of the shared lock on the
-      // buffer pool, retrieve the size of the block's file.
-      num_slots = file_manager_->numSlots(block);
     }
   }
+  // To be safe, release the block's shard after 'eviction_lock' destructs.
+  lock_manager_.release(block);
 
   // Note that there is no way for the block to be evicted between the call to
   // loadBlock and the call to EvictionPolicy::blockReferenced from
@@ -517,33 +506,16 @@ MutableBlockReference StorageManager::getBlockInternal(
       SpinSharedMutexSharedLock<false> read_lock(blocks_shared_mutex_);
       std::unordered_map<block_id, BlockHandle>::iterator it = blocks_.find(block);
       if (it != blocks_.end()) {
-        DCHECK(!it->second.block->isBlob());
+        DEBUG_ASSERT(!it->second.block->isBlob());
         ret = MutableBlockReference(static_cast<StorageBlock*>(it->second.block), eviction_policy_.get());
         return ret;
       }
     }
-
-    // Call a best-effort method to evict blocks until the size of our buffer
-    // pool falls below the current buffer pool size plus the size of the
-    // block we are going to retrieve.
-    makeRoomForBlock(num_slots);
-
     // No other thread loaded the block before us.
-    // But going forward be careful as there is a potential self-deadlock
-    // situation here -- we are holding an Exclusive lock (io_lock)
-    //   and getting ready to go into the call chain
-    //   "MutableBlockReference"/"loadBlock" -> "loadBlockOrBlob"
-    //       -> "allocateSlots" -> "makeRoomForBlock"
-    //   In "makeRoomForBlock," we will acquire an exclusive lock via the call
-    //   "eviction_lock(*lock_manager_.get(block_index))"
-    //   This situation could lead to a self-deadlock as block_index could
-    //   hash to the same position in the "ShardedLockManager" as "block."
-    //   To deal with this case, we pass the block information for "block"
-    //   though the call chain, and check for a collision in the the
-    //   "ShardedLockManager" in the function "makeRoomForBlock."
-    //   If a collision is detected we avoid a self-deadlock.
     ret = MutableBlockReference(loadBlock(block, relation, numa_node), eviction_policy_.get());
   }
+  // To be safe, release the block's shard after 'io_lock' destructs.
+  lock_manager_.release(block);
 
   return ret;
 }
@@ -556,10 +528,12 @@ MutableBlobReference StorageManager::getBlobInternal(const block_id blob,
     SpinSharedMutexSharedLock<false> read_lock(blocks_shared_mutex_);
     std::unordered_map<block_id, BlockHandle>::iterator it = blocks_.find(blob);
     if (it != blocks_.end()) {
-      DCHECK(it->second.block->isBlob());
+      DEBUG_ASSERT(it->second.block->isBlob());
       ret = MutableBlobReference(static_cast<StorageBlob*>(it->second.block), eviction_policy_.get());
     }
   }
+  // To be safe, release the blob's shard after 'eviction_lock' destructs.
+  lock_manager_.release(blob);
 
   if (!ret.valid()) {
     SpinSharedMutexExclusiveLock<false> io_lock(*lock_manager_.get(blob));
@@ -572,7 +546,7 @@ MutableBlobReference StorageManager::getBlobInternal(const block_id blob,
       SpinSharedMutexSharedLock<false> read_lock(blocks_shared_mutex_);
       std::unordered_map<block_id, BlockHandle>::iterator it = blocks_.find(blob);
       if (it != blocks_.end()) {
-        DCHECK(it->second.block->isBlob());
+        DEBUG_ASSERT(it->second.block->isBlob());
         ret = MutableBlobReference(static_cast<StorageBlob*>(it->second.block), eviction_policy_.get());
         return ret;
       }
@@ -580,6 +554,8 @@ MutableBlobReference StorageManager::getBlobInternal(const block_id blob,
     // No other thread loaded the blob before us.
     ret = MutableBlobReference(loadBlob(blob, numa_node), eviction_policy_.get());
   }
+  // To be safe, release the blob's shard after 'io_lock' destructs.
+  lock_manager_.release(blob);
 
   return ret;
 }
@@ -590,23 +566,53 @@ void StorageManager::makeRoomForBlock(const size_t slots) {
     EvictionPolicy::Status status = eviction_policy_->chooseBlockToEvict(&block_index);
 
     if (status == EvictionPolicy::Status::kOk) {
-      SpinSharedMutexExclusiveLock<false> eviction_lock(*lock_manager_.get(block_index));
+      bool has_collision = false;
+      SpinSharedMutexExclusiveLock<false> eviction_lock(*lock_manager_.get(block_index, &has_collision));
+      if (has_collision) {
+        // We have a collision in the shared lock manager, where some callers
+        // of this function (i.e., getBlockInternal or getBlobInternal) has
+        // acquired an exclusive lock, and we are trying to evict a block that
+        // hashes to the same location. This will cause a deadlock.
+
+        // For now simply treat this situation as the case where there is not
+        // enough memory and we temporarily go over the memory limit.
+        break;
+      }
+
       StorageBlockBase* block;
       {
         SpinSharedMutexSharedLock<false> read_lock(blocks_shared_mutex_);
         if (blocks_.find(block_index) == blocks_.end()) {
           // another thread must have jumped in and evicted it before us
+
+          // NOTE(zuyu): It is ok to release the shard for a block or blob,
+          // before 'eviction_lock' destructs, because we will never encounter a
+          // self-deadlock in a single thread, and in multiple-thread case some
+          // thread will block but not deadlock if there is a shard collision.
+          lock_manager_.release(block_index);
           continue;
         }
         block = blocks_[block_index].block;
       }
       if (eviction_policy_->getRefCount(block->getID()) > 0) {
         // Someone sneaked in and referenced the block before we could evict it.
+
+        // NOTE(zuyu): It is ok to release the shard for a block or blob, before
+        // before 'eviction_lock' destructs, because we will never encounter a
+        // self-deadlock in a single thread, and in multiple-thread case some
+        // thread will block but not deadlock if there is a shard collision.
+        lock_manager_.release(block_index);
         continue;
       }
-      if (saveBlockOrBlobInternal(block->getID(), false)) {
+      if (saveBlockOrBlob(block->getID())) {
         evictBlockOrBlob(block->getID());
       }  // else : Someone sneaked in and evicted the block before we could.
+
+      // NOTE(zuyu): It is ok to release the shard for a block or blob, before
+      // before 'eviction_lock' destructs, because we will never encounter a
+      // self-deadlock in a single thread, and in multiple-thread case some
+      // thread will block but not deadlock if there is a shard collision.
+      lock_manager_.release(block_index);
     } else {
       // If status was not ok, then we must not have been able to evict enough
       // blocks; therefore, we return anyway, temporarily going over the memory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/storage/StorageManager.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.hpp b/storage/StorageManager.hpp
index dd67177..dab33f6 100644
--- a/storage/StorageManager.hpp
+++ b/storage/StorageManager.hpp
@@ -84,7 +84,7 @@ class StorageManager {
    *        storage.
    * @param max_memory_usage The maximum amount of memory that the storage
    *                         manager should use for cached blocks in slots. If
-   *                         a block is requested that is not currently in
+   *                         an block is requested that is not currently in
    *                         memory and there are already max_memory_usage slots
    *                         in use in memory, then the storage manager will
    *                         attempt to evict enough blocks to make room for the
@@ -224,7 +224,6 @@ class StorageManager {
 
   /**
    * @brief Save a block or blob in memory to the persistent storage.
-   * @details Obtains a read lock on the shard containing the saved block.
    *
    * @param block The id of the block or blob to save.
    * @param force Force the block to the persistent storage, even if it is not
@@ -358,20 +357,8 @@ class StorageManager {
   // Allocate a buffer to hold the specified number of slots. The memory
   // returned will be zeroed-out, and mapped using large pages if the system
   // supports it.
-  // Note if the last parameter "locked_block_id" is set to something other than
-  // "kInvalidBlockId," then it means that the caller has acquired
-  // a lock in the sharded lock manager for that block. Thus, if a block needs
-  // to be evicted by the EvictionPolicy in the "makeRoomForBlock" call, and
-  // if the block to be evicted happens to hash to the same entry in the
-  // sharded lock manager, then the Eviction policy needs to pick a different
-  // block for eviction.
-  // The key point is that if "locked_block_id" is not "kInvalidBlockId," then
-  // the caller of allocateSlots, e.g. loadBlock, will have acquired a lock
-  // in the sharded lock manager for the block "locked_block_id."
   void* allocateSlots(const std::size_t num_slots,
-                      const int numa_node,
-                      // const block_id locked_block_id = kInvalidBlockId);
-                      const block_id locked_block_id);
+                      const int numa_node);
 
   // Deallocate a buffer allocated by allocateSlots(). This must be used
   // instead of free(), because the underlying implementation of
@@ -380,20 +367,6 @@ class StorageManager {
                        const std::size_t num_slots);
 
   /**
-   * @brief Save a block or blob in memory to the persistent storage.
-   * 
-   * @param block The id of the block or blob to save.
-   * @param force Force the block to the persistent storage, even if it is not
-   *        dirty (by default, only actually write dirty blocks to the
-   *        persistent storage).
-   * 
-   * @return False if the block is not found in the memory. True if the block is
-   *         successfully saved to the persistent storage OR the block is clean
-   *         and force is false.
-   */
-  bool saveBlockOrBlobInternal(const block_id block, const bool force);
-
-  /**
    * @brief Evict a block or blob from memory.
    * @note The block is NOT automatically saved, so call saveBlock() first if
    *       necessary.
@@ -434,7 +407,7 @@ class StorageManager {
    *
    * @param slots Number of slots to make room for.
    */
-  void makeRoomForBlock(const size_t slots);
+  void makeRoomForBlock(const std::size_t slots);
 
   /**
    * @brief Load a block from the persistent storage into memory.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/storage/tests/StorageManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/StorageManager_unittest.cpp b/storage/tests/StorageManager_unittest.cpp
index 60537a9..4c252a1 100644
--- a/storage/tests/StorageManager_unittest.cpp
+++ b/storage/tests/StorageManager_unittest.cpp
@@ -205,4 +205,46 @@ TEST(StorageManagerTest, DifferentNUMANodeBlobTestWithEviction) {
 }
 #endif  // QUICKSTEP_HAVE_LIBNUMA
 
+// Trigger an eviction from the same shard in StorageManager's
+// ShardedLockManager while attempting to load a blob. Previously, a bug
+// existed that caused a self-deadlock in such situations. This test reproduces
+// the issue and validates the fix.
+TEST(StorageManagerTest, EvictFromSameShardTest) {
+  // Set up a StorageManager with a soft memory limit of only one slot.
+  StorageManager storage_manager("eviction_test_storage", 1);
+
+  // Create a blob.
+  const block_id blob_a_id = storage_manager.createBlob(1);
+
+  // Blob "a" is now memory-resident in StorageManager, but has a reference
+  // count of zero.
+  EXPECT_TRUE(storage_manager.blockOrBlobIsLoaded(blob_a_id));
+  EXPECT_EQ(kSlotSizeBytes, storage_manager.getMemorySize());
+
+  // Manually alter 'block_index_' inside 'storage_manager' so that the next
+  // block_id generated will be in the same shard as 'blob_id_a'.
+  storage_manager.block_index_.fetch_add(StorageManager::kLockManagerNumShards - 1);
+
+  // Create another blob and verify that it is in the same shard.
+  const block_id blob_b_id = storage_manager.createBlob(1);
+  EXPECT_EQ(storage_manager.lock_manager_.get(blob_a_id),
+            storage_manager.lock_manager_.get(blob_b_id));
+
+  // Creating a second blob should have triggered an eviction that kicked
+  // blob A out.
+  EXPECT_FALSE(storage_manager.blockOrBlobIsLoaded(blob_a_id));
+  EXPECT_TRUE(storage_manager.blockOrBlobIsLoaded(blob_b_id));
+  EXPECT_EQ(kSlotSizeBytes, storage_manager.getMemorySize());
+
+  // Try and get a reference to blob A. Blob A must be reloaded from disk.
+  // This will trigger an eviction of blob B. This is the point where the
+  // self-deadlock bug could be observed.
+  BlobReference blob_a_ref = storage_manager.getBlob(blob_a_id);
+
+  // Reaching this point means we have not self-deadlocked. Now clean up.
+  blob_a_ref.release();
+  storage_manager.deleteBlockOrBlobFile(blob_a_id);
+  storage_manager.deleteBlockOrBlobFile(blob_b_id);
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 30f01ef..4ff9254 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -243,6 +243,7 @@ target_link_libraries(quickstep_utility_SortConfiguration_proto
                       quickstep_expressions_Expressions_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_utility_ShardedLockManager
+                      quickstep_storage_StorageConstants
                       quickstep_threading_SharedMutex
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_StringUtil

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/44235508/utility/ShardedLockManager.hpp
----------------------------------------------------------------------
diff --git a/utility/ShardedLockManager.hpp b/utility/ShardedLockManager.hpp
index 0248882..1d59acb 100644
--- a/utility/ShardedLockManager.hpp
+++ b/utility/ShardedLockManager.hpp
@@ -1,6 +1,6 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2015-2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -22,6 +22,7 @@
 #include <cstddef>
 #include <functional>
 
+#include "storage/StorageConstants.hpp"
 #include "threading/SharedMutex.hpp"
 #include "utility/Macros.hpp"
 
@@ -51,16 +52,59 @@ class ShardedLockManager {
 
   /**
    * @brief Get the SharedMutex corresponding to the provided key.
-   * @param  key The key to map to a SharedMutex.
-   * @return     The corresponding SharedMutex.
+   * @param key The key to map to a SharedMutex.
+   * @param has_collision Whether accessing the given key would result in a
+   *        hash collision. Used in StorageManager::makeRoomForBlock only.
+   * @return The corresponding SharedMutex if there is no collision; otherwise,
+   *         the collision SharedMutex.
    */
-  SharedMutexT *get(const T key) {
-    return &shards[hash_(key) % N];
+  SharedMutexT* get(const T key, bool *has_collision = nullptr) {
+    const std::size_t shard = hash_(key) % N;
+
+    if (has_collision != nullptr) {
+      // In StorageManager::makeRoomForBlock, check whether the evicting block
+      // or blob has a shard collision with existing referenced shards.
+      SpinSharedMutexSharedLock<false> read_lock(shards_mutex_);
+      if (shards_.find(shard) != shards_.end()) {
+        *has_collision = true;
+        return &collision_mutex_;
+      }
+    }
+
+    {
+      SpinSharedMutexExclusiveLock<false> write_lock(shards_mutex_);
+
+      // Check one more time for the evicting block or blob if there is a shard
+      // collision.
+      if (has_collision != nullptr && shards_.find(shard) != shards_.end()) {
+        *has_collision = true;
+        return &collision_mutex_;
+      }
+
+      shards_.insert(shard);
+    }
+    return &sharded_mutexes_[shard];
+  }
+
+  /**
+   * @brief Release the shard corresponding to the provided key.
+   * @param key The key to compute the shard.
+   */
+  void release(const T key) {
+    SpinSharedMutexExclusiveLock<false> write_lock(shards_mutex_);
+    shards_.erase(hash_(key) % N);
   }
 
  private:
   std::hash<T> hash_;
-  std::array<SharedMutexT, N> shards;
+  std::array<SharedMutexT, N> sharded_mutexes_;
+
+  // The placeholder mutex used whenever there is a hash collision.
+  SharedMutexT collision_mutex_;
+
+  // Bookkeep all shards referenced by StorageManager in multiple threads.
+  std::unordered_set<std::size_t> shards_;
+  alignas(kCacheLineBytes) mutable SpinSharedMutex<false> shards_mutex_;
 
   DISALLOW_COPY_AND_ASSIGN(ShardedLockManager);
 };


[15/30] incubator-quickstep git commit: Fixed the lock problem with AggregationHandleSum::mergeStates (#189)

Posted by zu...@apache.org.
Fixed the lock problem with AggregationHandleSum::mergeStates (#189)

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

Branch: refs/heads/master
Commit: 43b926056b53e9e2ab49aec4e4399dab9c37cd19
Parents: ff3293b
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Apr 25 15:18:28 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:34 2016 -0700

----------------------------------------------------------------------
 expressions/aggregation/AggregationHandleSum.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/43b92605/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 410fecb..14421d2 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -144,7 +144,7 @@ void AggregationHandleSum::mergeStates(
   const AggregationStateSum &sum_source = static_cast<const AggregationStateSum&>(source);
   AggregationStateSum *sum_destination = static_cast<AggregationStateSum*>(destination);
 
-  SpinMutexLock(sum_destination->mutex_);
+  SpinMutexLock lock(sum_destination->mutex_);
   sum_destination->sum_ = merge_operator_->applyToTypedValues(sum_destination->sum_,
                                                               sum_source.sum_);
   sum_destination->null_ = sum_destination->null_ && sum_source.null_;


[10/30] incubator-quickstep git commit: Merge pull request #170 from pivotalsoftware/quickstep_date_support

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index b884861..72fa9ef 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -87,78 +87,84 @@ extern int quickstep_yydebug;
     TOKEN_CREATE = 297,
     TOKEN_DATE = 298,
     TOKEN_DATETIME = 299,
-    TOKEN_DECIMAL = 300,
-    TOKEN_DEFAULT = 301,
-    TOKEN_DELETE = 302,
-    TOKEN_DELIMITER = 303,
-    TOKEN_DESC = 304,
-    TOKEN_DISTINCT = 305,
-    TOKEN_DOUBLE = 306,
-    TOKEN_DROP = 307,
-    TOKEN_ELSE = 308,
-    TOKEN_END = 309,
-    TOKEN_ESCAPE_STRINGS = 310,
-    TOKEN_EXISTS = 311,
-    TOKEN_EXTRACT = 312,
-    TOKEN_FALSE = 313,
-    TOKEN_FIRST = 314,
-    TOKEN_FLOAT = 315,
-    TOKEN_FOREIGN = 316,
-    TOKEN_FROM = 317,
-    TOKEN_FULL = 318,
-    TOKEN_GROUP = 319,
-    TOKEN_HASH = 320,
-    TOKEN_HAVING = 321,
-    TOKEN_IN = 322,
-    TOKEN_INDEX = 323,
-    TOKEN_INNER = 324,
-    TOKEN_INSERT = 325,
-    TOKEN_INTEGER = 326,
-    TOKEN_INTERVAL = 327,
-    TOKEN_INTO = 328,
-    TOKEN_JOIN = 329,
-    TOKEN_KEY = 330,
-    TOKEN_LAST = 331,
-    TOKEN_LEFT = 332,
-    TOKEN_LIMIT = 333,
-    TOKEN_LONG = 334,
-    TOKEN_NULL = 335,
-    TOKEN_NULLS = 336,
-    TOKEN_OFF = 337,
-    TOKEN_ON = 338,
-    TOKEN_ORDER = 339,
-    TOKEN_OUTER = 340,
-    TOKEN_PARTITION = 341,
-    TOKEN_PARTITIONS = 342,
-    TOKEN_PERCENT = 343,
-    TOKEN_PRIMARY = 344,
-    TOKEN_QUIT = 345,
-    TOKEN_RANGE = 346,
-    TOKEN_REAL = 347,
-    TOKEN_REFERENCES = 348,
-    TOKEN_RIGHT = 349,
-    TOKEN_ROW_DELIMITER = 350,
-    TOKEN_SELECT = 351,
-    TOKEN_SET = 352,
-    TOKEN_SMA = 353,
-    TOKEN_SMALLINT = 354,
-    TOKEN_TABLE = 355,
-    TOKEN_THEN = 356,
-    TOKEN_TIME = 357,
-    TOKEN_TIMESTAMP = 358,
-    TOKEN_TRUE = 359,
-    TOKEN_TUPLESAMPLE = 360,
-    TOKEN_UNIQUE = 361,
-    TOKEN_UPDATE = 362,
-    TOKEN_USING = 363,
-    TOKEN_VALUES = 364,
-    TOKEN_VARCHAR = 365,
-    TOKEN_WHEN = 366,
-    TOKEN_WHERE = 367,
-    TOKEN_WITH = 368,
-    TOKEN_YEARMONTH = 369,
-    TOKEN_EOF = 370,
-    TOKEN_LEX_ERROR = 371
+    TOKEN_DAY = 300,
+    TOKEN_DECIMAL = 301,
+    TOKEN_DEFAULT = 302,
+    TOKEN_DELETE = 303,
+    TOKEN_DELIMITER = 304,
+    TOKEN_DESC = 305,
+    TOKEN_DISTINCT = 306,
+    TOKEN_DOUBLE = 307,
+    TOKEN_DROP = 308,
+    TOKEN_ELSE = 309,
+    TOKEN_END = 310,
+    TOKEN_ESCAPE_STRINGS = 311,
+    TOKEN_EXISTS = 312,
+    TOKEN_EXTRACT = 313,
+    TOKEN_FALSE = 314,
+    TOKEN_FIRST = 315,
+    TOKEN_FLOAT = 316,
+    TOKEN_FOREIGN = 317,
+    TOKEN_FROM = 318,
+    TOKEN_FULL = 319,
+    TOKEN_GROUP = 320,
+    TOKEN_HASH = 321,
+    TOKEN_HAVING = 322,
+    TOKEN_HOUR = 323,
+    TOKEN_IN = 324,
+    TOKEN_INDEX = 325,
+    TOKEN_INNER = 326,
+    TOKEN_INSERT = 327,
+    TOKEN_INTEGER = 328,
+    TOKEN_INTERVAL = 329,
+    TOKEN_INTO = 330,
+    TOKEN_JOIN = 331,
+    TOKEN_KEY = 332,
+    TOKEN_LAST = 333,
+    TOKEN_LEFT = 334,
+    TOKEN_LIMIT = 335,
+    TOKEN_LONG = 336,
+    TOKEN_MINUTE = 337,
+    TOKEN_MONTH = 338,
+    TOKEN_NULL = 339,
+    TOKEN_NULLS = 340,
+    TOKEN_OFF = 341,
+    TOKEN_ON = 342,
+    TOKEN_ORDER = 343,
+    TOKEN_OUTER = 344,
+    TOKEN_PARTITION = 345,
+    TOKEN_PARTITIONS = 346,
+    TOKEN_PERCENT = 347,
+    TOKEN_PRIMARY = 348,
+    TOKEN_QUIT = 349,
+    TOKEN_RANGE = 350,
+    TOKEN_REAL = 351,
+    TOKEN_REFERENCES = 352,
+    TOKEN_RIGHT = 353,
+    TOKEN_ROW_DELIMITER = 354,
+    TOKEN_SECOND = 355,
+    TOKEN_SELECT = 356,
+    TOKEN_SET = 357,
+    TOKEN_SMA = 358,
+    TOKEN_SMALLINT = 359,
+    TOKEN_TABLE = 360,
+    TOKEN_THEN = 361,
+    TOKEN_TIME = 362,
+    TOKEN_TIMESTAMP = 363,
+    TOKEN_TRUE = 364,
+    TOKEN_TUPLESAMPLE = 365,
+    TOKEN_UNIQUE = 366,
+    TOKEN_UPDATE = 367,
+    TOKEN_USING = 368,
+    TOKEN_VALUES = 369,
+    TOKEN_VARCHAR = 370,
+    TOKEN_WHEN = 371,
+    TOKEN_WHERE = 372,
+    TOKEN_WITH = 373,
+    TOKEN_YEAR = 374,
+    TOKEN_YEARMONTH = 375,
+    TOKEN_EOF = 376,
+    TOKEN_LEX_ERROR = 377
   };
 #endif
 
@@ -250,7 +256,7 @@ union YYSTYPE
   quickstep::ParseOrderBy *opt_order_by_clause_;
   bool *order_direction_;
   quickstep::ParseLimit *opt_limit_clause_;
-  
+
   quickstep::ParseSample *opt_sample_clause_;
 
   quickstep::PtrList<quickstep::ParseOrderByItem> *order_commalist_;
@@ -259,7 +265,7 @@ union YYSTYPE
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
 
-#line 263 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 269 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/parser/tests/TPCH.test
----------------------------------------------------------------------
diff --git a/parser/tests/TPCH.test b/parser/tests/TPCH.test
index 12d85ce..7f4d911 100644
--- a/parser/tests/TPCH.test
+++ b/parser/tests/TPCH.test
@@ -532,7 +532,7 @@ FROM
     SELECT
       n1.n_name AS supp_nation,
       n2.n_name AS cust_nation,
-      EXTRACT(year FROM l_shipdate) AS l_year,
+      EXTRACT(YEAR FROM l_shipdate) AS l_year,
       l_extendedprice * (1 - l_discount) AS volume
     FROM
       supplier,
@@ -596,7 +596,7 @@ SelectStatement
           | +-SelectListItem[alias=cust_nation]
           | | +-AttributeReference[attribute_name=n_name,relation_name=n2]
           | +-SelectListItem[alias=l_year]
-          | | +-Extract[unit=year]
+          | | +-Extract[unit=YEAR]
           | |   +-date_expression=AttributeReference[attribute_name=l_shipdate]
           | +-SelectListItem[alias=volume]
           |   +-Multiply
@@ -673,7 +673,7 @@ SELECT
 FROM
   (
     SELECT
-      EXTRACT(year FROM o_orderdate) AS o_year,
+      EXTRACT(YEAR FROM o_orderdate) AS o_year,
       l_extendedprice * (1 - l_discount) AS volume,
       n2.n_name AS nation
     FROM
@@ -734,7 +734,7 @@ SelectStatement
         +-Select
           +-select_clause=SelectList
           | +-SelectListItem[alias=o_year]
-          | | +-Extract[unit=year]
+          | | +-Extract[unit=YEAR]
           | |   +-date_expression=AttributeReference[attribute_name=o_orderdate]
           | +-SelectListItem[alias=volume]
           | | +-Multiply
@@ -858,7 +858,7 @@ SelectStatement
           | +-SelectListItem[alias=nation]
           | | +-AttributeReference[attribute_name=n_name]
           | +-SelectListItem[alias=o_year]
-          | | +-Extract[unit=year]
+          | | +-Extract[unit=YEAR]
           | |   +-date_expression=AttributeReference[attribute_name=o_orderdate]
           | +-SelectListItem[alias=amount]
           |   +-Subtract

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/query_optimizer/tests/execution_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Select.test b/query_optimizer/tests/execution_generator/Select.test
index 47c1077..438546c 100644
--- a/query_optimizer/tests/execution_generator/Select.test
+++ b/query_optimizer/tests/execution_generator/Select.test
@@ -662,11 +662,11 @@ FROM dates;
 +-------------------------------+
 ==
 
-SELECT EXTRACT(hour FROM value + INTERVAL '1 hour')
+SELECT EXTRACT(HOUR FROM value + INTERVAL '1 hour')
 FROM dates;
 --
 +-----------------------------------------------------+
-|EXTRACT(hour FROM (value+DatetimeInterval('1 hour')))|
+|EXTRACT(HOUR FROM (value+DatetimeInterval('1 hour')))|
 +-----------------------------------------------------+
 |                                                   11|
 |                                                   12|


[18/30] incubator-quickstep git commit: Fixed the deadlock when loading while evicting. (#196)

Posted by zu...@apache.org.
Fixed the deadlock when loading while evicting. (#196)

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

Branch: refs/heads/master
Commit: 30a157769406f76fc91ccd619b14db24bcd6e585
Parents: 9372b94
Author: Zuyu ZHANG <zu...@users.noreply.github.com>
Authored: Wed Apr 27 22:59:25 2016 -0700
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 storage/StorageManager.cpp     | 20 ++++++++++++------
 utility/CMakeLists.txt         |  2 ++
 utility/ShardedLockManager.hpp | 41 +++++++++++++++++++++++++------------
 3 files changed, 44 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30a15776/storage/StorageManager.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.cpp b/storage/StorageManager.cpp
index a3f265d..b98a28c 100644
--- a/storage/StorageManager.cpp
+++ b/storage/StorageManager.cpp
@@ -494,12 +494,16 @@ MutableBlockReference StorageManager::getBlockInternal(
   // To be safe, release the block's shard after 'eviction_lock' destructs.
   lock_manager_.release(block);
 
+  if (ret.valid()) {
+    return ret;
+  }
+
   // Note that there is no way for the block to be evicted between the call to
   // loadBlock and the call to EvictionPolicy::blockReferenced from
   // MutableBlockReference's constructor; this is because EvictionPolicy
   // doesn't know about the block until blockReferenced is called, so
   // chooseBlockToEvict shouldn't return the block.
-  if (!ret.valid()) {
+  do {
     SpinSharedMutexExclusiveLock<false> io_lock(*lock_manager_.get(block));
     {
       // Check one more time if the block got loaded in memory by someone else.
@@ -508,12 +512,12 @@ MutableBlockReference StorageManager::getBlockInternal(
       if (it != blocks_.end()) {
         DEBUG_ASSERT(!it->second.block->isBlob());
         ret = MutableBlockReference(static_cast<StorageBlock*>(it->second.block), eviction_policy_.get());
-        return ret;
+        break;
       }
     }
     // No other thread loaded the block before us.
     ret = MutableBlockReference(loadBlock(block, relation, numa_node), eviction_policy_.get());
-  }
+  } while (false);
   // To be safe, release the block's shard after 'io_lock' destructs.
   lock_manager_.release(block);
 
@@ -535,7 +539,11 @@ MutableBlobReference StorageManager::getBlobInternal(const block_id blob,
   // To be safe, release the blob's shard after 'eviction_lock' destructs.
   lock_manager_.release(blob);
 
-  if (!ret.valid()) {
+  if (ret.valid()) {
+    return ret;
+  }
+
+  do {
     SpinSharedMutexExclusiveLock<false> io_lock(*lock_manager_.get(blob));
     // Note that there is no way for the block to be evicted between the call to
     // loadBlob and the call to EvictionPolicy::blockReferenced from
@@ -548,12 +556,12 @@ MutableBlobReference StorageManager::getBlobInternal(const block_id blob,
       if (it != blocks_.end()) {
         DEBUG_ASSERT(it->second.block->isBlob());
         ret = MutableBlobReference(static_cast<StorageBlob*>(it->second.block), eviction_policy_.get());
-        return ret;
+        break;
       }
     }
     // No other thread loaded the blob before us.
     ret = MutableBlobReference(loadBlob(blob, numa_node), eviction_policy_.get());
-  }
+  } while (false);
   // To be safe, release the blob's shard after 'io_lock' destructs.
   lock_manager_.release(blob);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30a15776/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 4ff9254..bb59f65 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -244,7 +244,9 @@ target_link_libraries(quickstep_utility_SortConfiguration_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_utility_ShardedLockManager
                       quickstep_storage_StorageConstants
+                      quickstep_threading_Mutex
                       quickstep_threading_SharedMutex
+                      quickstep_threading_SpinSharedMutex
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_StringUtil
                       glog)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30a15776/utility/ShardedLockManager.hpp
----------------------------------------------------------------------
diff --git a/utility/ShardedLockManager.hpp b/utility/ShardedLockManager.hpp
index 1d59acb..e3eba85 100644
--- a/utility/ShardedLockManager.hpp
+++ b/utility/ShardedLockManager.hpp
@@ -21,9 +21,12 @@
 #include <array>
 #include <cstddef>
 #include <functional>
+#include <unordered_map>
 
 #include "storage/StorageConstants.hpp"
+#include "threading/Mutex.hpp"
 #include "threading/SharedMutex.hpp"
+#include "threading/SpinSharedMutex.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -64,24 +67,29 @@ class ShardedLockManager {
     if (has_collision != nullptr) {
       // In StorageManager::makeRoomForBlock, check whether the evicting block
       // or blob has a shard collision with existing referenced shards.
-      SpinSharedMutexSharedLock<false> read_lock(shards_mutex_);
-      if (shards_.find(shard) != shards_.end()) {
+      SpinSharedMutexSharedLock<false> read_lock(shard_count_mutex_);
+      if (shard_count_.find(shard) != shard_count_.end()) {
         *has_collision = true;
         return &collision_mutex_;
       }
     }
 
     {
-      SpinSharedMutexExclusiveLock<false> write_lock(shards_mutex_);
+      SpinSharedMutexExclusiveLock<false> write_lock(shard_count_mutex_);
 
       // Check one more time for the evicting block or blob if there is a shard
       // collision.
-      if (has_collision != nullptr && shards_.find(shard) != shards_.end()) {
-        *has_collision = true;
-        return &collision_mutex_;
+      auto it = shard_count_.find(shard);
+      if (it != shard_count_.end()) {
+        if (has_collision != nullptr) {
+          *has_collision = true;
+          return &collision_mutex_;
+        }
+
+        ++it->second;
+      } else {
+        shard_count_.emplace(shard, 1);
       }
-
-      shards_.insert(shard);
     }
     return &sharded_mutexes_[shard];
   }
@@ -91,8 +99,13 @@ class ShardedLockManager {
    * @param key The key to compute the shard.
    */
   void release(const T key) {
-    SpinSharedMutexExclusiveLock<false> write_lock(shards_mutex_);
-    shards_.erase(hash_(key) % N);
+    SpinSharedMutexExclusiveLock<false> write_lock(shard_count_mutex_);
+    auto it = shard_count_.find(hash_(key) % N);
+    DCHECK(it != shard_count_.end());
+
+    if (--it->second == 0) {
+      shard_count_.erase(it);
+    }
   }
 
  private:
@@ -102,9 +115,11 @@ class ShardedLockManager {
   // The placeholder mutex used whenever there is a hash collision.
   SharedMutexT collision_mutex_;
 
-  // Bookkeep all shards referenced by StorageManager in multiple threads.
-  std::unordered_set<std::size_t> shards_;
-  alignas(kCacheLineBytes) mutable SpinSharedMutex<false> shards_mutex_;
+  // Count all shards referenced by StorageManager in multiple threads.
+  // The key is the shard, while the value is the count. If the count equals to
+  // zero, we delete the shard entry.
+  std::unordered_map<std::size_t, std::size_t> shard_count_;
+  alignas(kCacheLineBytes) mutable SpinSharedMutex<false> shard_count_mutex_;
 
   DISALLOW_COPY_AND_ASSIGN(ShardedLockManager);
 };


[22/30] incubator-quickstep git commit: Adds support for multiple SharedSubplanReference. (#199)

Posted by zu...@apache.org.
Adds support for multiple SharedSubplanReference. (#199)

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

Branch: refs/heads/master
Commit: 517bf5ecc9e270ff00bb1c8daf0ddf73bf631a5f
Parents: d384da1
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Apr 28 16:09:32 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp          |  13 ++
 query_optimizer/ExecutionGenerator.hpp          |  11 +-
 query_optimizer/cost_model/CMakeLists.txt       |   1 +
 query_optimizer/cost_model/SimpleCostModel.cpp  |   9 +
 query_optimizer/cost_model/SimpleCostModel.hpp  |   8 +-
 .../logical/SharedSubplanReference.cpp          |   7 +-
 .../logical/SharedSubplanReference.hpp          |  34 +++-
 .../physical/SharedSubplanReference.cpp         |   5 +
 .../physical/SharedSubplanReference.hpp         |  34 +++-
 query_optimizer/resolver/Resolver.cpp           |  25 ++-
 query_optimizer/strategy/OneToOne.cpp           |   1 +
 .../tests/execution_generator/Select.test       |  25 +++
 .../tests/logical_generator/Select.test         | 106 ++++++++++-
 .../tests/physical_generator/Select.test        | 165 ++++++++++++++++-
 query_optimizer/tests/resolver/Select.test      | 181 +++++++++++++++----
 15 files changed, 542 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index abcdd6c..c34f084 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -154,6 +154,9 @@ void ExecutionGenerator::generatePlan(const P::PhysicalPtr &physical_plan) {
   CHECK(P::SomeTopLevelPlan::MatchesWithConditionalCast(physical_plan, &top_level_physical_plan_))
       << "The physical plan must be rooted by a TopLevelPlan";
 
+  cost_model_.reset(
+      new cost::SimpleCostModel(top_level_physical_plan_->shared_subplans()));
+
   const CatalogRelation *result_relation = nullptr;
 
   try {
@@ -550,6 +553,16 @@ void ExecutionGenerator::convertSharedSubplanReference(const physical::SharedSub
           top_level_physical_plan_->shared_subplan_at(physical_plan->subplan_id()));
   if (found_it != physical_to_output_relation_map_.end()) {
     physical_to_output_relation_map_.emplace(physical_plan, found_it->second);
+
+    // Propagate the (ExprId -> CatalogAttribute) mapping.
+    const std::vector<E::AttributeReferencePtr> &referenced_attributes =
+        physical_plan->referenced_attributes();
+    const std::vector<E::AttributeReferencePtr> &output_attributes =
+        physical_plan->output_attributes();
+    for (std::size_t i = 0; i < referenced_attributes.size(); ++i) {
+      attribute_substitution_map_[output_attributes[i]->id()] =
+          attribute_substitution_map_[referenced_attributes[i]->id()];
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index df47b31..7c563d4 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015-2016 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -104,8 +106,6 @@ class ExecutionGenerator {
 #ifdef QUICKSTEP_DISTRIBUTED
     catalog_database_cache_proto_ = DCHECK_NOTNULL(query_handle->getCatalogDatabaseCacheProtoMutable());
 #endif
-
-    setupCostModel();
   }
 
   /**
@@ -163,13 +163,6 @@ class ExecutionGenerator {
   void generatePlanInternal(const physical::PhysicalPtr &physical_plan);
 
   /**
-   * @brief Sets up the cost model.
-   */
-  void setupCostModel() {
-    cost_model_.reset(new cost::SimpleCostModel());
-  }
-
-  /**
    * @brief Finds the CatalogRelationInfo from <physical_to_output_relation_map_>
    *        by the physical node \p physical. Returns NULL if not found.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index e561fc8..6697d52 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -31,6 +31,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_SimpleCostModel
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_SharedSubplanReference
                       quickstep_queryoptimizer_physical_TableGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index 291fb9d..48f76fa 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -27,6 +29,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
 #include "query_optimizer/physical/Selection.hpp"
+#include "query_optimizer/physical/SharedSubplanReference.hpp"
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
@@ -63,6 +66,12 @@ std::size_t SimpleCostModel::estimateCardinality(
     case P::PhysicalType::kAggregate:
       return estimateCardinalityForAggregate(
           std::static_pointer_cast<const P::Aggregate>(physical_plan));
+    case P::PhysicalType::kSharedSubplanReference: {
+      const P::SharedSubplanReferencePtr shared_subplan_reference =
+          std::static_pointer_cast<const P::SharedSubplanReference>(physical_plan);
+      return estimateCardinality(
+          shared_subplans_[shared_subplan_reference->subplan_id()]);
+    }
     default:
       LOG(FATAL) << "Unsupported physical plan:" << physical_plan->toString();
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/cost_model/SimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.hpp b/query_optimizer/cost_model/SimpleCostModel.hpp
index e53887d..9862198 100644
--- a/query_optimizer/cost_model/SimpleCostModel.hpp
+++ b/query_optimizer/cost_model/SimpleCostModel.hpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -19,6 +21,7 @@
 #define QUERY_OPTIMIZER_COST_MODEL_SIMPLE_COST_MODEL_HPP_
 
 #include <cstddef>
+#include <vector>
 
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
@@ -47,7 +50,8 @@ class SimpleCostModel : public CostModel {
   /**
    * @brief Constructor.
    */
-  SimpleCostModel() {}
+  explicit SimpleCostModel(const std::vector<physical::PhysicalPtr> &shared_subplans)
+      : shared_subplans_(shared_subplans) {}
 
   std::size_t estimateCardinality(
       const physical::PhysicalPtr &physical_plan) override;
@@ -84,6 +88,8 @@ class SimpleCostModel : public CostModel {
   std::size_t estimateCardinalityForAggregate(
       const physical::AggregatePtr &physical_plan);
 
+  const std::vector<physical::PhysicalPtr> &shared_subplans_;
+
   DISALLOW_COPY_AND_ASSIGN(SimpleCostModel);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/logical/SharedSubplanReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/SharedSubplanReference.cpp b/query_optimizer/logical/SharedSubplanReference.cpp
index 8a5bec9..b5d3143 100644
--- a/query_optimizer/logical/SharedSubplanReference.cpp
+++ b/query_optimizer/logical/SharedSubplanReference.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -41,13 +43,16 @@ void SharedSubplanReference::getFieldStringItems(
   inline_field_names->push_back("subplan_id");
   inline_field_values->push_back(std::to_string(subplan_id_));
 
+  container_child_field_names->push_back("referenced_attributes");
+  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(referenced_attributes_));
+
   container_child_field_names->push_back("output_attributes");
   container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(output_attributes_));
 }
 
 LogicalPtr SharedSubplanReference::copyWithNewChildren(const std::vector<LogicalPtr> &new_children) const {
   DCHECK(new_children.empty());
-  return Create(subplan_id_, output_attributes_);
+  return Create(subplan_id_, referenced_attributes_, output_attributes_);
 }
 
 }  // namespace logical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/logical/SharedSubplanReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/SharedSubplanReference.hpp b/query_optimizer/logical/SharedSubplanReference.hpp
index 55035d1..8f0e37b 100644
--- a/query_optimizer/logical/SharedSubplanReference.hpp
+++ b/query_optimizer/logical/SharedSubplanReference.hpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -28,6 +30,8 @@
 #include "query_optimizer/logical/LogicalType.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 namespace optimizer {
 namespace logical {
@@ -60,7 +64,14 @@ class SharedSubplanReference : public Logical {
   }
 
   /**
-   * @return The output attributes of the shared subplan.
+   * @return The attributes from the referenced shared subplan.
+   */
+  const std::vector<expressions::AttributeReferencePtr>& referenced_attributes() const {
+    return referenced_attributes_;
+  }
+
+  /**
+   * @return The output attributes of this shared subplan reference.
    */
   const std::vector<expressions::AttributeReferencePtr>& output_attributes() const {
     return output_attributes_;
@@ -81,7 +92,7 @@ class SharedSubplanReference : public Logical {
   LogicalPtr copyWithNewChildren(const std::vector<LogicalPtr> &new_children) const override;
 
   std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override {
-    return output_attributes_;
+    return referenced_attributes_;
   }
 
   /**
@@ -89,21 +100,30 @@ class SharedSubplanReference : public Logical {
    *
    * @param subplan_id The ID of the shared subplan, which is the index of the referenced
    *        shared subplan in <shared_subplans_> of the TopLevelPlan.
-   * @param output_attributes The output attributes of the shared subplan.
+   * @param referenced_attributes The attributes from the referenced shared subplan.
+   * @param output_attributes The output attributes of this shared subplan reference.
    * @return An immutable SharedSubplanReference.
    */
-  static SharedSubplanReferencePtr Create(int subplan_id,
-                                          const std::vector<expressions::AttributeReferencePtr> &output_attributes) {
-    return SharedSubplanReferencePtr(new SharedSubplanReference(subplan_id, output_attributes));
+  static SharedSubplanReferencePtr Create(
+      int subplan_id,
+      const std::vector<expressions::AttributeReferencePtr> &referenced_attributes,
+      const std::vector<expressions::AttributeReferencePtr> &output_attributes) {
+    return SharedSubplanReferencePtr(
+        new SharedSubplanReference(subplan_id, referenced_attributes, output_attributes));
   }
 
  private:
   SharedSubplanReference(int subplan_id,
+                         const std::vector<expressions::AttributeReferencePtr> &referenced_attributes,
                          const std::vector<expressions::AttributeReferencePtr> &output_attributes)
     : subplan_id_(subplan_id),
-      output_attributes_(output_attributes) {}
+      referenced_attributes_(referenced_attributes),
+      output_attributes_(output_attributes) {
+    DCHECK_EQ(output_attributes_.size(), referenced_attributes_.size());
+  }
 
   int subplan_id_;
+  std::vector<expressions::AttributeReferencePtr> referenced_attributes_;
   std::vector<expressions::AttributeReferencePtr> output_attributes_;
 
   DISALLOW_COPY_AND_ASSIGN(SharedSubplanReference);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/physical/SharedSubplanReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/SharedSubplanReference.cpp b/query_optimizer/physical/SharedSubplanReference.cpp
index ce851ac..eb33d32 100644
--- a/query_optimizer/physical/SharedSubplanReference.cpp
+++ b/query_optimizer/physical/SharedSubplanReference.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -40,6 +42,9 @@ void SharedSubplanReference::getFieldStringItems(
   inline_field_names->push_back("subplan_id");
   inline_field_values->push_back(std::to_string(subplan_id_));
 
+  container_child_field_names->push_back("referenced_attributes");
+  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(referenced_attributes_));
+
   container_child_field_names->push_back("output_attributes");
   container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(output_attributes_));
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/physical/SharedSubplanReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/SharedSubplanReference.hpp b/query_optimizer/physical/SharedSubplanReference.hpp
index 5005598..4439256 100644
--- a/query_optimizer/physical/SharedSubplanReference.hpp
+++ b/query_optimizer/physical/SharedSubplanReference.hpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -63,7 +65,14 @@ class SharedSubplanReference : public physical::Physical {
   }
 
   /**
-   * @return The output attributes of the shared subplan.
+   * @return The attributes from the referenced shared subplan.
+   */
+  const std::vector<expressions::AttributeReferencePtr>& referenced_attributes() const {
+    return referenced_attributes_;
+  }
+
+  /**
+   * @return The output attributes of this shared subplan reference.
    */
   const std::vector<expressions::AttributeReferencePtr>& output_attributes() const {
     return output_attributes_;
@@ -83,11 +92,11 @@ class SharedSubplanReference : public physical::Physical {
 
   PhysicalPtr copyWithNewChildren(const std::vector<PhysicalPtr> &new_children) const override {
     DCHECK(new_children.empty());
-    return Create(subplan_id_, output_attributes_);
+    return Create(subplan_id_, referenced_attributes_, output_attributes_);
   }
 
   std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override {
-    return output_attributes_;
+    return referenced_attributes_;
   }
 
   bool maybeCopyWithPrunedExpressions(
@@ -101,21 +110,30 @@ class SharedSubplanReference : public physical::Physical {
    *
    * @param subplan_id The ID of the shared subplan, which is the index of
    *        the referenced shared subplan in <shared_subplans_> of the TopLevelPlan.
-   * @param output_attributes The output attributes of the shared subplan.
+   * @param referenced_attributes The attributes from the referenced shared subplan.
+   * @param output_attributes The output attributes of this shared subplan reference.
    * @return An immutable SharedSubplanReference.
    */
-  static SharedSubplanReferencePtr Create(int subplan_id,
-                                          const std::vector<expressions::AttributeReferencePtr> &output_attributes) {
-    return SharedSubplanReferencePtr(new SharedSubplanReference(subplan_id, output_attributes));
+  static SharedSubplanReferencePtr Create(
+      int subplan_id,
+      const std::vector<expressions::AttributeReferencePtr> &referenced_attributes,
+      const std::vector<expressions::AttributeReferencePtr> &output_attributes) {
+    return SharedSubplanReferencePtr(
+        new SharedSubplanReference(subplan_id, referenced_attributes, output_attributes));
   }
 
  private:
   SharedSubplanReference(int subplan_id,
+                         const std::vector<expressions::AttributeReferencePtr> &referenced_attributes,
                          const std::vector<expressions::AttributeReferencePtr> &output_attributes)
     : subplan_id_(subplan_id),
-      output_attributes_(output_attributes) {}
+      referenced_attributes_(referenced_attributes),
+      output_attributes_(output_attributes) {
+    DCHECK_EQ(output_attributes_.size(), referenced_attributes_.size());
+  }
 
   int subplan_id_;
+  std::vector<expressions::AttributeReferencePtr> referenced_attributes_;
   std::vector<expressions::AttributeReferencePtr> output_attributes_;
 
   DISALLOW_COPY_AND_ASSIGN(SharedSubplanReference);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 1cf5c28..45ecf33 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -1541,9 +1541,28 @@ L::LogicalPtr Resolver::resolveSimpleTableReference(
       with_queries_info_.with_query_name_to_vector_position.find(lower_table_name);
   if (subplan_it != with_queries_info_.with_query_name_to_vector_position.end()) {
     with_queries_info_.unreferenced_query_indexes.erase(subplan_it->second);
-    return L::SharedSubplanReference::Create(
-        subplan_it->second,
-        with_queries_info_.with_query_plans[subplan_it->second]->getOutputAttributes());
+
+    const std::vector<E::AttributeReferencePtr> with_query_attributes =
+        with_queries_info_.with_query_plans[subplan_it->second]->getOutputAttributes();
+
+    // Create a vector of new attributes to delegate the original output attributes
+    // from the WITH query, to avoid (ExprId -> CatalogAttribute) mapping collision
+    // later in ExecutionGenerator when there are multiple SharedSubplanReference's
+    // referencing a same shared subplan.
+    std::vector<E::AttributeReferencePtr> delegator_attributes;
+    for (const E::AttributeReferencePtr &attribute : with_query_attributes) {
+      delegator_attributes.emplace_back(
+          E::AttributeReference::Create(context_->nextExprId(),
+                                        attribute->attribute_name(),
+                                        attribute->attribute_alias(),
+                                        attribute->relation_name(),
+                                        attribute->getValueType(),
+                                        attribute->scope()));
+    }
+
+    return L::SharedSubplanReference::Create(subplan_it->second,
+                                             with_query_attributes,
+                                             delegator_attributes);
   }
 
   // Then look up the name in the database.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/strategy/OneToOne.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/OneToOne.cpp b/query_optimizer/strategy/OneToOne.cpp
index d4d79f9..7f59151 100644
--- a/query_optimizer/strategy/OneToOne.cpp
+++ b/query_optimizer/strategy/OneToOne.cpp
@@ -84,6 +84,7 @@ bool OneToOne::generatePlan(const L::LogicalPtr &logical_input,
       const L::SharedSubplanReferencePtr shared_subplan_reference =
           std::static_pointer_cast<const L::SharedSubplanReference>(logical_input);
       *physical_output = P::SharedSubplanReference::Create(shared_subplan_reference->subplan_id(),
+                                                           shared_subplan_reference->referenced_attributes(),
                                                            shared_subplan_reference->output_attributes());
       return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/tests/execution_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Select.test b/query_optimizer/tests/execution_generator/Select.test
index 3a64c9b..a08b012 100644
--- a/query_optimizer/tests/execution_generator/Select.test
+++ b/query_optimizer/tests/execution_generator/Select.test
@@ -891,6 +891,31 @@ FROM (
 +-----------+--------------------+-----------+------------------------+
 ==
 
+# Same shared subplan referenced multiple times.
+WITH t(x, y) AS (
+  SELECT i % 5, i
+  FROM generate_series(1, 20) AS g(i)
+)
+SELECT *
+FROM t
+WHERE t.y = (
+  SELECT MAX(y)
+  FROM t t1
+  WHERE t.x = t1.x
+)
+ORDER BY x;
+--
++-----------+-----------+
+|x          |y          |
++-----------+-----------+
+|          0|         20|
+|          1|         16|
+|          2|         17|
+|          3|         18|
+|          4|         19|
++-----------+-----------+
+==
+
 # TODO(team): Fix Issue #9 to enable COUNT(*).
 SELECT COUNT(long_col)
 FROM test,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/tests/logical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Select.test b/query_optimizer/tests/logical_generator/Select.test
index 6dff3e7..3c152e8 100644
--- a/query_optimizer/tests/logical_generator/Select.test
+++ b/query_optimizer/tests/logical_generator/Select.test
@@ -502,22 +502,29 @@ TopLevelPlan
 | | | |   type=VarChar(20) NULL]
 | | | +-right=Filter
 | | | | +-input=SharedSubplanReference[subplan_id=0]
+| | | | | +-referenced_attributes=
+| | | | | | +-AttributeReference[id=0,name=subquery_int0,relation=subquery,
+| | | | | | | type=Int NULL]
+| | | | | | +-AttributeReference[id=7,name=subquery_long,relation=subquery,
+| | | | | | | type=Long]
+| | | | | | +-AttributeReference[id=12,name=subquery_int1,relation=subquery,
+| | | | | |   type=Int NULL]
 | | | | | +-output_attributes=
-| | | | |   +-AttributeReference[id=0,name=subquery_int0,relation=subquery,
+| | | | |   +-AttributeReference[id=36,name=subquery_int0,relation=subquery,
 | | | | |   | type=Int NULL]
-| | | | |   +-AttributeReference[id=7,name=subquery_long,relation=subquery,
+| | | | |   +-AttributeReference[id=37,name=subquery_long,relation=subquery,
 | | | | |   | type=Long]
-| | | | |   +-AttributeReference[id=12,name=subquery_int1,relation=subquery,
+| | | | |   +-AttributeReference[id=38,name=subquery_int1,relation=subquery,
 | | | | |     type=Int NULL]
 | | | | +-filter_predicate=Equal
-| | | |   +-AttributeReference[id=12,name=subquery_int1,relation=subquery,
+| | | |   +-AttributeReference[id=38,name=subquery_int1,relation=subquery,
 | | | |   | type=Int NULL]
-| | | |   +-AttributeReference[id=7,name=subquery_long,relation=subquery,
+| | | |   +-AttributeReference[id=37,name=subquery_long,relation=subquery,
 | | | |     type=Long]
 | | | +-left_join_attributes=
 | | | | +-AttributeReference[id=30,name=int_col,relation=c,type=Int NULL]
 | | | +-right_join_attributes=
-| | |   +-AttributeReference[id=0,name=subquery_int0,relation=subquery,
+| | |   +-AttributeReference[id=36,name=subquery_int0,relation=subquery,
 | | |     type=Int NULL]
 | | +-join_predicate=Literal[value=true]
 | +-project_list=
@@ -572,11 +579,14 @@ select * from subquery
 TopLevelPlan
 +-plan=Project
 | +-input=SharedSubplanReference[subplan_id=0]
+| | +-referenced_attributes=
+| | | +-AttributeReference[id=7,name=,alias=(int_col+2),relation=subquery,
+| | |   type=Int NULL]
 | | +-output_attributes=
-| |   +-AttributeReference[id=7,name=,alias=(int_col+2),relation=subquery,
+| |   +-AttributeReference[id=8,name=,alias=(int_col+2),relation=subquery,
 | |     type=Int NULL]
 | +-project_list=
-|   +-AttributeReference[id=7,name=,alias=(int_col+2),relation=subquery,
+|   +-AttributeReference[id=8,name=,alias=(int_col+2),relation=subquery,
 |     type=Int NULL]
 +-shared_subplans=
 | +-Project
@@ -596,7 +606,7 @@ TopLevelPlan
 |         | +-Literal[value=1,type=Int]
 |         +-Literal[value=2,type=Int]
 +-output_attributes=
-  +-AttributeReference[id=7,name=,alias=(int_col+2),relation=subquery,
+  +-AttributeReference[id=8,name=,alias=(int_col+2),relation=subquery,
     type=Int NULL]
 ==
 
@@ -1266,3 +1276,81 @@ SELECT x + (
 FROM b;
 --
 ERROR: Nested queries can only reference attributes in the outer query one level above
+==
+
+# Same shared subplan referenced multiple times.
+WITH t(x, y) AS (
+  SELECT i % 5, i
+  FROM generate_series(1, 20) AS g(i)
+)
+SELECT *
+FROM t
+WHERE t.y = (
+  SELECT MAX(y)
+  FROM t t1
+  WHERE t.x = t1.x
+)
+ORDER BY x;
+--
+TopLevelPlan
++-plan=Project
+| +-input=Sort[is_ascending=[true],nulls_first=[false]]
+| | +-input=Filter
+| | | +-input=HashJoin
+| | | | +-left=SharedSubplanReference[subplan_id=0]
+| | | | | +-referenced_attributes=
+| | | | | | +-AttributeReference[id=3,name=x,relation=,type=Int]
+| | | | | | +-AttributeReference[id=4,name=y,relation=,type=Int]
+| | | | | +-output_attributes=
+| | | | |   +-AttributeReference[id=5,name=x,relation=,type=Int]
+| | | | |   +-AttributeReference[id=6,name=y,relation=,type=Int]
+| | | | +-right=Project
+| | | | | +-input=Aggregate
+| | | | | | +-input=SharedSubplanReference[subplan_id=0]
+| | | | | | | +-referenced_attributes=
+| | | | | | | | +-AttributeReference[id=3,name=x,relation=,type=Int]
+| | | | | | | | +-AttributeReference[id=4,name=y,relation=,type=Int]
+| | | | | | | +-output_attributes=
+| | | | | | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | | | | |   +-AttributeReference[id=8,name=y,relation=,type=Int]
+| | | | | | +-grouping_expressions=
+| | | | | | | +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | | | | +-aggregate_expressions=
+| | | | | |   +-Alias[id=9,name=,alias=$aggregate0,relation=$aggregate,
+| | | | | |     type=Int NULL]
+| | | | | |     +-AggregateFunction[function=MAX]
+| | | | | |       +-AttributeReference[id=8,name=y,relation=,type=Int]
+| | | | | +-project_list=
+| | | | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | | |   +-Alias[id=9,name=,alias=MAX(y),relation=,type=Int NULL]
+| | | | |     +-AttributeReference[id=9,name=,alias=$aggregate0,
+| | | | |       relation=$aggregate,type=Int NULL]
+| | | | +-left_join_attributes=
+| | | | | +-AttributeReference[id=5,name=x,relation=,type=Int]
+| | | | +-right_join_attributes=
+| | | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | +-filter_predicate=Equal
+| | |   +-AttributeReference[id=6,name=y,relation=,type=Int]
+| | |   +-AttributeReference[id=9,name=,alias=MAX(y),relation=,type=Int NULL]
+| | +-sort_expressions=
+| |   +-AttributeReference[id=5,name=x,relation=,type=Int]
+| +-project_list=
+|   +-AttributeReference[id=5,name=x,relation=,type=Int]
+|   +-AttributeReference[id=6,name=y,relation=,type=Int]
++-shared_subplans=
+| +-Project
+|   +-input=TableGenerator[function_name=generate_series,table_alias=g]
+|   | +-AttributeReference[id=0,name=generate_series,alias=g,
+|   |   relation=generate_series,type=Int]
+|   +-project_list=
+|     +-Alias[id=3,name=x,relation=,type=Int]
+|     | +-Modulo
+|     |   +-AttributeReference[id=0,name=generate_series,alias=g,
+|     |   | relation=generate_series,type=Int]
+|     |   +-Literal[value=5,type=Int]
+|     +-Alias[id=4,name=y,relation=,type=Int]
+|       +-AttributeReference[id=0,name=generate_series,alias=g,
+|         relation=generate_series,type=Int]
++-output_attributes=
+  +-AttributeReference[id=5,name=x,relation=,type=Int]
+  +-AttributeReference[id=6,name=y,relation=,type=Int]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/tests/physical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index 62d09f5..3365206 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -1542,12 +1542,16 @@ select int_col from subquery
 TopLevelPlan
 +-plan=Project
 | +-input=SharedSubplanReference[subplan_id=0]
+| | +-referenced_attributes=
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
 | | +-output_attributes=
-| |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
-| |   +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| |   +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
+| |   +-AttributeReference[id=7,name=long_col,relation=test,type=Long]
+| |   +-AttributeReference[id=8,name=double_col,relation=test,type=Double NULL]
 | +-project_list=
-|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
 +-shared_subplans=
 | +-Project
 |   +-input=TableReference[relation_name=Test,relation_alias=test]
@@ -1563,17 +1567,21 @@ TopLevelPlan
 |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 |     +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
 +-output_attributes=
-  +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+  +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
 [Physical Plan]
 TopLevelPlan
 +-plan=Selection
 | +-input=SharedSubplanReference[subplan_id=0]
+| | +-referenced_attributes=
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
 | | +-output_attributes=
-| |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
-| |   +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| |   +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
+| |   +-AttributeReference[id=7,name=long_col,relation=test,type=Long]
+| |   +-AttributeReference[id=8,name=double_col,relation=test,type=Double NULL]
 | +-project_expressions=
-|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
 +-shared_subplans=
 | +-Selection
 |   +-input=TableReference[relation=Test,alias=test]
@@ -1589,7 +1597,7 @@ TopLevelPlan
 |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 |     +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
 +-output_attributes=
-  +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+  +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
 ==
 
 SELECT COUNT(DISTINCT int_col), SUM(float_col)
@@ -2731,3 +2739,140 @@ TopLevelPlan
 +-output_attributes=
   +-AttributeReference[id=8,name=,alias=(x*SubqueryExpression),relation=,
     type=Long NULL]
+==
+
+# Same shared subplan referenced multiple times.
+WITH t(x, y) AS (
+  SELECT i % 5, i
+  FROM generate_series(1, 20) AS g(i)
+)
+SELECT *
+FROM t
+WHERE t.y = (
+  SELECT MAX(y)
+  FROM t t1
+  WHERE t.x = t1.x
+)
+ORDER BY x;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=Sort[is_ascending=[true],nulls_first=[false]]
+| | +-input=Filter
+| | | +-input=HashJoin
+| | | | +-left=SharedSubplanReference[subplan_id=0]
+| | | | | +-referenced_attributes=
+| | | | | | +-AttributeReference[id=3,name=x,relation=,type=Int]
+| | | | | | +-AttributeReference[id=4,name=y,relation=,type=Int]
+| | | | | +-output_attributes=
+| | | | |   +-AttributeReference[id=5,name=x,relation=,type=Int]
+| | | | |   +-AttributeReference[id=6,name=y,relation=,type=Int]
+| | | | +-right=Project
+| | | | | +-input=Aggregate
+| | | | | | +-input=SharedSubplanReference[subplan_id=0]
+| | | | | | | +-referenced_attributes=
+| | | | | | | | +-AttributeReference[id=3,name=x,relation=,type=Int]
+| | | | | | | | +-AttributeReference[id=4,name=y,relation=,type=Int]
+| | | | | | | +-output_attributes=
+| | | | | | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | | | | |   +-AttributeReference[id=8,name=y,relation=,type=Int]
+| | | | | | +-grouping_expressions=
+| | | | | | | +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | | | | +-aggregate_expressions=
+| | | | | |   +-Alias[id=9,name=,alias=$aggregate0,relation=$aggregate,
+| | | | | |     type=Int NULL]
+| | | | | |     +-AggregateFunction[function=MAX]
+| | | | | |       +-AttributeReference[id=8,name=y,relation=,type=Int]
+| | | | | +-project_list=
+| | | | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | | |   +-Alias[id=9,name=,alias=MAX(y),relation=,type=Int NULL]
+| | | | |     +-AttributeReference[id=9,name=,alias=$aggregate0,
+| | | | |       relation=$aggregate,type=Int NULL]
+| | | | +-left_join_attributes=
+| | | | | +-AttributeReference[id=5,name=x,relation=,type=Int]
+| | | | +-right_join_attributes=
+| | | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | +-filter_predicate=Equal
+| | |   +-AttributeReference[id=6,name=y,relation=,type=Int]
+| | |   +-AttributeReference[id=9,name=,alias=MAX(y),relation=,type=Int NULL]
+| | +-sort_expressions=
+| |   +-AttributeReference[id=5,name=x,relation=,type=Int]
+| +-project_list=
+|   +-AttributeReference[id=5,name=x,relation=,type=Int]
+|   +-AttributeReference[id=6,name=y,relation=,type=Int]
++-shared_subplans=
+| +-Project
+|   +-input=TableGenerator[function_name=generate_series,table_alias=g]
+|   | +-AttributeReference[id=0,name=generate_series,alias=g,
+|   |   relation=generate_series,type=Int]
+|   +-project_list=
+|     +-Alias[id=3,name=x,relation=,type=Int]
+|     | +-Modulo
+|     |   +-AttributeReference[id=0,name=generate_series,alias=g,
+|     |   | relation=generate_series,type=Int]
+|     |   +-Literal[value=5,type=Int]
+|     +-Alias[id=4,name=y,relation=,type=Int]
+|       +-AttributeReference[id=0,name=generate_series,alias=g,
+|         relation=generate_series,type=Int]
++-output_attributes=
+  +-AttributeReference[id=5,name=x,relation=,type=Int]
+  +-AttributeReference[id=6,name=y,relation=,type=Int]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=Sort[is_ascending=[true],nulls_first=[false]]
+| | +-input=HashJoin
+| | | +-left=SharedSubplanReference[subplan_id=0]
+| | | | +-referenced_attributes=
+| | | | | +-AttributeReference[id=3,name=x,relation=,type=Int]
+| | | | | +-AttributeReference[id=4,name=y,relation=,type=Int]
+| | | | +-output_attributes=
+| | | |   +-AttributeReference[id=5,name=x,relation=,type=Int]
+| | | |   +-AttributeReference[id=6,name=y,relation=,type=Int]
+| | | +-right=Aggregate
+| | | | +-input=SharedSubplanReference[subplan_id=0]
+| | | | | +-referenced_attributes=
+| | | | | | +-AttributeReference[id=3,name=x,relation=,type=Int]
+| | | | | | +-AttributeReference[id=4,name=y,relation=,type=Int]
+| | | | | +-output_attributes=
+| | | | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | | |   +-AttributeReference[id=8,name=y,relation=,type=Int]
+| | | | +-grouping_expressions=
+| | | | | +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | | | +-aggregate_expressions=
+| | | |   +-Alias[id=9,name=,alias=$aggregate0,relation=$aggregate,type=Int NULL]
+| | | |     +-AggregateFunction[function=MAX]
+| | | |       +-AttributeReference[id=8,name=y,relation=,type=Int]
+| | | +-project_expressions=
+| | | | +-AttributeReference[id=5,name=x,relation=,type=Int]
+| | | | +-AttributeReference[id=6,name=y,relation=,type=Int]
+| | | +-left_join_attributes=
+| | | | +-AttributeReference[id=5,name=x,relation=,type=Int]
+| | | | +-AttributeReference[id=6,name=y,relation=,type=Int]
+| | | +-right_join_attributes=
+| | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | |   +-AttributeReference[id=9,name=,alias=$aggregate0,relation=$aggregate,
+| | |     type=Int NULL]
+| | +-sort_attributes=
+| |   +-AttributeReference[id=5,name=x,relation=,type=Int]
+| +-project_expressions=
+|   +-AttributeReference[id=5,name=x,relation=,type=Int]
+|   +-AttributeReference[id=6,name=y,relation=,type=Int]
++-shared_subplans=
+| +-Selection
+|   +-input=TableGenerator[function_name=generate_series,table_alias=g]
+|   | +-AttributeReference[id=0,name=generate_series,alias=g,
+|   |   relation=generate_series,type=Int]
+|   +-project_expressions=
+|     +-Alias[id=3,name=x,relation=,type=Int]
+|     | +-Modulo
+|     |   +-AttributeReference[id=0,name=generate_series,alias=g,
+|     |   | relation=generate_series,type=Int]
+|     |   +-Literal[value=5,type=Int]
+|     +-Alias[id=4,name=y,relation=,type=Int]
+|       +-AttributeReference[id=0,name=generate_series,alias=g,
+|         relation=generate_series,type=Int]
++-output_attributes=
+  +-AttributeReference[id=5,name=x,relation=,type=Int]
+  +-AttributeReference[id=6,name=y,relation=,type=Int]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/517bf5ec/query_optimizer/tests/resolver/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Select.test b/query_optimizer/tests/resolver/Select.test
index 00ff18a..9897934 100644
--- a/query_optimizer/tests/resolver/Select.test
+++ b/query_optimizer/tests/resolver/Select.test
@@ -1723,11 +1723,14 @@ select subquery_col from subquery
 TopLevelPlan
 +-plan=Project
 | +-input=SharedSubplanReference[subplan_id=0]
+| | +-referenced_attributes=
+| | | +-AttributeReference[id=0,name=subquery_col,relation=subquery,
+| | |   type=Int NULL]
 | | +-output_attributes=
-| |   +-AttributeReference[id=0,name=subquery_col,relation=subquery,
+| |   +-AttributeReference[id=6,name=subquery_col,relation=subquery,
 | |     type=Int NULL]
 | +-project_list=
-|   +-AttributeReference[id=0,name=subquery_col,relation=subquery,type=Int NULL]
+|   +-AttributeReference[id=6,name=subquery_col,relation=subquery,type=Int NULL]
 +-shared_subplans=
 | +-Project
 |   +-input=TableReference[relation_name=Test,relation_alias=test]
@@ -1742,7 +1745,7 @@ TopLevelPlan
 |     +-Alias[id=0,name=subquery_col,relation=subquery,type=Int NULL]
 |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
 +-output_attributes=
-  +-AttributeReference[id=0,name=subquery_col,relation=subquery,type=Int NULL]
+  +-AttributeReference[id=6,name=subquery_col,relation=subquery,type=Int NULL]
 ==
 
 # Multiple WITH queries.
@@ -1755,14 +1758,18 @@ TopLevelPlan
 +-plan=Project
 | +-input=MultiwayCartesianJoin
 | | +-SharedSubplanReference[subplan_id=2]
+| | | +-referenced_attributes=
+| | | | +-AttributeReference[id=10,name=int_col,relation=subquery3,type=Int NULL]
 | | | +-output_attributes=
-| | |   +-AttributeReference[id=7,name=int_col,relation=subquery3,type=Int NULL]
+| | |   +-AttributeReference[id=11,name=int_col,relation=subquery3,type=Int NULL]
 | | +-SharedSubplanReference[subplan_id=1]
+| |   +-referenced_attributes=
+| |   | +-AttributeReference[id=7,name=int_col,relation=subquery2,type=Int NULL]
 | |   +-output_attributes=
-| |     +-AttributeReference[id=6,name=int_col,relation=subquery2,type=Int NULL]
+| |     +-AttributeReference[id=12,name=int_col,relation=subquery2,type=Int NULL]
 | +-project_list=
-|   +-AttributeReference[id=7,name=int_col,relation=subquery3,type=Int NULL]
-|   +-AttributeReference[id=6,name=int_col,relation=subquery2,type=Int NULL]
+|   +-AttributeReference[id=11,name=int_col,relation=subquery3,type=Int NULL]
+|   +-AttributeReference[id=12,name=int_col,relation=subquery2,type=Int NULL]
 +-shared_subplans=
 | +-Project
 | | +-input=TableReference[relation_name=Test,relation_alias=test]
@@ -1777,31 +1784,38 @@ TopLevelPlan
 | |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
 | +-Project
 | | +-input=SharedSubplanReference[subplan_id=0]
+| | | +-referenced_attributes=
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
 | | | +-output_attributes=
-| | |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
 | | +-project_list=
-| |   +-Alias[id=6,name=int_col,relation=subquery2,type=Int NULL]
+| |   +-Alias[id=7,name=int_col,relation=subquery2,type=Int NULL]
 | |     +-Add
-| |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |       +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
 | |       +-Literal[value=5,type=Int]
 | +-Project
 |   +-input=MultiwayCartesianJoin
 |   | +-SharedSubplanReference[subplan_id=1]
+|   | | +-referenced_attributes=
+|   | | | +-AttributeReference[id=7,name=int_col,relation=subquery2,
+|   | | |   type=Int NULL]
 |   | | +-output_attributes=
-|   | |   +-AttributeReference[id=6,name=int_col,relation=subquery2,
+|   | |   +-AttributeReference[id=8,name=int_col,relation=subquery2,
 |   | |     type=Int NULL]
 |   | +-SharedSubplanReference[subplan_id=0]
+|   |   +-referenced_attributes=
+|   |   | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
 |   |   +-output_attributes=
-|   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |     +-AttributeReference[id=9,name=int_col,relation=test,type=Int NULL]
 |   +-project_list=
-|     +-Alias[id=7,name=int_col,relation=subquery3,type=Int NULL]
+|     +-Alias[id=10,name=int_col,relation=subquery3,type=Int NULL]
 |       +-Add
-|         +-AttributeReference[id=6,name=int_col,relation=subquery2,
+|         +-AttributeReference[id=8,name=int_col,relation=subquery2,
 |         | type=Int NULL]
 |         +-Literal[value=6,type=Int]
 +-output_attributes=
-  +-AttributeReference[id=7,name=int_col,relation=subquery3,type=Int NULL]
-  +-AttributeReference[id=6,name=int_col,relation=subquery2,type=Int NULL]
+  +-AttributeReference[id=11,name=int_col,relation=subquery3,type=Int NULL]
+  +-AttributeReference[id=12,name=int_col,relation=subquery2,type=Int NULL]
 ==
 
 with subquery(int_col_alias, long_col_alias) as (select int_col, long_col from test)
@@ -1810,11 +1824,14 @@ select int_col_alias from subquery
 TopLevelPlan
 +-plan=Project
 | +-input=SharedSubplanReference[subplan_id=0]
+| | +-referenced_attributes=
+| | | +-AttributeReference[id=6,name=int_col_alias,relation=,type=Int NULL]
+| | | +-AttributeReference[id=7,name=long_col_alias,relation=,type=Long]
 | | +-output_attributes=
-| |   +-AttributeReference[id=6,name=int_col_alias,relation=,type=Int NULL]
-| |   +-AttributeReference[id=7,name=long_col_alias,relation=,type=Long]
+| |   +-AttributeReference[id=8,name=int_col_alias,relation=,type=Int NULL]
+| |   +-AttributeReference[id=9,name=long_col_alias,relation=,type=Long]
 | +-project_list=
-|   +-AttributeReference[id=6,name=int_col_alias,relation=,type=Int NULL]
+|   +-AttributeReference[id=8,name=int_col_alias,relation=,type=Int NULL]
 +-shared_subplans=
 | +-Project
 |   +-input=Project
@@ -1835,7 +1852,7 @@ TopLevelPlan
 |     +-Alias[id=7,name=long_col_alias,relation=,type=Long]
 |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 +-output_attributes=
-  +-AttributeReference[id=6,name=int_col_alias,relation=,type=Int NULL]
+  +-AttributeReference[id=8,name=int_col_alias,relation=,type=Int NULL]
 ==
 
 with subquery as (select int_col, long_col from test)
@@ -1845,16 +1862,19 @@ TopLevelPlan
 +-plan=Project
 | +-input=Project
 | | +-input=SharedSubplanReference[subplan_id=0]
+| | | +-referenced_attributes=
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 | | | +-output_attributes=
-| | |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | |   +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
+| | |   +-AttributeReference[id=7,name=long_col,relation=test,type=Long]
 | | +-project_list=
-| |   +-Alias[id=6,name=int_col_alias,relation=,type=Int NULL]
-| |   | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| |   +-Alias[id=7,name=long_col_alias,relation=,type=Long]
-| |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   +-Alias[id=8,name=int_col_alias,relation=,type=Int NULL]
+| |   | +-AttributeReference[id=6,name=int_col,relation=test,type=Int NULL]
+| |   +-Alias[id=9,name=long_col_alias,relation=,type=Long]
+| |     +-AttributeReference[id=7,name=long_col,relation=test,type=Long]
 | +-project_list=
-|   +-AttributeReference[id=6,name=int_col_alias,relation=,type=Int NULL]
+|   +-AttributeReference[id=8,name=int_col_alias,relation=,type=Int NULL]
 +-shared_subplans=
 | +-Project
 |   +-input=TableReference[relation_name=Test,relation_alias=test]
@@ -1869,7 +1889,7 @@ TopLevelPlan
 |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
 |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 +-output_attributes=
-  +-AttributeReference[id=6,name=int_col_alias,relation=,type=Int NULL]
+  +-AttributeReference[id=8,name=int_col_alias,relation=,type=Int NULL]
 ==
 
 with subquery as (select int_col int_col_alias, long_col from test)
@@ -1878,12 +1898,16 @@ select int_col_alias from subquery
 TopLevelPlan
 +-plan=Project
 | +-input=SharedSubplanReference[subplan_id=0]
+| | +-referenced_attributes=
+| | | +-AttributeReference[id=0,name=int_col_alias,relation=subquery,
+| | | | type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 | | +-output_attributes=
-| |   +-AttributeReference[id=0,name=int_col_alias,relation=subquery,
+| |   +-AttributeReference[id=6,name=int_col_alias,relation=subquery,
 | |   | type=Int NULL]
-| |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   +-AttributeReference[id=7,name=long_col,relation=test,type=Long]
 | +-project_list=
-|   +-AttributeReference[id=0,name=int_col_alias,relation=subquery,type=Int NULL]
+|   +-AttributeReference[id=6,name=int_col_alias,relation=subquery,type=Int NULL]
 +-shared_subplans=
 | +-Project
 |   +-input=TableReference[relation_name=Test,relation_alias=test]
@@ -1899,7 +1923,7 @@ TopLevelPlan
 |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
 |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 +-output_attributes=
-  +-AttributeReference[id=0,name=int_col_alias,relation=subquery,type=Int NULL]
+  +-AttributeReference[id=6,name=int_col_alias,relation=subquery,type=Int NULL]
 ==
 
 # A WITH query cannot reference a subsequent WITH query.
@@ -1936,10 +1960,12 @@ select 1 from test
 TopLevelPlan
 +-plan=Project
 | +-input=SharedSubplanReference[subplan_id=0]
+| | +-referenced_attributes=
+| | | +-AttributeReference[id=6,name=,alias=1,relation=test,type=Int]
 | | +-output_attributes=
-| |   +-AttributeReference[id=6,name=,alias=1,relation=test,type=Int]
+| |   +-AttributeReference[id=7,name=,alias=1,relation=test,type=Int]
 | +-project_list=
-|   +-Alias[id=7,name=,alias=1,relation=,type=Int]
+|   +-Alias[id=8,name=,alias=1,relation=,type=Int]
 |     +-Literal[value=1,type=Int]
 +-shared_subplans=
 | +-Project
@@ -1955,7 +1981,7 @@ TopLevelPlan
 |     +-Alias[id=6,name=,alias=1,relation=test,type=Int]
 |       +-Literal[value=1,type=Int]
 +-output_attributes=
-  +-AttributeReference[id=7,name=,alias=1,relation=,type=Int]
+  +-AttributeReference[id=8,name=,alias=1,relation=,type=Int]
 ==
 
 # Identifies with special characters.
@@ -2937,3 +2963,88 @@ TopLevelPlan
 +-output_attributes=
   +-AttributeReference[id=11,name=,alias=(x+SubqueryExpression),relation=,
     type=Long NULL]
+==
+
+# Same shared subplan referenced multiple times.
+WITH t(x, y) AS (
+  SELECT i % 5, i
+  FROM generate_series(1, 20) AS g(i)
+)
+SELECT *
+FROM t
+WHERE t.y = (
+  SELECT MAX(y)
+  FROM t t1
+  WHERE t.x = t1.x
+)
+ORDER BY x;
+--
+TopLevelPlan
++-plan=Project
+| +-input=Sort[is_ascending=[true],nulls_first=[false]]
+| | +-input=Filter
+| | | +-input=SharedSubplanReference[subplan_id=0]
+| | | | +-referenced_attributes=
+| | | | | +-AttributeReference[id=3,name=x,relation=,type=Int]
+| | | | | +-AttributeReference[id=4,name=y,relation=,type=Int]
+| | | | +-output_attributes=
+| | | |   +-AttributeReference[id=5,name=x,relation=,type=Int]
+| | | |   +-AttributeReference[id=6,name=y,relation=,type=Int]
+| | | +-filter_predicate=Equal
+| | |   +-AttributeReference[id=6,name=y,relation=,type=Int]
+| | |   +-SubqueryExpression
+| | |     +-subquery=Project
+| | |       +-input=Aggregate
+| | |       | +-input=Filter
+| | |       | | +-input=SharedSubplanReference[subplan_id=0]
+| | |       | | | +-referenced_attributes=
+| | |       | | | | +-AttributeReference[id=3,name=x,relation=,type=Int]
+| | |       | | | | +-AttributeReference[id=4,name=y,relation=,type=Int]
+| | |       | | | +-output_attributes=
+| | |       | | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | |       | | |   +-AttributeReference[id=8,name=y,relation=,type=Int]
+| | |       | | +-filter_predicate=Equal
+| | |       | |   +-AttributeReference[id=5,name=x,relation=,type=Int,
+| | |       | |   | is_outer_reference=true]
+| | |       | |   +-AttributeReference[id=7,name=x,relation=,type=Int]
+| | |       | +-grouping_expressions=
+| | |       | | +-[]
+| | |       | +-aggregate_expressions=
+| | |       |   +-Alias[id=9,name=,alias=$aggregate0,relation=$aggregate,
+| | |       |     type=Int NULL]
+| | |       |     +-AggregateFunction[function=MAX]
+| | |       |       +-AttributeReference[id=8,name=y,relation=,type=Int]
+| | |       +-project_list=
+| | |         +-Alias[id=9,name=,alias=MAX(y),relation=,type=Int NULL]
+| | |           +-AttributeReference[id=9,name=,alias=$aggregate0,
+| | |             relation=$aggregate,type=Int NULL]
+| | +-sort_expressions=
+| |   +-AttributeReference[id=5,name=x,relation=,type=Int]
+| +-project_list=
+|   +-AttributeReference[id=5,name=x,relation=,type=Int]
+|   +-AttributeReference[id=6,name=y,relation=,type=Int]
++-shared_subplans=
+| +-Project
+|   +-input=Project
+|   | +-input=Project
+|   | | +-input=TableGenerator[function_name=generate_series,table_alias=g]
+|   | | | +-AttributeReference[id=0,name=generate_series,alias=g,
+|   | | |   relation=generate_series,type=Int]
+|   | | +-project_list=
+|   | |   +-Alias[id=1,name=i,relation=,type=Int]
+|   | |     +-AttributeReference[id=0,name=generate_series,alias=g,
+|   | |       relation=generate_series,type=Int]
+|   | +-project_list=
+|   |   +-Alias[id=2,name=,alias=(i%5),relation=t,type=Int]
+|   |   | +-Modulo
+|   |   |   +-AttributeReference[id=1,name=i,relation=,type=Int]
+|   |   |   +-Literal[value=5,type=Int]
+|   |   +-AttributeReference[id=1,name=i,relation=,type=Int]
+|   +-project_list=
+|     +-Alias[id=3,name=x,relation=,type=Int]
+|     | +-AttributeReference[id=2,name=,alias=(i%5),relation=t,type=Int]
+|     +-Alias[id=4,name=y,relation=,type=Int]
+|       +-AttributeReference[id=1,name=i,relation=,type=Int]
++-output_attributes=
+  +-AttributeReference[id=5,name=x,relation=,type=Int]
+  +-AttributeReference[id=6,name=y,relation=,type=Int]


[08/30] incubator-quickstep git commit: Adds support for scalar subqueries (#185)

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index d625dc7..32149a1 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -678,16 +678,16 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  47
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   975
+#define YYLAST   1072
 
 /* YYNTOKENS -- Number of terminals.  */
 #define YYNTOKENS  128
 /* YYNNTS -- Number of nonterminals.  */
 #define YYNNTS  94
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  254
+#define YYNRULES  255
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  470
+#define YYNSTATES  471
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
@@ -764,13 +764,13 @@ static const yytype_uint16 yyrline[] =
     1343,  1346,  1349,  1354,  1357,  1360,  1366,  1369,  1374,  1379,
     1388,  1393,  1402,  1407,  1410,  1415,  1418,  1423,  1429,  1435,
     1438,  1441,  1444,  1447,  1450,  1456,  1465,  1468,  1473,  1476,
-    1481,  1484,  1489,  1492,  1495,  1498,  1501,  1504,  1509,  1513,
-    1517,  1520,  1525,  1530,  1533,  1538,  1542,  1548,  1553,  1557,
-    1563,  1568,  1571,  1576,  1580,  1586,  1589,  1592,  1595,  1607,
-    1611,  1630,  1645,  1649,  1655,  1658,  1663,  1667,  1674,  1677,
-    1680,  1683,  1686,  1689,  1692,  1695,  1698,  1701,  1706,  1717,
-    1720,  1725,  1728,  1731,  1737,  1741,  1747,  1750,  1758,  1761,
-    1764,  1767,  1773,  1778,  1783
+    1481,  1484,  1489,  1492,  1495,  1498,  1501,  1504,  1507,  1512,
+    1516,  1520,  1523,  1528,  1533,  1536,  1541,  1545,  1551,  1556,
+    1560,  1566,  1571,  1574,  1579,  1583,  1589,  1592,  1595,  1598,
+    1610,  1614,  1633,  1648,  1652,  1658,  1661,  1666,  1670,  1677,
+    1680,  1683,  1686,  1689,  1692,  1695,  1698,  1701,  1704,  1709,
+    1720,  1723,  1728,  1731,  1734,  1740,  1744,  1750,  1753,  1761,
+    1764,  1767,  1770,  1776,  1781,  1786
 };
 #endif
 
@@ -863,10 +863,10 @@ static const yytype_uint16 yytoknum[] =
 };
 # endif
 
-#define YYPACT_NINF -223
+#define YYPACT_NINF -216
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-223)))
+  (!!((Yystate) == (-216)))
 
 #define YYTABLE_NINF -128
 
@@ -877,53 +877,54 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-      63,  -223,  -223,   -56,   229,   -15,     4,   -51,    14,  -223,
-      36,   229,   229,  -223,    96,   124,  -223,  -223,  -223,  -223,
-    -223,  -223,  -223,  -223,  -223,  -223,    97,  -223,    48,   101,
-     229,  -223,  -223,   133,   229,   229,   229,   229,   229,  -223,
-    -223,   493,    23,    79,  -223,   199,    60,  -223,  -223,  -223,
-     179,  -223,  -223,  -223,  -223,    27,   260,   181,   165,   158,
-    -223,   123,  -223,  -223,   289,   293,  -223,  -223,  -223,   526,
-     178,  -223,   232,  -223,  -223,   180,  -223,  -223,   306,  -223,
-    -223,  -223,  -223,  -223,  -223,   188,   242,   735,   320,   273,
-     248,  -223,   221,    22,  -223,  -223,  -223,  -223,  -223,  -223,
-    -223,   768,    -5,   229,   229,   223,   229,   229,   115,   194,
-     237,   229,   229,   405,  -223,  -223,   234,   229,  -223,  -223,
-    -223,   405,    51,   -22,  -223,   377,  -223,   229,  -223,   378,
-    -223,     7,  -223,    21,   158,   735,  -223,  -223,   229,   735,
-    -223,  -223,  -223,  -223,   735,   293,  -223,   229,   272,   -64,
-    -223,   375,  -223,   287,  -223,   138,  -223,   287,   229,    58,
-     229,   229,   265,  -223,   266,  -223,   148,   851,   614,   223,
-     405,   384,   385,  -223,  -223,   349,   379,   856,   155,    16,
-     735,    -2,  -223,   735,  -223,   336,   276,   331,   277,  -223,
-      15,   195,   111,  -223,   278,   195,    52,   334,  -223,  -223,
-      22,  -223,  -223,   280,   735,  -223,   261,   161,   229,  -223,
-     735,   281,  -223,   229,  -223,  -223,   283,   328,   333,   290,
-    -223,  -223,  -223,   117,   229,   303,    58,   229,  -223,   151,
-    -223,  -223,     5,    64,   405,   405,    25,  -223,  -223,  -223,
-    -223,  -223,  -223,  -223,  -223,   735,   298,   735,    13,  -223,
-     166,   309,   735,    38,  -223,   358,   261,  -223,  -223,   735,
-    -223,   129,   229,  -223,  -223,   335,  -223,   338,   339,   345,
-      21,  -223,   423,   424,   195,   393,   364,  -223,   173,  -223,
-     735,  -223,   261,  -223,  -223,   405,   312,   316,   229,   436,
-      24,   182,  -223,   190,   415,   157,  -223,   317,   326,  -223,
-     360,   324,   856,  -223,   369,   229,  -223,  -223,   151,  -223,
-    -223,   385,  -223,  -223,  -223,   735,   327,   246,   647,  -223,
-     261,   365,  -223,  -223,   856,   340,   261,   735,  -223,    26,
-    -223,  -223,  -223,  -223,  -223,    21,   111,   366,   367,  -223,
-     735,   405,   368,  -223,   261,    12,   229,   229,   192,  -223,
-    -223,  -223,  -223,  -223,  -223,  -223,   153,  -223,   229,  -223,
-    -223,  -223,  -223,   341,    58,   422,   371,  -223,   405,  -223,
-    -223,   344,  -223,   250,   647,  -223,   735,   198,  -223,  -223,
-     856,   261,  -223,   381,  -223,  -223,   337,   384,   431,   389,
-    -223,   204,   207,  -223,   468,    24,  -223,   229,  -223,  -223,
-     350,   435,  -223,    30,   229,   735,   210,   261,  -223,   213,
-     405,   735,   469,  -223,   380,  -223,  -223,  -223,   225,  -223,
-    -223,  -223,  -223,    11,   229,   118,  -223,   352,   261,  -223,
-    -223,   384,   353,  -223,   186,  -223,   229,  -223,   229,  -223,
-    -223,   229,  -223,   227,  -223,  -223,   356,  -223,   735,  -223,
-    -223,   397,   361,  -223,   247,  -223,   229,  -223,   -13,  -223,
-     229,  -223,   252,  -223,  -223,   257,   392,  -223,   479,  -223
+     198,  -216,  -216,   -46,   175,   -19,    46,   -27,    20,  -216,
+      32,   175,   175,  -216,    88,   127,  -216,  -216,  -216,  -216,
+    -216,  -216,  -216,  -216,  -216,  -216,    -8,  -216,   115,   114,
+     175,  -216,  -216,    54,   175,   175,   175,   175,   175,  -216,
+    -216,   527,    52,    34,  -216,   135,    67,  -216,  -216,  -216,
+     191,  -216,  -216,  -216,  -216,    37,   197,   117,   152,   180,
+    -216,    86,  -216,  -216,   293,   294,  -216,  -216,  -216,   560,
+     188,  -216,   236,  -216,  -216,   202,  -216,  -216,   312,  -216,
+    -216,  -216,  -216,  -216,  -216,   212,   260,   648,   334,   274,
+     214,  -216,  -216,   164,    -1,  -216,  -216,  -216,  -216,  -216,
+    -216,  -216,   769,    23,   175,   175,   224,   175,   175,    28,
+     171,   233,   175,   175,   439,  -216,  -216,   228,   175,  -216,
+    -216,  -216,   439,    27,   -17,  -216,   346,  -216,   175,  -216,
+     347,  -216,   231,     7,  -216,    15,   180,   802,  -216,  -216,
+     175,   802,  -216,  -216,  -216,  -216,   802,   294,  -216,   175,
+     273,    56,  -216,   348,  -216,   272,  -216,   101,  -216,   272,
+     175,   118,   175,   175,   250,  -216,   251,  -216,   111,   885,
+     681,   224,   406,   370,   374,  -216,  -216,  1000,   359,   890,
+     131,    17,   802,   -11,  -216,   802,  -216,   325,   261,   318,
+     262,  -216,  -216,    40,    75,   124,  -216,   263,    75,   -13,
+     321,  -216,  -216,    -1,  -216,  -216,   266,   802,  -216,   219,
+     133,   175,  -216,   802,  -216,   175,  -216,  -216,   269,   315,
+     316,   275,  -216,  -216,  -216,   144,   175,   287,   118,   175,
+    -216,    82,  -216,  -216,    -3,   350,   439,   439,    58,  -216,
+    -216,  -216,  -216,  -216,  -216,  -216,  -216,   802,   278,   802,
+      25,  -216,   137,   289,   802,    47,  -216,   345,   219,  -216,
+    -216,   802,  -216,    89,   175,  -216,  -216,   317,  -216,   319,
+     324,   326,    15,  -216,   399,   401,    75,   375,   352,  -216,
+     139,  -216,   802,  -216,   219,  -216,   439,   300,   301,   175,
+     412,     3,   141,  -216,   145,   391,    45,  -216,   303,   308,
+    -216,   349,   306,   890,  -216,   351,   175,  -216,  -216,    82,
+    -216,  -216,   374,  -216,  -216,  -216,   802,   313,   187,   648,
+    -216,   219,   356,  -216,  -216,   890,   327,   219,   802,  -216,
+      21,  -216,  -216,  -216,  -216,  -216,    15,   124,   339,   343,
+    -216,   802,   439,   353,  -216,   219,    12,   175,   175,   149,
+    -216,  -216,  -216,  -216,  -216,  -216,  -216,   193,  -216,   175,
+    -216,  -216,  -216,  -216,   328,   118,   414,   367,  -216,   439,
+    -216,  -216,   335,  -216,   221,   648,  -216,   802,   156,  -216,
+    -216,   890,   219,  -216,   376,  -216,  -216,   337,   370,   425,
+     383,  -216,   158,   162,  -216,   464,     3,  -216,   175,  -216,
+    -216,   354,   430,  -216,    19,   175,   802,   166,   219,  -216,
+     179,   439,   802,   467,  -216,   378,  -216,  -216,  -216,   183,
+    -216,  -216,  -216,  -216,    11,   175,   -14,  -216,   360,   219,
+    -216,  -216,   370,   358,  -216,    31,  -216,   175,  -216,   175,
+    -216,  -216,   175,  -216,   194,  -216,  -216,   355,  -216,   802,
+    -216,  -216,   400,   368,  -216,   199,  -216,   175,  -216,    38,
+    -216,   175,  -216,   205,  -216,  -216,   208,   385,  -216,   486,
+    -216
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -931,83 +932,84 @@ static const yytype_int16 yypact[] =
      means the default is an error.  */
 static const yytype_uint8 yydefact[] =
 {
-       0,     6,   254,     0,     0,     0,     0,     0,     0,    18,
+       0,     6,   255,     0,     0,     0,     0,     0,     0,    18,
      112,     0,     0,     7,     0,     0,    15,     8,    10,    11,
-      13,    14,     9,    17,    12,    16,     0,   105,     0,   252,
-       0,   246,   247,     0,     0,     0,     0,     0,     0,   113,
+      13,    14,     9,    17,    12,    16,     0,   105,     0,   253,
+       0,   247,   248,     0,     0,     0,     0,     0,     0,   113,
      114,     0,     0,   107,   108,     0,   145,     1,     3,     2,
-       0,   106,     5,     4,   253,     0,     0,     0,     0,   166,
-      25,     0,   219,   216,     0,   238,   115,    40,    29,     0,
+       0,   106,     5,     4,   254,     0,     0,     0,     0,   166,
+      25,     0,   220,   217,     0,   239,   115,    40,    29,     0,
        0,    30,    31,    34,    36,     0,    37,    39,     0,    41,
-     215,    35,    38,    32,    33,     0,     0,     0,     0,     0,
-     116,   117,   121,   187,   189,   191,   194,   195,   196,   193,
-     192,     0,   224,     0,     0,     0,     0,     0,     0,     0,
-      94,     0,     0,     0,   101,   167,     0,     0,    91,   217,
-     218,     0,     0,   211,   208,     0,    43,     0,   220,     0,
-      44,     0,   221,     0,   166,     0,   239,   240,     0,     0,
-     120,   242,   243,   241,     0,     0,   190,     0,     0,   166,
-     103,     0,   109,     0,   110,     0,   244,     0,     0,     0,
-       0,     0,     0,    93,    66,    27,     0,     0,     0,     0,
-       0,   168,   170,   172,   174,     0,   192,     0,     0,     0,
-       0,   211,   205,     0,   209,     0,     0,     0,     0,   197,
-       0,     0,   147,   136,   122,   141,   124,   149,   118,   119,
-     186,   188,   225,     0,     0,   198,   213,     0,     0,   100,
-       0,     0,   146,     0,    92,    19,     0,     0,     0,     0,
-      20,    21,    22,     0,     0,     0,    64,     0,    42,    56,
-     173,   181,     0,     0,     0,     0,     0,   228,   230,   231,
-     232,   233,   229,   234,   236,     0,     0,     0,     0,   222,
-       0,     0,     0,     0,   206,     0,   212,   204,    45,     0,
-      46,   127,     0,   137,   143,   133,   128,   129,   131,     0,
-       0,   140,     0,     0,   139,     0,   151,   199,     0,   200,
-       0,   102,   104,   123,   245,     0,     0,     0,     0,     0,
-       0,     0,   226,     0,   224,     0,    63,    65,    68,    28,
-       0,     0,     0,    47,     0,     0,    49,    55,    57,    26,
-     180,   169,   171,   235,   237,     0,     0,     0,     0,   182,
-     179,     0,   178,    90,     0,     0,   210,     0,   203,     0,
-     142,   144,   134,   130,   132,     0,   148,     0,     0,   138,
-       0,     0,   153,   201,   214,     0,     0,     0,     0,    96,
-     250,   251,   249,   248,    97,    95,     0,    67,     0,    83,
-      84,    85,    86,    87,     0,     0,    70,    48,     0,    51,
-      50,     0,    54,     0,     0,   184,     0,     0,   177,   223,
-       0,   207,   202,     0,   125,   126,   150,   152,     0,   155,
-      61,     0,     0,    58,     0,     0,   227,     0,    24,    62,
-       0,     0,    23,     0,     0,     0,     0,   175,   183,     0,
-       0,     0,     0,   111,     0,    59,    98,    99,     0,    74,
-      76,    77,    78,     0,     0,     0,    52,     0,   176,   185,
-      89,   135,   154,   157,   160,   156,     0,    88,     0,    82,
-      80,     0,    79,     0,    72,    73,     0,    53,     0,   161,
-     162,   163,     0,    75,     0,    69,     0,   158,     0,   159,
-       0,    81,     0,   164,   165,     0,     0,    60,     0,    71
+     216,    35,    38,    32,    33,     0,     0,     0,     0,     0,
+     116,   117,   198,   121,   187,   189,   191,   194,   195,   196,
+     193,   192,     0,   225,     0,     0,     0,     0,     0,     0,
+       0,    94,     0,     0,     0,   101,   167,     0,     0,    91,
+     218,   219,     0,     0,   212,   209,     0,    43,     0,   221,
+       0,    44,     0,     0,   222,     0,   166,     0,   240,   241,
+       0,     0,   120,   243,   244,   242,     0,     0,   190,     0,
+       0,   166,   103,     0,   109,     0,   110,     0,   245,     0,
+       0,     0,     0,     0,     0,    93,    66,    27,     0,     0,
+       0,     0,     0,   168,   170,   172,   174,     0,   192,     0,
+       0,     0,     0,   212,   206,     0,   210,     0,     0,     0,
+       0,   123,   197,     0,     0,   147,   136,   122,   141,   124,
+     149,   118,   119,   186,   188,   226,     0,     0,   199,   214,
+       0,     0,   100,     0,   146,     0,    92,    19,     0,     0,
+       0,     0,    20,    21,    22,     0,     0,     0,    64,     0,
+      42,    56,   173,   181,     0,     0,     0,     0,     0,   229,
+     231,   232,   233,   234,   230,   235,   237,     0,     0,     0,
+       0,   223,     0,     0,     0,     0,   207,     0,   213,   205,
+      45,     0,    46,   127,     0,   137,   143,   133,   128,   129,
+     131,     0,     0,   140,     0,     0,   139,     0,   151,   200,
+       0,   201,     0,   102,   104,   246,     0,     0,     0,     0,
+       0,     0,     0,   227,     0,   225,     0,    63,    65,    68,
+      28,     0,     0,     0,    47,     0,     0,    49,    55,    57,
+      26,   180,   169,   171,   236,   238,     0,     0,     0,     0,
+     182,   179,     0,   178,    90,     0,     0,   211,     0,   204,
+       0,   142,   144,   134,   130,   132,     0,   148,     0,     0,
+     138,     0,     0,   153,   202,   215,     0,     0,     0,     0,
+      96,   251,   252,   250,   249,    97,    95,     0,    67,     0,
+      83,    84,    85,    86,    87,     0,     0,    70,    48,     0,
+      51,    50,     0,    54,     0,     0,   184,     0,     0,   177,
+     224,     0,   208,   203,     0,   125,   126,   150,   152,     0,
+     155,    61,     0,     0,    58,     0,     0,   228,     0,    24,
+      62,     0,     0,    23,     0,     0,     0,     0,   175,   183,
+       0,     0,     0,     0,   111,     0,    59,    98,    99,     0,
+      74,    76,    77,    78,     0,     0,     0,    52,     0,   176,
+     185,    89,   135,   154,   157,   160,   156,     0,    88,     0,
+      82,    80,     0,    79,     0,    72,    73,     0,    53,     0,
+     161,   162,   163,     0,    75,     0,    69,     0,   158,     0,
+     159,     0,    81,     0,   164,   165,     0,     0,    60,     0,
+      71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -223,  -223,  -223,  -223,  -223,  -223,  -223,  -223,  -121,  -223,
-     321,   183,  -223,  -223,  -222,  -223,  -223,  -223,  -223,  -223,
-    -223,    68,    49,  -223,  -223,  -223,  -223,  -223,  -223,  -223,
-    -223,  -223,  -223,  -223,  -223,   285,  -223,  -223,  -223,   390,
-       9,  -223,  -223,  -223,   370,  -223,  -100,  -223,  -223,  -149,
-     160,  -143,    -9,  -223,  -223,  -223,  -223,  -223,  -223,    53,
-    -223,  -223,   107,  -223,  -120,   262,   268,   342,   -30,   372,
-     362,   403,  -123,  -223,  -223,  -223,   347,  -223,   394,   355,
-    -192,  -161,   127,  -107,  -223,  -223,  -223,  -223,  -223,  -115,
-      -4,   113,  -223,  -223
+    -216,  -216,  -216,  -216,  -216,  -216,  -216,  -216,  -102,  -216,
+     329,   185,  -216,  -216,  -215,  -216,  -216,  -216,  -216,  -216,
+    -216,    71,    60,  -216,  -216,  -216,  -216,  -216,  -216,  -216,
+    -216,  -216,  -216,  -216,  -216,   286,  -216,  -216,  -216,   397,
+      10,  -216,  -216,  -216,   369,  -216,   -97,  -216,  -216,  -138,
+     169,  -161,   -10,  -216,  -216,  -216,  -216,  -216,  -216,    59,
+    -216,  -216,   -91,  -216,  -121,   276,   280,   341,   -30,   377,
+     363,   417,  -123,  -216,  -216,  -216,   344,  -216,   396,   357,
+    -203,  -163,   147,  -108,  -216,  -216,  -216,  -216,  -216,  -115,
+      -4,   126,  -216,  -216
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
-      -1,    14,    15,    16,    17,    18,    19,    20,   165,   166,
-      88,   307,   308,   309,   220,   297,   298,   225,   366,   402,
-     446,   418,   419,   420,   421,   422,   363,   398,    21,    22,
-     163,   291,    23,    24,   149,   150,    25,    26,    43,    44,
-     211,    41,    89,    90,    91,   134,   191,   274,   269,   192,
-     193,   263,   264,   194,   276,   342,   389,   413,   432,   433,
-     451,   459,   114,   115,   171,   172,   173,   174,   175,    93,
-      94,    95,    96,    97,    98,   181,   182,   123,   124,   185,
-     207,    99,   250,   100,   293,   247,   101,   139,   144,   155,
-     102,   354,    28,    29
+      -1,    14,    15,    16,    17,    18,    19,    20,   167,   168,
+      88,   308,   309,   310,   222,   298,   299,   227,   367,   403,
+     447,   419,   420,   421,   422,   423,   364,   399,    21,    22,
+     165,   292,    23,    24,   151,   152,    25,    26,    43,    44,
+     132,    41,    89,    90,    91,   136,    92,   276,   271,   195,
+     196,   265,   266,   197,   278,   343,   390,   414,   433,   434,
+     452,   460,   115,   116,   173,   174,   175,   176,   177,    94,
+      95,    96,    97,    98,    99,   183,   184,   124,   125,   187,
+     210,   100,   252,   101,   294,   249,   102,   141,   146,   157,
+     103,   355,    28,    29
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
@@ -1015,206 +1017,226 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      33,   179,   178,    45,   296,   154,   176,    42,    46,    27,
-     195,    92,   278,   234,   176,    31,   249,    32,   439,    31,
-     234,    32,   147,   321,   234,    31,    55,    32,   136,   137,
-      57,    58,    59,    60,    61,    51,   183,   215,   234,   122,
-     440,   261,   313,   314,   315,   141,   142,   136,   137,    30,
-     232,   463,   271,   113,    37,   108,   183,   131,    34,   136,
-     137,   176,   208,   176,     1,    39,     2,   195,   464,   231,
-     118,    36,   136,   137,   236,   237,   238,   239,   240,   241,
-     242,   243,   244,   245,   109,   136,   137,   350,   140,   272,
-      35,    40,    38,     3,   121,    45,    47,   316,   322,   151,
-      46,   216,   156,   157,    54,    92,   299,   164,   167,     4,
-       5,   351,   352,   156,   180,     6,    10,   292,   206,   148,
-       7,   336,   252,   187,   217,   103,   377,   176,   176,   196,
-     310,   339,   189,   353,   199,   441,   246,   390,     8,   190,
-     233,   369,   399,   202,   327,   190,   319,   195,   386,   143,
-     253,   382,   218,   256,   167,   426,   221,   222,     9,   158,
-     159,   300,   273,   379,    10,   345,   214,   180,    52,   219,
-     289,    53,    50,   348,   206,    11,   148,   290,   176,   265,
-     282,    12,   406,    13,   106,   266,   196,    46,   444,   189,
-    -127,    46,   359,   267,   301,   360,   361,   265,    10,    31,
-      56,    32,   302,   266,   151,   104,   394,   136,   137,   284,
-     268,   267,   195,   395,   445,   317,   375,   320,   449,   249,
-     294,   387,   326,   167,    10,    31,   262,    32,   268,   329,
-     105,   391,   392,    31,   176,    32,   303,   116,   160,   161,
-     450,   197,   136,   137,    48,   304,    49,   117,   403,   305,
-     344,   396,   138,   331,   330,   376,   209,   107,    46,   405,
-     362,   176,   306,   212,   213,   110,   196,   136,   137,   111,
-      46,   136,   137,   226,   227,   113,    31,    62,    32,    63,
-     251,   213,   136,   137,   156,   373,   279,   280,   206,   112,
-     431,   323,   324,    64,    65,   203,   119,   381,   343,   280,
-     120,   371,   125,   176,   127,    67,    68,   355,   356,   126,
-     206,   128,   129,    69,    70,   357,   358,   393,   213,   130,
-      71,    72,    73,   408,   280,   132,   454,   204,    74,   414,
-     213,   196,   415,   213,    75,   429,   280,    76,   430,   324,
-     133,   462,   156,   156,   206,   465,   407,   153,    77,    78,
-     437,   438,   455,   438,   294,   162,    79,    80,   177,   236,
-     237,   238,   239,   240,   241,   242,   243,   244,   245,    81,
-     136,   137,   461,   213,   135,   428,    82,   466,   213,    83,
-      84,   434,   467,   213,   186,   188,   210,    85,    10,   223,
-     224,    86,   234,   423,   235,   257,    87,   205,   259,   248,
-     427,   258,   260,   275,   270,   277,   283,   285,   286,    31,
-      62,    32,    63,   287,   288,   168,   295,   328,   434,   442,
-     423,   246,   318,   325,   335,   332,    64,    65,   333,   334,
-     337,   338,   452,   340,   423,   341,   346,   156,    67,    68,
-     347,   349,   147,   364,   365,   367,    69,    70,   368,   370,
-     378,   374,   156,    71,    72,    73,   156,   388,   400,   384,
-     385,    74,   401,   280,   380,   397,   169,    75,   404,   410,
-      76,   411,   412,   416,   424,   425,   435,   447,   436,   448,
-     456,    77,    78,   458,   468,   460,   469,   453,   229,    79,
-      80,   372,   443,   281,   152,   383,   311,    31,    62,    32,
-      63,   457,    81,   312,   146,   198,   201,   409,   417,    82,
-     230,   200,    83,    84,    64,    65,    66,   184,     0,     0,
-      85,     0,     0,     0,    86,     0,    67,    68,   254,   170,
-      31,    62,    32,    63,    69,    70,   255,     0,     0,     0,
-       0,    71,    72,    73,     0,     0,     0,    64,    65,    74,
-       0,     0,     0,     0,     0,    75,     0,     0,    76,    67,
-      68,     0,     0,     0,     0,     0,     0,    69,    70,    77,
-      78,     0,     0,     0,    71,    72,    73,    79,    80,     0,
-       0,     0,    74,     0,     0,     0,     0,     0,    75,     0,
-      81,    76,     0,     0,     0,     0,     0,    82,     0,     0,
-      83,    84,    77,    78,     0,     0,     0,     0,    85,     0,
-      79,    80,    86,     0,     0,     0,     0,    87,    31,    62,
-      32,    63,     0,    81,     0,     0,     0,     0,     0,     0,
-      82,     0,     0,    83,    84,    64,    65,     0,     0,     0,
-       0,    85,   121,     0,     0,    86,     0,    67,    68,     0,
-      87,    31,    62,    32,    63,    69,    70,     0,     0,     0,
-       0,     0,    71,    72,    73,     0,     0,     0,    64,    65,
-      74,     0,     0,     0,     0,   169,    75,     0,     0,    76,
-      67,    68,     0,     0,     0,     0,     0,     0,    69,    70,
-      77,    78,     0,     0,     0,    71,    72,    73,    79,    80,
-       0,     0,     0,    74,     0,     0,     0,     0,     0,    75,
-       0,    81,    76,     0,     0,     0,     0,     0,    82,     0,
-       0,    83,    84,    77,    78,     0,     0,     0,     0,    85,
-       0,    79,    80,    86,     0,     0,     0,     0,   170,    31,
-      62,    32,    63,     0,    81,     0,     0,     0,    10,     0,
-       0,    82,     0,     0,    83,    84,    64,    65,     0,     0,
-       0,     0,    85,     0,     0,     0,    86,     0,    67,    68,
-       0,    87,    31,    62,    32,    63,    69,    70,     0,     0,
-       0,     0,     0,    71,    72,    73,     0,     0,     0,    64,
-     145,    74,     0,     0,     0,     0,     0,    75,     0,     0,
-      76,    67,    68,     0,     0,     0,     0,     0,     0,    69,
-      70,    77,    78,     0,     0,     0,    71,    72,    73,    79,
-      80,     0,     0,     0,    74,     0,     0,     0,     0,     0,
-      75,     0,    81,    76,     0,     0,     0,     0,     0,    82,
-       0,     0,    83,    84,    77,    78,     0,     0,     0,     0,
-      85,     0,    79,    80,    86,     0,     0,     0,     0,    87,
-       0,    62,     0,    63,     0,    81,     0,     0,     0,     0,
-       0,     0,    82,     0,     0,    83,    84,    64,   145,     0,
-       0,     0,     0,    85,    67,    68,     0,    86,     0,    67,
-      68,     0,    87,    70,     0,     0,     0,     0,    70,    71,
-      72,    73,     0,     0,    71,    72,    73,    74,     0,     0,
-       0,     0,    74,     0,     0,     0,    76,     0,     0,     0,
-       0,    76,     0,     0,     0,     0,     0,    77,   228,     0,
-       0,     0,    77,    78,     0,    79,     0,     0,     0,     0,
-      79,    80,     0,     0,     0,     0,     0,     0,    81,     0,
-       0,     0,     0,    81,     0,    82,     0,     0,    83,    84,
-      82,     0,     0,    83,    84,     0,    85,     0,     0,     0,
-      86,    85,     0,     0,     0,    86
+      33,   181,    45,   180,   280,   236,   178,    42,    46,   156,
+      27,    93,   198,   297,   178,    31,   251,    32,   440,    31,
+     236,    32,   143,   144,   274,   236,    55,   236,   138,   139,
+      57,    58,    59,    60,    61,   322,    51,   273,   194,   123,
+     441,   185,   138,   139,    31,   200,    32,   185,   138,   139,
+     149,   234,   138,   139,    34,   263,   445,   133,   217,    30,
+     212,    39,   178,   450,   178,   109,   351,    50,   138,   139,
+     198,   119,   160,   161,   233,   314,   315,   316,    37,    31,
+     360,    32,   446,   361,   362,   451,    35,    40,    47,   142,
+     352,   353,   301,    10,   110,    45,   194,   275,    38,   122,
+     153,    46,   464,   158,   159,   182,   264,    93,   166,   169,
+     323,   150,   354,    36,   158,   340,   378,    54,   293,   465,
+     209,    56,   311,   254,   189,   302,   145,   300,   178,   178,
+     317,   199,   192,   303,   337,   442,   202,   391,   387,   193,
+     370,    10,   235,   182,   427,   205,   383,   150,   363,   198,
+     400,   320,   255,   328,   104,   258,   169,   267,   223,   224,
+     105,   218,   380,   268,   193,   346,   106,   304,    31,   216,
+      32,   269,   407,   114,   349,   194,   305,   209,   178,    31,
+     306,    32,   211,   284,   219,   138,   139,    10,   270,   199,
+      46,   107,   267,   307,    46,   140,   377,   290,   268,     1,
+     117,     2,   111,  -127,   291,   112,   269,   153,   138,   139,
+     118,   285,   220,   198,   331,   162,   163,   318,   251,   321,
+     376,   388,   295,   270,   327,   169,   214,   215,     3,   221,
+     406,   330,   392,   393,   178,    52,   228,   229,    53,   194,
+     138,   139,   138,   139,     4,     5,   395,    48,   404,    49,
+       6,   397,   345,   396,   332,     7,   253,   215,   281,   282,
+      46,   178,   324,   325,   344,   282,   356,   357,   199,   108,
+     358,   359,    46,     8,   394,   215,   113,    31,    62,    32,
+      63,   409,   282,   415,   215,   158,   374,   416,   215,   209,
+     432,   430,   282,     9,    64,    65,   206,   114,   382,    10,
+     120,   121,   372,   178,   431,   325,    67,    68,   438,   439,
+      11,   209,   126,   127,    69,    70,    12,   129,    13,   456,
+     439,    71,    72,    73,   462,   215,   128,   455,   207,    74,
+     467,   215,   199,   468,   215,    75,   130,   131,    76,   134,
+     137,   135,   463,   158,   158,   209,   466,   408,   155,    77,
+      78,   164,   179,   188,   190,   295,   191,    79,    80,   213,
+     238,   239,   240,   241,   242,   243,   244,   245,   246,   247,
+      81,   138,   139,    10,   225,   226,   429,    82,   236,   250,
+      83,    84,   435,   237,   259,   261,   260,   262,    85,   272,
+     277,   279,    86,   286,   424,   287,   288,    87,   208,   289,
+     296,   428,   319,   326,   329,   336,   338,   333,   339,   334,
+      31,    62,    32,    63,   335,   341,   170,   350,   149,   435,
+     443,   424,   248,   342,   347,   348,   366,    64,    65,   365,
+     369,   371,   385,   453,   368,   424,   386,   375,   158,    67,
+      68,   379,   389,    31,    62,    32,    63,    69,    70,   170,
+     401,   381,   398,   158,    71,    72,    73,   158,   402,   405,
+      64,    65,    74,   282,   411,   412,   413,   171,    75,   417,
+     426,    76,    67,    68,   436,   192,   437,   469,   425,   457,
+      69,    70,    77,    78,   449,   448,   459,    71,    72,    73,
+      79,    80,   461,   470,   373,    74,   444,   283,   231,   454,
+     171,    75,   154,    81,    76,   384,   201,    10,   458,   204,
+      82,   232,   312,    83,    84,    77,    78,   313,   203,   148,
+     186,    85,   418,    79,    80,    86,     0,   256,   410,     0,
+     172,    31,    62,    32,    63,     0,    81,     0,     0,     0,
+     257,     0,     0,    82,     0,     0,    83,    84,    64,    65,
+      66,     0,     0,     0,    85,     0,     0,     0,    86,     0,
+      67,    68,     0,   172,    31,    62,    32,    63,    69,    70,
+       0,     0,     0,     0,     0,    71,    72,    73,     0,     0,
+       0,    64,    65,    74,     0,     0,     0,     0,     0,    75,
+       0,     0,    76,    67,    68,     0,     0,     0,     0,     0,
+       0,    69,    70,    77,    78,     0,     0,     0,    71,    72,
+      73,    79,    80,     0,     0,     0,    74,     0,     0,     0,
+       0,     0,    75,     0,    81,    76,     0,     0,     0,     0,
+       0,    82,     0,     0,    83,    84,    77,    78,     0,     0,
+       0,     0,    85,     0,    79,    80,    86,     0,     0,     0,
+       0,    87,    31,    62,    32,    63,     0,    81,     0,     0,
+       0,     0,     0,     0,    82,     0,     0,    83,    84,    64,
+      65,     0,     0,     0,     0,    85,   122,     0,     0,    86,
+       0,    67,    68,     0,    87,    31,    62,    32,    63,    69,
+      70,     0,     0,     0,     0,     0,    71,    72,    73,     0,
+       0,     0,    64,    65,    74,     0,     0,     0,     0,     0,
+      75,     0,     0,    76,    67,    68,     0,     0,     0,     0,
+       0,     0,    69,    70,    77,    78,     0,     0,     0,    71,
+      72,    73,    79,    80,     0,     0,     0,    74,     0,     0,
+       0,     0,   171,    75,     0,    81,    76,     0,     0,    10,
+       0,     0,    82,     0,     0,    83,    84,    77,    78,     0,
+       0,     0,     0,    85,     0,    79,    80,    86,     0,     0,
+       0,     0,    87,    31,    62,    32,    63,     0,    81,     0,
+       0,     0,     0,     0,     0,    82,     0,     0,    83,    84,
+      64,   147,     0,     0,     0,     0,    85,     0,     0,     0,
+      86,     0,    67,    68,     0,   172,    31,    62,    32,    63,
+      69,    70,     0,     0,     0,     0,     0,    71,    72,    73,
+       0,     0,     0,    64,    65,    74,     0,     0,     0,     0,
+       0,    75,     0,     0,    76,    67,    68,     0,     0,     0,
+       0,     0,     0,    69,    70,    77,    78,     0,     0,     0,
+      71,    72,    73,    79,    80,     0,     0,     0,    74,     0,
+       0,     0,     0,     0,    75,     0,    81,    76,     0,     0,
+       0,     0,     0,    82,     0,     0,    83,    84,    77,    78,
+       0,     0,     0,     0,    85,     0,    79,    80,    86,     0,
+       0,     0,     0,    87,     0,    62,     0,    63,     0,    81,
+       0,     0,     0,     0,     0,     0,    82,     0,     0,    83,
+      84,    64,   147,     0,     0,     0,     0,    85,    67,    68,
+       0,    86,     0,    67,    68,     0,    87,    70,     0,     0,
+       0,     0,    70,    71,    72,    73,     0,     0,    71,    72,
+      73,    74,     0,     0,     0,     0,    74,     0,     0,     0,
+      76,     0,     0,     0,     0,    76,     0,     0,     0,     0,
+       0,    77,   230,     0,     0,     0,    77,    78,     0,    79,
+       0,     0,     0,     0,    79,    80,     0,     0,     0,     0,
+       0,     0,    81,     0,     0,     0,     0,    81,     0,    82,
+       0,     0,    83,    84,    82,     0,     0,    83,    84,     0,
+      85,     0,     0,     0,    86,    85,     0,     0,     0,    86,
+     238,   239,   240,   241,   242,   243,   244,   245,   246,   247,
+       0,   138,   139,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,   248
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   121,   117,    12,   226,   105,   113,    11,    12,     0,
-     133,    41,   204,     8,   121,     4,   177,     6,     7,     4,
-       8,     6,    27,    10,     8,     4,    30,     6,    21,    22,
-      34,    35,    36,    37,    38,    26,    58,   158,     8,    69,
-      29,   190,    17,    18,    19,    23,    24,    21,    22,   105,
-     170,    64,   195,   117,   105,    28,    58,    87,    73,    21,
-      22,   168,   126,   170,     1,    29,     3,   190,    81,   169,
-      61,    67,    21,    22,    10,    11,    12,    13,    14,    15,
-      16,    17,    18,    19,    57,    21,    22,    63,    92,    37,
-     105,    55,    78,    30,   116,   104,     0,    72,    85,   103,
-     104,    43,   106,   107,     3,   135,   227,   111,   112,    46,
-      47,    87,    88,   117,   116,    52,   101,   224,   148,   124,
-      57,   270,   106,   127,    66,   102,   318,   234,   235,   133,
-     125,   274,   125,   109,   138,   124,    72,   125,    75,   124,
-     170,   302,   364,   147,   106,   124,   246,   270,   340,   127,
-     180,   125,    94,   183,   158,   125,   160,   161,    95,    44,
-      45,    10,   110,   324,   101,   285,   157,   116,   120,   111,
-      53,   123,    75,   288,   204,   112,   124,    60,   285,    68,
-     210,   118,   374,   120,   124,    74,   190,   191,    70,   125,
-      79,   195,    35,    82,    43,    38,    39,    68,   101,     4,
-      67,     6,    51,    74,   208,   126,    53,    21,    22,   213,
-      99,    82,   335,    60,    96,   245,   316,   247,    32,   380,
-     224,   341,   252,   227,   101,     4,    31,     6,    99,   259,
-      31,   346,   347,     4,   341,     6,    85,   114,    44,    45,
-      54,   134,    21,    22,   120,    94,   122,   124,   368,    98,
-     280,   358,    31,   262,   125,     9,   149,    78,   262,     9,
-     103,   368,   111,   125,   126,     5,   270,    21,    22,    88,
-     274,    21,    22,   125,   126,   117,     4,     5,     6,     7,
-     125,   126,    21,    22,   288,   315,   125,   126,   318,   124,
-     410,   125,   126,    21,    22,    23,     7,   327,   125,   126,
-       7,   305,   124,   410,   124,    33,    34,   125,   126,    77,
-     340,     5,   124,    41,    42,   125,   126,   125,   126,    77,
-      48,    49,    50,   125,   126,     5,   441,    55,    56,   125,
-     126,   335,   125,   126,    62,   125,   126,    65,   125,   126,
-      67,   456,   346,   347,   374,   460,   376,   124,    76,    77,
-     125,   126,   125,   126,   358,   118,    84,    85,   124,    10,
-      11,    12,    13,    14,    15,    16,    17,    18,    19,    97,
-      21,    22,   125,   126,   126,   405,   104,   125,   126,   107,
-     108,   411,   125,   126,     7,     7,    11,   115,   101,   124,
-     124,   119,     8,   397,     9,    59,   124,   125,    67,    20,
-     404,   125,   125,    69,   126,   125,   125,   124,    80,     4,
-       5,     6,     7,    80,   124,    10,   113,    59,   448,   423,
-     424,    72,   124,   114,    79,    90,    21,    22,    90,    90,
-       7,     7,   436,    40,   438,    71,   124,   441,    33,    34,
-     124,     5,    27,   126,   118,    85,    41,    42,   124,    80,
-      85,   124,   456,    48,    49,    50,   460,    89,    36,    93,
-      93,    56,    91,   126,   124,   124,    61,    62,   124,    88,
-      65,    40,    83,     5,   124,    40,     7,   125,    98,   126,
-     124,    76,    77,    86,    92,   124,     7,   438,   167,    84,
-      85,   308,   424,   208,   104,   335,   234,     4,     5,     6,
-       7,   448,    97,   235,   101,   135,   144,   380,   395,   104,
-     168,   139,   107,   108,    21,    22,    23,   123,    -1,    -1,
-     115,    -1,    -1,    -1,   119,    -1,    33,    34,   181,   124,
-       4,     5,     6,     7,    41,    42,   181,    -1,    -1,    -1,
-      -1,    48,    49,    50,    -1,    -1,    -1,    21,    22,    56,
-      -1,    -1,    -1,    -1,    -1,    62,    -1,    -1,    65,    33,
-      34,    -1,    -1,    -1,    -1,    -1,    -1,    41,    42,    76,
-      77,    -1,    -1,    -1,    48,    49,    50,    84,    85,    -1,
-      -1,    -1,    56,    -1,    -1,    -1,    -1,    -1,    62,    -1,
-      97,    65,    -1,    -1,    -1,    -1,    -1,   104,    -1,    -1,
-     107,   108,    76,    77,    -1,    -1,    -1,    -1,   115,    -1,
-      84,    85,   119,    -1,    -1,    -1,    -1,   124,     4,     5,
-       6,     7,    -1,    97,    -1,    -1,    -1,    -1,    -1,    -1,
-     104,    -1,    -1,   107,   108,    21,    22,    -1,    -1,    -1,
-      -1,   115,   116,    -1,    -1,   119,    -1,    33,    34,    -1,
-     124,     4,     5,     6,     7,    41,    42,    -1,    -1,    -1,
-      -1,    -1,    48,    49,    50,    -1,    -1,    -1,    21,    22,
-      56,    -1,    -1,    -1,    -1,    61,    62,    -1,    -1,    65,
-      33,    34,    -1,    -1,    -1,    -1,    -1,    -1,    41,    42,
-      76,    77,    -1,    -1,    -1,    48,    49,    50,    84,    85,
-      -1,    -1,    -1,    56,    -1,    -1,    -1,    -1,    -1,    62,
-      -1,    97,    65,    -1,    -1,    -1,    -1,    -1,   104,    -1,
-      -1,   107,   108,    76,    77,    -1,    -1,    -1,    -1,   115,
-      -1,    84,    85,   119,    -1,    -1,    -1,    -1,   124,     4,
-       5,     6,     7,    -1,    97,    -1,    -1,    -1,   101,    -1,
-      -1,   104,    -1,    -1,   107,   108,    21,    22,    -1,    -1,
-      -1,    -1,   115,    -1,    -1,    -1,   119,    -1,    33,    34,
-      -1,   124,     4,     5,     6,     7,    41,    42,    -1,    -1,
-      -1,    -1,    -1,    48,    49,    50,    -1,    -1,    -1,    21,
-      22,    56,    -1,    -1,    -1,    -1,    -1,    62,    -1,    -1,
-      65,    33,    34,    -1,    -1,    -1,    -1,    -1,    -1,    41,
-      42,    76,    77,    -1,    -1,    -1,    48,    49,    50,    84,
-      85,    -1,    -1,    -1,    56,    -1,    -1,    -1,    -1,    -1,
-      62,    -1,    97,    65,    -1,    -1,    -1,    -1,    -1,   104,
-      -1,    -1,   107,   108,    76,    77,    -1,    -1,    -1,    -1,
-     115,    -1,    84,    85,   119,    -1,    -1,    -1,    -1,   124,
-      -1,     5,    -1,     7,    -1,    97,    -1,    -1,    -1,    -1,
-      -1,    -1,   104,    -1,    -1,   107,   108,    21,    22,    -1,
-      -1,    -1,    -1,   115,    33,    34,    -1,   119,    -1,    33,
-      34,    -1,   124,    42,    -1,    -1,    -1,    -1,    42,    48,
-      49,    50,    -1,    -1,    48,    49,    50,    56,    -1,    -1,
-      -1,    -1,    56,    -1,    -1,    -1,    65,    -1,    -1,    -1,
-      -1,    65,    -1,    -1,    -1,    -1,    -1,    76,    77,    -1,
-      -1,    -1,    76,    77,    -1,    84,    -1,    -1,    -1,    -1,
-      84,    85,    -1,    -1,    -1,    -1,    -1,    -1,    97,    -1,
-      -1,    -1,    -1,    97,    -1,   104,    -1,    -1,   107,   108,
-     104,    -1,    -1,   107,   108,    -1,   115,    -1,    -1,    -1,
-     119,   115,    -1,    -1,    -1,   119
+       4,   122,    12,   118,   207,     8,   114,    11,    12,   106,
+       0,    41,   135,   228,   122,     4,   179,     6,     7,     4,
+       8,     6,    23,    24,    37,     8,    30,     8,    21,    22,
+      34,    35,    36,    37,    38,    10,    26,   198,   135,    69,
+      29,    58,    21,    22,     4,   136,     6,    58,    21,    22,
+      27,   172,    21,    22,    73,   193,    70,    87,   160,   105,
+     151,    29,   170,    32,   172,    28,    63,    75,    21,    22,
+     193,    61,    44,    45,   171,    17,    18,    19,   105,     4,
+      35,     6,    96,    38,    39,    54,   105,    55,     0,    93,
+      87,    88,    10,   101,    57,   105,   193,   110,    78,   116,
+     104,   105,    64,   107,   108,   116,    31,   137,   112,   113,
+      85,   124,   109,    67,   118,   276,   319,     3,   226,    81,
+     150,    67,   125,   106,   128,    43,   127,   229,   236,   237,
+      72,   135,   125,    51,   272,   124,   140,   125,   341,   124,
+     303,   101,   172,   116,   125,   149,   125,   124,   103,   272,
+     365,   248,   182,   106,   102,   185,   160,    68,   162,   163,
+     126,    43,   325,    74,   124,   286,    31,    85,     4,   159,
+       6,    82,   375,   117,   289,   272,    94,   207,   286,     4,
+      98,     6,   126,   213,    66,    21,    22,   101,    99,   193,
+     194,   124,    68,   111,   198,    31,     9,    53,    74,     1,
+     114,     3,     5,    79,    60,    88,    82,   211,    21,    22,
+     124,   215,    94,   336,   125,    44,    45,   247,   381,   249,
+     317,   342,   226,    99,   254,   229,   125,   126,    30,   111,
+       9,   261,   347,   348,   342,   120,   125,   126,   123,   336,
+      21,    22,    21,    22,    46,    47,    53,   120,   369,   122,
+      52,   359,   282,    60,   264,    57,   125,   126,   125,   126,
+     264,   369,   125,   126,   125,   126,   125,   126,   272,    78,
+     125,   126,   276,    75,   125,   126,   124,     4,     5,     6,
+       7,   125,   126,   125,   126,   289,   316,   125,   126,   319,
+     411,   125,   126,    95,    21,    22,    23,   117,   328,   101,
+       7,     7,   306,   411,   125,   126,    33,    34,   125,   126,
+     112,   341,   124,    77,    41,    42,   118,     5,   120,   125,
+     126,    48,    49,    50,   125,   126,   124,   442,    55,    56,
+     125,   126,   336,   125,   126,    62,   124,    77,    65,     5,
+     126,    67,   457,   347,   348,   375,   461,   377,   124,    76,
+      77,   118,   124,     7,     7,   359,   125,    84,    85,    11,
+      10,    11,    12,    13,    14,    15,    16,    17,    18,    19,
+      97,    21,    22,   101,   124,   124,   406,   104,     8,    20,
+     107,   108,   412,     9,    59,    67,   125,   125,   115,   126,
+      69,   125,   119,   124,   398,    80,    80,   124,   125,   124,
+     113,   405,   124,   114,    59,    79,     7,    90,     7,    90,
+       4,     5,     6,     7,    90,    40,    10,     5,    27,   449,
+     424,   425,    72,    71,   124,   124,   118,    21,    22,   126,
+     124,    80,    93,   437,    85,   439,    93,   124,   442,    33,
+      34,    85,    89,     4,     5,     6,     7,    41,    42,    10,
+      36,   124,   124,   457,    48,    49,    50,   461,    91,   124,
+      21,    22,    56,   126,    88,    40,    83,    61,    62,     5,
+      40,    65,    33,    34,     7,   125,    98,    92,   124,   124,
+      41,    42,    76,    77,   126,   125,    86,    48,    49,    50,
+      84,    85,   124,     7,   309,    56,   425,   211,   169,   439,
+      61,    62,   105,    97,    65,   336,   137,   101,   449,   146,
+     104,   170,   236,   107,   108,    76,    77,   237,   141,   102,
+     124,   115,   396,    84,    85,   119,    -1,   183,   381,    -1,
+     124,     4,     5,     6,     7,    -1,    97,    -1,    -1,    -1,
+     183,    -1,    -1,   104,    -1,    -1,   107,   108,    21,    22,
+      23,    -1,    -1,    -1,   115,    -1,    -1,    -1,   119,    -1,
+      33,    34,    -1,   124,     4,     5,     6,     7,    41,    42,
+      -1,    -1,    -1,    -1,    -1,    48,    49,    50,    -1,    -1,
+      -1,    21,    22,    56,    -1,    -1,    -1,    -1,    -1,    62,
+      -1,    -1,    65,    33,    34,    -1,    -1,    -1,    -1,    -1,
+      -1,    41,    42,    76,    77,    -1,    -1,    -1,    48,    49,
+      50,    84,    85,    -1,    -1,    -1,    56,    -1,    -1,    -1,
+      -1,    -1,    62,    -1,    97,    65,    -1,    -1,    -1,    -1,
+      -1,   104,    -1,    -1,   107,   108,    76,    77,    -1,    -1,
+      -1,    -1,   115,    -1,    84,    85,   119,    -1,    -1,    -1,
+      -1,   124,     4,     5,     6,     7,    -1,    97,    -1,    -1,
+      -1,    -1,    -1,    -1,   104,    -1,    -1,   107,   108,    21,
+      22,    -1,    -1,    -1,    -1,   115,   116,    -1,    -1,   119,
+      -1,    33,    34,    -1,   124,     4,     5,     6,     7,    41,
+      42,    -1,    -1,    -1,    -1,    -1,    48,    49,    50,    -1,
+      -1,    -1,    21,    22,    56,    -1,    -1,    -1,    -1,    -1,
+      62,    -1,    -1,    65,    33,    34,    -1,    -1,    -1,    -1,
+      -1,    -1,    41,    42,    76,    77,    -1,    -1,    -1,    48,
+      49,    50,    84,    85,    -1,    -1,    -1,    56,    -1,    -1,
+      -1,    -1,    61,    62,    -1,    97,    65,    -1,    -1,   101,
+      -1,    -1,   104,    -1,    -1,   107,   108,    76,    77,    -1,
+      -1,    -1,    -1,   115,    -1,    84,    85,   119,    -1,    -1,
+      -1,    -1,   124,     4,     5,     6,     7,    -1,    97,    -1,
+      -1,    -1,    -1,    -1,    -1,   104,    -1,    -1,   107,   108,
+      21,    22,    -1,    -1,    -1,    -1,   115,    -1,    -1,    -1,
+     119,    -1,    33,    34,    -1,   124,     4,     5,     6,     7,
+      41,    42,    -1,    -1,    -1,    -1,    -1,    48,    49,    50,
+      -1,    -1,    -1,    21,    22,    56,    -1,    -1,    -1,    -1,
+      -1,    62,    -1,    -1,    65,    33,    34,    -1,    -1,    -1,
+      -1,    -1,    -1,    41,    42,    76,    77,    -1,    -1,    -1,
+      48,    49,    50,    84,    85,    -1,    -1,    -1,    56,    -1,
+      -1,    -1,    -1,    -1,    62,    -1,    97,    65,    -1,    -1,
+      -1,    -1,    -1,   104,    -1,    -1,   107,   108,    76,    77,
+      -1,    -1,    -1,    -1,   115,    -1,    84,    85,   119,    -1,
+      -1,    -1,    -1,   124,    -1,     5,    -1,     7,    -1,    97,
+      -1,    -1,    -1,    -1,    -1,    -1,   104,    -1,    -1,   107,
+     108,    21,    22,    -1,    -1,    -1,    -1,   115,    33,    34,
+      -1,   119,    -1,    33,    34,    -1,   124,    42,    -1,    -1,
+      -1,    -1,    42,    48,    49,    50,    -1,    -1,    48,    49,
+      50,    56,    -1,    -1,    -1,    -1,    56,    -1,    -1,    -1,
+      65,    -1,    -1,    -1,    -1,    65,    -1,    -1,    -1,    -1,
+      -1,    76,    77,    -1,    -1,    -1,    76,    77,    -1,    84,
+      -1,    -1,    -1,    -1,    84,    85,    -1,    -1,    -1,    -1,
+      -1,    -1,    97,    -1,    -1,    -1,    -1,    97,    -1,   104,
+      -1,    -1,   107,   108,   104,    -1,    -1,   107,   108,    -1,
+     115,    -1,    -1,    -1,   119,   115,    -1,    -1,    -1,   119,
+      10,    11,    12,    13,    14,    15,    16,    17,    18,    19,
+      -1,    21,    22,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    72
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
@@ -1230,44 +1252,45 @@ static const yytype_uint8 yystos[] =
      218,   218,     5,     7,    21,    22,    23,    33,    34,    41,
       42,    48,    49,    50,    56,    62,    65,    76,    77,    84,
       85,    97,   104,   107,   108,   115,   119,   124,   138,   170,
-     171,   172,   196,   197,   198,   199,   200,   201,   202,   209,
-     211,   214,   218,   102,   126,    31,   124,    78,    28,    57,
-       5,    88,   124,   117,   190,   191,   114,   124,   168,     7,
-       7,   116,   196,   205,   206,   124,    77,   124,     5,   124,
-      77,   196,     5,    67,   173,   126,    21,    22,    31,   215,
-     218,    23,    24,   127,   216,    22,   199,    27,   124,   162,
-     163,   218,   167,   124,   174,   217,   218,   218,    44,    45,
-      44,    45,   118,   158,   218,   136,   137,   218,    10,    61,
-     124,   192,   193,   194,   195,   196,   211,   124,   217,   192,
-     116,   203,   204,    58,   206,   207,     7,   218,     7,   125,
-     124,   174,   177,   178,   181,   200,   218,   190,   172,   218,
-     197,   198,   218,    23,    55,   125,   196,   208,   126,   190,
-      11,   168,   125,   126,   168,   136,    43,    66,    94,   111,
-     142,   218,   218,   124,   124,   145,   125,   126,    77,   138,
-     195,   174,   192,   196,     8,     9,    10,    11,    12,    13,
-      14,    15,    16,    17,    18,    19,    72,   213,    20,   209,
-     210,   125,   106,   196,   204,   207,   196,    59,   125,    67,
-     125,   177,    31,   179,   180,    68,    74,    82,    99,   176,
-     126,   179,    37,   110,   175,    69,   182,   125,   208,   125,
-     126,   163,   196,   125,   218,   124,    80,    80,   124,    53,
-      60,   159,   211,   212,   218,   113,   142,   143,   144,   136,
-      10,    43,    51,    85,    94,    98,   111,   139,   140,   141,
-     125,   193,   194,    17,    18,    19,    72,   196,   124,   174,
-     196,    10,    85,   125,   126,   114,   196,   106,    59,   196,
-     125,   180,    90,    90,    90,    79,   177,     7,     7,   179,
-      40,    71,   183,   125,   196,   192,   124,   124,   217,     5,
-      63,    87,    88,   109,   219,   125,   126,   125,   126,    35,
-      38,    39,   103,   154,   126,   118,   146,    85,   124,   209,
-      80,   218,   139,   196,   124,   174,     9,   208,    85,   209,
-     124,   196,   125,   178,    93,    93,   208,   192,    89,   184,
-     125,   217,   217,   125,    53,    60,   211,   124,   155,   142,
-      36,    91,   147,   192,   124,     9,   208,   196,   125,   210,
-      88,    40,    83,   185,   125,   125,     5,   219,   149,   150,
-     151,   152,   153,   218,   124,    40,   125,   218,   196,   125,
-     125,   192,   186,   187,   196,     7,    98,   125,   126,     7,
-      29,   124,   218,   149,    70,    96,   148,   125,   126,    32,
-      54,   188,   218,   150,   217,   125,   124,   187,    86,   189,
-     124,   125,   217,    64,    81,   217,   125,   125,    92,     7
+     171,   172,   174,   196,   197,   198,   199,   200,   201,   202,
+     209,   211,   214,   218,   102,   126,    31,   124,    78,    28,
+      57,     5,    88,   124,   117,   190,   191,   114,   124,   168,
+       7,     7,   116,   196,   205,   206,   124,    77,   124,     5,
+     124,    77,   168,   196,     5,    67,   173,   126,    21,    22,
+      31,   215,   218,    23,    24,   127,   216,    22,   199,    27,
+     124,   162,   163,   218,   167,   124,   174,   217,   218,   218,
+      44,    45,    44,    45,   118,   158,   218,   136,   137,   218,
+      10,    61,   124,   192,   193,   194,   195,   196,   211,   124,
+     217,   192,   116,   203,   204,    58,   206,   207,     7,   218,
+       7,   125,   125,   124,   174,   177,   178,   181,   200,   218,
+     190,   172,   218,   197,   198,   218,    23,    55,   125,   196,
+     208,   126,   190,    11,   125,   126,   168,   136,    43,    66,
+      94,   111,   142,   218,   218,   124,   124,   145,   125,   126,
+      77,   138,   195,   174,   192,   196,     8,     9,    10,    11,
+      12,    13,    14,    15,    16,    17,    18,    19,    72,   213,
+      20,   209,   210,   125,   106,   196,   204,   207,   196,    59,
+     125,    67,   125,   177,    31,   179,   180,    68,    74,    82,
+      99,   176,   126,   179,    37,   110,   175,    69,   182,   125,
+     208,   125,   126,   163,   196,   218,   124,    80,    80,   124,
+      53,    60,   159,   211,   212,   218,   113,   142,   143,   144,
+     136,    10,    43,    51,    85,    94,    98,   111,   139,   140,
+     141,   125,   193,   194,    17,    18,    19,    72,   196,   124,
+     174,   196,    10,    85,   125,   126,   114,   196,   106,    59,
+     196,   125,   180,    90,    90,    90,    79,   177,     7,     7,
+     179,    40,    71,   183,   125,   196,   192,   124,   124,   217,
+       5,    63,    87,    88,   109,   219,   125,   126,   125,   126,
+      35,    38,    39,   103,   154,   126,   118,   146,    85,   124,
+     209,    80,   218,   139,   196,   124,   174,     9,   208,    85,
+     209,   124,   196,   125,   178,    93,    93,   208,   192,    89,
+     184,   125,   217,   217,   125,    53,    60,   211,   124,   155,
+     142,    36,    91,   147,   192,   124,     9,   208,   196,   125,
+     210,    88,    40,    83,   185,   125,   125,     5,   219,   149,
+     150,   151,   152,   153,   218,   124,    40,   125,   218,   196,
+     125,   125,   192,   186,   187,   196,     7,    98,   125,   126,
+       7,    29,   124,   218,   149,    70,    96,   148,   125,   126,
+      32,    54,   188,   218,   150,   217,   125,   124,   187,    86,
+     189,   124,   125,   217,    64,    81,   217,   125,   125,    92,
+       7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
@@ -1292,13 +1315,13 @@ static const yytype_uint8 yyr1[] =
      188,   188,   188,   189,   189,   189,   190,   190,   191,   192,
      192,   193,   193,   194,   194,   195,   195,   195,   195,   195,
      195,   195,   195,   195,   195,   195,   196,   196,   197,   197,
-     198,   198,   199,   199,   199,   199,   199,   199,   200,   200,
-     200,   200,   201,   202,   202,   203,   203,   204,   205,   205,
-     206,   207,   207,   208,   208,   209,   209,   209,   209,   209,
-     209,   209,   210,   210,   211,   211,   212,   212,   213,   213,
-     213,   213,   213,   213,   213,   213,   213,   213,   214,   215,
-     215,   216,   216,   216,   217,   217,   218,   218,   219,   219,
-     219,   219,   220,   221,   221
+     198,   198,   199,   199,   199,   199,   199,   199,   199,   200,
+     200,   200,   200,   201,   202,   202,   203,   203,   204,   205,
+     205,   206,   207,   207,   208,   208,   209,   209,   209,   209,
+     209,   209,   209,   210,   210,   211,   211,   212,   212,   213,
+     213,   213,   213,   213,   213,   213,   213,   213,   213,   214,
+     215,   215,   216,   216,   216,   217,   217,   218,   218,   219,
+     219,   219,   219,   220,   221,   221
 };
 
   /* YYR2[YYN] -- Number of symbols on the right hand side of rule YYN.  */
@@ -1323,13 +1346,13 @@ static const yytype_uint8 yyr2[] =
        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,     1,     1,     3,     3,     4,
-       4,     5,     6,     5,     4,     1,     2,     4,     1,     2,
-       4,     0,     2,     1,     3,     1,     1,     2,     2,     1,
-       2,     2,     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
+       2,     1,     1,     1,     1,     1,     1,     3,     1,     3,
+       4,     4,     5,     6,     5,     4,     1,     2,     4,     1,
+       2,     4,     0,     2,     1,     3,     1,     1,     2,     2,
+       1,     2,     2,     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
 };
 
 
@@ -1832,7 +1855,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1836 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1859 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 4: /* TOKEN_NAME  */
@@ -1842,7 +1865,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1846 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1869 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 5: /* TOKEN_STRING_SINGLE_QUOTED  */
@@ -1852,7 +1875,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1856 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1879 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 6: /* TOKEN_STRING_DOUBLE_QUOTED  */
@@ -1862,7 +1885,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1866 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1889 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 7: /* TOKEN_UNSIGNED_NUMVAL  */
@@ -1872,7 +1895,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).numeric_literal_value_);
   }
 }
-#line 1876 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1899 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 130: /* sql_statement  */
@@ -1882,7 +1905,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).statement_);
   }
 }
-#line 1886 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1909 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 131: /* quit_statement  */
@@ -1892,7 +1915,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).quit_statement_);
   }
 }
-#line 1896 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1919 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 132: /* alter_table_statement  */
@@ -1902,7 +1925,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).statement_);
   }
 }
-#line 1906 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1929 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 133: /* create_table_statement  */
@@ -1912,7 +1935,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).create_table_statement_);
   }
 }
-#line 1916 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1939 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 134: /* create_index_statement  */
@@ -1922,7 +1945,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).statement_);
   }
 }
-#line 1926 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1949 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 135: /* drop_table_statement  */
@@ -1932,7 +1955,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).drop_table_statement_);
   }
 }
-#line 1936 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1959 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 136: /* column_def  */
@@ -1942,7 +1965,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_definition_);
   }
 }
-#line 1946 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1969 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 137: /* column_def_commalist  */
@@ -1952,7 +1975,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_definition_list_);
   }
 }
-#line 1956 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1979 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 138: /* data_type  */
@@ -1962,7 +1985,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).data_type_);
   }
 }
-#line 1966 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1989 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 139: /* column_constraint_def  */
@@ -1972,7 +1995,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).column_constraint_);
   }
 }
-#line 1976 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1999 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 140: /* column_constraint_def_list  */
@@ -1982,7 +2005,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 1986 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2009 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 141: /* opt_column_constraint_def_list  */
@@ -1992,7 +2015,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 1996 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2019 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 145: /* opt_column_list  */
@@ -2002,7 +2025,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_list_);
   }
 }
-#line 2006 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2029 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 146: /* opt_block_properties  */
@@ -2012,7 +2035,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).block_properties_);
   }
 }
-#line 2016 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2039 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 147: /* opt_partition_clause  */
@@ -2022,7 +2045,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).partition_clause_);
   }
 }
-#line 2026 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2049 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 148: /* partition_type  */
@@ -2032,7 +2055,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2036 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2059 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 149: /* key_value_list  */
@@ -2042,7 +2065,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2046 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2069 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 150: /* key_value  */
@@ -2052,7 +2075,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_value_);
   }
 }
-#line 2056 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2079 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 151: /* key_string_value  */
@@ -2062,7 +2085,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_string_value_);
   }
 }
-#line 2066 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2089 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 152: /* key_string_list  */
@@ -2072,7 +2095,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_string_list_);
   }
 }
-#line 2076 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2099 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 153: /* key_integer_value  */
@@ -2082,7 +2105,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_integer_value_);
   }
 }
-#line 2086 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2109 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 154: /* index_type  */
@@ -2092,7 +2115,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2096 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2119 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 155: /* opt_index_properties  */
@@ -2102,7 +2125,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2106 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2129 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 156: /* insert_statement  */
@@ -2112,7 +2135,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).insert_statement_);
   }
 }
-#line 2116 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2139 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 157: /* copy_from_statement  */
@@ -2122,7 +2145,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).copy_from_statement_);
   }
 }
-#line 2126 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2149 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 158: /* opt_copy_from_params  */
@@ -2132,7 +2155,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).copy_from_params_);
   }
 }
-#line 2136 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2159 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 159: /* copy_from_params  */
@@ -2142,7 +2165,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).copy_from_params_);
   }
 }
-#line 2146 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2169 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 160: /* update_statement  */
@@ -2152,7 +2175,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).update_statement_);
   }
 }
-#line 2156 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2179 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 161: /* delete_statement  */
@@ -2162,7 +2185,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).delete_statement_);
   }
 }
-#line 2166 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2189 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 162: /* assignment_list  */
@@ -2172,7 +2195,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).assignment_list_);
   }
 }
-#line 2176 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2199 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 163: /* assignment_item  */
@@ -2182,7 +2205,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).assignment_);
   }
 }
-#line 2186 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2209 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 164: /* select_statement  */
@@ -2192,7 +2215,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).select_statement_);
   }
 }
-#line 2196 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2219 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 165: /* with_clause  */
@@ -2202,7 +2225,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).with_list_);
   }
 }
-#line 2206 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2229 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 166: /* with_list  */
@@ -2212,7 +2235,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).with_list_);
   }
 }
-#line 2216 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2239 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 167: /* with_list_element  */
@@ -2222,7 +2245,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).with_list_element_);
   }
 }
-#line 2226 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2249 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 168: /* select_query  */
@@ -2232,7 +2255,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).select_query_);
   }
 }
-#line 2236 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2259 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 170: /* selection  */
@@ -2242,7 +2265,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).selection_);
   }
 }
-#line 2246 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2269 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 171: /* selection_item_commalist  */
@@ -2252,7 +2275,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).selection_list_);
   }
 }
-#line 2256 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2279 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 172: /* selection_item  */
@@ -2262,7 +2285,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).selection_item_);
   }
 }
-#line 2266 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2289 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 173: /* from_clause  */
@@ -2272,7 +2295,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_list_);
   }
 }
-#line 2276 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2299 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 174: /* subquery_expression  */
@@ -2282,7 +2305,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).subquery_expression_);
   }
 }
-#line 2286 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2309 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 175: /* opt_sample_clause  */
@@ -2292,13 +2315,13 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_sample_clause_);
   }
 }
-#line 2296 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2319 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 176: /* join_type  */
 #line 564 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2302 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2325 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 177: /* joined_table_reference  */
@@ -2308,7 +2331,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_);
   }
 }
-#line 2312 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2335 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 178: /* table_reference  */
@@ -2318,7 +2341,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_);
   }
 }
-#line 2322 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2345 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 179: /* table_reference_signature  */
@@ -2328,7 +2351,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_signature_);
   }
 }
-#line 2332 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2355 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 180: /* table_reference_signature_primary  */
@@ -2338,7 +2361,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_signature_);
   }
 }
-#line 2342 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2365 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 181: /* joined_table_reference_commalist  */
@@ -2348,7 +2371,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_list_);
   }
 }
-#line 2352 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2375 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 182: /* opt_group_by_clause  */
@@ -2358,7 +2381,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_group_by_clause_);
   }
 }
-#line 2362 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2385 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 183: /* opt_having_clause  */
@@ -2368,7 +2391,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_having_clause_);
   }
 }
-#line 2372 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2395 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 184: /* opt_order_by_clause  */
@@ -2378,7 +2401,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_order_by_clause_);
   }
 }
-#line 2382 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2405 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 185: /* opt_limit_clause  */
@@ -2388,7 +2411,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_limit_clause_);
   }
 }
-#line 2392 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2415 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 186: /* order_commalist  */
@@ -2398,7 +2421,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).order_commalist_);
   }
 }
-#line 2402 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2425 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 187: /* order_item  */
@@ -2408,7 +2431,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).order_item_);
   }
 }
-#line 2412 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2435 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 188: /* opt_order_direction  */
@@ -2418,7 +2441,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).order_direction_);
   }
 }
-#line 2422 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2445 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 189: /* opt_nulls_first  */
@@ -2428,7 +2451,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).order_direction_);
   }
 }
-#line 2432 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2455 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 190: /* opt_where_clause  */
@@ -2438,7 +2461,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2442 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2465 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 191: /* where_clause  */
@@ -2448,7 +2471,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2452 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2475 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 192: /* or_expression  */
@@ -2458,7 +2481,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2462 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2485 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 193: /* and_expression  */
@@ -2468,7 +2491,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2472 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2495 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 194: /* not_expression  */
@@ -2478,7 +2501,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2482 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2505 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 195: /* predicate_expression_base  */
@@ -2488,7 +2511,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2492 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2515 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 196: /* add_expression  */
@@ -2498,7 +2521,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2502 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2525 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 197: /* multiply_expression  */
@@ -2508,7 +2531,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2512 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2535 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 198: /* unary_expression  */
@@ -2518,7 +2541,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2522 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2545 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 199: /* expression_base  */
@@ -2528,7 +2551,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2532 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2555 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 200: /* function_call  */
@@ -2538,7 +2561,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).function_call_);
   }
 }
-#line 2542 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2565 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 201: /* extract_function  */
@@ -2548,7 +2571,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2552 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2575 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 202: /* case_expression  */
@@ -2558,7 +2581,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2562 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2585 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 203: /* simple_when_clause_list  */
@@ -2568,7 +2591,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).simple_when_clause_list_);
   }
 }
-#line 2572 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2595 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 204: /* simple_when_clause  */
@@ -2578,7 +2601,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).simple_when_clause_);
   }
 }
-#line 2582 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2605 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 205: /* searched_when_clause_list  */
@@ -2588,7 +2611,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).searched_when_clause_list_);
   }
 }
-#line 2592 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2615 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 206: /* searched_when_clause  */
@@ -2598,7 +2621,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).searched_when_clause_);
   }
 }
-#line 2602 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2625 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 207: /* opt_else_clause  */
@@ -2608,7 +2631,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2612 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2635 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 208: /* expression_list  */
@@ -2618,7 +2641,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_list_);
   }
 }
-#line 2622 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2645 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 209: /* literal_value  */
@@ -2628,7 +2651,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).literal_value_);
   }
 }
-#line 2632 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2655 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 210: /* literal_value_commalist  */
@@ -2638,7 +2661,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).literal_value_list_);
   }
 }
-#line 2642 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2665 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 211: /* attribute_ref  */
@@ -2648,7 +2671,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_);
   }
 }
-#line 2652 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2675 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 212: /* attribute_ref_list  */
@@ -2658,31 +2681,31 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_list_);
   }
 }
-#line 2662 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2685 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 213: /* comparison_operation  */
 #line 561 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2668 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2691 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 214: /* unary_operation  */
 #line 562 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2674 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2697 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 215: /* add_operation  */
 #line 563 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2680 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2703 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 216: /* multiply_operation  */
 #line 563 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2686 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2709 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 217: /* name_commalist  */
@@ -2692,7 +2715,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_list_);
   }
 }
-#line 2696 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2719 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 218: /* any_name  */
@@ -2702,13 +2725,13 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2706 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2729 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 219: /* boolean_value  */
 #line 560 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2712 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2735 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 220: /* command  */
@@ -2718,7 +2741,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).command_);
   }
 }
-#line 2722 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2745 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 221: /* command_argument_list  */
@@ -2728,7 +2751,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).command_argument_list_);
   }
 }
-#line 2732 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2755 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
 
@@ -3025,7 +3048,7 @@ yyreduce:
     *parsedStatement = (yyvsp[-1].statement_);
     YYACCEPT;
   }
-#line 3029 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3052 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 3:
@@ -3034,7 +3057,7 @@ yyreduce:
     *parsedStatement = (yyvsp[-1].statement_);
     YYACCEPT;
   }
-#line 3038 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3061 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 4:
@@ -3043,7 +3066,7 @@ yyreduce:
     *parsedStatement = (yyvsp[-1].command_);
     YYACCEPT;
   }
-#line 3047 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3070 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 5:
@@ -3052,7 +3075,7 @@ yyreduce:
     *parsedStatement = (yyvsp[-1].command_);
     YYACCEPT;
   }
-#line 3056 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3079 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 6:
@@ -3060,7 +3083,7 @@ yyreduce:
     {
     YYABORT;
   }
-#line 3064 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3087 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 7:
@@ -3069,7 +3092,7 @@ yyreduce:
     // Regular yyparse() return codes are non-negative, so use a negative one here.
     return -1;
   }
-#line 3073 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3096 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 8:
@@ -3077,7 +3100,7 @@ yyreduce:
     {
     (yyval.statement_) = (yyvsp[0].statement_);
   }
-#line 3081 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3104 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 9:
@@ -3085,7 +3108,7 @@ yyreduce:
     {
     (yyval.statement_) = (yyvsp[0].copy_from_statement_);
   }
-#line 3089 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3112 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 10:
@@ -3093,7 +3116,7 @@ yyreduce:
     {
     (yyval.statement_) = (yyvsp[0].create_table_statement_);
   }
-#line 3097 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3120 "SqlParser_gen.cpp" /* yacc.c:1661  */


<TRUNCATED>


[12/30] incubator-quickstep git commit: Merge pull request #170 from pivotalsoftware/quickstep_date_support

Posted by zu...@apache.org.
Merge pull request #170 from pivotalsoftware/quickstep_date_support

Quickstep date support

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

Branch: refs/heads/master
Commit: 0f79a9af25f38687bb9ddd506a858b1ce7926b45
Parents: 7187c86
Author: Rogers Jeffrey Leo John <ro...@gmail.com>
Authored: Fri Apr 22 13:07:04 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:34 2016 -0700

----------------------------------------------------------------------
 parser/CMakeLists.txt                           |    3 +-
 parser/ParseLiteralValue.cpp                    |    1 +
 parser/SqlLexer.lpp                             |    6 +
 parser/SqlParser.ypp                            |   44 +-
 parser/preprocessed/SqlLexer_gen.cpp            | 1248 ++++----
 parser/preprocessed/SqlLexer_gen.hpp            |    2 +-
 parser/preprocessed/SqlParser_gen.cpp           | 2796 +++++++++---------
 parser/preprocessed/SqlParser_gen.hpp           |  154 +-
 parser/tests/TPCH.test                          |   10 +-
 .../tests/execution_generator/Select.test       |    4 +-
 10 files changed, 2238 insertions(+), 2030 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index d35f3be..9738c2c 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -199,7 +199,8 @@ target_link_libraries(quickstep_parser_ParseLiteralValue
                       quickstep_types_TypedValue
                       quickstep_types_VarCharType
                       quickstep_types_YearMonthIntervalType
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      quickstep_utility_SqlError)
 target_link_libraries(quickstep_parser_ParseOrderBy
                       quickstep_parser_ParseExpression
                       quickstep_parser_ParseTreeNode

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/parser/ParseLiteralValue.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseLiteralValue.cpp b/parser/ParseLiteralValue.cpp
index 1462244..f839bed 100644
--- a/parser/ParseLiteralValue.cpp
+++ b/parser/ParseLiteralValue.cpp
@@ -37,6 +37,7 @@
 #include "types/Type.hpp"
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
+#include "utility/SqlError.hpp"
 
 #include "glog/logging.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index 3043322..a399723 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -191,6 +191,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "create"           return TOKEN_CREATE;
   "date"             return TOKEN_DATE;
   "datetime"         return TOKEN_DATETIME;
+  "day"              return TOKEN_DAY;
   "decimal"          return TOKEN_DECIMAL;
   "default"          return TOKEN_DEFAULT;
   "delete"           return TOKEN_DELETE;
@@ -214,6 +215,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "group"            return TOKEN_GROUP;
   "hash"             return TOKEN_HASH;
   "having"           return TOKEN_HAVING;
+  "hour"             return TOKEN_HOUR;
   "in"               return TOKEN_IN;
   "index"            return TOKEN_INDEX;
   "inner"            return TOKEN_INNER;
@@ -230,6 +232,8 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "like"             return TOKEN_LIKE;
   "limit"            return TOKEN_LIMIT;
   "long"             return TOKEN_LONG;
+  "minute"           return TOKEN_MINUTE;
+  "month"            return TOKEN_MONTH;
   "not"              return TOKEN_NOT;
   "null"             return TOKEN_NULL;
   "nulls"            return TOKEN_NULLS;
@@ -249,6 +253,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "regexp"           return TOKEN_REGEXP;
   "right"            return TOKEN_RIGHT;
   "row_delimiter"    return TOKEN_ROW_DELIMITER;
+  "second"           return TOKEN_SECOND;
   "select"           return TOKEN_SELECT;
   "set"              return TOKEN_SET;
   "sma"              return TOKEN_SMA;
@@ -267,6 +272,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "when"             return TOKEN_WHEN;
   "where"            return TOKEN_WHERE;
   "with"             return TOKEN_WITH;
+  "year"             return TOKEN_YEAR;
   "yearmonth"        return TOKEN_YEARMONTH;
 
   "="                return TOKEN_EQ;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index a8ec821..1202d66 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -197,7 +197,7 @@ typedef void* yyscan_t;
   quickstep::ParseOrderBy *opt_order_by_clause_;
   bool *order_direction_;
   quickstep::ParseLimit *opt_limit_clause_;
-  
+
   quickstep::ParseSample *opt_sample_clause_;
 
   quickstep::PtrList<quickstep::ParseOrderByItem> *order_commalist_;
@@ -256,6 +256,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_CREATE;
 %token TOKEN_DATE;
 %token TOKEN_DATETIME;
+%token TOKEN_DAY;
 %token TOKEN_DECIMAL;
 %token TOKEN_DEFAULT;
 %token TOKEN_DELETE;
@@ -278,6 +279,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_GROUP;
 %token TOKEN_HASH;
 %token TOKEN_HAVING;
+%token TOKEN_HOUR;
 %token TOKEN_IN;
 %token TOKEN_INDEX;
 %token TOKEN_INNER;
@@ -291,6 +293,8 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_LEFT;
 %token TOKEN_LIMIT;
 %token TOKEN_LONG;
+%token TOKEN_MINUTE;
+%token TOKEN_MONTH;
 %token TOKEN_NOT;
 %token TOKEN_NULL;
 %token TOKEN_NULLS;
@@ -310,6 +314,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_REGEXP;
 %token TOKEN_RIGHT;
 %token TOKEN_ROW_DELIMITER;
+%token TOKEN_SECOND;
 %token TOKEN_SELECT;
 %token TOKEN_SET;
 %token TOKEN_SMA;
@@ -328,12 +333,14 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_WHEN;
 %token TOKEN_WHERE;
 %token TOKEN_WITH;
+%token TOKEN_YEAR;
 %token TOKEN_YEARMONTH;
 %token TOKEN_EOF;
 %token TOKEN_LEX_ERROR;
 
 %type <string_value_>
   any_name
+  datetime_unit
   index_type
   partition_type
 
@@ -1525,7 +1532,7 @@ function_call:
   };
 
 extract_function:
-  TOKEN_EXTRACT '(' any_name TOKEN_FROM add_expression ')' {
+  TOKEN_EXTRACT '(' datetime_unit TOKEN_FROM add_expression ')' {
     $$ = new quickstep::ParseExtractFunction(@1.first_line, @1.first_column, $3, $5);
   };
 
@@ -1630,6 +1637,19 @@ literal_value:
       YYERROR;
     }
   }
+  | TOKEN_INTERVAL TOKEN_STRING_SINGLE_QUOTED datetime_unit {
+    quickstep::StringParseLiteralValue *parse_value;
+    const std::string &datetime_type_value = $3->value();
+    if (quickstep::StringParseLiteralValue::ParseAmbiguousInterval(
+        &($2->append((" " + datetime_type_value).c_str(), datetime_type_value.length() + 1)),
+        &parse_value)) {
+      $$ = parse_value;
+    } else {
+      $$ = nullptr;
+      quickstep_yyerror(&@3, yyscanner, nullptr, "Failed to parse literal as specified type");
+      YYERROR;
+    }
+  }
   | data_type TOKEN_STRING_SINGLE_QUOTED {
     quickstep::StringParseLiteralValue *parse_value
         = new quickstep::StringParseLiteralValue($2, &($1->getType()));
@@ -1642,6 +1662,26 @@ literal_value:
     } else {
       $$ = parse_value;
     }
+  }
+
+datetime_unit:
+  TOKEN_YEAR {
+     $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("YEAR"));
+  }
+  | TOKEN_MONTH {
+     $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("MONTH"));
+  }
+  | TOKEN_DAY {
+     $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("DAY"));
+  }
+  | TOKEN_HOUR {
+     $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("HOUR"));
+  }
+  | TOKEN_MINUTE {
+     $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("MINUTE"));
+  }
+  | TOKEN_SECOND {
+     $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("SECOND"));
   };
 
 literal_value_commalist:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index 553a7d9..d836988 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -381,8 +381,8 @@ static void yy_fatal_error (yyconst char msg[] ,yyscan_t yyscanner );
 	*yy_cp = '\0'; \
 	yyg->yy_c_buf_p = yy_cp;
 
-#define YY_NUM_RULES 144
-#define YY_END_OF_BUFFER 145
+#define YY_NUM_RULES 150
+#define YY_END_OF_BUFFER 151
 /* This struct is not used in this scanner,
    but its presence is necessary. */
 struct yy_trans_info
@@ -390,66 +390,68 @@ struct yy_trans_info
 	flex_int32_t yy_verify;
 	flex_int32_t yy_nxt;
 	};
-static yyconst flex_int16_t yy_accept[527] =
+static yyconst flex_int16_t yy_accept[545] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
-        0,    0,  145,    2,    2,  143,  143,  142,  141,  143,
-      120,  116,  119,  116,  116,  139,  112,  109,  113,  138,
-      138,  138,  138,  138,  138,  138,  138,  138,  138,  138,
-      138,  138,  138,  138,  138,  138,  138,  138,  138,  138,
-      138,  138,  138,  117,    4,    5,    5,    3,  135,  135,
-      132,  136,  136,  130,  137,  137,  134,    1,  142,  110,
-      140,  139,  139,  139,    0,  114,  111,  115,  138,  138,
-      138,  138,   10,  138,  138,  138,   22,  138,  138,  138,
-      138,  138,  138,  138,  138,  138,  138,  118,  138,  138,
-
-      138,  138,  138,  138,  138,  138,  138,  138,  138,  138,
-       55,   63,  138,  138,  138,  138,  138,  138,  138,  138,
-      138,   75,   76,  138,  138,  138,  138,  138,  138,  138,
-      138,  138,  138,  138,  138,  138,  138,  138,  138,  138,
-      138,  138,  138,  138,  138,  138,    4,    5,    3,  135,
-      131,  136,  129,  129,  121,  123,  124,  125,  126,  127,
-      128,  129,  137,  133,  140,  139,    0,  139,    6,    7,
-      138,    9,   11,  138,  138,   15,  138,  138,  138,  138,
-      138,  138,  138,  138,  138,  138,  138,  138,  138,  138,
-      138,  138,  138,  138,   42,  138,  138,  138,  138,  138,
-
-      138,  138,  138,  138,  138,  138,  138,  138,  138,  138,
-       59,  138,   65,  138,  138,  138,  138,  138,   71,  138,
-       74,  138,  138,  138,  138,  138,  138,  138,  138,  138,
-      138,  138,  138,  138,   91,   92,  138,  138,  138,  138,
-      138,  138,  138,  138,  138,  138,  138,  138,  138,  121,
-      123,  122,  138,  138,  138,  138,  138,  138,  138,   20,
-       23,  138,  138,  138,   28,  138,  138,   30,  138,  138,
-      138,  138,   36,  138,  138,   40,   41,  138,  138,  138,
-      138,  138,  138,  138,   50,   51,  138,   53,  138,  138,
-      138,  138,  138,   62,   64,   66,   67,   68,  138,   70,
-
-       72,  138,  138,  138,  138,  138,   83,  138,   85,  138,
-      138,  138,  138,  138,  138,  138,   95,   96,   98,  138,
-      138,  138,  138,  138,  138,  105,  138,  107,  138,  121,
-      122,    8,  138,  138,  138,  138,  138,  138,  138,   25,
-      138,  138,  138,  138,  138,  138,  138,  138,  138,  138,
-      138,  138,  138,  138,  138,   46,   47,   48,  138,   52,
-      138,   56,   57,  138,  138,  138,   69,   73,   77,   78,
-      138,  138,  138,   84,  138,  138,   88,  138,  138,  138,
-       94,  138,  138,  138,  138,  102,  138,  138,  106,  138,
-      138,  138,   14,  138,  138,  138,  138,  138,   26,  138,
-
-       29,  138,  138,  138,  138,   34,  138,  138,  138,   39,
-      138,   44,  138,  138,   54,   58,  138,  138,  138,  138,
-      138,  138,   87,  138,   90,  138,  138,  138,  100,  101,
-      103,  138,  138,  138,   13,  138,  138,  138,  138,  138,
-      138,   21,  138,   32,   33,  138,  138,  138,  138,   45,
-       49,   60,  138,  138,   81,   82,  138,  138,  138,  138,
-      138,  104,  138,  138,  138,  138,  138,  138,  138,  138,
-       31,  138,  138,   38,  138,   61,  138,  138,  138,   93,
-      138,  138,  138,   12,  138,  138,  138,  138,   24,  138,
-       35,  138,  138,   79,  138,  138,   97,  138,  108,   16,
-
-      138,  138,  138,   27,   37,  138,   80,   86,  138,  138,
-      138,   18,   19,  138,  138,   99,  138,  138,  138,  138,
-      138,   89,  138,   43,   17,    0
+        0,    0,  151,    2,    2,  149,  149,  148,  147,  149,
+      126,  122,  125,  122,  122,  145,  118,  115,  119,  144,
+      144,  144,  144,  144,  144,  144,  144,  144,  144,  144,
+      144,  144,  144,  144,  144,  144,  144,  144,  144,  144,
+      144,  144,  144,  144,  123,    4,    5,    5,    3,  141,
+      141,  138,  142,  142,  136,  143,  143,  140,    1,  148,
+      116,  146,  145,  145,  145,    0,  120,  117,  121,  144,
+      144,  144,  144,   10,  144,  144,  144,   22,  144,  144,
+      144,  144,  144,  144,  144,  144,  144,  144,  124,  144,
+
+      144,  144,  144,  144,  144,  144,  144,  144,  144,  144,
+      144,  144,   57,   65,  144,  144,  144,  144,  144,  144,
+      144,  144,  144,  144,  144,   79,   80,  144,  144,  144,
+      144,  144,  144,  144,  144,  144,  144,  144,  144,  144,
+      144,  144,  144,  144,  144,  144,  144,  144,  144,  144,
+        4,    5,    3,  141,  137,  142,  135,  135,  127,  129,
+      130,  131,  132,  133,  134,  135,  143,  139,  146,  145,
+        0,  145,    6,    7,  144,    9,   11,  144,  144,   15,
+      144,  144,  144,  144,  144,  144,  144,  144,  144,  144,
+       32,  144,  144,  144,  144,  144,  144,  144,  144,   43,
+
+      144,  144,  144,  144,  144,  144,  144,  144,  144,  144,
+      144,  144,  144,  144,  144,  144,   61,  144,   67,  144,
+      144,  144,  144,  144,  144,  144,   75,  144,   78,  144,
+      144,  144,  144,  144,  144,  144,  144,  144,  144,  144,
+      144,  144,  144,   96,   97,  144,  144,  144,  144,  144,
+      144,  144,  144,  144,  144,  144,  144,  144,  127,  129,
+      128,  144,  144,  144,  144,  144,  144,  144,   20,   23,
+      144,  144,  144,   28,  144,  144,   30,  144,  144,  144,
+      144,   37,  144,  144,   41,   42,  144,  144,  144,  144,
+      144,  144,  144,   51,   52,  144,   54,  144,   56,  144,
+
+      144,  144,  144,   64,   66,   68,   69,   70,  144,   72,
+      144,  144,   76,  144,  144,  144,  144,  144,   87,  144,
+       89,  144,  144,  144,  144,  144,  144,  144,  144,  100,
+      101,  103,  144,  144,  144,  144,  144,  144,  110,  144,
+      112,  113,  127,  128,    8,  144,  144,  144,  144,  144,
+      144,  144,   25,  144,  144,  144,  144,  144,  144,  144,
+      144,  144,  144,  144,  144,  144,  144,  144,   47,   48,
+       49,  144,   53,  144,   58,   59,  144,  144,  144,   71,
+      144,   74,   77,   81,   82,  144,  144,  144,   88,  144,
+      144,   92,  144,  144,  144,  144,   99,  144,  144,  144,
+
+      144,  107,  144,  144,  111,  144,  144,  144,   14,  144,
+      144,  144,  144,  144,   26,  144,   29,  144,  144,  144,
+      144,   35,  144,  144,  144,   40,  144,   45,  144,  144,
+       55,   60,  144,  144,   73,  144,  144,  144,  144,   91,
+      144,   94,   95,  144,  144,  144,  105,  106,  108,  144,
+      144,  144,   13,  144,  144,  144,  144,  144,  144,   21,
+      144,   33,   34,  144,  144,  144,  144,   46,   50,   62,
+      144,  144,   85,   86,  144,  144,  144,  144,  144,  109,
+      144,  144,  144,  144,  144,  144,  144,  144,   31,  144,
+      144,   39,  144,   63,  144,  144,  144,   98,  144,  144,
+
+      144,   12,  144,  144,  144,  144,   24,  144,   36,  144,
+      144,   83,  144,  144,  102,  144,  114,   16,  144,  144,
+      144,   27,   38,  144,   84,   90,  144,  144,  144,   18,
+       19,  144,  144,  104,  144,  144,  144,  144,  144,   93,
+      144,   44,   17,    0
     } ;
 
 static yyconst YY_CHAR yy_ec[256] =
@@ -496,282 +498,288 @@ static yyconst YY_CHAR yy_meta[72] =
         8
     } ;
 
-static yyconst flex_uint16_t yy_base[542] =
+static yyconst flex_uint16_t yy_base[560] =
     {   0,
         0,    1,   46,    0,  117,  163,    2,    3,  128,  132,
-        6,   10,  260, 1177, 1177,    0, 1177,   13, 1177,  241,
-     1177, 1177, 1177,  239,    6,  130,    4, 1177,  202,  124,
-      161,  170,  178,  207,  260,   92,  110,  161,   97,  108,
-      219,    0,  153,  221,  176,  108,  232,  171,  276,  272,
-      129,  221,  177, 1177,  184,    4,   19,    0,    0,    0,
-      146,    0,    0,  340,    0,    0,  145,    0,   22, 1177,
-        0,  249,  284,  334,   18, 1177, 1177, 1177,    0,  232,
-      262,  234,  270,  267,  285,  278,    0,  276,  307,  331,
-      291,  307,  299,  347,  313,  312,  325, 1177,  325,  345,
-
-      348,  343,  343,  338,  342,  347,  352,  358,  362,  378,
-      394,    0,  381,  366,  382,  396,  392,  390,  387,  397,
-      404,    0,  407,  392,  397,  398,  408,  409,  407,  447,
-      415,  400,  437,  434,  453,  451,  445,  438,  444,  452,
-      458,  454,  453,  461,  447,  467,  148,   29,    0,    0,
-     1177,    0, 1177, 1177,   22,   24, 1177, 1177, 1177, 1177,
-     1177,    0,    0, 1177,    0,  474,   26,   28,    0,    0,
-      467,    0,  468,  451,  466,  453,  478,  475,  480,  496,
-      485,  488,  486,  511,  493,  509,  506,  515,  512,  515,
-      499,  518,  507,  519,    0,  524,  507,  511,  511,  512,
-
-      531,  528,  521,  525,  517,  531,  542,  551,  552,  559,
-      560,  555,    0,  550,  551,  567,  564,  567,    0,  564,
-        0,  572,  573,  559,  578,  569,  563,  577,  575,  583,
-      584,  582,   98,  586,    0,  580,  583,  582,  592,  604,
-      602,  598,  621,  609,  606,  625,  615,  622,  613,   30,
-      125,    0,  614,  620,  630,  622,  632,  628,  627,    0,
-      640,  631,  632,  626,    0,  627,  630,  629,  637,  632,
-      634,  642,  662,  663,  661,    0,    0,  664,  661,  684,
-      681,  667,  668,  680,    0,    0,  674,    0,  677,  668,
-      675,  676,  688,    0,    0,    0,    0,    0,  677,    0,
-
-      679,  681,  682,  694,  699,  704,    0,  702,    0,  690,
-      687,  692,  709,  722,  718,  726,    0,  719,    0,  734,
-      722,  724,  738,  741,  739,    0,  743,    0,  736,  136,
-     1177,    0,  746,  746,  732,  752,  739,  750,  754,    0,
-      747,  744,  758,  759,  756,  765,  757,  765,  762,  769,
-      774,  784,  791,  778,  798,    0,    0,    0,  795,    0,
-      796,    0,    0,  784,  800,  784,    0,    0,    0,    0,
-      787,  794,  791,    0,  805,  795,    0,  808,  794,  806,
-        0,  796,  800,  815,  816,    0,  803,  822,    0,  809,
-      818,  814,    0,  807,  823,  845,  838,  834,    0,  854,
-
-        0,  854,  847,  849,  842,    0,  843,  860,  862,    0,
-       93,    0,  846,  853,    0,    0,  850,  868,  862,  852,
-      848,  860,    0,  865,    0,  864,  878,  879,    0,    0,
-        0,  863,  868,  871,    0,  877,  872,  886,  892,  900,
-      903,    0,  908,    0,    0,  912,  909,  899,  901,    0,
-        0,    0,  909,  907,    0,    0,  920,  915,  905,  913,
-      914,    0,  909,  923,  917,  916,  919,  916,  919,  924,
-        0,  921,  926,    0,  923,    0,  930,  940,  944,    0,
-      946,  947,  962,    0,  964,  970,  964,  972,    0,  958,
-        0,  972,  962,  962,  963,  974,    0,  972,    0,    0,
-
-      967,  982,  970,    0,    0,  980,    0,    0,  970,  988,
-      974,    0,    0,  981,  991,    0,  988,  991,  983,  997,
-      984,    0,  996,    0,    0, 1177, 1061, 1071, 1081, 1091,
-     1101, 1105, 1108, 1114, 1122, 1132, 1142, 1152, 1162, 1167,
-     1169
+        6,   10,  257, 1212, 1212,    0, 1212,   13, 1212,  233,
+     1212, 1212, 1212,  208,    6,  130,    4, 1212,  195,  124,
+      161,  170,  178,  207,  260,   92,  167,  161,   96,  107,
+      219,  214,  212,  224,  236,   92,  279,  171,  278,  281,
+      128,  227,    0,  125, 1212,  184,    4,   19,    0,    0,
+        0,  146,    0,    0,  343,    0,    0,  145,    0,   22,
+     1212,    0,  297,  316,  338,   18, 1212, 1212, 1212,    0,
+      170,  227,  173,  178,  224,  299,  270,    0,  270,  335,
+      330,  286,  320,  327,  376,  308,  316,  326, 1212,  335,
+
+      351,  355,  371,  348,  346,  353,  359,  370,  382,  383,
+      380,  379,  399,    0,  392,  379,  386,  401,  399,  401,
+      402,  407,  402,  413,  420,    0,  431,  417,  420,  422,
+      434,  437,  435,  451,  446,  433,  456,  459,  459,  457,
+      450,  444,  454,  462,  469,  465,  465,  474,  460,  483,
+      148,   29,    0,    0, 1212,    0, 1212, 1212,   22,   24,
+     1212, 1212, 1212, 1212, 1212,    0,    0, 1212,    0,  515,
+       26,   28,    0,    0,  488,    0,  490,  473,  489,  478,
+      501,  502,  496,  512,  496,  499,  494,  520,  503,  521,
+        0,  519,  528,  526,  529,  514,  535,  522,  534,    0,
+
+      539,  522,  524,  532,  534,  553,  551,  546,  550,  544,
+      564,  564,  556,  570,  571,  572,  574,  564,    0,  561,
+      564,  581,  578,  583,  571,  573,    0,  583,    0,  591,
+      592,  578,  596,  587,  589,  604,  600,  609,  612,  612,
+       98,  608,  625,    0,  619,  620,  619,  629,  630,  624,
+      620,  638,  628,  623,  642,  633,  640,  632,   30,  125,
+        0,  635,  640,  650,  642,  652,  647,  654,    0,  668,
+      659,  659,  655,    0,  658,  663,  668,  676,  669,  671,
+      679,  688,  685,  683,    0,    0,  681,  680,  701,  698,
+      685,  686,  699,    0,    0,  693,    0,  697,    0,  688,
+
+      695,  696,  709,    0,    0,    0,    0,    0,  695,    0,
+      705,  720,  711,  715,  718,  730,  741,  746,    0,  743,
+        0,  731,  726,  731,  748,  739,  752,  746,  755,    0,
+      742,    0,  758,  743,  746,  760,  764,  762,    0,  766,
+        0,  759,  136, 1212,    0,  769,  769,  763,  784,  772,
+      780,  791,    0,  783,  786,  800,  801,  798,  807,  797,
+      805,  802,  799,  802,  813,  814,  802,  819,    0,    0,
+        0,  817,    0,  818,    0,    0,  807,  823,  807,    0,
+      825,    0,    0,    0,    0,  811,  818,  823,    0,  838,
+      828,    0,  841,  845,  832,  846,    0,  842,  844,  859,
+
+      860,    0,  847,  866,    0,  853,  860,  857,    0,  852,
+      858,  876,  870,  860,    0,  881,    0,  878,  872,  874,
+      867,    0,  868,  885,  887,    0,   93,    0,  879,  887,
+        0,    0,  884,  903,    0,  898,  890,  888,  906,    0,
+      909,    0,    0,  908,  922,  923,    0,    0,    0,  907,
+      912,  913,    0,  920,  917,  921,  923,  932,  929,    0,
+      935,    0,    0,  936,  934,  924,  926,    0,    0,    0,
+      934,  932,    0,    0,  945,  948,  939,  947,  949,    0,
+      945,  961,  957,  962,  963,  960,  963,  968,    0,  965,
+      970,    0,  965,    0,  973,  985,  979,    0,  977,  979,
+
+      988,    0,  991,  994,  989,  997,    0,  983,    0,  997,
+      987,  987,  996, 1008,    0, 1006,    0,    0, 1002, 1018,
+     1008,    0,    0, 1020,    0,    0, 1016, 1032, 1018,    0,
+        0, 1025, 1035,    0, 1032, 1035, 1025, 1040, 1029,    0,
+     1031,    0,    0, 1212, 1096, 1106, 1116, 1126, 1136, 1140,
+     1143, 1149, 1157, 1167, 1177, 1187, 1197, 1202, 1204
     } ;
 
-static yyconst flex_int16_t yy_def[542] =
+static yyconst flex_int16_t yy_def[560] =
     {   0,
-      527,  527,  526,    3,  528,  528,  529,  529,  530,  530,
-      531,  531,  526,  526,  526,  532,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  526,  526,  526,  526,  534,  535,  535,
-      526,  536,  536,  537,  538,  538,  526,  532,  526,  526,
-      539,  526,  526,  526,  526,  526,  526,  526,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  526,  533,  533,
-
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  526,  526,  534,  535,
-      526,  536,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  540,  538,  526,  539,  526,  526,  526,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  526,
-      526,  541,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  526,
-      526,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,  533,  533,  533,  533,  533,
-      533,  533,  533,  533,  533,    0,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526
+      545,  545,  544,    3,  546,  546,  547,  547,  548,  548,
+      549,  549,  544,  544,  544,  550,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  544,  544,  544,  544,  552,  553,
+      553,  544,  554,  554,  555,  556,  556,  544,  550,  544,
+      544,  557,  544,  544,  544,  544,  544,  544,  544,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  544,  551,
+
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      544,  544,  552,  553,  544,  554,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  558,  556,  544,  557,  544,
+      544,  544,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  544,  544,
+      559,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  544,  544,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,  551,  551,  551,  551,  551,  551,  551,
+      551,  551,  551,    0,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544
     } ;
 
-static yyconst flex_uint16_t yy_nxt[1249] =
+static yyconst flex_uint16_t yy_nxt[1284] =
     {   0,
-      526,  526,   15,   15,   60,   60,  148,  148,   66,   61,
-       61,   67,   66,  526,   69,   67,   69,   72,   72,   76,
-       77,  148,  148,   69,  526,   69,  167,  167,  526,  168,
-      168,  148,  148,  250,  251,  251,  251,  168,  168,  168,
-      168,  330,  251,  526,   16,   16,   17,   18,   19,   18,
+      544,  544,   15,   15,   61,   61,  152,  152,   67,   62,
+       62,   68,   67,  544,   70,   68,   70,   73,   73,   77,
+       78,  152,  152,   70,  544,   70,  171,  171,  544,  172,
+      172,  152,  152,  259,  260,  260,  260,  172,  172,  172,
+      172,  343,  260,  544,   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,
-       47,   48,   49,   50,   51,   52,   42,   53,   42,   54,
+       47,   48,   49,   50,   51,   52,   53,   54,   53,   55,
        17,   17,   30,   31,   32,   33,   34,   35,   36,   37,
 
        38,   39,   40,   41,   42,   43,   44,   45,   46,   47,
-       48,   49,   50,   51,   52,   42,   53,   17,   55,   56,
-       57,   17,   17,   17,   17,   17,  109,  110,  113,  114,
-       63,   17,   17,   17,   63,   61,  251,  251,  449,   61,
-       73,   74,   74,  313,   80,  128,  143,  251,  251,  147,
-      164,   75,   81,  151,   82,  109,  110,  113,  114,   83,
-       17,   17,   17,   17,   55,   56,   57,   17,   17,   17,
-       17,   17,   64,   80,  128,  143,   64,   17,   17,   17,
-       75,   81,   84,   82,  119,  147,   85,   88,   83,   86,
-      120,  111,  133,  125,   89,   93,  112,  126,  146,   94,
-
-      134,   90,   87,   95,   91,   92,   17,   17,   17,   96,
-      127,   84,   97,  119,   98,   85,   88,   78,   86,  120,
-      111,  133,  125,   89,   93,  112,  126,  146,   94,  134,
-       90,   87,   95,   91,   92,   99,  115,  100,   96,  127,
-      116,   97,  101,  121,  117,  144,  145,  102,   71,  129,
-      118,  122,  169,  130,  172,  123,   70,  131,  124,  526,
-       72,   72,  526,  132,   99,  115,  100,  526,  526,  116,
-       75,  101,  121,  117,  144,  145,  102,  103,  129,  118,
-      122,  169,  130,  172,  123,  104,  131,  124,  105,  173,
-      170,  106,  132,  135,  107,  166,  166,  108,  171,   75,
-
-      136,  137,  140,  174,  141,   75,  103,  142,  175,  177,
-      138,  178,  184,  139,  104,  526,  526,  105,  173,  170,
-      106,  176,  135,  107,  179,  185,  108,  171,  180,  136,
-      137,  140,  174,  141,   75,  186,  142,  175,  177,  138,
-      178,  184,  139,  154,   73,   74,   74,  526,  191,  192,
-      176,  155,  156,  179,  185,   75,  193,  180,  157,  181,
-      194,  182,  158,  183,  186,  195,  187,  196,  197,  188,
-      159,  199,  200,  201,  160,  189,  161,  191,  192,  198,
-      162,  202,  190,  203,   75,  193,  204,  157,  181,  194,
-      182,  158,  183,  205,  195,  187,  196,  197,  188,  159,
-
-      199,  200,  201,  160,  189,  161,  212,  213,  198,  162,
-      202,  190,  203,  206,  208,  204,  207,  214,  215,  216,
-      218,  217,  205,  219,  209,  220,  221,  222,  223,  210,
-      211,  224,  225,  226,  227,  212,  213,  228,  232,  233,
-      526,  526,  206,  208,  526,  207,  214,  215,  216,  218,
-      217,  236,  219,  209,  220,  221,  222,  223,  210,  211,
-      224,  225,  226,  227,  229,  234,  228,  232,  233,  230,
-      231,  237,  238,  235,  239,  240,  241,  242,  243,  244,
-      236,  245,  247,  248,  249,  166,  166,  246,  253,  254,
-      255,  256,  257,  229,  234,   75,  260,  258,  230,  231,
-
-      237,  238,  235,  239,  240,  241,  242,  243,  244,  259,
-      245,  247,  248,  249,  261,  262,  246,  253,  254,  255,
-      256,  257,  263,  264,   75,  260,  258,  265,  266,  267,
-      268,  269,  270,  271,  273,  274,  275,  272,  259,  276,
-      277,  278,  279,  261,  262,  280,  281,  282,  283,  284,
-      285,  263,  264,  286,  287,  288,  265,  266,  267,  268,
-      269,  270,  271,  273,  274,  275,  272,  289,  276,  277,
-      278,  279,  290,  291,  280,  281,  282,  283,  284,  285,
-      292,  293,  286,  287,  288,  295,  296,  297,  298,  299,
-      300,  294,  301,  302,  303,  304,  289,  305,  306,  307,
-
-      308,  290,  291,  309,  310,  311,  312,  314,  315,  292,
-      293,  316,  317,  318,  295,  296,  297,  298,  299,  300,
-      294,  301,  302,  303,  304,  319,  305,  306,  307,  308,
-      320,  321,  309,  310,  311,  312,  314,  315,  322,  323,
-      316,  317,  318,  324,  325,  326,  328,  329,  332,  327,
-      333,  334,  335,  336,  319,  337,  338,  339,  340,  320,
-      321,  341,  342,  343,  344,  345,  346,  322,  323,  347,
-      348,  349,  324,  325,  326,  328,  329,  332,  327,  333,
-      334,  335,  336,  350,  337,  338,  339,  340,  351,  352,
-      341,  342,  343,  344,  345,  346,  353,  354,  347,  348,
-
-      349,  355,  356,  357,  358,  359,  360,  361,  362,  363,
-      364,  365,  350,  367,  368,  369,  370,  351,  352,  371,
-      372,  373,  366,  374,  375,  353,  354,  376,  377,  378,
-      355,  356,  357,  358,  359,  360,  361,  362,  363,  364,
-      365,  379,  367,  368,  369,  370,  380,  381,  371,  372,
-      373,  366,  374,  375,  382,  383,  376,  377,  378,  384,
-      385,  386,  387,  388,  389,  390,  391,  392,  393,  394,
-      379,  395,  397,  398,  396,  380,  381,  399,  400,  401,
-      402,  403,  404,  382,  383,  405,  406,  407,  384,  385,
-      386,  387,  388,  389,  390,  391,  392,  393,  394,  408,
-
-      395,  397,  398,  396,  409,  410,  399,  400,  401,  402,
-      403,  404,  411,  412,  405,  406,  407,  413,  414,  415,
-      416,  417,  418,  419,  420,  421,  422,  423,  408,  424,
-      425,  426,  427,  409,  410,  428,  429,  430,  431,  432,
-      433,  411,  412,  434,  435,  436,  413,  414,  415,  416,
-      417,  418,  419,  420,  421,  422,  423,  437,  424,  425,
-      426,  427,  438,  439,  428,  429,  430,  431,  432,  433,
-      440,  441,  434,  435,  436,  442,  443,  444,  445,  446,
-      447,  448,  450,  451,  452,  453,  437,  454,  455,  456,
-      457,  438,  439,  458,  459,  460,  461,  462,  463,  440,
-
-      441,  464,  465,  466,  442,  443,  444,  445,  446,  447,
-      448,  450,  451,  452,  453,  467,  454,  455,  456,  457,
-      468,  469,  458,  459,  460,  461,  462,  463,  470,  471,
-      464,  465,  466,  472,  473,  474,  475,  476,  477,  478,
-      479,  480,  481,  482,  467,  483,  484,  485,  486,  468,
-      469,  487,  488,  489,  490,  491,  492,  470,  471,  493,
-      494,  495,  472,  473,  474,  475,  476,  477,  478,  479,
-      480,  481,  482,  496,  483,  484,  485,  486,  497,  498,
-      487,  488,  489,  490,  491,  492,  499,  500,  493,  494,
-      495,  501,  502,  503,  504,  505,  506,  507,  508,  509,
-
-      510,  511,  496,  512,  513,  514,  515,  497,  498,  516,
-      517,  518,  519,  520,  521,  499,  500,  522,  523,  524,
-      501,  502,  503,  504,  505,  506,  507,  508,  509,  510,
-      511,  525,  512,  513,  514,  515,  526,  526,  516,  517,
-      518,  519,  520,  521,  526,  526,  522,  523,  524,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      525,   14,   14,   14,   14,   14,   14,   14,   14,   14,
-       14,   58,   58,   58,   58,   58,   58,   58,   58,   58,
-       58,   59,   59,   59,   59,   59,   59,   59,   59,   59,
-       59,   62,   62,   62,   62,   62,   62,   62,   62,   62,
-
-       62,   65,   65,   65,   65,   65,   65,   65,   65,   65,
-       65,   68,   68,   79,   79,   79,  526,   79,  149,  149,
-      149,  149,  150,  150,  150,  526,  150,  150,  150,  150,
-      150,  150,  152,  152,  152,  526,  152,  152,  152,  152,
-      526,  152,  153,  153,  153,  153,  153,  153,  153,  153,
-      153,  153,  163,  163,  526,  163,  163,  163,  163,  163,
-      163,  163,  165,  526,  165,  165,  165,  165,  165,  165,
-      165,  165,  252,  252,  331,  331,   13,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526
+       48,   49,   50,   51,   52,   53,   54,   17,   56,   57,
+       58,   17,   17,   17,   17,   17,  110,  115,  116,  132,
+       64,   17,   17,   17,   64,   62,  260,  260,  467,   62,
+       74,   75,   75,  325,   81,  147,  150,  260,  260,  151,
+      168,   76,   82,  155,   83,  110,  115,  116,  132,   84,
+       17,   17,   17,   17,   56,   57,   58,   17,   17,   17,
+       17,   17,   65,   81,  147,  150,   65,   17,   17,   17,
+       76,   82,   85,   83,  111,  151,   86,   89,   84,   87,
+      173,  113,  137,  176,   90,   94,  114,  177,  112,   95,
+
+      138,   91,   88,   96,   92,   93,   17,   17,   17,   97,
+       79,   85,   98,  111,   99,   86,   89,   72,   87,  173,
+      113,  137,  176,   90,   94,  114,  177,  112,   95,  138,
+       91,   88,   96,   92,   93,  100,  117,  101,   97,  121,
+      118,   98,  102,  123,  119,  122,  125,  103,   71,  124,
+      120,  148,  149,  129,  126,  174,  544,  130,  127,  544,
+      178,  128,  544,  175,  100,  117,  101,  544,  121,  118,
+      131,  102,  123,  119,  122,  125,  103,  104,  124,  120,
+      148,  149,  129,  126,  174,  105,  130,  127,  106,  178,
+      128,  107,  175,  544,  108,  139,  133,  109,  544,  131,
+
+      134,  181,  140,  141,  135,  182,  104,  188,   73,   73,
+      136,  144,  142,  145,  105,  143,  146,  106,   76,  544,
+      107,  544,  179,  108,  139,  133,  109,  170,  170,  134,
+      181,  140,  141,  135,  182,  180,  188,   76,  189,  136,
+      144,  142,  145,  196,  143,  146,  158,   76,   74,   75,
+       75,  179,  183,  197,  159,  160,  184,  198,  185,   76,
+      186,  161,  187,  190,  180,  162,   76,  189,  191,  544,
+      199,  200,  196,  163,  201,  544,  204,  164,  544,  165,
+      205,  183,  197,  166,  206,  184,  198,  185,   76,  186,
+      161,  187,  190,  207,  162,  192,  202,  191,  193,  199,
+
+      200,  208,  163,  201,  194,  204,  164,  203,  165,  205,
+      209,  195,  166,  206,  210,  211,  213,  218,  212,  214,
+      219,  220,  207,  221,  192,  202,  222,  193,  223,  215,
+      208,  224,  225,  194,  216,  217,  203,  226,  227,  209,
+      195,  228,  229,  210,  211,  213,  218,  212,  214,  219,
+      220,  230,  221,  231,  232,  222,  233,  223,  215,  234,
+      224,  225,  235,  216,  217,  236,  226,  227,  237,  240,
+      228,  229,  241,  238,  239,  242,  245,  246,  247,  248,
+      230,  249,  231,  232,  243,  233,  250,  251,  234,  252,
+      253,  235,  244,  254,  236,  256,  257,  237,  240,  255,
+
+      258,  241,  238,  239,  242,  245,  246,  247,  248,  262,
+      249,  263,  264,  243,  265,  250,  251,  266,  252,  253,
+      267,  244,  254,  269,  256,  257,  170,  170,  255,  258,
+      270,  271,  268,  272,  273,  274,   76,  275,  262,  276,
+      263,  264,  277,  265,  278,  279,  266,  280,  282,  267,
+      283,  281,  269,  284,  285,  286,  287,  288,  289,  270,
+      271,  268,  272,  273,  274,   76,  275,  290,  276,  291,
+      292,  277,  293,  278,  279,  294,  280,  282,  295,  283,
+      281,  296,  284,  285,  286,  287,  288,  289,  297,  298,
+      299,  300,  301,  302,  305,  303,  290,  306,  291,  292,
+
+      307,  293,  308,  309,  294,  304,  310,  295,  311,  312,
+      296,  313,  314,  315,  316,  317,  318,  297,  298,  299,
+      300,  301,  302,  305,  303,  319,  306,  320,  321,  307,
+      322,  308,  309,  323,  304,  310,  324,  311,  312,  326,
+      313,  314,  315,  316,  317,  318,  327,  328,  329,  330,
+      331,  332,  333,  334,  319,  335,  320,  321,  336,  322,
+      337,  338,  323,  339,  341,  324,  342,  340,  326,  345,
+      346,  347,  348,  349,  350,  327,  328,  329,  330,  331,
+      332,  333,  334,  351,  335,  352,  353,  336,  354,  337,
+      338,  355,  339,  341,  356,  342,  340,  357,  345,  346,
+
+      347,  348,  349,  350,  358,  359,  360,  361,  362,  363,
+      364,  365,  351,  366,  352,  353,  367,  354,  368,  369,
+      355,  370,  371,  356,  372,  373,  357,  374,  375,  376,
+      377,  380,  378,  358,  359,  360,  361,  362,  363,  364,
+      365,  381,  366,  379,  382,  367,  383,  368,  369,  384,
+      370,  371,  385,  372,  373,  386,  374,  375,  376,  377,
+      380,  378,  387,  388,  389,  390,  391,  392,  393,  394,
+      381,  395,  379,  382,  396,  383,  397,  398,  384,  399,
+      400,  385,  401,  402,  386,  403,  404,  405,  406,  407,
+      408,  387,  388,  389,  390,  391,  392,  393,  394,  409,
+
+      395,  410,  413,  396,  411,  397,  398,  412,  399,  400,
+      414,  401,  402,  415,  403,  404,  405,  406,  407,  408,
+      416,  417,  418,  419,  420,  421,  422,  423,  409,  424,
+      410,  413,  425,  411,  426,  427,  412,  428,  429,  414,
+      430,  431,  415,  432,  433,  434,  435,  436,  437,  416,
+      417,  418,  419,  420,  421,  422,  423,  438,  424,  439,
+      440,  425,  441,  426,  427,  442,  428,  429,  443,  430,
+      431,  444,  432,  433,  434,  435,  436,  437,  445,  446,
+      447,  448,  449,  450,  451,  452,  438,  453,  439,  440,
+      454,  441,  455,  456,  442,  457,  458,  443,  459,  460,
+
+      444,  461,  462,  463,  464,  465,  466,  445,  446,  447,
+      448,  449,  450,  451,  452,  468,  453,  469,  470,  454,
+      471,  455,  456,  472,  457,  458,  473,  459,  460,  474,
+      461,  462,  463,  464,  465,  466,  475,  476,  477,  478,
+      479,  480,  481,  482,  468,  483,  469,  470,  484,  471,
+      485,  486,  472,  487,  488,  473,  489,  490,  474,  491,
+      492,  493,  494,  495,  496,  475,  476,  477,  478,  479,
+      480,  481,  482,  497,  483,  498,  499,  484,  500,  485,
+      486,  501,  487,  488,  502,  489,  490,  503,  491,  492,
+      493,  494,  495,  496,  504,  505,  506,  507,  508,  509,
+
+      510,  511,  497,  512,  498,  499,  513,  500,  514,  515,
+      501,  516,  517,  502,  518,  519,  503,  520,  521,  522,
+      523,  524,  525,  504,  505,  506,  507,  508,  509,  510,
+      511,  526,  512,  527,  528,  513,  529,  514,  515,  530,
+      516,  517,  531,  518,  519,  532,  520,  521,  522,  523,
+      524,  525,  533,  534,  535,  536,  537,  538,  539,  540,
+      526,  541,  527,  528,  542,  529,  543,  544,  530,  544,
+      544,  531,  544,  544,  532,  544,  544,  544,  544,  544,
+      544,  533,  534,  535,  536,  537,  538,  539,  540,  544,
+      541,  544,  544,  542,  544,  543,   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,  544,   80,  153,  153,  153,  153,  154,  154,  154,
+      544,  154,  154,  154,  154,  154,  154,  156,  156,  156,
+      544,  156,  156,  156,  156,  544,  156,  157,  157,  157,
+      157,  157,  157,  157,  157,  157,  157,  167,  167,  544,
+      167,  167,  167,  167,  167,  167,  167,  169,  544,  169,
+
+      169,  169,  169,  169,  169,  169,  169,  261,  261,  344,
+      344,   13,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544
     } ;
 
-static yyconst flex_int16_t yy_chk[1249] =
+static yyconst flex_int16_t yy_chk[1284] =
     {   0,
-        0,    0,    1,    2,    7,    8,   56,   56,   11,    7,
+        0,    0,    1,    2,    7,    8,   57,   57,   11,    7,
         8,   11,   12,    0,   18,   12,   18,   25,   25,   27,
-       27,   57,   57,   69,    0,   69,   75,   75,    0,   75,
-       75,  148,  148,  155,  155,  156,  156,  167,  167,  168,
-      168,  250,  250,    0,    1,    2,    3,    3,    3,    3,
+       27,   58,   58,   70,    0,   70,   76,   76,    0,   76,
+       76,  152,  152,  159,  159,  160,  160,  171,  171,  172,
+      172,  259,  259,    0,    1,    2,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
@@ -780,134 +788,138 @@ static yyconst flex_int16_t yy_chk[1249] =
 
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    5,    5,    5,
-        5,    5,    5,    5,    5,    5,   36,   37,   39,   40,
-        9,    5,    5,    5,   10,    9,  251,  251,  411,   10,
-       26,   26,   26,  233,   30,   46,   51,  330,  330,  147,
-       67,   26,   30,   61,   30,   36,   37,   39,   40,   30,
+        5,    5,    5,    5,    5,    5,   36,   39,   40,   46,
+        9,    5,    5,    5,   10,    9,  260,  260,  427,   10,
+       26,   26,   26,  241,   30,   51,   54,  343,  343,  151,
+       68,   26,   30,   62,   30,   36,   39,   40,   46,   30,
         5,    5,    5,    6,    6,    6,    6,    6,    6,    6,
-        6,    6,    9,   30,   46,   51,   10,    6,    6,    6,
-       26,   30,   31,   30,   43,   55,   31,   32,   30,   31,
-       43,   38,   48,   45,   32,   33,   38,   45,   53,   33,
+        6,    6,    9,   30,   51,   54,   10,    6,    6,    6,
+       26,   30,   31,   30,   37,   56,   31,   32,   30,   31,
+       81,   38,   48,   83,   32,   33,   38,   84,   37,   33,
 
        48,   32,   31,   33,   32,   32,    6,    6,    6,   33,
-       45,   31,   33,   43,   34,   31,   32,   29,   31,   43,
-       38,   48,   45,   32,   33,   38,   45,   53,   33,   48,
-       32,   31,   33,   32,   32,   34,   41,   34,   33,   45,
-       41,   33,   34,   44,   41,   52,   52,   34,   24,   47,
-       41,   44,   80,   47,   82,   44,   20,   47,   44,   13,
-       72,   72,    0,   47,   34,   41,   34,    0,    0,   41,
-       72,   34,   44,   41,   52,   52,   34,   35,   47,   41,
-       44,   80,   47,   82,   44,   35,   47,   44,   35,   83,
-       81,   35,   47,   49,   35,   73,   73,   35,   81,   72,
-
-       49,   49,   50,   84,   50,   73,   35,   50,   85,   86,
-       49,   88,   91,   49,   35,    0,    0,   35,   83,   81,
-       35,   85,   49,   35,   89,   92,   35,   81,   89,   49,
-       49,   50,   84,   50,   73,   93,   50,   85,   86,   49,
-       88,   91,   49,   64,   74,   74,   74,    0,   95,   96,
-       85,   64,   64,   89,   92,   74,   97,   89,   64,   90,
-       99,   90,   64,   90,   93,  100,   94,  101,  102,   94,
-       64,  103,  104,  105,   64,   94,   64,   95,   96,  102,
-       64,  106,   94,  107,   74,   97,  108,   64,   90,   99,
-       90,   64,   90,  109,  100,   94,  101,  102,   94,   64,
-
-      103,  104,  105,   64,   94,   64,  113,  114,  102,   64,
-      106,   94,  107,  110,  111,  108,  110,  115,  116,  117,
-      118,  117,  109,  119,  111,  120,  121,  123,  124,  111,
-      111,  125,  126,  127,  128,  113,  114,  129,  131,  132,
-        0,    0,  110,  111,    0,  110,  115,  116,  117,  118,
-      117,  134,  119,  111,  120,  121,  123,  124,  111,  111,
-      125,  126,  127,  128,  130,  133,  129,  131,  132,  130,
-      130,  135,  136,  133,  137,  138,  139,  140,  141,  142,
-      134,  143,  144,  145,  146,  166,  166,  143,  171,  173,
-      174,  175,  176,  130,  133,  166,  178,  177,  130,  130,
-
-      135,  136,  133,  137,  138,  139,  140,  141,  142,  177,
-      143,  144,  145,  146,  179,  180,  143,  171,  173,  174,
-      175,  176,  181,  182,  166,  178,  177,  183,  184,  185,
-      186,  187,  188,  189,  190,  191,  192,  189,  177,  193,
-      194,  196,  197,  179,  180,  198,  199,  200,  201,  202,
-      203,  181,  182,  204,  205,  206,  183,  184,  185,  186,
-      187,  188,  189,  190,  191,  192,  189,  207,  193,  194,
-      196,  197,  208,  209,  198,  199,  200,  201,  202,  203,
-      210,  211,  204,  205,  206,  212,  214,  215,  216,  217,
-      218,  211,  220,  222,  223,  224,  207,  225,  226,  227,
-
-      228,  208,  209,  229,  230,  231,  232,  234,  236,  210,
-      211,  237,  238,  239,  212,  214,  215,  216,  217,  218,
-      211,  220,  222,  223,  224,  240,  225,  226,  227,  228,
-      241,  242,  229,  230,  231,  232,  234,  236,  243,  244,
-      237,  238,  239,  245,  246,  247,  248,  249,  253,  247,
-      254,  255,  256,  257,  240,  258,  259,  261,  262,  241,
-      242,  263,  264,  266,  267,  268,  269,  243,  244,  270,
-      271,  272,  245,  246,  247,  248,  249,  253,  247,  254,
-      255,  256,  257,  273,  258,  259,  261,  262,  274,  275,
-      263,  264,  266,  267,  268,  269,  278,  279,  270,  271,
-
-      272,  280,  281,  282,  283,  284,  287,  289,  290,  291,
-      292,  293,  273,  299,  301,  302,  303,  274,  275,  304,
-      305,  306,  293,  308,  310,  278,  279,  311,  312,  313,
-      280,  281,  282,  283,  284,  287,  289,  290,  291,  292,
-      293,  314,  299,  301,  302,  303,  315,  316,  304,  305,
-      306,  293,  308,  310,  318,  320,  311,  312,  313,  321,
-      322,  323,  324,  325,  327,  329,  333,  334,  335,  336,
-      314,  337,  338,  339,  337,  315,  316,  341,  342,  343,
-      344,  345,  346,  318,  320,  347,  348,  349,  321,  322,
-      323,  324,  325,  327,  329,  333,  334,  335,  336,  350,
-
-      337,  338,  339,  337,  351,  352,  341,  342,  343,  344,
-      345,  346,  353,  354,  347,  348,  349,  355,  359,  361,
-      364,  365,  366,  371,  372,  373,  375,  376,  350,  378,
-      379,  380,  382,  351,  352,  383,  384,  385,  387,  388,
-      390,  353,  354,  391,  392,  394,  355,  359,  361,  364,
-      365,  366,  371,  372,  373,  375,  376,  395,  378,  379,
-      380,  382,  396,  397,  383,  384,  385,  387,  388,  390,
-      398,  400,  391,  392,  394,  402,  403,  404,  405,  407,
-      408,  409,  413,  414,  417,  418,  395,  419,  420,  421,
-      422,  396,  397,  424,  426,  427,  428,  432,  433,  398,
-
-      400,  434,  436,  437,  402,  403,  404,  405,  407,  408,
-      409,  413,  414,  417,  418,  438,  419,  420,  421,  422,
-      439,  440,  424,  426,  427,  428,  432,  433,  441,  443,
-      434,  436,  437,  446,  447,  448,  449,  453,  454,  457,
-      458,  459,  460,  461,  438,  463,  464,  465,  466,  439,
-      440,  467,  468,  469,  470,  472,  473,  441,  443,  475,
-      477,  478,  446,  447,  448,  449,  453,  454,  457,  458,
-      459,  460,  461,  479,  463,  464,  465,  466,  481,  482,
-      467,  468,  469,  470,  472,  473,  483,  485,  475,  477,
-      478,  486,  487,  488,  490,  492,  493,  494,  495,  496,
-
-      498,  501,  479,  502,  503,  506,  509,  481,  482,  510,
-      511,  514,  515,  517,  518,  483,  485,  519,  520,  521,
-      486,  487,  488,  490,  492,  493,  494,  495,  496,  498,
-      501,  523,  502,  503,  506,  509,    0,    0,  510,  511,
-      514,  515,  517,  518,    0,    0,  519,  520,  521,    0,
-        0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
-      523,  527,  527,  527,  527,  527,  527,  527,  527,  527,
-      527,  528,  528,  528,  528,  528,  528,  528,  528,  528,
-      528,  529,  529,  529,  529,  529,  529,  529,  529,  529,
-      529,  530,  530,  530,  530,  530,  530,  530,  530,  530,
-
-      530,  531,  531,  531,  531,  531,  531,  531,  531,  531,
-      531,  532,  532,  533,  533,  533,    0,  533,  534,  534,
-      534,  534,  535,  535,  535,    0,  535,  535,  535,  535,
-      535,  535,  536,  536,  536,    0,  536,  536,  536,  536,
-        0,  536,  537,  537,  537,  537,  537,  537,  537,  537,
-      537,  537,  538,  538,    0,  538,  538,  538,  538,  538,
-      538,  538,  539,    0,  539,  539,  539,  539,  539,  539,
-      539,  539,  540,  540,  541,  541,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526,  526,  526,
-      526,  526,  526,  526,  526,  526,  526,  526
+       29,   31,   33,   37,   34,   31,   32,   24,   31,   81,
+       38,   48,   83,   32,   33,   38,   84,   37,   33,   48,
+       32,   31,   33,   32,   32,   34,   41,   34,   33,   42,
+       41,   33,   34,   43,   41,   42,   44,   34,   20,   43,
+       41,   52,   52,   45,   44,   82,   13,   45,   44,    0,
+       85,   44,    0,   82,   34,   41,   34,    0,   42,   41,
+       45,   34,   43,   41,   42,   44,   34,   35,   43,   41,
+       52,   52,   45,   44,   82,   35,   45,   44,   35,   85,
+       44,   35,   82,    0,   35,   49,   47,   35,    0,   45,
+
+       47,   87,   49,   49,   47,   89,   35,   92,   73,   73,
+       47,   50,   49,   50,   35,   49,   50,   35,   73,    0,
+       35,    0,   86,   35,   49,   47,   35,   74,   74,   47,
+       87,   49,   49,   47,   89,   86,   92,   74,   93,   47,
+       50,   49,   50,   96,   49,   50,   65,   73,   75,   75,
+       75,   86,   90,   97,   65,   65,   90,   98,   91,   75,
+       91,   65,   91,   94,   86,   65,   74,   93,   94,    0,
+      100,  101,   96,   65,  102,    0,  104,   65,    0,   65,
+      105,   90,   97,   65,  106,   90,   98,   91,   75,   91,
+       65,   91,   94,  107,   65,   95,  103,   94,   95,  100,
+
+      101,  108,   65,  102,   95,  104,   65,  103,   65,  105,
+      109,   95,   65,  106,  110,  111,  112,  115,  111,  113,
+      116,  117,  107,  118,   95,  103,  119,   95,  119,  113,
+      108,  120,  121,   95,  113,  113,  103,  122,  123,  109,
+       95,  124,  125,  110,  111,  112,  115,  111,  113,  116,
+      117,  127,  118,  128,  129,  119,  130,  119,  113,  131,
+      120,  121,  132,  113,  113,  133,  122,  123,  134,  135,
+      124,  125,  136,  134,  134,  137,  138,  139,  140,  141,
+      127,  142,  128,  129,  137,  130,  143,  144,  131,  145,
+      146,  132,  137,  147,  133,  148,  149,  134,  135,  147,
+
+      150,  136,  134,  134,  137,  138,  139,  140,  141,  175,
+      142,  177,  178,  137,  179,  143,  144,  180,  145,  146,
+      181,  137,  147,  182,  148,  149,  170,  170,  147,  150,
+      183,  184,  181,  185,  186,  187,  170,  188,  175,  189,
+      177,  178,  190,  179,  192,  193,  180,  194,  195,  181,
+      196,  194,  182,  197,  198,  199,  201,  202,  203,  183,
+      184,  181,  185,  186,  187,  170,  188,  204,  189,  205,
+      206,  190,  207,  192,  193,  208,  194,  195,  209,  196,
+      194,  210,  197,  198,  199,  201,  202,  203,  211,  212,
+      213,  214,  215,  216,  218,  217,  204,  220,  205,  206,
+
+      221,  207,  222,  223,  208,  217,  224,  209,  225,  226,
+      210,  228,  230,  231,  232,  233,  234,  211,  212,  213,
+      214,  215,  216,  218,  217,  235,  220,  236,  237,  221,
+      238,  222,  223,  239,  217,  224,  240,  225,  226,  242,
+      228,  230,  231,  232,  233,  234,  243,  245,  246,  247,
+      248,  249,  250,  251,  235,  252,  236,  237,  253,  238,
+      254,  255,  239,  256,  257,  240,  258,  256,  242,  262,
+      263,  264,  265,  266,  267,  243,  245,  246,  247,  248,
+      249,  250,  251,  268,  252,  270,  271,  253,  272,  254,
+      255,  273,  256,  257,  275,  258,  256,  276,  262,  263,
+
+      264,  265,  266,  267,  277,  278,  279,  280,  281,  282,
+      283,  284,  268,  287,  270,  271,  288,  272,  289,  290,
+      273,  291,  292,  275,  293,  296,  276,  298,  300,  301,
+      302,  309,  303,  277,  278,  279,  280,  281,  282,  283,
+      284,  311,  287,  303,  312,  288,  313,  289,  290,  314,
+      291,  292,  315,  293,  296,  316,  298,  300,  301,  302,
+      309,  303,  317,  318,  320,  322,  323,  324,  325,  326,
+      311,  327,  303,  312,  328,  313,  329,  331,  314,  333,
+      334,  315,  335,  336,  316,  337,  338,  340,  342,  346,
+      347,  317,  318,  320,  322,  323,  324,  325,  326,  348,
+
+      327,  349,  351,  328,  350,  329,  331,  350,  333,  334,
+      352,  335,  336,  354,  337,  338,  340,  342,  346,  347,
+      355,  356,  357,  358,  359,  360,  361,  362,  348,  363,
+      349,  351,  364,  350,  365,  366,  350,  367,  368,  352,
+      372,  374,  354,  377,  378,  379,  381,  386,  387,  355,
+      356,  357,  358,  359,  360,  361,  362,  388,  363,  390,
+      391,  364,  393,  365,  366,  394,  367,  368,  395,  372,
+      374,  396,  377,  378,  379,  381,  386,  387,  398,  399,
+      400,  401,  403,  404,  406,  407,  388,  408,  390,  391,
+      410,  393,  411,  412,  394,  413,  414,  395,  416,  418,
+
+      396,  419,  420,  421,  423,  424,  425,  398,  399,  400,
+      401,  403,  404,  406,  407,  429,  408,  430,  433,  410,
+      434,  411,  412,  436,  413,  414,  437,  416,  418,  438,
+      419,  420,  421,  423,  424,  425,  439,  441,  444,  445,
+      446,  450,  451,  452,  429,  454,  430,  433,  455,  434,
+      456,  457,  436,  458,  459,  437,  461,  464,  438,  465,
+      466,  467,  471,  472,  475,  439,  441,  444,  445,  446,
+      450,  451,  452,  476,  454,  477,  478,  455,  479,  456,
+      457,  481,  458,  459,  482,  461,  464,  483,  465,  466,
+      467,  471,  472,  475,  484,  485,  486,  487,  488,  490,
+
+      491,  493,  476,  495,  477,  478,  496,  479,  497,  499,
+      481,  500,  501,  482,  503,  504,  483,  505,  506,  508,
+      510,  511,  512,  484,  485,  486,  487,  488,  490,  491,
+      493,  513,  495,  514,  516,  496,  519,  497,  499,  520,
+      500,  501,  521,  503,  504,  524,  505,  506,  508,  510,
+      511,  512,  527,  528,  529,  532,  533,  535,  536,  537,
+      513,  538,  514,  516,  539,  519,  541,    0,  520,    0,
+        0,  521,    0,    0,  524,    0,    0,    0,    0,    0,
+        0,  527,  528,  529,  532,  533,  535,  536,  537,    0,
+      538,    0,    0,  539,    0,  541,  545,  545,  545,  545,
+
+      545,  545,  545,  545,  545,  545,  546,  546,  546,  546,
+      546,  546,  546,  546,  546,  546,  547,  547,  547,  547,
+      547,  547,  547,  547,  547,  547,  548,  548,  548,  548,
+      548,  548,  548,  548,  548,  548,  549,  549,  549,  549,
+      549,  549,  549,  549,  549,  549,  550,  550,  551,  551,
+      551,    0,  551,  552,  552,  552,  552,  553,  553,  553,
+        0,  553,  553,  553,  553,  553,  553,  554,  554,  554,
+        0,  554,  554,  554,  554,    0,  554,  555,  555,  555,
+      555,  555,  555,  555,  555,  555,  555,  556,  556,    0,
+      556,  556,  556,  556,  556,  556,  556,  557,    0,  557,
+
+      557,  557,  557,  557,  557,  557,  557,  558,  558,  559,
+      559,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544,  544,  544,  544,  544,  544,  544,  544,
+      544,  544,  544
     } ;
 
 /* Table of booleans, true if rule could match eol. */
-static yyconst flex_int32_t yy_rule_can_match_eol[145] =
+static yyconst flex_int32_t yy_rule_can_match_eol[151] =
     {   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, 
@@ -915,8 +927,8 @@ static yyconst flex_int32_t yy_rule_can_match_eol[145] =
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
-    0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 1, 1, 0, 0, 
-    0, 1, 0, 0, 0,     };
+    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 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.
@@ -1033,7 +1045,7 @@ class UnaryOperation;
 
 
 
-#line 1037 "SqlLexer_gen.cpp"
+#line 1049 "SqlLexer_gen.cpp"
 
 #define INITIAL 0
 #define CONDITION_SQL 1
@@ -1324,7 +1336,7 @@ YY_DECL
 #line 128 "../SqlLexer.lpp"
 
 
-#line 1328 "SqlLexer_gen.cpp"
+#line 1340 "SqlLexer_gen.cpp"
 
 	while ( /*CONSTCOND*/1 )		/* loops until end-of-file is reached */
 		{
@@ -1351,13 +1363,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 >= 527 )
+				if ( yy_current_state >= 545 )
 					yy_c = yy_meta[(unsigned int) yy_c];
 				}
 			yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
 			++yy_cp;
 			}
-		while ( yy_current_state != 526 );
+		while ( yy_current_state != 544 );
 		yy_cp = yyg->yy_last_accepting_cpos;
 		yy_current_state = yyg->yy_last_accepting_state;
 
@@ -1576,27 +1588,27 @@ return TOKEN_DATETIME;
 case 32:
 YY_RULE_SETUP
 #line 194 "../SqlLexer.lpp"
-return TOKEN_DECIMAL;
+return TOKEN_DAY;
 	YY_BREAK
 case 33:
 YY_RULE_SETUP
 #line 195 "../SqlLexer.lpp"
-return TOKEN_DEFAULT;
+return TOKEN_DECIMAL;
 	YY_BREAK
 case 34:
 YY_RULE_SETUP
 #line 196 "../SqlLexer.lpp"
-return TOKEN_DELETE;
+return TOKEN_DEFAULT;
 	YY_BREAK
 case 35:
 YY_RULE_SETUP
 #line 197 "../SqlLexer.lpp"
-return TOKEN_DELIMITER;
+return TOKEN_DELETE;
 	YY_BREAK
 case 36:
 YY_RULE_SETUP
 #line 198 "../SqlLexer.lpp"
-return TOKEN_DESC;
+return TOKEN_DELIMITER;
 	YY_BREAK
 case 37:
 YY_RULE_SETUP
@@ -1606,426 +1618,456 @@ return TOKEN_DESC;
 case 38:
 YY_RULE_SETUP
 #line 200 "../SqlLexer.lpp"
-return TOKEN_DISTINCT;
+return TOKEN_DESC;
 	YY_BREAK
 case 39:
 YY_RULE_SETUP
 #line 201 "../SqlLexer.lpp"
-return TOKEN_DOUBLE;
+return TOKEN_DISTINCT;
 	YY_BREAK
 case 40:
 YY_RULE_SETUP
 #line 202 "../SqlLexer.lpp"
-return TOKEN_DROP;
+return TOKEN_DOUBLE;
 	YY_BREAK
 case 41:
 YY_RULE_SETUP
 #line 203 "../SqlLexer.lpp"
-return TOKEN_ELSE;
+return TOKEN_DROP;
 	YY_BREAK
 case 42:
 YY_RULE_SETUP
 #line 204 "../SqlLexer.lpp"
-return TOKEN_END;
+return TOKEN_ELSE;
 	YY_BREAK
 case 43:
 YY_RULE_SETUP
 #line 205 "../SqlLexer.lpp"
-return TOKEN_ESCAPE_STRINGS;
+return TOKEN_END;
 	YY_BREAK
 case 44:
 YY_RULE_SETUP
 #line 206 "../SqlLexer.lpp"
-return TOKEN_EXISTS;
+return TOKEN_ESCAPE_STRINGS;
 	YY_BREAK
 case 45:
 YY_RULE_SETUP
 #line 207 "../SqlLexer.lpp"
-return TOKEN_EXTRACT;
+return TOKEN_EXISTS;
 	YY_BREAK
 case 46:
 YY_RULE_SETUP
 #line 208 "../SqlLexer.lpp"
-return TOKEN_FALSE;
+return TOKEN_EXTRACT;
 	YY_BREAK
 case 47:
 YY_RULE_SETUP
 #line 209 "../SqlLexer.lpp"
-return TOKEN_FIRST;
+return TOKEN_FALSE;
 	YY_BREAK
 case 48:
 YY_RULE_SETUP
 #line 210 "../SqlLexer.lpp"
-return TOKEN_FLOAT;
+return TOKEN_FIRST;
 	YY_BREAK
 case 49:
 YY_RULE_SETUP
 #line 211 "../SqlLexer.lpp"
-return TOKEN_FOREIGN;
+return TOKEN_FLOAT;
 	YY_BREAK
 case 50:
 YY_RULE_SETUP
 #line 212 "../SqlLexer.lpp"
-return TOKEN_FROM;
+return TOKEN_FOREIGN;
 	YY_BREAK
 case 51:
 YY_RULE_SETUP
 #line 213 "../SqlLexer.lpp"
-return TOKEN_FULL;
+return TOKEN_FROM;
 	YY_BREAK
 case 52:
 YY_RULE_SETUP
 #line 214 "../SqlLexer.lpp"
-return TOKEN_GROUP;
+return TOKEN_FULL;
 	YY_BREAK
 case 53:
 YY_RULE_SETUP
 #line 215 "../SqlLexer.lpp"
-return TOKEN_HASH;
+return TOKEN_GROUP;
 	YY_BREAK
 case 54:
 YY_RULE_SETUP
 #line 216 "../SqlLexer.lpp"
-return TOKEN_HAVING;
+return TOKEN_HASH;
 	YY_BREAK
 case 55:
 YY_RULE_SETUP
 #line 217 "../SqlLexer.lpp"
-return TOKEN_IN;
+return TOKEN_HAVING;
 	YY_BREAK
 case 56:
 YY_RULE_SETUP
 #line 218 "../SqlLexer.lpp"
-return TOKEN_INDEX;
+return TOKEN_HOUR;
 	YY_BREAK
 case 57:
 YY_RULE_SETUP
 #line 219 "../SqlLexer.lpp"
-return TOKEN_INNER;
+return TOKEN_IN;
 	YY_BREAK
 case 58:
 YY_RULE_SETUP
 #line 220 "../SqlLexer.lpp"
-return TOKEN_INSERT;
+return TOKEN_INDEX;
 	YY_BREAK
 case 59:
 YY_RULE_SETUP
 #line 221 "../SqlLexer.lpp"
-return TOKEN_INTEGER;
+return TOKEN_INNER;
 	YY_BREAK
 case 60:
 YY_RULE_SETUP
 #line 222 "../SqlLexer.lpp"
-return TOKEN_INTEGER;
+return TOKEN_INSERT;
 	YY_BREAK
 case 61:
 YY_RULE_SETUP
 #line 223 "../SqlLexer.lpp"
-return TOKEN_INTERVAL;
+return TOKEN_INTEGER;
 	YY_BREAK
 case 62:
 YY_RULE_SETUP
 #line 224 "../SqlLexer.lpp"
-return TOKEN_INTO;
+return TOKEN_INTEGER;
 	YY_BREAK
 case 63:
 YY_RULE_SETUP
 #line 225 "../SqlLexer.lpp"
-return TOKEN_IS;
+return TOKEN_INTERVAL;
 	YY_BREAK
 case 64:
 YY_RULE_SETUP
 #line 226 "../SqlLexer.lpp"
-return TOKEN_JOIN;
+return TOKEN_INTO;
 	YY_BREAK
 case 65:
 YY_RULE_SETUP
 #line 227 "../SqlLexer.lpp"
-return TOKEN_KEY;
+return TOKEN_IS;
 	YY_BREAK
 case 66:
 YY_RULE_SETUP
 #line 228 "../SqlLexer.lpp"
-return TOKEN_LAST;
+return TOKEN_JOIN;
 	YY_BREAK
 case 67:
 YY_RULE_SETUP
 #line 229 "../SqlLexer.lpp"
-return TOKEN_LEFT;
+return TOKEN_KEY;
 	YY_BREAK
 case 68:
 YY_RULE_SETUP
 #line 230 "../SqlLexer.lpp"
-return TOKEN_LIKE;
+return TOKEN_LAST;
 	YY_BREAK
 case 69:
 YY_RULE_SETUP
 #line 231 "../SqlLexer.lpp"
-return TOKEN_LIMIT;
+return TOKEN_LEFT;
 	YY_BREAK
 case 70:
 YY_RULE_SETUP
 #line 232 "../SqlLexer.lpp"
-return TOKEN_LONG;
+return TOKEN_LIKE;
 	YY_BREAK
 case 71:
 YY_RULE_SETUP
 #line 233 "../SqlLexer.lpp"
-return TOKEN_NOT;
+return TOKEN_LIMIT;
 	YY_BREAK
 case 72:
 YY_RULE_SETUP
 #line 234 "../SqlLexer.lpp"
-return TOKEN_NULL;
+return TOKEN_LONG;
 	YY_BREAK
 case 73:
 YY_RULE_SETUP
 #line 235 "../SqlLexer.lpp"
-return TOKEN_NULLS;
+return TOKEN_MINUTE;
 	YY_BREAK
 case 74:
 YY_RULE_SETUP
 #line 236 "../SqlLexer.lpp"
-return TOKEN_OFF;
+return TOKEN_MONTH;
 	YY_BREAK
 case 75:
 YY_RULE_SETUP
 #line 237 "../SqlLexer.lpp"
-return TOKEN_ON;
+return TOKEN_NOT;
 	YY_BREAK
 case 76:
 YY_RULE_SETUP
 #line 238 "../SqlLexer.lpp"
-return TOKEN_OR;
+return TOKEN_NULL;
 	YY_BREAK
 case 77:
 YY_RULE_SETUP
 #line 239 "../SqlLexer.lpp"
-return TOKEN_ORDER;
+return TOKEN_NULLS;
 	YY_BREAK
 case 78:
 YY_RULE_SETUP
 #line 240 "../SqlLexer.lpp"
-return TOKEN_OUTER;
+return TOKEN_OFF;
 	YY_BREAK
 case 79:
 YY_RULE_SETUP
 #line 241 "../SqlLexer.lpp"
-return TOKEN_PARTITION;
+return TOKEN_ON;
 	YY_BREAK
 case 80:
 YY_RULE_SETUP
 #line 242 "../SqlLexer.lpp"
-return TOKEN_PARTITIONS;
+return TOKEN_OR;
 	YY_BREAK
 case 81:
 YY_RULE_SETUP
 #line 243 "../SqlLexer.lpp"
-return TOKEN_PERCENT;
+return TOKEN_ORDER;
 	YY_BREAK
 case 82:
 YY_RULE_SETUP
 #line 244 "../SqlLexer.lpp"
-return TOKEN_PRIMARY;
+return TOKEN_OUTER;
 	YY_BREAK
 case 83:
 YY_RULE_SETUP
 #line 245 "../SqlLexer.lpp"
-return TOKEN_QUIT;
+return TOKEN_PARTITION;
 	YY_BREAK
 case 84:
 YY_RULE_SETUP
 #line 246 "../SqlLexer.lpp"
-return TOKEN_RANGE;
+return TOKEN_PARTITIONS;
 	YY_BREAK
 case 85:
 YY_RULE_SETUP
 #line 247 "../SqlLexer.lpp"
-return TOKEN_REAL;
+return TOKEN_PERCENT;
 	YY_BREAK
 case 86:
 YY_RULE_SETUP
 #line 248 "../SqlLexer.lpp"
-return TOKEN_REFERENCES;
+return TOKEN_PRIMARY;
 	YY_BREAK
 case 87:
 YY_RULE_SETUP
 #line 249 "../SqlLexer.lpp"
-return TOKEN_REGEXP;
+return TOKEN_QUIT;
 	YY_BREAK
 case 88:
 YY_RULE_SETUP
 #line 250 "../SqlLexer.lpp"
-return TOKEN_RIGHT;
+return TOKEN_RANGE;
 	YY_BREAK
 case 89:
 YY_RULE_SETUP
 #line 251 "../SqlLexer.lpp"
-return TOKEN_ROW_DELIMITER;
+return TOKEN_REAL;
 	YY_BREAK
 case 90:
 YY_RULE_SETUP
 #line 252 "../SqlLexer.lpp"
-return TOKEN_SELECT;
+return TOKEN_REFERENCES;
 	YY_BREAK
 case 91:
 YY_RULE_SETUP
 #line 253 "../SqlLexer.lpp"
-return TOKEN_SET;
+return TOKEN_REGEXP;
 	YY_BREAK
 case 92:
 YY_RULE_SETUP
 #line 254 "../SqlLexer.lpp"
-return TOKEN_SMA;
+return TOKEN_RIGHT;
 	YY_BREAK
 case 93:
 YY_RULE_SETUP
 #line 255 "../SqlLexer.lpp"
-return TOKEN_SMALLINT;
+return TOKEN_ROW_DELIMITER;
 	YY_BREAK
 case 94:
 YY_RULE_SETUP
 #line 256 "../SqlLexer.lpp"
-return TOKEN_TABLE;
+return TOKEN_SECOND;
 	YY_BREAK
 case 95:
 YY_RULE_SETUP
 #line 257 "../SqlLexer.lpp"
-return TOKEN_THEN;
+return TOKEN_SELECT;
 	YY_BREAK
 case 96:
 YY_RULE_SETUP
 #line 258 "../SqlLexer.lpp"
-return TOKEN_TIME;
+return TOKEN_SET;
 	YY_BREAK
 case 97:
 YY_RULE_SETUP
 #line 259 "../SqlLexer.lpp"
-return TOKEN_TIMESTAMP;
+return TOKEN_SMA;
 	YY_BREAK
 case 98:
 YY_RULE_SETUP
 #line 260 "../SqlLexer.lpp"
-return TOKEN_TRUE;
+return TOKEN_SMALLINT;
 	YY_BREAK
 case 99:
 YY_RULE_SETUP
 #line 261 "../SqlLexer.lpp"
-return TOKEN_TUPLESAMPLE;
+return TOKEN_TABLE;
 	YY_BREAK
 case 100:
 YY_RULE_SETUP
 #line 262 "../SqlLexer.lpp"
-return TOKEN_UNIQUE;
+return TOKEN_THEN;
 	YY_BREAK
 case 101:
 YY_RULE_SETUP
 #line 263 "../SqlLexer.lpp"
-return TOKEN_UPDATE;
+return TOKEN_TIME;
 	YY_BREAK
 case 102:
 YY_RULE_SETUP
 #line 264 "../SqlLexer.lpp"
-return TOKEN_USING;
+return TOKEN_TIMESTAMP;
 	YY_BREAK
 case 103:
 YY_RULE_SETUP
 #line 265 "../SqlLexer.lpp"
-return TOKEN_VALUES;
+return TOKEN_TRUE;
 	YY_BREAK
 case 104:
 YY_RULE_SETUP
 #line 266 "../SqlLexer.lpp"
-return TOKEN_VARCHAR;
+return TOKEN_TUPLESAMPLE;
 	YY_BREAK
 case 105:
 YY_RULE_SETUP
 #line 267 "../SqlLexer.lpp"
-return TOKEN_WHEN;
+return TOKEN_UNIQUE;
 	YY_BREAK
 case 106:
 YY_RULE_SETUP
 #line 268 "../SqlLexer.lpp"
-return TOKEN_WHERE;
+return TOKEN_UPDATE;
 	YY_BREAK
 case 107:
 YY_RULE_SETUP
 #line 269 "../SqlLexer.lpp"
-return TOKEN_WITH;
+return TOKEN_USING;
 	YY_BREAK
 case 108:
 YY_RULE_SETUP
 #line 270 "../SqlLexer.lpp"
-return TOKEN_YEARMONTH;
+return TOKEN_VALUES;
 	YY_BREAK
 case 109:
 YY_RULE_SETUP
-#line 272 "../SqlLexer.lpp"
-return TOKEN_EQ;
+#line 271 "../SqlLexer.lpp"
+return TOKEN_VARCHAR;
 	YY_BREAK
 case 110:
 YY_RULE_SETUP
-#line 273 "../SqlLexer.lpp"
-return TOKEN_NEQ;
+#line 272 "../SqlLexer.lpp"
+return TOKEN_WHEN;
 	YY_BREAK
 case 111:
 YY_RULE_SETUP
-#line 274 "../SqlLexer.lpp"
-return TOKEN_NEQ;
+#line 273 "../SqlLexer.lpp"
+return TOKEN_WHERE;
 	YY_BREAK
 case 112:
 YY_RULE_SETUP
-#line 275 "../SqlLexer.lpp"
-return TOKEN_LT;
+#line 274 "../SqlLexer.lpp"
+return TOKEN_WITH;
 	YY_BREAK
 case 113:
 YY_RULE_SETUP
-#line 276 "../SqlLexer.lpp"
-return TOKEN_GT;
+#line 275 "../SqlLexer.lpp"
+return TOKEN_YEAR;
 	YY_BREAK
 case 114:
 YY_RULE_SETUP
-#line 277 "../SqlLexer.lpp"
-return TOKEN_LEQ;
+#line 276 "../SqlLexer.lpp"
+return TOKEN_YEARMONTH;
 	YY_BREAK
 case 115:
 YY_RULE_SETUP
 #line 278 "../SqlLexer.lpp"
-return TOKEN_GEQ;
+return TOKEN_EQ;
 	YY_BREAK
 case 116:
 YY_RULE_SETUP
-#line 280 "../SqlLexer.lpp"
-return yytext[0];
+#line 279 "../SqlLexer.lpp"
+return TOKEN_NEQ;
 	YY_BREAK
 case 117:
 YY_RULE_SETUP
+#line 280 "../SqlLexer.lpp"
+return TOKEN_NEQ;
+	YY_BREAK
+case 118:
+YY_RULE_SETUP
 #line 281 "../SqlLexer.lpp"
+return TOKEN_LT;
+	YY_BREAK
+case 119:
+YY_RULE_SETUP
+#line 282 "../SqlLexer.lpp"
+return TOKEN_GT;
+	YY_BREAK
+case 120:
+YY_RULE_SETUP
+#line 283 "../SqlLexer.lpp"
+return TOKEN_LEQ;
+	YY_BREAK
+case 121:
+YY_RULE_SETUP
+#line 284 "../SqlLexer.lpp"
+return TOKEN_GEQ;
+	YY_BREAK
+case 122:
+YY_RULE_SETUP
+#line 286 "../SqlLexer.lpp"
+return yytext[0];
+	YY_BREAK
+case 123:
+YY_RULE_SETUP
+#line 287 "../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 118:
+case 124:
 YY_RULE_SETUP
-#line 287 "../SqlLexer.lpp"
+#line 293 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED_ESCAPED);
   }
 	YY_BREAK
-case 119:
+case 125:
 YY_RULE_SETUP
-#line 292 "../SqlLexer.lpp"
+#line 298 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED);
   }
 	YY_BREAK
-case 120:
+case 126:
 YY_RULE_SETUP
-#line 297 "../SqlLexer.lpp"
+#line 303 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_DOUBLE_QUOTED);
@@ -2037,7 +2079,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 306 "../SqlLexer.lpp"
+#line 312 "../SqlLexer.lpp"
 {
     delete yylval->string_value_;
     BEGIN(INITIAL);
@@ -2048,9 +2090,9 @@ case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
 
 /* Process escape sequences. */
 
-case 121:
+case 127:
 YY_RULE_SETUP
-#line 316 "../SqlLexer.lpp"
+#line 322 "../SqlLexer.lpp"
 {
     /* Octal code */
     unsigned int code;
@@ -2064,9 +2106,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 122:
+case 128:
 YY_RULE_SETUP
-#line 328 "../SqlLexer.lpp"
+#line 334 "../SqlLexer.lpp"
 {
     /* Hexadecimal code */
     unsigned int code;
@@ -2074,9 +2116,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 123:
+case 129:
 YY_RULE_SETUP
-#line 334 "../SqlLexer.lpp"
+#line 340 "../SqlLexer.lpp"
 {
     /* A numeric escape sequence that isn't correctly specified. */
     delete yylval->string_value_;
@@ -2085,58 +2127,58 @@ YY_RULE_SETUP
     return TOKEN_LEX_ERROR;
   }
 	YY_BREAK
-case 124:
+case 130:
 YY_RULE_SETUP
-#line 341 "../SqlLexer.lpp"
+#line 347 "../SqlLexer.lpp"
 {
     /* Backspace */
     yylval->string_value_->push_back('\b');
   }
 	YY_BREAK
-case 125:
+case 131:
 YY_RULE_SETUP
-#line 345 "../SqlLexer.lpp"
+#line 351 "../SqlLexer.lpp"
 {
     /* Form-feed */
     yylval->string_value_->push_back('\f');
   }
 	YY_BREAK
-case 126:
+case 132:
 YY_RULE_SETUP
-#line 349 "../SqlLexer.lpp"
+#line 355 "../SqlLexer.lpp"
 {
     /* Newline */
     yylval->string_value_->push_back('\n');
   }
 	YY_BREAK
-case 127:
+case 133:
 YY_RULE_SETUP
-#line 353 "../SqlLexer.lpp"
+#line 359 "../SqlLexer.lpp"
 {
     /* Carriage-return */
     yylval->string_value_->push_back('\r');
   }
 	YY_BREAK
-case 128:
+case 134:
 YY_RULE_SETUP
-#line 357 "../SqlLexer.lpp"
+#line 363 "../SqlLexer.lpp"
 {
     /* Horizontal Tab */
     yylval->string_value_->push_back('\t');
   }
 	YY_BREAK
-case 129:
-/* rule 129 can match eol */
+case 135:
+/* rule 135 can match eol */
 YY_RULE_SETUP
-#line 361 "../SqlLexer.lpp"
+#line 367 "../SqlLexer.lpp"
 {
     /* Any other character (including actual newline or carriage return) */
     yylval->string_value_->push_back(yytext[1]);
   }
 	YY_BREAK
-case 130:
+case 136:
 YY_RULE_SETUP
-#line 365 "../SqlLexer.lpp"
+#line 371 "../SqlLexer.lpp"
 {
     /* This should only be encountered right before an EOF. */
     delete yylval->string_value_;
@@ -2147,17 +2189,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 131:
+case 137:
 YY_RULE_SETUP
-#line 375 "../SqlLexer.lpp"
+#line 381 "../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 132:
+case 138:
 YY_RULE_SETUP
-#line 379 "../SqlLexer.lpp"
+#line 385 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2166,17 +2208,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 133:
+case 139:
 YY_RULE_SETUP
-#line 387 "../SqlLexer.lpp"
+#line 393 "../SqlLexer.lpp"
 {
     /* Two quotes in a row become a single quote (this is specified by the SQL standard). */
     yylval->string_value_->push_back('"');
   }
 	YY_BREAK
-case 134:
+case 140:
 YY_RULE_SETUP
-#line 391 "../SqlLexer.lpp"
+#line 397 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2184,94 +2226,94 @@ YY_RULE_SETUP
   }
 	YY_BREAK
 
-case 135:
-/* rule 135 can match eol */
+case 141:
+/* rule 141 can match eol */
 YY_RULE_SETUP
-#line 398 "../SqlLexer.lpp"
+#line 404 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 136:
-/* rule 136 can match eol */
+case 142:
+/* rule 142 can match eol */
 YY_RULE_SETUP
-#line 403 "../SqlLexer.lpp"
+#line 409 "../SqlLexer.lpp"
 {
   /* Scan up to a quote or escape sequence. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 137:
-/* rule 137 can match eol */
+case 143:
+/* rule 143 can match eol */
 YY_RULE_SETUP
-#line 408 "../SqlLexer.lpp"
+#line 414 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
 
-case 138:
+case 144:
 YY_RULE_SETUP
-#line 414 "../SqlLexer.lpp"
+#line 420 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(
         yylloc->first_line, yylloc->first_column, std::string(yytext, yyleng));
     return TOKEN_NAME;
   }
 	YY_BREAK
-case 139:
+case 145:
 YY_RULE_SETUP
-#line 420 "../SqlLexer.lpp"
+#line 426 "../SqlLexer.lpp"
 {
     yylval->numeric_literal_value_ = new quickstep::NumericParseLiteralValue(
         yylloc->first_line, yylloc->first_column, yytext);
     return TOKEN_UNSIGNED_NUMVAL;
   }
 	YY_BREAK
-case 140:
+case 146:
 YY_RULE_SETUP
-#line 426 "../SqlLexer.lpp"
+#line 432 "../SqlLexer.lpp"
 /* comment */
 	YY_BREAK
-case 141:
-/* rule 141 can match eol */
+case 147:
+/* rule 147 can match eol */
 YY_RULE_SETUP
-#line 428 "../SqlLexer.lpp"
+#line 434 "../SqlLexer.lpp"
 { yycolumn = 0; }
 	YY_BREAK
-case 142:
+case 148:
 YY_RULE_SETUP
-#line 430 "../SqlLexer.lpp"
+#line 436 "../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 434 "../SqlLexer.lpp"
+#line 440 "../SqlLexer.lpp"
 {
   /* All conditions except for mutli-state string extracting conditions. */
   BEGIN(INITIAL);
   return TOKEN_EOF;
 }
 	YY_BREAK
-case 143:
+case 149:
 YY_RULE_SETUP
-#line 440 "../SqlLexer.lpp"
+#line 446 "../SqlLexer.lpp"
 {
   BEGIN(INITIAL);
   quickstep_yyerror(NULL, yyscanner, NULL, "illegal character");
   return TOKEN_LEX_ERROR;
 }
 	YY_BREAK
-case 144:
+case 150:
 YY_RULE_SETUP
-#line 446 "../SqlLexer.lpp"
+#line 452 "../SqlLexer.lpp"
 YY_FATAL_ERROR( "flex scanner jammed" );
 	YY_BREAK
-#line 2275 "SqlLexer_gen.cpp"
+#line 2317 "SqlLexer_gen.cpp"
 
 	case YY_END_OF_BUFFER:
 		{
@@ -2565,7 +2607,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 >= 527 )
+			if ( yy_current_state >= 545 )
 				yy_c = yy_meta[(unsigned int) yy_c];
 			}
 		yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
@@ -2594,11 +2636,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 >= 527 )
+		if ( yy_current_state >= 545 )
 			yy_c = yy_meta[(unsigned int) yy_c];
 		}
 	yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
-	yy_is_jam = (yy_current_state == 526);
+	yy_is_jam = (yy_current_state == 544);
 
 	(void)yyg;
 	return yy_is_jam ? 0 : yy_current_state;
@@ -3432,7 +3474,7 @@ void quickstep_yyfree (void * ptr , yyscan_t yyscanner)
 
 #define YYTABLES_NAME "yytables"
 
-#line 446 "../SqlLexer.lpp"
+#line 452 "../SqlLexer.lpp"
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f79a9af/parser/preprocessed/SqlLexer_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.hpp b/parser/preprocessed/SqlLexer_gen.hpp
index b30d697..d629f04 100644
--- a/parser/preprocessed/SqlLexer_gen.hpp
+++ b/parser/preprocessed/SqlLexer_gen.hpp
@@ -360,7 +360,7 @@ extern int quickstep_yylex \
 #undef YY_DECL
 #endif
 
-#line 446 "../SqlLexer.lpp"
+#line 452 "../SqlLexer.lpp"
 
 
 #line 367 "SqlLexer_gen.hpp"


[07/30] incubator-quickstep git commit: Adds support for scalar subqueries (#185)

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/parser/tests/Select.test
----------------------------------------------------------------------
diff --git a/parser/tests/Select.test b/parser/tests/Select.test
index 8a10a12..e70ee5c 100644
--- a/parser/tests/Select.test
+++ b/parser/tests/Select.test
@@ -543,16 +543,40 @@ SelectStatement
 # Subqueries are not supported yet in clauses other than the FROM clause.
 SELECT (select * FROM test) FROM test
 --
-ERROR: syntax error (1 : 9)
-SELECT (select * FROM test) FROM test
-        ^
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  |   +-SubqueryExpression
+  |     +-Select
+  |       +-select_clause=SelectStar
+  |       +-from_clause=
+  |         +-TableReference[table=test]
+  +-from_clause=
+    +-TableReference[table=test]
 ==
 
 SELECT 1 FROM test WHERE 1 > (select 1 FROM test)
 --
-ERROR: syntax error (1 : 31)
-SELECT 1 FROM test WHERE 1 > (select 1 FROM test)
-                              ^
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  |   +-Literal
+  |     +-NumericLiteral[numeric_string=1,float_like=false]
+  +-where_clause=Greater
+  | +-left_operand=Literal
+  | | +-NumericLiteral[numeric_string=1,float_like=false]
+  | +-right_operand=SubqueryExpression
+  |   +-Select
+  |     +-select_clause=SelectList
+  |     | +-SelectListItem
+  |     |   +-Literal
+  |     |     +-NumericLiteral[numeric_string=1,float_like=false]
+  |     +-from_clause=
+  |       +-TableReference[table=test]
+  +-from_clause=
+    +-TableReference[table=test]
 ==
 
 #

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/parser/tests/TPCH.test
----------------------------------------------------------------------
diff --git a/parser/tests/TPCH.test b/parser/tests/TPCH.test
index 2d12df5..12d85ce 100644
--- a/parser/tests/TPCH.test
+++ b/parser/tests/TPCH.test
@@ -148,9 +148,97 @@ ORDER BY
   p_partkey
 LIMIT 100
 --
-ERROR: syntax error (25 : 5)
-    SELECT
-    ^
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=s_acctbal]
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=s_name]
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=n_name]
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=p_partkey]
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=p_mfgr]
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=s_address]
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=s_phone]
+  | +-SelectListItem
+  |   +-AttributeReference[attribute_name=s_comment]
+  +-where_clause=And
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=p_partkey]
+  | | +-right_operand=AttributeReference[attribute_name=ps_partkey]
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+  | | +-right_operand=AttributeReference[attribute_name=ps_suppkey]
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=p_size]
+  | | +-right_operand=Literal
+  | |   +-NumericLiteral[numeric_string=48,float_like=false]
+  | +-Like
+  | | +-left_operand=AttributeReference[attribute_name=p_type]
+  | | +-right_operand=Literal
+  | |   +-StringLiteral[value=%NICKEL]
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+  | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=n_regionkey]
+  | | +-right_operand=AttributeReference[attribute_name=r_regionkey]
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=r_name]
+  | | +-right_operand=Literal
+  | |   +-StringLiteral[value=ASIA]
+  | +-Equal
+  |   +-left_operand=AttributeReference[attribute_name=ps_supplycost]
+  |   +-right_operand=SubqueryExpression
+  |     +-Select
+  |       +-select_clause=SelectList
+  |       | +-SelectListItem
+  |       |   +-FunctionCall[name=MIN]
+  |       |     +-AttributeReference[attribute_name=ps_supplycost]
+  |       +-where_clause=And
+  |       | +-Equal
+  |       | | +-left_operand=AttributeReference[attribute_name=p_partkey]
+  |       | | +-right_operand=AttributeReference[attribute_name=ps_partkey]
+  |       | +-Equal
+  |       | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+  |       | | +-right_operand=AttributeReference[attribute_name=ps_suppkey]
+  |       | +-Equal
+  |       | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+  |       | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
+  |       | +-Equal
+  |       | | +-left_operand=AttributeReference[attribute_name=n_regionkey]
+  |       | | +-right_operand=AttributeReference[attribute_name=r_regionkey]
+  |       | +-Equal
+  |       |   +-left_operand=AttributeReference[attribute_name=r_name]
+  |       |   +-right_operand=Literal
+  |       |     +-StringLiteral[value=ASIA]
+  |       +-from_clause=
+  |         +-TableReference[table=partsupp]
+  |         +-TableReference[table=supplier]
+  |         +-TableReference[table=nation]
+  |         +-TableReference[table=region]
+  +-order_by=OrderBy
+  | +-OrderByItem[is_asc=false,nulls_first=true]
+  | | +-AttributeReference[attribute_name=s_acctbal]
+  | +-OrderByItem[is_asc=true,nulls_first=false]
+  | | +-AttributeReference[attribute_name=n_name]
+  | +-OrderByItem[is_asc=true,nulls_first=false]
+  | | +-AttributeReference[attribute_name=s_name]
+  | +-OrderByItem[is_asc=true,nulls_first=false]
+  |   +-AttributeReference[attribute_name=p_partkey]
+  +-limit=LIMIT
+  | +-NumericLiteral[numeric_string=100,float_like=false]
+  +-from_clause=
+    +-TableReference[table=part]
+    +-TableReference[table=supplier]
+    +-TableReference[table=partsupp]
+    +-TableReference[table=nation]
+    +-TableReference[table=region]
 ==
 
 # Query 3
@@ -949,9 +1037,70 @@ GROUP BY
 ORDER BY
   value DESC
 --
-ERROR: syntax error (15 : 7)
-      SELECT
-      ^
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=ps_partkey]
+  | +-SelectListItem[alias=value]
+  |   +-FunctionCall[name=SUM]
+  |     +-Multiply
+  |       +-left_operand=AttributeReference[attribute_name=ps_supplycost]
+  |       +-right_operand=AttributeReference[attribute_name=ps_availqty]
+  +-where_clause=And
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=ps_suppkey]
+  | | +-right_operand=AttributeReference[attribute_name=s_suppkey]
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+  | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
+  | +-Equal
+  |   +-left_operand=AttributeReference[attribute_name=n_name]
+  |   +-right_operand=Literal
+  |     +-StringLiteral[value=INDONESIA]
+  +-group_by=GroupBy
+  | +-AttributeReference[attribute_name=ps_partkey]
+  +-having=HAVING
+  | +-Greater
+  |   +-left_operand=FunctionCall[name=SUM]
+  |   | +-Multiply
+  |   |   +-left_operand=AttributeReference[attribute_name=ps_supplycost]
+  |   |   +-right_operand=AttributeReference[attribute_name=ps_availqty]
+  |   +-right_operand=SubqueryExpression
+  |     +-Select
+  |       +-select_clause=SelectList
+  |       | +-SelectListItem
+  |       |   +-Multiply
+  |       |     +-left_operand=FunctionCall[name=SUM]
+  |       |     | +-Multiply
+  |       |     |   +-left_operand=AttributeReference[
+  |       |     |   | attribute_name=ps_supplycost]
+  |       |     |   +-right_operand=AttributeReference[
+  |       |     |     attribute_name=ps_availqty]
+  |       |     +-right_operand=Literal
+  |       |       +-NumericLiteral[numeric_string=0.0000010000,float_like=true]
+  |       +-where_clause=And
+  |       | +-Equal
+  |       | | +-left_operand=AttributeReference[attribute_name=ps_suppkey]
+  |       | | +-right_operand=AttributeReference[attribute_name=s_suppkey]
+  |       | +-Equal
+  |       | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+  |       | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
+  |       | +-Equal
+  |       |   +-left_operand=AttributeReference[attribute_name=n_name]
+  |       |   +-right_operand=Literal
+  |       |     +-StringLiteral[value=INDONESIA]
+  |       +-from_clause=
+  |         +-TableReference[table=partsupp]
+  |         +-TableReference[table=supplier]
+  |         +-TableReference[table=nation]
+  +-order_by=OrderBy
+  | +-OrderByItem[is_asc=false,nulls_first=true]
+  |   +-AttributeReference[attribute_name=value]
+  +-from_clause=
+    +-TableReference[table=partsupp]
+    +-TableReference[table=supplier]
+    +-TableReference[table=nation]
 ==
 
 # Query 12
@@ -1230,9 +1379,72 @@ FROM
 ORDER BY
   s_suppkey
 --
-ERROR: syntax error (23 : 1)
-SELECT
-^
+SelectStatement
++-select_query=Select
+| +-select_clause=SelectList
+| | +-SelectListItem
+| | | +-AttributeReference[attribute_name=s_suppkey]
+| | +-SelectListItem
+| | | +-AttributeReference[attribute_name=s_name]
+| | +-SelectListItem
+| | | +-AttributeReference[attribute_name=s_address]
+| | +-SelectListItem
+| | | +-AttributeReference[attribute_name=s_phone]
+| | +-SelectListItem
+| |   +-AttributeReference[attribute_name=total_revenue]
+| +-where_clause=And
+| | +-Equal
+| | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+| | | +-right_operand=AttributeReference[attribute_name=supplier_no]
+| | +-Equal
+| |   +-left_operand=AttributeReference[attribute_name=total_revenue]
+| |   +-right_operand=SubqueryExpression
+| |     +-Select
+| |       +-select_clause=SelectList
+| |       | +-SelectListItem
+| |       |   +-FunctionCall[name=MAX]
+| |       |     +-AttributeReference[attribute_name=total_revenue]
+| |       +-from_clause=
+| |         +-TableReference[table=revenue]
+| +-order_by=OrderBy
+| | +-OrderByItem[is_asc=true,nulls_first=false]
+| |   +-AttributeReference[attribute_name=s_suppkey]
+| +-from_clause=
+|   +-TableReference[table=supplier]
+|   +-TableReference[table=revenue]
++-with_clause=
+  +-SubqueryTable
+    +-table_signature=TableSignature[table_alias=revenue,
+    | columns=(supplier_no, total_revenue)]
+    +-SubqueryExpression
+      +-Select
+        +-select_clause=SelectList
+        | +-SelectListItem
+        | | +-AttributeReference[attribute_name=l_suppkey]
+        | +-SelectListItem
+        |   +-FunctionCall[name=sum]
+        |     +-Multiply
+        |       +-left_operand=AttributeReference[attribute_name=l_extendedprice]
+        |       +-right_operand=Subtract
+        |         +-left_operand=Literal
+        |         | +-NumericLiteral[numeric_string=1,float_like=false]
+        |         +-right_operand=AttributeReference[attribute_name=l_discount]
+        +-where_clause=And
+        | +-GreaterOrEqual
+        | | +-left_operand=AttributeReference[attribute_name=l_shipdate]
+        | | +-right_operand=Literal
+        | |   +-StringLiteral[value=1996-11-01,explicit_type=Datetime]
+        | +-Less
+        |   +-left_operand=AttributeReference[attribute_name=l_shipdate]
+        |   +-right_operand=Add
+        |     +-left_operand=Literal
+        |     | +-StringLiteral[value=1996-11-01,explicit_type=Datetime]
+        |     +-right_operand=Literal
+        |       +-StringLiteral[value=3 month,explicit_type=YearMonthInterval]
+        +-group_by=GroupBy
+        | +-AttributeReference[attribute_name=l_suppkey]
+        +-from_clause=
+          +-TableReference[table=lineitem]
 ==
 
 # Query 16
@@ -1361,9 +1573,46 @@ WHERE
       l_partkey = p_partkey
   )
 --
-ERROR: syntax error (11 : 5)
-    SELECT
-    ^
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem[alias=avg_yearly]
+  |   +-Divide
+  |     +-left_operand=FunctionCall[name=SUM]
+  |     | +-AttributeReference[attribute_name=l_extendedprice]
+  |     +-right_operand=Literal
+  |       +-NumericLiteral[numeric_string=7.0,float_like=true]
+  +-where_clause=And
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=p_partkey]
+  | | +-right_operand=AttributeReference[attribute_name=l_partkey]
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=p_brand]
+  | | +-right_operand=Literal
+  | |   +-StringLiteral[value=Brand#24]
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=p_container]
+  | | +-right_operand=Literal
+  | |   +-StringLiteral[value=JUMBO BOX]
+  | +-Less
+  |   +-left_operand=AttributeReference[attribute_name=l_quantity]
+  |   +-right_operand=SubqueryExpression
+  |     +-Select
+  |       +-select_clause=SelectList
+  |       | +-SelectListItem
+  |       |   +-Multiply
+  |       |     +-left_operand=Literal
+  |       |     | +-NumericLiteral[numeric_string=0.2,float_like=true]
+  |       |     +-right_operand=FunctionCall[name=AVG]
+  |       |       +-AttributeReference[attribute_name=l_quantity]
+  |       +-where_clause=Equal
+  |       | +-left_operand=AttributeReference[attribute_name=l_partkey]
+  |       | +-right_operand=AttributeReference[attribute_name=p_partkey]
+  |       +-from_clause=
+  |         +-TableReference[table=lineitem]
+  +-from_clause=
+    +-TableReference[table=lineitem]
+    +-TableReference[table=part]
 ==
 
 # Query 18
@@ -1694,9 +1943,87 @@ WHERE
 ORDER BY
   s_name
 --
-ERROR: syntax error (23 : 9)
-        SELECT
-        ^
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=s_name]
+  | +-SelectListItem
+  |   +-AttributeReference[attribute_name=s_address]
+  +-where_clause=And
+  | +-InTableQuery
+  | | +-test_expression=AttributeReference[attribute_name=s_suppkey]
+  | | +-table_query=SubqueryExpression
+  | |   +-Select
+  | |     +-select_clause=SelectList
+  | |     | +-SelectListItem
+  | |     |   +-AttributeReference[attribute_name=ps_suppkey]
+  | |     +-where_clause=And
+  | |     | +-InTableQuery
+  | |     | | +-test_expression=AttributeReference[attribute_name=ps_partkey]
+  | |     | | +-table_query=SubqueryExpression
+  | |     | |   +-Select
+  | |     | |     +-select_clause=SelectList
+  | |     | |     | +-SelectListItem
+  | |     | |     |   +-AttributeReference[attribute_name=p_partkey]
+  | |     | |     +-where_clause=Like
+  | |     | |     | +-left_operand=AttributeReference[attribute_name=p_name]
+  | |     | |     | +-right_operand=Literal
+  | |     | |     |   +-StringLiteral[value=sandy%]
+  | |     | |     +-from_clause=
+  | |     | |       +-TableReference[table=part]
+  | |     | +-Greater
+  | |     |   +-left_operand=AttributeReference[attribute_name=ps_availqty]
+  | |     |   +-right_operand=SubqueryExpression
+  | |     |     +-Select
+  | |     |       +-select_clause=SelectList
+  | |     |       | +-SelectListItem
+  | |     |       |   +-Multiply
+  | |     |       |     +-left_operand=Literal
+  | |     |       |     | +-NumericLiteral[numeric_string=0.5,float_like=true]
+  | |     |       |     +-right_operand=FunctionCall[name=SUM]
+  | |     |       |       +-AttributeReference[attribute_name=l_quantity]
+  | |     |       +-where_clause=And
+  | |     |       | +-Equal
+  | |     |       | | +-left_operand=AttributeReference[attribute_name=l_partkey]
+  | |     |       | | +-right_operand=AttributeReference[
+  | |     |       | |   attribute_name=ps_partkey]
+  | |     |       | +-Equal
+  | |     |       | | +-left_operand=AttributeReference[attribute_name=l_suppkey]
+  | |     |       | | +-right_operand=AttributeReference[
+  | |     |       | |   attribute_name=ps_suppkey]
+  | |     |       | +-GreaterOrEqual
+  | |     |       | | +-left_operand=AttributeReference[
+  | |     |       | | | attribute_name=l_shipdate]
+  | |     |       | | +-right_operand=Literal
+  | |     |       | |   +-StringLiteral[value=1993-01-01,explicit_type=Datetime]
+  | |     |       | +-Less
+  | |     |       |   +-left_operand=AttributeReference[
+  | |     |       |   | attribute_name=l_shipdate]
+  | |     |       |   +-right_operand=Add
+  | |     |       |     +-left_operand=Literal
+  | |     |       |     | +-StringLiteral[value=1993-01-01,
+  | |     |       |     |   explicit_type=Datetime]
+  | |     |       |     +-right_operand=Literal
+  | |     |       |       +-StringLiteral[value=1 year,
+  | |     |       |         explicit_type=YearMonthInterval]
+  | |     |       +-from_clause=
+  | |     |         +-TableReference[table=lineitem]
+  | |     +-from_clause=
+  | |       +-TableReference[table=partsupp]
+  | +-Equal
+  | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+  | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
+  | +-Equal
+  |   +-left_operand=AttributeReference[attribute_name=n_name]
+  |   +-right_operand=Literal
+  |     +-StringLiteral[value=GERMANY]
+  +-order_by=OrderBy
+  | +-OrderByItem[is_asc=true,nulls_first=false]
+  |   +-AttributeReference[attribute_name=s_name]
+  +-from_clause=
+    +-TableReference[table=supplier]
+    +-TableReference[table=nation]
 ==
 
 # Query 21
@@ -1870,6 +2197,106 @@ GROUP BY
 ORDER BY
   cntrycode
 --
-ERROR: syntax error (16 : 9)
-        SELECT
-        ^
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=cntrycode]
+  | +-SelectListItem[alias=numcust]
+  | | +-FunctionCall[name=COUNT,is_star=true]
+  | +-SelectListItem[alias=totacctbal]
+  |   +-FunctionCall[name=SUM]
+  |     +-AttributeReference[attribute_name=c_acctbal]
+  +-group_by=GroupBy
+  | +-AttributeReference[attribute_name=cntrycode]
+  +-order_by=OrderBy
+  | +-OrderByItem[is_asc=true,nulls_first=false]
+  |   +-AttributeReference[attribute_name=cntrycode]
+  +-from_clause=
+    +-SubqueryTable
+      +-table_signature=TableSignature[table_alias=custsale]
+      +-SubqueryExpression
+        +-Select
+          +-select_clause=SelectList
+          | +-SelectListItem[alias=cntrycode]
+          | | +-FunctionCall[name=SUBSTR]
+          | |   +-AttributeReference[attribute_name=c_phone]
+          | |   +-Literal
+          | |   | +-NumericLiteral[numeric_string=1,float_like=false]
+          | |   +-Literal
+          | |     +-NumericLiteral[numeric_string=2,float_like=false]
+          | +-SelectListItem
+          |   +-AttributeReference[attribute_name=c_acctbal]
+          +-where_clause=And
+          | +-InValueList
+          | | +-test_expression=FunctionCall[name=SUBSTR]
+          | | | +-AttributeReference[attribute_name=c_phone]
+          | | | +-Literal
+          | | | | +-NumericLiteral[numeric_string=1,float_like=false]
+          | | | +-Literal
+          | | |   +-NumericLiteral[numeric_string=2,float_like=false]
+          | | +-value_list=
+          | |   +-Literal
+          | |   | +-StringLiteral[value=27]
+          | |   +-Literal
+          | |   | +-StringLiteral[value=44]
+          | |   +-Literal
+          | |   | +-StringLiteral[value=34]
+          | |   +-Literal
+          | |   | +-StringLiteral[value=25]
+          | |   +-Literal
+          | |   | +-StringLiteral[value=30]
+          | |   +-Literal
+          | |   | +-StringLiteral[value=33]
+          | |   +-Literal
+          | |     +-StringLiteral[value=23]
+          | +-Greater
+          | | +-left_operand=AttributeReference[attribute_name=c_acctbal]
+          | | +-right_operand=SubqueryExpression
+          | |   +-Select
+          | |     +-select_clause=SelectList
+          | |     | +-SelectListItem
+          | |     |   +-FunctionCall[name=AVG]
+          | |     |     +-AttributeReference[attribute_name=c_acctbal]
+          | |     +-where_clause=And
+          | |     | +-Greater
+          | |     | | +-left_operand=AttributeReference[attribute_name=c_acctbal]
+          | |     | | +-right_operand=Literal
+          | |     | |   +-NumericLiteral[numeric_string=0.00,float_like=true]
+          | |     | +-InValueList
+          | |     |   +-test_expression=FunctionCall[name=SUBSTR]
+          | |     |   | +-AttributeReference[attribute_name=c_phone]
+          | |     |   | +-Literal
+          | |     |   | | +-NumericLiteral[numeric_string=1,float_like=false]
+          | |     |   | +-Literal
+          | |     |   |   +-NumericLiteral[numeric_string=2,float_like=false]
+          | |     |   +-value_list=
+          | |     |     +-Literal
+          | |     |     | +-StringLiteral[value=27]
+          | |     |     +-Literal
+          | |     |     | +-StringLiteral[value=44]
+          | |     |     +-Literal
+          | |     |     | +-StringLiteral[value=34]
+          | |     |     +-Literal
+          | |     |     | +-StringLiteral[value=25]
+          | |     |     +-Literal
+          | |     |     | +-StringLiteral[value=30]
+          | |     |     +-Literal
+          | |     |     | +-StringLiteral[value=33]
+          | |     |     +-Literal
+          | |     |       +-StringLiteral[value=23]
+          | |     +-from_clause=
+          | |       +-TableReference[table=customer]
+          | +-Not
+          |   +-Exists
+          |     +-subquery=SubqueryExpression
+          |       +-Select
+          |         +-select_clause=SelectStar
+          |         +-where_clause=Equal
+          |         | +-left_operand=AttributeReference[attribute_name=o_custkey]
+          |         | +-right_operand=AttributeReference[
+          |         |   attribute_name=c_custkey]
+          |         +-from_clause=
+          |           +-TableReference[table=orders]
+          +-from_clause=
+            +-TableReference[table=customer]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/query_optimizer/expressions/SubqueryExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SubqueryExpression.cpp b/query_optimizer/expressions/SubqueryExpression.cpp
index 0459212..3a33fe1 100644
--- a/query_optimizer/expressions/SubqueryExpression.cpp
+++ b/query_optimizer/expressions/SubqueryExpression.cpp
@@ -40,9 +40,14 @@ namespace expressions {
 }
 
 std::vector<AttributeReferencePtr> SubqueryExpression::getReferencedAttributes() const {
-  // SubqueryExpression should be eliminated before any place that needs
-  // a call of getReferencedAttributes.
-  LOG(FATAL) << "SubqueryExpression::getReferencedAttributes() is not implemented";
+  // Note(jianqiao): Here simply return an empty set so that we skip the validation
+  // for this expression at the end of the resolving phase (otherwise we need to
+  // revise the Validate() function to deal with OUTER scoped attributes). Note
+  // that SubqueryExpression will always be eliminated by UnnestSubqueries as the
+  // first logical optimization pass in LogicalGenerator. So any dangling attribute
+  // will still be detected by Validate() at the end of the logical optimization
+  // phase.
+  return {};
 }
 
 void SubqueryExpression::getFieldStringItems(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index f70dae0..8ddcf3f 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -1997,8 +1997,12 @@ E::ScalarPtr Resolver::resolveExpression(
           expression_resolution_info);
     }
     case ParseExpression::kSubqueryExpression: {
-      THROW_SQL_ERROR_AT(&parse_expression)
-          << "Subquery expression in a non-FROM clause is not supported yet";
+      const std::vector<const Type*> type_hints = { type_hint };
+      return resolveSubqueryExpression(
+          static_cast<const ParseSubqueryExpression&>(parse_expression),
+          &type_hints,
+          expression_resolution_info,
+          true /* has_single_column */);
     }
     case ParseExpression::kExtract: {
       const ParseExtractFunction &parse_extract =

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index bc70d2d..7032af5 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -135,6 +135,7 @@ target_link_libraries(quickstep_queryoptimizer_rules_UnnestSubqueries
                       quickstep_queryoptimizer_logical_HashJoin
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_logical_LogicalType
+                      quickstep_queryoptimizer_logical_MultiwayCartesianJoin
                       quickstep_queryoptimizer_logical_PatternMatcher
                       quickstep_queryoptimizer_logical_Project
                       quickstep_queryoptimizer_logical_TopLevelPlan

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/query_optimizer/rules/UnnestSubqueries.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/UnnestSubqueries.cpp b/query_optimizer/rules/UnnestSubqueries.cpp
index 7852577..d22ab8e 100644
--- a/query_optimizer/rules/UnnestSubqueries.cpp
+++ b/query_optimizer/rules/UnnestSubqueries.cpp
@@ -18,7 +18,9 @@
 #include "query_optimizer/rules/UnnestSubqueries.hpp"
 
 #include <algorithm>
+#include <functional>
 #include <memory>
+#include <set>
 #include <utility>
 #include <vector>
 
@@ -41,6 +43,7 @@
 #include "query_optimizer/logical/HashJoin.hpp"
 #include "query_optimizer/logical/Logical.hpp"
 #include "query_optimizer/logical/LogicalType.hpp"
+#include "query_optimizer/logical/MultiwayCartesianJoin.hpp"
 #include "query_optimizer/logical/PatternMatcher.hpp"
 #include "query_optimizer/logical/Project.hpp"
 #include "query_optimizer/logical/TopLevelPlan.hpp"
@@ -62,7 +65,8 @@ struct CorrelatedQueryInfo {
   enum class JoinType {
     kInnerJoin = 0,
     kLeftSemiJoin,
-    kLeftAntiJoin
+    kLeftAntiJoin,
+    kCartesianJoin
   };
 
   CorrelatedQueryInfo(const JoinType join_type_in,
@@ -464,9 +468,29 @@ L::LogicalPtr UnnestSubqueriesForNonRootLogical::eliminateNestedScalarQueries(co
         new_child = node->children()[0];
       }
 
-      for (CorrelatedQueryInfo &correlated_query_info : correlated_query_info_vec) {
-        DCHECK(!correlated_query_info.probe_join_attributes.empty());
+      // Join uncorrelated subqueries early.
+      L::LogicalPtr uncorrelated_query_child;
+      for (const CorrelatedQueryInfo &correlated_query_info : correlated_query_info_vec) {
+        if (correlated_query_info.join_type == CorrelatedQueryInfo::JoinType::kCartesianJoin) {
+          // The only case for this nested loop join is that it is an uncorrelated
+          // subquery which returns a scalar (single column and single row) result.
+          DCHECK(correlated_query_info.probe_join_attributes.empty());
+          DCHECK_EQ(0u, correlated_query_info.non_hash_join_predicates.size());
+          if (uncorrelated_query_child == nullptr) {
+            uncorrelated_query_child = correlated_query_info.correlated_query;
+          } else {
+            uncorrelated_query_child = L::MultiwayCartesianJoin::Create(
+                { uncorrelated_query_child, correlated_query_info.correlated_query });
+          }
+        }
+      }
+      if (uncorrelated_query_child != nullptr) {
+        new_child = L::MultiwayCartesianJoin::Create({ new_child, uncorrelated_query_child });
+      }
+
+      for (const CorrelatedQueryInfo &correlated_query_info : correlated_query_info_vec) {
         if (correlated_query_info.join_type == CorrelatedQueryInfo::JoinType::kInnerJoin) {
+          DCHECK(!correlated_query_info.probe_join_attributes.empty());
           DCHECK(correlated_query_info.non_hash_join_predicates.empty())
               << correlated_query_info.non_hash_join_predicates[0]->toString();
           new_child = L::HashJoin::Create(new_child,
@@ -475,7 +499,9 @@ L::LogicalPtr UnnestSubqueriesForNonRootLogical::eliminateNestedScalarQueries(co
                                           correlated_query_info.build_join_attributes,
                                           nullptr, /* residual_predicate */
                                           L::HashJoin::JoinType::kInnerJoin);
-        } else {
+        } else if (correlated_query_info.join_type == CorrelatedQueryInfo::JoinType::kLeftSemiJoin ||
+                   correlated_query_info.join_type == CorrelatedQueryInfo::JoinType::kLeftAntiJoin) {
+          DCHECK(!correlated_query_info.probe_join_attributes.empty());
           E::PredicatePtr filter_predicate;
           if (correlated_query_info.non_hash_join_predicates.size() > 1u) {
             filter_predicate = E::LogicalAnd::Create(correlated_query_info.non_hash_join_predicates);
@@ -545,16 +571,11 @@ E::ExpressionPtr UnnestSubqueriesForExpession::applyInternal(
       if (probe_join_attributes.empty()) {
         DCHECK(non_hash_join_predicates.empty());
         DCHECK_EQ(1u, new_subquery->getOutputAttributes().size()) << node->toString();
-        const E::AttributeReferencePtr new_outer_attribute_reference =
-            E::AttributeReference::Create(context_->nextExprId(),
-                                          output_attribute->attribute_name(),
-                                          output_attribute->attribute_alias(),
-                                          output_attribute->relation_name(),
-                                          output_attribute->getValueType(),
-                                          E::AttributeReferenceScope::kOuter);
-        uncorrelated_subqueries_->emplace(new_outer_attribute_reference->id(),
-                                          new_subquery);
-        return new_outer_attribute_reference;
+        correlated_query_info_vec_->emplace_back(CorrelatedQueryInfo::JoinType::kCartesianJoin,
+                                                 new_subquery,
+                                                 std::move(probe_join_attributes),
+                                                 std::move(build_join_attributes),
+                                                 std::move(non_hash_join_predicates));
       } else {
         correlated_query_info_vec_->emplace_back(CorrelatedQueryInfo::JoinType::kInnerJoin,
                                                  new_subquery,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/query_optimizer/tests/execution_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Select.test b/query_optimizer/tests/execution_generator/Select.test
index 0618ae2..47c1077 100644
--- a/query_optimizer/tests/execution_generator/Select.test
+++ b/query_optimizer/tests/execution_generator/Select.test
@@ -820,11 +820,56 @@ WHERE i NOT IN (
 +-----------+
 ==
 
-# TODO(team): Support uncorrelated queries.
-# SELECT COUNT(*)
-# FROM test
-# WHERE double_col < 0
-#   AND long_col > (SELECT AVG(long_col) FROM test)
+# Scalar subquery expression
+SELECT *
+FROM test
+WHERE double_col < 0
+  AND long_col > (SELECT AVG(long_col) FROM test);
+--
++-----------+--------------------+---------------+------------------------+--------------------+
+|int_col    |long_col            |float_col      |double_col              |char_col            |
++-----------+--------------------+---------------+------------------------+--------------------+
+|        -15|                 225|     3.87298346|     -58.094750193111253|        -15 3.872983|
+|        -17|                 289|     4.12310553|     -70.092795635500224|        -17 4.123106|
+|        -19|                 361|     4.35889912|      -82.81907992727281|        -19 4.358899|
+|        -21|                 441|      4.5825758|     -96.234089594072643|        -21 4.582576|
+|        -23|                 529|     4.79583168|     -110.30412503619254|        -23 4.795832|
++-----------+--------------------+---------------+------------------------+--------------------+
+==
+
+SELECT i + (
+  SELECT SUM(j)
+  FROM generate_series(1, 10) AS gs(j))
+FROM generate_series(1, 5) AS gs(i);
+--
++----------------------+
+|(i+SubqueryExpression)|
++----------------------+
+|                    56|
+|                    57|
+|                    58|
+|                    59|
+|                    60|
++----------------------+
+==
+
+SELECT i + (
+  SELECT SUM(j)
+  FROM generate_series(1, 10) AS gs1(j),
+       generate_series(1, 10) AS gs2(k)
+  WHERE i = k AND j <= k)
+FROM generate_series(1, 5) AS gs(i);
+--
++----------------------+
+|(i+SubqueryExpression)|
++----------------------+
+|                     2|
+|                     5|
+|                     9|
+|                    14|
+|                    20|
++----------------------+
+==
 
 # TODO(team): Fix Issue #9 to enable COUNT(*).
 SELECT COUNT(long_col)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/query_optimizer/tests/logical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Select.test b/query_optimizer/tests/logical_generator/Select.test
index c6d4201..6dff3e7 100644
--- a/query_optimizer/tests/logical_generator/Select.test
+++ b/query_optimizer/tests/logical_generator/Select.test
@@ -896,3 +896,373 @@ TopLevelPlan
 |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
 +-output_attributes=
   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+==
+
+# Scalar subquery expressions
+SELECT x + (SELECT SUM(y) FROM c)
+FROM b;
+--
+TopLevelPlan
++-plan=Project
+| +-input=NestedLoopsJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=Project
+| | | +-input=Aggregate
+| | | | +-input=TableReference[relation_name=c]
+| | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-grouping_expressions=
+| | | | | +-[]
+| | | | +-aggregate_expressions=
+| | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | |     type=Long NULL]
+| | | |     +-AggregateFunction[function=SUM]
+| | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | +-project_list=
+| | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | |     +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | |       type=Long NULL]
+| | +-join_predicate=Literal[value=true]
+| +-project_list=
+|   +-Alias[id=5,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+|     +-Add
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=5,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+==
+
+SELECT x + (SELECT SUM(y) FROM c WHERE b.w = c.x)
+FROM b;
+--
+TopLevelPlan
++-plan=Project
+| +-input=HashJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=Project
+| | | +-input=Aggregate
+| | | | +-input=TableReference[relation_name=c]
+| | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-grouping_expressions=
+| | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | +-aggregate_expressions=
+| | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | |     type=Long NULL]
+| | | |     +-AggregateFunction[function=SUM]
+| | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | +-project_list=
+| | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | |     +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | |       type=Long NULL]
+| | +-left_join_attributes=
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-right_join_attributes=
+| |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| +-project_list=
+|   +-Alias[id=5,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+|     +-Add
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=5,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+==
+
+SELECT *
+FROM b
+WHERE b.x > (SELECT SUM(y) FROM c);
+--
+TopLevelPlan
++-plan=Project
+| +-input=NestedLoopsJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=Project
+| | | +-input=Aggregate
+| | | | +-input=TableReference[relation_name=c]
+| | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-grouping_expressions=
+| | | | | +-[]
+| | | | +-aggregate_expressions=
+| | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | |     type=Long NULL]
+| | | |     +-AggregateFunction[function=SUM]
+| | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | +-project_list=
+| | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | |     +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | |       type=Long NULL]
+| | +-join_predicate=Greater
+| |   +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| |   +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| +-project_list=
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
+|   +-AttributeReference[id=1,name=x,relation=b,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=w,relation=b,type=Int]
+  +-AttributeReference[id=1,name=x,relation=b,type=Int]
+==
+
+SELECT *
+FROM b
+WHERE b.x > (SELECT SUM(y) FROM c WHERE b.w = c.x);
+--
+TopLevelPlan
++-plan=Project
+| +-input=Filter
+| | +-input=HashJoin
+| | | +-left=TableReference[relation_name=b]
+| | | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | | +-right=Project
+| | | | +-input=Aggregate
+| | | | | +-input=TableReference[relation_name=c]
+| | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Long NULL]
+| | | | |     +-AggregateFunction[function=SUM]
+| | | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-project_list=
+| | | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | | |     +-AttributeReference[id=4,name=,alias=$aggregate0,
+| | | |       relation=$aggregate,type=Long NULL]
+| | | +-left_join_attributes=
+| | | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-right_join_attributes=
+| | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | +-filter_predicate=Greater
+| |   +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| |   +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| +-project_list=
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
+|   +-AttributeReference[id=1,name=x,relation=b,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=w,relation=b,type=Int]
+  +-AttributeReference[id=1,name=x,relation=b,type=Int]
+==
+
+SELECT x + (
+  SELECT SUM(y) + (SELECT SUM(w) FROM a WHERE a.y > 10)
+  FROM c
+  WHERE b.w = c.x AND c.x < 10)
+FROM b;
+--
+TopLevelPlan
++-plan=Project
+| +-input=HashJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=Project
+| | | +-input=NestedLoopsJoin
+| | | | +-left=Aggregate
+| | | | | +-input=Filter
+| | | | | | +-input=TableReference[relation_name=c]
+| | | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | | | +-filter_predicate=Less
+| | | | | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | |   +-Literal[value=10,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Long NULL]
+| | | | |     +-AggregateFunction[function=SUM]
+| | | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-right=Project
+| | | | | +-input=Aggregate
+| | | | | | +-input=Filter
+| | | | | | | +-input=TableReference[relation_name=a]
+| | | | | | | | +-AttributeReference[id=5,name=w,relation=a,type=Int]
+| | | | | | | | +-AttributeReference[id=6,name=x,relation=a,type=Int]
+| | | | | | | | +-AttributeReference[id=7,name=y,relation=a,type=Int]
+| | | | | | | | +-AttributeReference[id=8,name=z,relation=a,type=Int]
+| | | | | | | +-filter_predicate=Greater
+| | | | | | |   +-AttributeReference[id=7,name=y,relation=a,type=Int]
+| | | | | | |   +-Literal[value=10,type=Int]
+| | | | | | +-grouping_expressions=
+| | | | | | | +-[]
+| | | | | | +-aggregate_expressions=
+| | | | | |   +-Alias[id=9,name=,alias=$aggregate0,relation=$aggregate,
+| | | | | |     type=Long NULL]
+| | | | | |     +-AggregateFunction[function=SUM]
+| | | | | |       +-AttributeReference[id=5,name=w,relation=a,type=Int]
+| | | | | +-project_list=
+| | | | |   +-Alias[id=9,name=,alias=SUM(w),relation=,type=Long NULL]
+| | | | |     +-AttributeReference[id=9,name=,alias=$aggregate0,
+| | | | |       relation=$aggregate,type=Long NULL]
+| | | | +-join_predicate=Literal[value=true]
+| | | +-project_list=
+| | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | |   +-Alias[id=10,name=,alias=(SUM(y)+SubqueryExpression),relation=,
+| | |     type=Long NULL]
+| | |     +-Add
+| | |       +-AttributeReference[id=4,name=,alias=$aggregate0,
+| | |       | relation=$aggregate,type=Long NULL]
+| | |       +-AttributeReference[id=9,name=,alias=SUM(w),relation=,
+| | |         type=Long NULL]
+| | +-left_join_attributes=
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-right_join_attributes=
+| |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| +-project_list=
+|   +-Alias[id=11,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+|     +-Add
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=10,name=,alias=(SUM(y)+SubqueryExpression),
+|         relation=,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=11,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+==
+
+SELECT x * (SELECT SUM(y) FROM c) + (SELECT AVG(y) FROM d)
+FROM b;
+--
+TopLevelPlan
++-plan=Project
+| +-input=NestedLoopsJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=NestedLoopsJoin
+| | | +-left=Project
+| | | | +-input=Aggregate
+| | | | | +-input=TableReference[relation_name=c]
+| | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-[]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Long NULL]
+| | | | |     +-AggregateFunction[function=SUM]
+| | | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-project_list=
+| | | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | | |     +-AttributeReference[id=4,name=,alias=$aggregate0,
+| | | |       relation=$aggregate,type=Long NULL]
+| | | +-right=Project
+| | | | +-input=Aggregate
+| | | | | +-input=TableReference[relation_name=d]
+| | | | | | +-AttributeReference[id=5,name=y,relation=d,type=Int]
+| | | | | | +-AttributeReference[id=6,name=z,relation=d,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-[]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Double NULL]
+| | | | |     +-AggregateFunction[function=AVG]
+| | | | |       +-AttributeReference[id=5,name=y,relation=d,type=Int]
+| | | | +-project_list=
+| | | |   +-Alias[id=7,name=,alias=AVG(y),relation=,type=Double NULL]
+| | | |     +-AttributeReference[id=7,name=,alias=$aggregate0,
+| | | |       relation=$aggregate,type=Double NULL]
+| | | +-join_predicate=Literal[value=true]
+| | +-join_predicate=Literal[value=true]
+| +-project_list=
+|   +-Alias[id=8,name=,alias=((x*SubqueryExpression)+SubqueryExpression),
+|     relation=,type=Double NULL]
+|     +-Add
+|       +-Multiply
+|       | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       | +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+|       +-AttributeReference[id=7,name=,alias=AVG(y),relation=,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=8,name=,
+    alias=((x*SubqueryExpression)+SubqueryExpression),relation=,type=Double NULL]
+==
+
+SELECT x * (SELECT SUM(y) FROM c)
+FROM b
+WHERE w < (SELECT AVG(y) FROM d);
+--
+TopLevelPlan
++-plan=Project
+| +-input=NestedLoopsJoin
+| | +-left=NestedLoopsJoin
+| | | +-left=TableReference[relation_name=b]
+| | | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | | +-right=Project
+| | | | +-input=Aggregate
+| | | | | +-input=TableReference[relation_name=d]
+| | | | | | +-AttributeReference[id=2,name=y,relation=d,type=Int]
+| | | | | | +-AttributeReference[id=3,name=z,relation=d,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-[]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Double NULL]
+| | | | |     +-AggregateFunction[function=AVG]
+| | | | |       +-AttributeReference[id=2,name=y,relation=d,type=Int]
+| | | | +-project_list=
+| | | |   +-Alias[id=4,name=,alias=AVG(y),relation=,type=Double NULL]
+| | | |     +-AttributeReference[id=4,name=,alias=$aggregate0,
+| | | |       relation=$aggregate,type=Double NULL]
+| | | +-join_predicate=Less
+| | |   +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | |   +-AttributeReference[id=4,name=,alias=AVG(y),relation=,type=Double NULL]
+| | +-right=Project
+| | | +-input=Aggregate
+| | | | +-input=TableReference[relation_name=c]
+| | | | | +-AttributeReference[id=5,name=x,relation=c,type=Int]
+| | | | | +-AttributeReference[id=6,name=y,relation=c,type=Int]
+| | | | +-grouping_expressions=
+| | | | | +-[]
+| | | | +-aggregate_expressions=
+| | | |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,
+| | | |     type=Long NULL]
+| | | |     +-AggregateFunction[function=SUM]
+| | | |       +-AttributeReference[id=6,name=y,relation=c,type=Int]
+| | | +-project_list=
+| | |   +-Alias[id=7,name=,alias=SUM(y),relation=,type=Long NULL]
+| | |     +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+| | |       type=Long NULL]
+| | +-join_predicate=Literal[value=true]
+| +-project_list=
+|   +-Alias[id=8,name=,alias=(x*SubqueryExpression),relation=,type=Long NULL]
+|     +-Multiply
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=7,name=,alias=SUM(y),relation=,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=8,name=,alias=(x*SubqueryExpression),relation=,
+    type=Long NULL]
+==
+
+SELECT x + (SELECT SUM(y) FROM c WHERE b.w > c.x)
+FROM b;
+--
+ERROR: Non-equality join predicate is not allowed in scalar subqueries
+==
+
+SELECT x + (SELECT SUM(y) FROM c WHERE b.w = c.x AND b.x > c.y)
+FROM b;
+--
+ERROR: Non-equality join predicate is not allowed in scalar subqueries
+==
+
+SELECT x + (
+  SELECT SUM(y) + (
+    SELECT SUM(w)
+    FROM a WHERE a.y = b.x)
+  FROM c
+  WHERE b.w = c.x AND c.x < 10)
+FROM b;
+--
+ERROR: Nested queries can only reference attributes in the outer query one level above

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/query_optimizer/tests/physical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index 2fa3720..b405bc9 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -2113,3 +2113,617 @@ TopLevelPlan
 |   +-AttributeReference[id=7,name=long_col,relation=test,type=Long]
 +-output_attributes=
   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+==
+
+
+# Scalar subquery expressions
+SELECT x + (SELECT SUM(y) FROM c)
+FROM b;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=NestedLoopsJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=Project
+| | | +-input=Aggregate
+| | | | +-input=TableReference[relation_name=c]
+| | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-grouping_expressions=
+| | | | | +-[]
+| | | | +-aggregate_expressions=
+| | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | |     type=Long NULL]
+| | | |     +-AggregateFunction[function=SUM]
+| | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | +-project_list=
+| | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | |     +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | |       type=Long NULL]
+| | +-join_predicate=Literal[value=true]
+| +-project_list=
+|   +-Alias[id=5,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+|     +-Add
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=5,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=NestedLoopsJoin
+| +-left=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-right=Aggregate
+| | +-input=TableReference[relation=c]
+| | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| +-join_predicate=Literal[value=true]
+| +-project_expressions=
+|   +-Alias[id=5,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+|     +-Add
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+|         type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=5,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+==
+
+SELECT x + (SELECT SUM(y) FROM c WHERE b.w = c.x)
+FROM b;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=HashJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=Project
+| | | +-input=Aggregate
+| | | | +-input=TableReference[relation_name=c]
+| | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-grouping_expressions=
+| | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | +-aggregate_expressions=
+| | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | |     type=Long NULL]
+| | | |     +-AggregateFunction[function=SUM]
+| | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | +-project_list=
+| | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | |     +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | |       type=Long NULL]
+| | +-left_join_attributes=
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-right_join_attributes=
+| |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| +-project_list=
+|   +-Alias[id=5,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+|     +-Add
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=5,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=HashJoin
+| +-left=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-right=Aggregate
+| | +-input=TableReference[relation=c]
+| | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | +-grouping_expressions=
+| | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | +-aggregate_expressions=
+| |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| +-project_expressions=
+| | +-Alias[id=5,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+| |   +-Add
+| |     +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| |     +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| |       type=Long NULL]
+| +-left_join_attributes=
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| +-right_join_attributes=
+|   +-AttributeReference[id=2,name=x,relation=c,type=Int]
++-output_attributes=
+  +-AttributeReference[id=5,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+==
+
+SELECT *
+FROM b
+WHERE b.x > (SELECT SUM(y) FROM c);
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=NestedLoopsJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=Project
+| | | +-input=Aggregate
+| | | | +-input=TableReference[relation_name=c]
+| | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-grouping_expressions=
+| | | | | +-[]
+| | | | +-aggregate_expressions=
+| | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | |     type=Long NULL]
+| | | |     +-AggregateFunction[function=SUM]
+| | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | +-project_list=
+| | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | |     +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | |       type=Long NULL]
+| | +-join_predicate=Greater
+| |   +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| |   +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| +-project_list=
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
+|   +-AttributeReference[id=1,name=x,relation=b,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=w,relation=b,type=Int]
+  +-AttributeReference[id=1,name=x,relation=b,type=Int]
+[Physical Plan]
+TopLevelPlan
++-plan=NestedLoopsJoin
+| +-left=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-right=Aggregate
+| | +-input=TableReference[relation=c]
+| | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| +-join_predicate=Greater
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| |   type=Long NULL]
+| +-project_expressions=
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
+|   +-AttributeReference[id=1,name=x,relation=b,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=w,relation=b,type=Int]
+  +-AttributeReference[id=1,name=x,relation=b,type=Int]
+==
+
+SELECT *
+FROM b
+WHERE b.x > (SELECT SUM(y) FROM c WHERE b.w = c.x);
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=Filter
+| | +-input=HashJoin
+| | | +-left=TableReference[relation_name=b]
+| | | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | | +-right=Project
+| | | | +-input=Aggregate
+| | | | | +-input=TableReference[relation_name=c]
+| | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Long NULL]
+| | | | |     +-AggregateFunction[function=SUM]
+| | | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-project_list=
+| | | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | | |     +-AttributeReference[id=4,name=,alias=$aggregate0,
+| | | |       relation=$aggregate,type=Long NULL]
+| | | +-left_join_attributes=
+| | | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-right_join_attributes=
+| | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | +-filter_predicate=Greater
+| |   +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| |   +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| +-project_list=
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
+|   +-AttributeReference[id=1,name=x,relation=b,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=w,relation=b,type=Int]
+  +-AttributeReference[id=1,name=x,relation=b,type=Int]
+[Physical Plan]
+TopLevelPlan
++-plan=HashJoin
+| +-left=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-right=Aggregate
+| | +-input=TableReference[relation=c]
+| | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | +-grouping_expressions=
+| | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | +-aggregate_expressions=
+| |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| +-residual_predicate=Greater
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| |   type=Long NULL]
+| +-project_expressions=
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-left_join_attributes=
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| +-right_join_attributes=
+|   +-AttributeReference[id=2,name=x,relation=c,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=w,relation=b,type=Int]
+  +-AttributeReference[id=1,name=x,relation=b,type=Int]
+==
+
+SELECT x + (
+  SELECT SUM(y) + (SELECT SUM(w) FROM a WHERE a.y > 10)
+  FROM c
+  WHERE b.w = c.x AND c.x < 10)
+FROM b;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=HashJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=Project
+| | | +-input=NestedLoopsJoin
+| | | | +-left=Aggregate
+| | | | | +-input=Filter
+| | | | | | +-input=TableReference[relation_name=c]
+| | | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | | | +-filter_predicate=Less
+| | | | | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | |   +-Literal[value=10,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Long NULL]
+| | | | |     +-AggregateFunction[function=SUM]
+| | | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-right=Project
+| | | | | +-input=Aggregate
+| | | | | | +-input=Filter
+| | | | | | | +-input=TableReference[relation_name=a]
+| | | | | | | | +-AttributeReference[id=5,name=w,relation=a,type=Int]
+| | | | | | | | +-AttributeReference[id=6,name=x,relation=a,type=Int]
+| | | | | | | | +-AttributeReference[id=7,name=y,relation=a,type=Int]
+| | | | | | | | +-AttributeReference[id=8,name=z,relation=a,type=Int]
+| | | | | | | +-filter_predicate=Greater
+| | | | | | |   +-AttributeReference[id=7,name=y,relation=a,type=Int]
+| | | | | | |   +-Literal[value=10,type=Int]
+| | | | | | +-grouping_expressions=
+| | | | | | | +-[]
+| | | | | | +-aggregate_expressions=
+| | | | | |   +-Alias[id=9,name=,alias=$aggregate0,relation=$aggregate,
+| | | | | |     type=Long NULL]
+| | | | | |     +-AggregateFunction[function=SUM]
+| | | | | |       +-AttributeReference[id=5,name=w,relation=a,type=Int]
+| | | | | +-project_list=
+| | | | |   +-Alias[id=9,name=,alias=SUM(w),relation=,type=Long NULL]
+| | | | |     +-AttributeReference[id=9,name=,alias=$aggregate0,
+| | | | |       relation=$aggregate,type=Long NULL]
+| | | | +-join_predicate=Literal[value=true]
+| | | +-project_list=
+| | |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | |   +-Alias[id=10,name=,alias=(SUM(y)+SubqueryExpression),relation=,
+| | |     type=Long NULL]
+| | |     +-Add
+| | |       +-AttributeReference[id=4,name=,alias=$aggregate0,
+| | |       | relation=$aggregate,type=Long NULL]
+| | |       +-AttributeReference[id=9,name=,alias=SUM(w),relation=,
+| | |         type=Long NULL]
+| | +-left_join_attributes=
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-right_join_attributes=
+| |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| +-project_list=
+|   +-Alias[id=11,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+|     +-Add
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=10,name=,alias=(SUM(y)+SubqueryExpression),
+|         relation=,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=11,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=HashJoin
+| +-left=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-right=NestedLoopsJoin
+| | +-left=Aggregate
+| | | +-input=TableReference[relation=c]
+| | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | +-filter_predicate=Less
+| | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | +-Literal[value=10,type=Int]
+| | | +-grouping_expressions=
+| | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| | |     +-AggregateFunction[function=SUM]
+| | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | +-right=Aggregate
+| | | +-input=TableReference[relation=a]
+| | | | +-AttributeReference[id=5,name=w,relation=a,type=Int]
+| | | | +-AttributeReference[id=6,name=x,relation=a,type=Int]
+| | | | +-AttributeReference[id=7,name=y,relation=a,type=Int]
+| | | | +-AttributeReference[id=8,name=z,relation=a,type=Int]
+| | | +-filter_predicate=Greater
+| | | | +-AttributeReference[id=7,name=y,relation=a,type=Int]
+| | | | +-Literal[value=10,type=Int]
+| | | +-grouping_expressions=
+| | | | +-[]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=9,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| | |     +-AggregateFunction[function=SUM]
+| | |       +-AttributeReference[id=5,name=w,relation=a,type=Int]
+| | +-join_predicate=Literal[value=true]
+| | +-project_expressions=
+| |   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| |   +-Alias[id=10,name=,alias=(SUM(y)+SubqueryExpression),relation=,
+| |     type=Long NULL]
+| |     +-Add
+| |       +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| |       | type=Long NULL]
+| |       +-AttributeReference[id=9,name=,alias=$aggregate0,relation=$aggregate,
+| |         type=Long NULL]
+| +-project_expressions=
+| | +-Alias[id=11,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+| |   +-Add
+| |     +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| |     +-AttributeReference[id=10,name=,alias=(SUM(y)+SubqueryExpression),
+| |       relation=,type=Long NULL]
+| +-left_join_attributes=
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| +-right_join_attributes=
+|   +-AttributeReference[id=2,name=x,relation=c,type=Int]
++-output_attributes=
+  +-AttributeReference[id=11,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+==
+
+SELECT x * (SELECT SUM(y) FROM c) + (SELECT AVG(y) FROM d)
+FROM b;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=NestedLoopsJoin
+| | +-left=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=NestedLoopsJoin
+| | | +-left=Project
+| | | | +-input=Aggregate
+| | | | | +-input=TableReference[relation_name=c]
+| | | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-[]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Long NULL]
+| | | | |     +-AggregateFunction[function=SUM]
+| | | | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | | +-project_list=
+| | | |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| | | |     +-AttributeReference[id=4,name=,alias=$aggregate0,
+| | | |       relation=$aggregate,type=Long NULL]
+| | | +-right=Project
+| | | | +-input=Aggregate
+| | | | | +-input=TableReference[relation_name=d]
+| | | | | | +-AttributeReference[id=5,name=y,relation=d,type=Int]
+| | | | | | +-AttributeReference[id=6,name=z,relation=d,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-[]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Double NULL]
+| | | | |     +-AggregateFunction[function=AVG]
+| | | | |       +-AttributeReference[id=5,name=y,relation=d,type=Int]
+| | | | +-project_list=
+| | | |   +-Alias[id=7,name=,alias=AVG(y),relation=,type=Double NULL]
+| | | |     +-AttributeReference[id=7,name=,alias=$aggregate0,
+| | | |       relation=$aggregate,type=Double NULL]
+| | | +-join_predicate=Literal[value=true]
+| | +-join_predicate=Literal[value=true]
+| +-project_list=
+|   +-Alias[id=8,name=,alias=((x*SubqueryExpression)+SubqueryExpression),
+|     relation=,type=Double NULL]
+|     +-Add
+|       +-Multiply
+|       | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       | +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+|       +-AttributeReference[id=7,name=,alias=AVG(y),relation=,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=8,name=,
+    alias=((x*SubqueryExpression)+SubqueryExpression),relation=,type=Double NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=NestedLoopsJoin
+| +-left=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-right=NestedLoopsJoin
+| | +-left=Aggregate
+| | | +-input=TableReference[relation=c]
+| | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | | +-grouping_expressions=
+| | | | +-[]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| | |     +-AggregateFunction[function=SUM]
+| | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| | +-right=Aggregate
+| | | +-input=TableReference[relation=d]
+| | | | +-AttributeReference[id=5,name=y,relation=d,type=Int]
+| | | | +-AttributeReference[id=6,name=z,relation=d,type=Int]
+| | | +-grouping_expressions=
+| | | | +-[]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,
+| | |     type=Double NULL]
+| | |     +-AggregateFunction[function=AVG]
+| | |       +-AttributeReference[id=5,name=y,relation=d,type=Int]
+| | +-join_predicate=Literal[value=true]
+| | +-project_expressions=
+| |   +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| |   | type=Long NULL]
+| |   +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+| |     type=Double NULL]
+| +-join_predicate=Literal[value=true]
+| +-project_expressions=
+|   +-Alias[id=8,name=,alias=((x*SubqueryExpression)+SubqueryExpression),
+|     relation=,type=Double NULL]
+|     +-Add
+|       +-Multiply
+|       | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       | +-AttributeReference[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+|       +-AttributeReference[id=7,name=,alias=AVG(y),relation=,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=8,name=,
+    alias=((x*SubqueryExpression)+SubqueryExpression),relation=,type=Double NULL]
+==
+
+SELECT x * (SELECT SUM(y) FROM c)
+FROM b
+WHERE w < (SELECT AVG(y) FROM d);
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=NestedLoopsJoin
+| | +-left=NestedLoopsJoin
+| | | +-left=TableReference[relation_name=b]
+| | | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | | +-right=Project
+| | | | +-input=Aggregate
+| | | | | +-input=TableReference[relation_name=d]
+| | | | | | +-AttributeReference[id=2,name=y,relation=d,type=Int]
+| | | | | | +-AttributeReference[id=3,name=z,relation=d,type=Int]
+| | | | | +-grouping_expressions=
+| | | | | | +-[]
+| | | | | +-aggregate_expressions=
+| | | | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | | | |     type=Double NULL]
+| | | | |     +-AggregateFunction[function=AVG]
+| | | | |       +-AttributeReference[id=2,name=y,relation=d,type=Int]
+| | | | +-project_list=
+| | | |   +-Alias[id=4,name=,alias=AVG(y),relation=,type=Double NULL]
+| | | |     +-AttributeReference[id=4,name=,alias=$aggregate0,
+| | | |       relation=$aggregate,type=Double NULL]
+| | | +-join_predicate=Less
+| | |   +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | |   +-AttributeReference[id=4,name=,alias=AVG(y),relation=,type=Double NULL]
+| | +-right=Project
+| | | +-input=Aggregate
+| | | | +-input=TableReference[relation_name=c]
+| | | | | +-AttributeReference[id=5,name=x,relation=c,type=Int]
+| | | | | +-AttributeReference[id=6,name=y,relation=c,type=Int]
+| | | | +-grouping_expressions=
+| | | | | +-[]
+| | | | +-aggregate_expressions=
+| | | |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,
+| | | |     type=Long NULL]
+| | | |     +-AggregateFunction[function=SUM]
+| | | |       +-AttributeReference[id=6,name=y,relation=c,type=Int]
+| | | +-project_list=
+| | |   +-Alias[id=7,name=,alias=SUM(y),relation=,type=Long NULL]
+| | |     +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+| | |       type=Long NULL]
+| | +-join_predicate=Literal[value=true]
+| +-project_list=
+|   +-Alias[id=8,name=,alias=(x*SubqueryExpression),relation=,type=Long NULL]
+|     +-Multiply
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=7,name=,alias=SUM(y),relation=,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=8,name=,alias=(x*SubqueryExpression),relation=,
+    type=Long NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=NestedLoopsJoin
+| +-left=NestedLoopsJoin
+| | +-left=TableReference[relation=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-right=Aggregate
+| | | +-input=TableReference[relation=d]
+| | | | +-AttributeReference[id=2,name=y,relation=d,type=Int]
+| | | | +-AttributeReference[id=3,name=z,relation=d,type=Int]
+| | | +-grouping_expressions=
+| | | | +-[]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | |     type=Double NULL]
+| | |     +-AggregateFunction[function=AVG]
+| | |       +-AttributeReference[id=2,name=y,relation=d,type=Int]
+| | +-join_predicate=Less
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| | |   type=Double NULL]
+| | +-project_expressions=
+| |   +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-right=Aggregate
+| | +-input=TableReference[relation=c]
+| | | +-AttributeReference[id=5,name=x,relation=c,type=Int]
+| | | +-AttributeReference[id=6,name=y,relation=c,type=Int]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=6,name=y,relation=c,type=Int]
+| +-join_predicate=Literal[value=true]
+| +-project_expressions=
+|   +-Alias[id=8,name=,alias=(x*SubqueryExpression),relation=,type=Long NULL]
+|     +-Multiply
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+|         type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=8,name=,alias=(x*SubqueryExpression),relation=,
+    type=Long NULL]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b3d9156/query_optimizer/tests/resolver/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Select.test b/query_optimizer/tests/resolver/Select.test
index 82bdb55..00ff18a 100644
--- a/query_optimizer/tests/resolver/Select.test
+++ b/query_optimizer/tests/resolver/Select.test
@@ -2793,3 +2793,147 @@ WHERE i IN (
 ERROR: Subquery must return exactly one column (3 : 12)
 WHERE i IN (
            ^
+==
+
+# Scalar subquery expressions
+SELECT x + (SELECT SUM(y) FROM c)
+FROM b;
+--
+TopLevelPlan
++-plan=Project
+| +-input=TableReference[relation_name=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-project_list=
+|   +-Alias[id=5,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+|     +-Add
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-SubqueryExpression
+|         +-subquery=Project
+|           +-input=Aggregate
+|           | +-input=TableReference[relation_name=c]
+|           | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|           | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+|           | +-grouping_expressions=
+|           | | +-[]
+|           | +-aggregate_expressions=
+|           |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+|           |     type=Long NULL]
+|           |     +-AggregateFunction[function=SUM]
+|           |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+|           +-project_list=
+|             +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+|               +-AttributeReference[id=4,name=,alias=$aggregate0,
+|                 relation=$aggregate,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=5,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]
+==
+
+SELECT *
+FROM b
+WHERE b.x > (SELECT SUM(y) FROM c);
+--
+TopLevelPlan
++-plan=Project
+| +-input=Filter
+| | +-input=TableReference[relation_name=b]
+| | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| | +-filter_predicate=Greater
+| |   +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| |   +-SubqueryExpression
+| |     +-subquery=Project
+| |       +-input=Aggregate
+| |       | +-input=TableReference[relation_name=c]
+| |       | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+| |       | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| |       | +-grouping_expressions=
+| |       | | +-[]
+| |       | +-aggregate_expressions=
+| |       |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| |       |     type=Long NULL]
+| |       |     +-AggregateFunction[function=SUM]
+| |       |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| |       +-project_list=
+| |         +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| |           +-AttributeReference[id=4,name=,alias=$aggregate0,
+| |             relation=$aggregate,type=Long NULL]
+| +-project_list=
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
+|   +-AttributeReference[id=1,name=x,relation=b,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=w,relation=b,type=Int]
+  +-AttributeReference[id=1,name=x,relation=b,type=Int]
+==
+
+SELECT x + (
+  SELECT SUM(y) + (SELECT SUM(w) FROM a WHERE a.y > 10)
+  FROM c
+  WHERE b.w = c.x AND c.x < 10)
+FROM b;
+--
+TopLevelPlan
++-plan=Project
+| +-input=TableReference[relation_name=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
+| +-project_list=
+|   +-Alias[id=11,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
+|     +-Add
+|       +-AttributeReference[id=1,name=x,relation=b,type=Int]
+|       +-SubqueryExpression
+|         +-subquery=Project
+|           +-input=Aggregate
+|           | +-input=Filter
+|           | | +-input=TableReference[relation_name=c]
+|           | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|           | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
+|           | | +-filter_predicate=And
+|           | |   +-Equal
+|           | |   | +-AttributeReference[id=0,name=w,relation=b,type=Int,
+|           | |   | | is_outer_reference=true]
+|           | |   | +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|           | |   +-Less
+|           | |     +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|           | |     +-Literal[value=10,type=Int]
+|           | +-grouping_expressions=
+|           | | +-[]
+|           | +-aggregate_expressions=
+|           |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,
+|           |     type=Long NULL]
+|           |     +-AggregateFunction[function=SUM]
+|           |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+|           +-project_list=
+|             +-Alias[id=10,name=,alias=(SUM(y)+SubqueryExpression),relation=,
+|               type=Long NULL]
+|               +-Add
+|                 +-AttributeReference[id=4,name=,alias=$aggregate0,
+|                 | relation=$aggregate,type=Long NULL]
+|                 +-SubqueryExpression
+|                   +-subquery=Project
+|                     +-input=Aggregate
+|                     | +-input=Filter
+|                     | | +-input=TableReference[relation_name=a]
+|                     | | | +-AttributeReference[id=5,name=w,relation=a,type=Int]
+|                     | | | +-AttributeReference[id=6,name=x,relation=a,type=Int]
+|                     | | | +-AttributeReference[id=7,name=y,relation=a,type=Int]
+|                     | | | +-AttributeReference[id=8,name=z,relation=a,type=Int]
+|                     | | +-filter_predicate=Greater
+|                     | |   +-AttributeReference[id=7,name=y,relation=a,type=Int]
+|                     | |   +-Literal[value=10,type=Int]
+|                     | +-grouping_expressions=
+|                     | | +-[]
+|                     | +-aggregate_expressions=
+|                     |   +-Alias[id=9,name=,alias=$aggregate0,
+|                     |     relation=$aggregate,type=Long NULL]
+|                     |     +-AggregateFunction[function=SUM]
+|                     |       +-AttributeReference[id=5,name=w,relation=a,
+|                     |         type=Int]
+|                     +-project_list=
+|                       +-Alias[id=9,name=,alias=SUM(w),relation=,type=Long NULL]
+|                         +-AttributeReference[id=9,name=,alias=$aggregate0,
+|                           relation=$aggregate,type=Long NULL]
++-output_attributes=
+  +-AttributeReference[id=11,name=,alias=(x+SubqueryExpression),relation=,
+    type=Long NULL]


[21/30] incubator-quickstep git commit: Fixes loss of alias problem with PullUpProjectExpressions (#197)

Posted by zu...@apache.org.
Fixes loss of alias problem with PullUpProjectExpressions (#197)

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

Branch: refs/heads/master
Commit: d384da1bfc4c924d39194c9d0356701ae029dcc5
Parents: 85edc76
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Apr 28 01:00:50 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 query_optimizer/rules/RuleHelper.cpp            | 18 +++++++++++-------
 .../tests/execution_generator/Select.test       | 20 ++++++++++++++++++++
 .../tests/physical_generator/Select.test        | 16 ++++++++++------
 3 files changed, 41 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d384da1b/query_optimizer/rules/RuleHelper.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/RuleHelper.cpp b/query_optimizer/rules/RuleHelper.cpp
index ce9ef9a..a273004 100644
--- a/query_optimizer/rules/RuleHelper.cpp
+++ b/query_optimizer/rules/RuleHelper.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -75,19 +77,21 @@ void PullUpProjectExpressions(
         E::ExpressionPtr updated_expression = rule.apply(expression_to_update);
 
         E::NamedExpressionPtr updated_named_expression;
-        // If the updated expression is not a NamedExpression or we have changed
-        // the ID, restore the ID by adding an Alias.
-        if (!E::SomeNamedExpression::MatchesWithConditionalCast(updated_expression,
-                                                                &updated_named_expression) ||
-            expression_to_update->id() != updated_named_expression->id()) {
+        if (E::SomeNamedExpression::MatchesWithConditionalCast(updated_expression,
+                                                               &updated_named_expression) &&
+            expression_to_update->id() == updated_named_expression->id() &&
+            expression_to_update->attribute_alias() == updated_named_expression->attribute_alias()) {
+          // Pull up directly if the updated expression has the same ExprId and
+          // attribute_alias as the original project expression.
+          (*project_expressions)[i] = updated_named_expression;
+        } else {
+          // Otherwise create an Alias to wrap the updated expression.
           (*project_expressions)[i] =
               E::Alias::Create(expression_to_update->id(),
                                updated_expression,
                                expression_to_update->attribute_name(),
                                expression_to_update->attribute_alias(),
                                expression_to_update->relation_name());
-        } else {
-          (*project_expressions)[i] = updated_named_expression;
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d384da1b/query_optimizer/tests/execution_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Select.test b/query_optimizer/tests/execution_generator/Select.test
index 438546c..3a64c9b 100644
--- a/query_optimizer/tests/execution_generator/Select.test
+++ b/query_optimizer/tests/execution_generator/Select.test
@@ -871,6 +871,26 @@ FROM generate_series(1, 5) AS gs(i);
 +----------------------+
 ==
 
+# This query is to test that the output columns have the correct alias name as specified.
+SELECT *
+FROM (
+  SELECT i, SUM(i) AS sum
+  FROM generate_series(1, 2) AS gs(i)
+  GROUP BY i
+) t1 JOIN (
+  SELECT j, AVG(j) AS avg
+  FROM generate_series(1, 2) AS gs(j)
+  GROUP BY j
+) t2 ON i = j;
+--
++-----------+--------------------+-----------+------------------------+
+|i          |sum                 |j          |avg                     |
++-----------+--------------------+-----------+------------------------+
+|          1|                   1|          1|                       1|
+|          2|                   2|          2|                       2|
++-----------+--------------------+-----------+------------------------+
+==
+
 # TODO(team): Fix Issue #9 to enable COUNT(*).
 SELECT COUNT(long_col)
 FROM test,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d384da1b/query_optimizer/tests/physical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index b405bc9..62d09f5 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -1166,7 +1166,8 @@ TopLevelPlan
 | | | | +-Add
 | | | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 | | | |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
-| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-Alias[id=1,name=col1,relation=,type=Long]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 | | | +-Alias[id=8,name=,alias=$groupby2,relation=$groupby,type=Float]
 | | |   +-Add
 | | |     +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
@@ -1304,7 +1305,8 @@ TopLevelPlan
 | | | +-Literal[value=2,type=Int]
 | | +-grouping_expressions=
 | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-Alias[id=4,name=subquery_col3,relation=subquery,type=Char(20)]
+| | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
 | | +-aggregate_expressions=
 | |   +-Alias[id=8,name=,alias=$aggregate0,relation=$aggregate,type=Long]
 | |   | +-AggregateFunction[function=COUNT]
@@ -2609,10 +2611,12 @@ TopLevelPlan
 | | |       +-AttributeReference[id=5,name=y,relation=d,type=Int]
 | | +-join_predicate=Literal[value=true]
 | | +-project_expressions=
-| |   +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
-| |   | type=Long NULL]
-| |   +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
-| |     type=Double NULL]
+| |   +-Alias[id=4,name=,alias=SUM(y),relation=,type=Long NULL]
+| |   | +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
+| |   |   type=Long NULL]
+| |   +-Alias[id=7,name=,alias=AVG(y),relation=,type=Double NULL]
+| |     +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+| |       type=Double NULL]
 | +-join_predicate=Literal[value=true]
 | +-project_expressions=
 |   +-Alias[id=8,name=,alias=((x*SubqueryExpression)+SubqueryExpression),


[02/30] incubator-quickstep git commit: Stop preloading the buffer pool when it is full. (#172)

Posted by zu...@apache.org.
Stop preloading the buffer pool when it is full. (#172)

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

Branch: refs/heads/master
Commit: 3ae0d1cedf73be738f3fae323e783e5564416026
Parents: cd93fd7
Author: Jignesh Patel <pa...@users.noreply.github.com>
Authored: Thu Apr 21 10:15:23 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:33 2016 -0700

----------------------------------------------------------------------
 cli/QuickstepCli.cpp        | 13 +++++++++----
 storage/PreloaderThread.cpp | 18 +++++++++++++++++-
 storage/StorageManager.hpp  | 16 ++++++++++++++++
 3 files changed, 42 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3ae0d1ce/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index b0b98a3..8dee1f7 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -169,7 +169,7 @@ int main(int argc, char* argv[]) {
            real_num_workers,
            (static_cast<double>(quickstep::FLAGS_buffer_pool_slots) * quickstep::kSlotSizeBytes)/quickstep::kAGigaByte);
   } else {
-    LOG(FATAL) << "Quickstep needs at least one worker thread";
+    LOG(FATAL) << "Quickstep needs at least one worker thread to run";
   }
 
 #ifdef QUICKSTEP_HAVE_FILE_MANAGER_HDFS
@@ -262,14 +262,19 @@ int main(int argc, char* argv[]) {
       DefaultsConfigurator::GetNumNUMANodesCoveredByWorkers(worker_cpu_affinities);
 
   if (quickstep::FLAGS_preload_buffer_pool) {
+    std::chrono::time_point<std::chrono::steady_clock> preload_start, preload_end;
+    preload_start = std::chrono::steady_clock::now();
+    printf("Preloading the buffer pool ... ");
+    fflush(stdout);
     quickstep::PreloaderThread preloader(*query_processor->getDefaultDatabase(),
                                          query_processor->getStorageManager(),
                                          worker_cpu_affinities.front());
-    printf("Preloading buffer pool... ");
-    fflush(stdout);
+
     preloader.start();
     preloader.join();
-    printf("DONE\n");
+    preload_end = std::chrono::steady_clock::now();
+    printf("in %g seconds\n",
+           std::chrono::duration<double>(preload_end - preload_start).count());
   }
 
   Foreman foreman(&bus,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3ae0d1ce/storage/PreloaderThread.cpp
----------------------------------------------------------------------
diff --git a/storage/PreloaderThread.cpp b/storage/PreloaderThread.cpp
index a870837..d5dc55b 100644
--- a/storage/PreloaderThread.cpp
+++ b/storage/PreloaderThread.cpp
@@ -34,12 +34,28 @@ void PreloaderThread::run() {
     ThreadUtil::BindToCPU(cpu_id_);
   }
 
+  const std::size_t num_slots = storage_manager_->getMaxBufferPoolSlots();
+  std::size_t blocks_loaded = 0;
+
   for (const CatalogRelation &relation : database_) {
     std::vector<block_id> blocks = relation.getBlocksSnapshot();
     for (block_id current_block_id : blocks) {
-      BlockReference current_block = storage_manager_->getBlock(current_block_id, relation);
+      try {
+        BlockReference current_block = storage_manager_->getBlock(current_block_id, relation);
+      } catch (...) {
+        LOG(ERROR) << "Error after loading " << blocks_loaded << "blocks\n";
+        throw;
+      }
+      ++blocks_loaded;
+      if (blocks_loaded == num_slots) {
+        // The buffer pool has filled up. But, some database blocks are not loaded.
+        printf(" The database is larger than the buffer pool. Only %lu blocks were loaded ",
+               blocks_loaded);
+        return;
+      }
     }
   }
+  printf(" Loaded %lu blocks ", blocks_loaded);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3ae0d1ce/storage/StorageManager.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.hpp b/storage/StorageManager.hpp
index b510532..dd67177 100644
--- a/storage/StorageManager.hpp
+++ b/storage/StorageManager.hpp
@@ -166,6 +166,22 @@ class StorageManager {
   }
 
   /**
+   * @brief Return the upper limit on the number of buffer pool slots that the
+   *        StorageManager can allocate. This number is specified during the
+   *        initialization of the StorageManager. The size of each slot is
+   *        kSlotSizeBytes.
+   * @note This information is provided for informational purposes. The memory
+   *       pool may grow larger than this upper limite temporarily, depending
+   *       on the path that is followed in a call to createBlock() or
+   *       loadBlock().
+   *
+   * @return The number of buffer pool slots managed by this StorageManager.
+   **/
+  std::size_t getMaxBufferPoolSlots() const {
+    return max_memory_usage_;
+  }
+
+  /**
    * @brief Create a new empty block.
    *
    * @param relation The relation which the new block will belong to (you must


[17/30] incubator-quickstep git commit: Partition aware selection using NUMA-awareness (#175)

Posted by zu...@apache.org.
Partition aware selection using NUMA-awareness (#175)

Link: https://github.com/pivotalsoftware/quickstep/pull/175

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

Branch: refs/heads/master
Commit: 52b758fc8f8e4c88cc17b81a398f469ede594069
Parents: 6f4dd8f
Author: Adalbert Gerald Soosai Raj <ad...@gmail.com>
Authored: Sun May 1 15:38:38 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 catalog/CMakeLists.txt                  |   5 +-
 catalog/CatalogRelation.hpp             |   9 ++
 relational_operators/CMakeLists.txt     |   5 ++
 relational_operators/SelectOperator.cpp | 129 +++++++++++++++++++++------
 relational_operators/SelectOperator.hpp | 115 +++++++++++++++++++++---
 5 files changed, 226 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/52b758fc/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index 8c89d7e..94da838 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -175,9 +175,12 @@ target_link_libraries(quickstep_catalog
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_IndexScheme
-                      quickstep_catalog_NUMAPlacementScheme
                       quickstep_catalog_PartitionScheme
                       quickstep_catalog_PartitionSchemeHeader)
+if(QUICKSTEP_HAVE_LIBNUMA)
+target_link_libraries(quickstep_catalog
+                      quickstep_catalog_NUMAPlacementScheme)
+endif()
 
 # Tests:
 add_executable(Catalog_unittest "${CMAKE_CURRENT_SOURCE_DIR}/tests/Catalog_unittest.cpp")

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/52b758fc/catalog/CatalogRelation.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelation.hpp b/catalog/CatalogRelation.hpp
index 4cc8d79..3701090 100644
--- a/catalog/CatalogRelation.hpp
+++ b/catalog/CatalogRelation.hpp
@@ -165,6 +165,15 @@ class CatalogRelation : public CatalogRelationSchema {
   }
 
   /**
+   * @brief Get the NUMA placement scheme of the relation.
+   *
+   * @return A pointer to a const NUMA placement scheme.
+   **/
+  const NUMAPlacementScheme* getNUMAPlacementSchemePtr() const {
+    return placement_scheme_.get();
+  }
+
+  /**
    * @brief Set the NUMA placement scheme for the catalog relation.
    *
    * @param placement_scheme The NUMA placement scheme object for the relation,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/52b758fc/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index a4600e6..eec5300 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -266,6 +266,7 @@ target_link_libraries(quickstep_relationaloperators_SelectOperator
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_catalog_PartitionSchemeHeader
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
@@ -276,6 +277,10 @@ target_link_libraries(quickstep_relationaloperators_SelectOperator
                       quickstep_storage_StorageManager
                       quickstep_utility_Macros
                       tmb)
+if(QUICKSTEP_HAVE_LIBNUMA)
+target_link_libraries(quickstep_relationaloperators_SelectOperator
+                      quickstep_catalog_NUMAPlacementScheme)
+endif()
 target_link_libraries(quickstep_relationaloperators_SortMergeRunOperator
                       glog
                       quickstep_catalog_CatalogRelation

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/52b758fc/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index 3cac199..69bb434 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -35,6 +35,93 @@ namespace quickstep {
 
 class Predicate;
 
+void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
+                                   StorageManager *storage_manager,
+                                   const Predicate *predicate,
+                                   const std::vector<std::unique_ptr<const Scalar>> *selection,
+                                   InsertDestination *output_destination) {
+  if (input_relation_is_stored_) {
+    for (const block_id input_block_id : input_relation_block_ids_) {
+      container->addNormalWorkOrder(
+          new SelectWorkOrder(input_relation_,
+                              input_block_id,
+                              predicate,
+                              simple_projection_,
+                              simple_selection_,
+                              selection,
+                              output_destination,
+                              storage_manager),
+          op_index_);
+    }
+  } else {
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addNormalWorkOrder(
+          new SelectWorkOrder(
+              input_relation_,
+              input_relation_block_ids_[num_workorders_generated_],
+              predicate,
+              simple_projection_,
+              simple_selection_,
+              selection,
+              output_destination,
+              storage_manager),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+  }
+}
+
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
+                                                 StorageManager *storage_manager,
+                                                 const Predicate *predicate,
+                                                 const std::vector<std::unique_ptr<const Scalar>> *selection,
+                                                 InsertDestination *output_destination) {
+  DCHECK(placement_scheme_ != nullptr);
+  const std::size_t num_partitions = input_relation_.getPartitionScheme().getPartitionSchemeHeader().getNumPartitions();
+  if (input_relation_is_stored_) {
+    for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
+      for (const block_id input_block_id :
+           input_relation_block_ids_in_partition_[part_id]) {
+        container->addNormalWorkOrder(
+            new SelectWorkOrder(
+                input_relation_,
+                input_block_id,
+                predicate,
+                simple_projection_,
+                simple_selection_,
+                selection,
+                output_destination,
+                storage_manager,
+                placement_scheme_->getNUMANodeForBlock(input_block_id)),
+            op_index_);
+      }
+    }
+  } else {
+    for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
+      while (num_workorders_generated_in_partition_[part_id] <
+             input_relation_block_ids_in_partition_[part_id].size()) {
+        block_id block_in_partition
+            = input_relation_block_ids_in_partition_[part_id][num_workorders_generated_in_partition_[part_id]];
+        container->addNormalWorkOrder(
+            new SelectWorkOrder(
+                input_relation_,
+                block_in_partition,
+                predicate,
+                simple_projection_,
+                simple_selection_,
+                selection,
+                output_destination,
+                storage_manager,
+                placement_scheme_->getNUMANodeForBlock(block_in_partition)),
+            op_index_);
+        ++num_workorders_generated_in_partition_[part_id];
+      }
+    }
+  }
+}
+#endif
+
 bool SelectOperator::getAllWorkOrders(
     WorkOrdersContainer *container,
     QueryContext *query_context,
@@ -54,35 +141,27 @@ bool SelectOperator::getAllWorkOrders(
 
   if (input_relation_is_stored_) {
     if (!started_) {
-      for (const block_id input_block_id : input_relation_block_ids_) {
-        container->addNormalWorkOrder(
-            new SelectWorkOrder(input_relation_,
-                                input_block_id,
-                                predicate,
-                                simple_projection_,
-                                simple_selection_,
-                                selection,
-                                output_destination,
-                                storage_manager),
-            op_index_);
+      if (input_relation_.hasPartitionScheme()) {
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+        if (input_relation_.hasNUMAPlacementScheme()) {
+          addPartitionAwareWorkOrders(container, storage_manager, predicate, selection, output_destination);
+        }
+#endif
+      } else {
+        addWorkOrders(container, storage_manager, predicate, selection, output_destination);
       }
       started_ = true;
     }
     return started_;
   } else {
-    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
-      container->addNormalWorkOrder(
-          new SelectWorkOrder(
-              input_relation_,
-              input_relation_block_ids_[num_workorders_generated_],
-              predicate,
-              simple_projection_,
-              simple_selection_,
-              selection,
-              output_destination,
-              storage_manager),
-          op_index_);
-      ++num_workorders_generated_;
+    if (input_relation_.hasPartitionScheme()) {
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+        if (input_relation_.hasNUMAPlacementScheme()) {
+          addPartitionAwareWorkOrders(container, storage_manager, predicate, selection, output_destination);
+        }
+#endif
+    } else {
+        addWorkOrders(container, storage_manager, predicate, selection, output_destination);
     }
     return done_feeding_input_relation_;
   }
@@ -90,7 +169,7 @@ bool SelectOperator::getAllWorkOrders(
 
 void SelectWorkOrder::execute() {
   BlockReference block(
-      storage_manager_->getBlock(input_block_id_, input_relation_));
+      storage_manager_->getBlock(input_block_id_, input_relation_, getPreferredNUMANodes()[0]));
 
   if (simple_projection_) {
     block->selectSimple(simple_selection_,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/52b758fc/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 3da496c..76f4cb6 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -24,6 +24,12 @@
 
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogTypedefs.hpp"
+
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+#include "catalog/NUMAPlacementScheme.hpp"
+#endif
+
+#include "catalog/PartitionSchemeHeader.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
@@ -87,7 +93,28 @@ class SelectOperator : public RelationalOperator {
         num_workorders_generated_(0),
         simple_projection_(false),
         input_relation_is_stored_(input_relation_is_stored),
-        started_(false) {}
+        started_(false) {
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+    placement_scheme_ = input_relation.getNUMAPlacementSchemePtr();
+#endif
+    if (input_relation.hasPartitionScheme()) {
+      const PartitionScheme &part_scheme = input_relation.getPartitionScheme();
+      const PartitionSchemeHeader &part_scheme_header = part_scheme.getPartitionSchemeHeader();
+      const std::size_t num_partitions = part_scheme_header.getNumPartitions();
+      input_relation_block_ids_in_partition_.resize(num_partitions);
+      num_workorders_generated_in_partition_.resize(num_partitions);
+      num_workorders_generated_in_partition_.assign(num_partitions, 0);
+      for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
+        if (input_relation_is_stored) {
+          input_relation_block_ids_in_partition_[part_id] =
+              part_scheme.getBlocksInPartition(part_id);
+        } else {
+          input_relation_block_ids_in_partition_[part_id] =
+              std::vector<block_id>();
+        }
+      }
+    }
+  }
 
   /**
    * @brief Constructor for selection with simple projection of attributes.
@@ -124,7 +151,28 @@ class SelectOperator : public RelationalOperator {
         num_workorders_generated_(0),
         simple_projection_(true),
         input_relation_is_stored_(input_relation_is_stored),
-        started_(false) {}
+        started_(false) {
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+    placement_scheme_ = input_relation.getNUMAPlacementSchemePtr();
+#endif
+    if (input_relation.hasPartitionScheme()) {
+      const PartitionScheme &part_scheme = input_relation.getPartitionScheme();
+      const PartitionSchemeHeader &part_scheme_header = part_scheme.getPartitionSchemeHeader();
+      const std::size_t num_partitions = part_scheme_header.getNumPartitions();
+      input_relation_block_ids_in_partition_.resize(num_partitions);
+      num_workorders_generated_in_partition_.resize(num_partitions);
+      num_workorders_generated_in_partition_.assign(num_partitions, 0);
+      for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
+        if (input_relation_is_stored) {
+          input_relation_block_ids_in_partition_[part_id] =
+              part_scheme.getBlocksInPartition(part_id);
+        } else {
+          input_relation_block_ids_in_partition_[part_id] =
+              std::vector<block_id>();
+        }
+      }
+    }
+  }
 
   ~SelectOperator() override {}
 
@@ -135,13 +183,33 @@ class SelectOperator : public RelationalOperator {
                         tmb::MessageBus *bus) override;
 
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
-    input_relation_block_ids_.push_back(input_block_id);
+    if (input_relation_.hasPartitionScheme()) {
+      const partition_id part_id =
+          input_relation_.getPartitionScheme().getPartitionForBlock(input_block_id);
+      input_relation_block_ids_in_partition_[part_id].push_back(input_block_id);
+    } else {
+      input_relation_block_ids_.push_back(input_block_id);
+    }
   }
 
+  // TODO(gerald): Each call to getPartitionForBlock() involves grabbing shared
+  // locks on each partition's mutex, checking if the block belongs to the
+  // partition. Instead, we can provide a method getPartitionsForBlocks() which
+  // accepts a list of blocks and returns corresponding list of their partition IDs.
+  // Therefore, once we grab a lock for a partition, we search for all the blocks
+  // and then release the lock.
   void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override {
-    input_relation_block_ids_.insert(input_relation_block_ids_.end(),
-                                     partially_filled_blocks->begin(),
-                                     partially_filled_blocks->end());
+    if (input_relation_.hasPartitionScheme()) {
+      for (auto it = partially_filled_blocks->begin(); it != partially_filled_blocks->end(); ++it) {
+        const partition_id part_id = input_relation_.getPartitionScheme().getPartitionForBlock((*it));
+        input_relation_block_ids_in_partition_[part_id].insert(input_relation_block_ids_in_partition_[part_id].end(),
+                                                               *it);
+      }
+    } else {
+      input_relation_block_ids_.insert(input_relation_block_ids_.end(),
+                                       partially_filled_blocks->begin(),
+                                       partially_filled_blocks->end());
+    }
   }
 
   QueryContext::insert_destination_id getInsertDestinationID() const override {
@@ -152,9 +220,20 @@ class SelectOperator : public RelationalOperator {
     return output_relation_.getID();
   }
 
+  void addWorkOrders(WorkOrdersContainer *container,
+                     StorageManager *storage_manager,
+                     const Predicate *predicate,
+                     const std::vector<std::unique_ptr<const Scalar>> *selection,
+                     InsertDestination *output_destination);
+
+  void addPartitionAwareWorkOrders(WorkOrdersContainer *container,
+                                   StorageManager *storage_manager,
+                                   const Predicate *predicate,
+                                   const std::vector<std::unique_ptr<const Scalar>> *selection,
+                                   InsertDestination *output_destination);
+
  private:
   const CatalogRelation &input_relation_;
-
   const CatalogRelation &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;
   const QueryContext::predicate_id predicate_index_;
@@ -163,12 +242,20 @@ class SelectOperator : public RelationalOperator {
   const std::vector<attribute_id> simple_selection_;
 
   std::vector<block_id> input_relation_block_ids_;
+  // A vector of vectors V where V[i] indicates the list of block IDs of the
+  // input relation that belong to the partition i.
+  std::vector<std::vector<block_id>> input_relation_block_ids_in_partition_;
 
   // A single workorder is generated for each block of input relation.
   std::vector<block_id>::size_type num_workorders_generated_;
+  // A single workorder is generated for each block in each partition of input relation.
+  std::vector<std::size_t> num_workorders_generated_in_partition_;
 
   const bool simple_projection_;
   const bool input_relation_is_stored_;
+#ifdef QUICKSTEP_HAVE_LIBNUMA
+  const NUMAPlacementScheme *placement_scheme_;
+#endif
   bool started_;
 
   DISALLOW_COPY_AND_ASSIGN(SelectOperator);
@@ -205,7 +292,8 @@ class SelectWorkOrder : public WorkOrder {
                   const std::vector<attribute_id> &simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
                   InsertDestination *output_destination,
-                  StorageManager *storage_manager)
+                  StorageManager *storage_manager,
+                  const numa_node_id numa_node = 0)
       : input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
@@ -213,7 +301,9 @@ class SelectWorkOrder : public WorkOrder {
         simple_selection_(simple_selection),
         selection_(selection),
         output_destination_(DCHECK_NOTNULL(output_destination)),
-        storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
+        storage_manager_(DCHECK_NOTNULL(storage_manager)) {
+    preferred_numa_nodes_.push_back(numa_node);
+  }
 
   /**
    * @brief Constructor for the distributed version.
@@ -241,7 +331,8 @@ class SelectWorkOrder : public WorkOrder {
                   std::vector<attribute_id> &&simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
                   InsertDestination *output_destination,
-                  StorageManager *storage_manager)
+                  StorageManager *storage_manager,
+                  const numa_node_id numa_node = 0)
       : input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
@@ -249,7 +340,9 @@ class SelectWorkOrder : public WorkOrder {
         simple_selection_(std::move(simple_selection)),
         selection_(selection),
         output_destination_(DCHECK_NOTNULL(output_destination)),
-        storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
+        storage_manager_(DCHECK_NOTNULL(storage_manager)) {
+    preferred_numa_nodes_.push_back(numa_node);
+  }
 
   ~SelectWorkOrder() override {}
 


[25/30] incubator-quickstep git commit: Set block slots correctly. (#202)

Posted by zu...@apache.org.
Set block slots correctly. (#202)

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

Branch: refs/heads/master
Commit: 2abc988dc2bf6d41ab4077a72fb1ff5bb57d7758
Parents: ae3ca33
Author: Zuyu ZHANG <zu...@users.noreply.github.com>
Authored: Fri Apr 29 12:01:19 2016 -0700
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed May 4 23:15:35 2016 -0700

----------------------------------------------------------------------
 query_optimizer/resolver/Resolver.cpp                | 13 +++++++++----
 query_optimizer/tests/physical_generator/Create.test |  6 +++---
 query_optimizer/tests/resolver/Create.test           | 15 ++++++++++++---
 storage/StorageConstants.hpp                         |  3 +--
 4 files changed, 25 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2abc988d/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 45ecf33..2667ee9 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -602,16 +602,21 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
   // Resolve the Block size (size -> # of slots).
   std::int64_t slots = kDefaultBlockSizeInSlots;
   if (block_properties->hasBlockSizeMb()) {
-    std::int64_t blocksizemb = block_properties->getBlockSizeMbValue();
-    if (blocksizemb == -1) {
+    const std::int64_t block_size_in_mega_bytes = block_properties->getBlockSizeMbValue();
+    if (block_size_in_mega_bytes == -1) {
       // Indicates an error condition if the property is present but getter returns -1.
       THROW_SQL_ERROR_AT(block_properties->getBlockSizeMb())
           << "The BLOCKSIZEMB property must be an integer.";
+    } else if ((block_size_in_mega_bytes * kAMegaByte) % kSlotSizeBytes != 0) {
+      THROW_SQL_ERROR_AT(block_properties->getBlockSizeMb())
+          << "The BLOCKSIZEMB property must be multiple times of "
+          << std::to_string(kSlotSizeBytes / kAMegaByte) << "MB.";
     }
-    slots = (blocksizemb * kAMegaByte) / kSlotSizeBytes;
+
+    slots = (block_size_in_mega_bytes * kAMegaByte) / kSlotSizeBytes;
     DLOG(INFO) << "Resolver using BLOCKSIZEMB of " << slots << " slots"
         << " which is " << (slots * kSlotSizeBytes) << " bytes versus"
-        << " user requested " << (blocksizemb * kAMegaByte) << " bytes.";
+        << " user requested " << (block_size_in_mega_bytes * kAMegaByte) << " bytes.";
     const std::uint64_t max_size_slots = kBlockSizeUpperBoundBytes / kSlotSizeBytes;
     const std::uint64_t min_size_slots = kBlockSizeLowerBoundBytes / kSlotSizeBytes;
     if (static_cast<std::uint64_t>(slots) < min_size_slots ||

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2abc988d/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 58e15fa..8e6c64b 100644
--- a/query_optimizer/tests/physical_generator/Create.test
+++ b/query_optimizer/tests/physical_generator/Create.test
@@ -1,5 +1,5 @@
 #   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+#   Copyright 2015-2016 Pivotal Software, Inc.
 #
 #   Licensed under the Apache License, Version 2.0 (the "License");
 #   you may not use this file except in compliance with the License.
@@ -86,8 +86,8 @@ TopLevelPlan
   +-AttributeReference[id=10,name=col11,relation=foo,type=Char(5) NULL]
   +-AttributeReference[id=11,name=col12,relation=foo,type=VarChar(5) NULL]
 ==
-CREATE TABLE foo (col1 INT) WITH BLOCKPROPERTIES 
-  (TYPE compressed_columnstore, SORT col1, COMPRESS ALL, BLOCKSIZEMB 5);
+CREATE TABLE foo (col1 INT) WITH BLOCKPROPERTIES
+  (TYPE compressed_columnstore, SORT col1, COMPRESS ALL, BLOCKSIZEMB 4);
 --
 [Optimized Logical Plan]
 TopLevelPlan

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2abc988d/query_optimizer/tests/resolver/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Create.test b/query_optimizer/tests/resolver/Create.test
index 18beacd..63f7ac9 100644
--- a/query_optimizer/tests/resolver/Create.test
+++ b/query_optimizer/tests/resolver/Create.test
@@ -1,5 +1,5 @@
 #   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+#   Copyright 2015-2016 Pivotal Software, Inc.
 #
 #   Licensed under the Apache License, Version 2.0 (the "License");
 #   you may not use this file except in compliance with the License.
@@ -123,7 +123,7 @@ BLOCKPROPERTIES (BLOCKSIZEMB 1...
 ==
 
 # Rowstores cannot have a sorted attribute.
-CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES 
+CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
 (TYPE rowstore, SORT attr);
 --
 ERROR: The SORT property does not apply to this block type. (2 : 22)
@@ -167,7 +167,7 @@ ERROR: The COMPRESS property does not apply to this block type. (2 : 7)
 ==
 
 # Compress property is required for compressed blocks.
-CREATE TABLE foo (attr INT) WITH 
+CREATE TABLE foo (attr INT) WITH
 BLOCKPROPERTIES (TYPE compressed_rowstore);
 --
 ERROR: The COMPRESS property must be specified as ALL or a list of attributes. (2 : 1)
@@ -202,6 +202,15 @@ ERROR: The BLOCKSIZEMB property must be an integer. (2 : 17)
                 ^
 ==
 
+# BLOCKSIZEMB property must be multiple times of the slot size.
+CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
+(TYPE rowstore, BLOCKSIZEMB 25);
+--
+ERROR: The BLOCKSIZEMB property must be multiple times of 2MB. (2 : 17)
+(TYPE rowstore, BLOCKSIZEMB 25);
+                ^
+==
+
 # BLOCKSIZEMB must be greater than the minimum (defined in StorageConstants.hpp).
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
 (TYPE rowstore, BLOCKSIZEMB 0);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2abc988d/storage/StorageConstants.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageConstants.hpp b/storage/StorageConstants.hpp
index de54345..154d2f7 100644
--- a/storage/StorageConstants.hpp
+++ b/storage/StorageConstants.hpp
@@ -44,8 +44,7 @@ const std::uint64_t kAMegaByte = (1 << 20);
 // the SQL clause BLOCKPROPERTIES.
 const std::uint64_t kBlockSizeUpperBoundBytes = kAGigaByte;
 
-// 2 Megabytes.
-const std::uint64_t kBlockSizeLowerBoundBytes = kAMegaByte << 1;
+const std::uint64_t kBlockSizeLowerBoundBytes = kSlotSizeBytes;
 
 // The default size of a new relation in terms of the number of slots.
 const std::uint64_t kDefaultBlockSizeInSlots = 1;