Merge branch 'master' into in-operator-support
diff --git a/build/catalog.pb.bin.orig b/build/catalog.pb.bin.orig
deleted file mode 100644
index 349a0a4..0000000
--- a/build/catalog.pb.bin.orig
+++ /dev/null
@@ -1,3 +0,0 @@
-
-	
-default
\ No newline at end of file
diff --git a/build/qsstor/.gitignore b/build/qsstor/.gitignore
deleted file mode 100644
index 1aabdb4..0000000
--- a/build/qsstor/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-# Empty .gitignore to force git to include this empty directory.
diff --git a/build/qsstor/catalog.pb.bin b/build/qsstor/catalog.pb.bin
deleted file mode 100644
index 349a0a4..0000000
--- a/build/qsstor/catalog.pb.bin
+++ /dev/null
@@ -1,3 +0,0 @@
-
-	
-default
\ No newline at end of file
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index ae3d88e..8c89d7e 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -179,13 +179,6 @@
                       quickstep_catalog_PartitionScheme
                       quickstep_catalog_PartitionSchemeHeader)
 
-# Catalog generator program.
-add_executable(quickstep_generate_initial_default_catalog GenerateInitialDefaultCatalog.cpp)
-target_link_libraries(quickstep_generate_initial_default_catalog
-                      quickstep_catalog_Catalog
-                      quickstep_catalog_CatalogDatabase
-                      quickstep_catalog_Catalog_proto)
-
 # Tests:
 add_executable(Catalog_unittest "${CMAKE_CURRENT_SOURCE_DIR}/tests/Catalog_unittest.cpp")
 target_link_libraries(Catalog_unittest
diff --git a/catalog/CatalogRelationSchema.cpp b/catalog/CatalogRelationSchema.cpp
index 35c8189..97c834f 100644
--- a/catalog/CatalogRelationSchema.cpp
+++ b/catalog/CatalogRelationSchema.cpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015-2016 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -25,6 +27,7 @@
 #include "catalog/Catalog.pb.h"
 #include "catalog/CatalogAttribute.hpp"
 #include "catalog/CatalogErrors.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "types/Type.hpp"
 #include "utility/PtrVector.hpp"
 #include "utility/StringUtil.hpp"
@@ -52,6 +55,7 @@
       min_variable_byte_length_(0),
       min_variable_byte_length_excluding_nullable_(0),
       estimated_variable_byte_length_(0),
+      max_byte_lengths_(proto.attributes_size()),
       current_nullable_attribute_index_(-1),
       current_variable_length_attribute_index_(-1) {
   DCHECK(ProtoIsValid(proto))
@@ -150,7 +154,7 @@
       }
       estimated_variable_byte_length_ += attr_type.estimateAverageByteLength();
     } else {
-      variable_length_attribute_indices_.push_back(-1);
+      variable_length_attribute_indices_.push_back(kInvalidCatalogId);
 
       fixed_length_attribute_offsets_.resize(new_attr->getID() + 1, fixed_byte_length_);
       fixed_byte_length_ += attr_type.maximumByteLength();
@@ -163,10 +167,16 @@
       ++num_nullable_attributes_;
       nullable_attribute_indices_.push_back(++current_nullable_attribute_index_);
     } else {
-      nullable_attribute_indices_.push_back(-1);
+      nullable_attribute_indices_.push_back(kInvalidCatalogId);
     }
 
-    return new_attr->getID();
+    const attribute_id attr_id = new_attr->getID();
+    // Ensure that we will not overrun the vector by resizing and initializing
+    // new entries to zero.
+    max_byte_lengths_.resize(attr_id + 1, 0);
+    max_byte_lengths_[attr_id] = attr_type.maximumByteLength();
+
+    return attr_id;
   }
 }
 
diff --git a/catalog/CatalogRelationSchema.hpp b/catalog/CatalogRelationSchema.hpp
index 054bd02..d773bc7 100644
--- a/catalog/CatalogRelationSchema.hpp
+++ b/catalog/CatalogRelationSchema.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -415,6 +417,16 @@
     return attr_vec_.end_skip();
   }
 
+  /**
+   * @brief Get a vector of the maximum byte lengths for each attribute.
+   * @details An index in the vector corresponds with the attribute's id.
+   *
+   * @return A vector of the maximum byte lengths for each attribute.
+   */
+  const std::vector<std::size_t>& getMaximumAttributeByteLengths() const {
+    return max_byte_lengths_;
+  }
+
  protected:
   /**
    * @brief Create a new relation.
@@ -511,6 +523,7 @@
               min_variable_byte_length_excluding_nullable_,
               estimated_variable_byte_length_;
   std::vector<std::size_t> fixed_length_attribute_offsets_;
+  std::vector<std::size_t> max_byte_lengths_;
 
   // Entries are -1 for non-nullable attributes.
   std::vector<int> nullable_attribute_indices_;
diff --git a/catalog/CatalogTypedefs.hpp b/catalog/CatalogTypedefs.hpp
index 233c6e8..213d91d 100644
--- a/catalog/CatalogTypedefs.hpp
+++ b/catalog/CatalogTypedefs.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—Madison.
  *
  *   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,10 @@
 // partition_id.
 const int kCatalogMaxID = INT_MAX;
 
+// Catalog ids use negative values as invalid ids. Mark -1 as constant invalid
+// id for the catalog ids.
+constexpr int kInvalidCatalogId = -1;
+
 /** @} */
 
 }  // namespace quickstep
diff --git a/catalog/GenerateInitialDefaultCatalog.cpp b/catalog/GenerateInitialDefaultCatalog.cpp
deleted file mode 100644
index 5fe99a5..0000000
--- a/catalog/GenerateInitialDefaultCatalog.cpp
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *
- *   Licensed under the Apache License, Version 2.0 (the "License");
- *   you may not use this file except in compliance with the License.
- *   You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- **/
-
-// This small program generates an initial catalog.pb.bin with an empty
-// "default" database.
-
-#include <cstdio>
-#include <fstream>
-
-#include "catalog/Catalog.hpp"
-#include "catalog/Catalog.pb.h"
-#include "catalog/CatalogDatabase.hpp"
-
-int main() {
-  std::ofstream catalog_file("catalog.pb.bin");
-  if (!catalog_file.good()) {
-    std::fprintf(stderr, "ERROR: Unable to open catalog.pb.bin for writing.\n");
-    return 1;
-  }
-
-  quickstep::Catalog catalog;
-  catalog.addDatabase(new quickstep::CatalogDatabase(nullptr, "default"));
-
-  if (!catalog.getProto().SerializeToOstream(&catalog_file)) {
-    std::fprintf(stderr, "ERROR: Unable to serialize catalog proto to file catalog.pb.bin\n");
-    return 1;
-  }
-
-  catalog_file.close();
-  return 0;
-}
diff --git a/catalog/tests/Catalog_unittest.cpp b/catalog/tests/Catalog_unittest.cpp
index 135b013..65593dc 100644
--- a/catalog/tests/Catalog_unittest.cpp
+++ b/catalog/tests/Catalog_unittest.cpp
@@ -214,6 +214,17 @@
            ++cit_expected, ++cit_checked) {
         CompareCatalogAttribute(*cit_expected, *cit_checked);
       }
+      const std::vector<std::size_t>& expected_max_lengths =
+          expected.getMaximumAttributeByteLengths();
+      const std::vector<std::size_t>& checked_max_lengths =
+          checked.getMaximumAttributeByteLengths();
+      ASSERT_EQ(expected_max_lengths.size(), checked_max_lengths.size());
+      for (attribute_id len_index = 0;
+           static_cast<std::size_t>(len_index) < expected_max_lengths.size();
+           ++len_index) {
+        EXPECT_EQ(expected_max_lengths[len_index],
+                  checked_max_lengths[len_index]);
+      }
     }
   }
 
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index ad04a91..b549514 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -25,6 +25,14 @@
 #include <string>
 #include <utility>
 #include <vector>
+#include <fstream>
+
+// TODO(jmp): If filesystem shows up in C++-17, we can switch to just using that.
+#ifdef QUICKSTEP_OS_WINDOWS
+#include <filesystem>
+#else
+#include <stdlib.h>
+#endif
 
 #include "cli/CliConfig.h"  // For QUICKSTEP_USE_LINENOISE.
 #include "cli/DropRelation.hpp"
@@ -127,13 +135,14 @@
             "accepting queries (should also set --buffer_pool_slots to be "
             "large enough to accomodate the entire database).");
 DEFINE_string(storage_path, kDefaultStoragePath,
-              "Filesystem path for quickstep database storage.");
+              "Filesystem path to store the Quickstep database.");
 DEFINE_string(worker_affinities, "",
               "A comma-separated list of CPU IDs to pin worker threads to "
               "(leaving this empty will cause all worker threads to inherit "
               "the affinity mask of the Quickstep process, which typically "
               "means that they will all be runable on any CPU according to "
               "the kernel's own scheduling policy).");
+DEFINE_bool(initialize_db, false, "If true, initialize a database.");
 }  // namespace quickstep
 
 int main(int argc, char* argv[]) {
@@ -189,13 +198,53 @@
 
   string catalog_path(fixed_storage_path);
   catalog_path.append("catalog.pb.bin");
+  if (quickstep::FLAGS_initialize_db) {  // Initialize the database
+    // TODO(jmp): Refactor the code in this file!
+    LOG(INFO) << "Initializing the database, creating a new catalog file and storage directory\n";
+
+    // Create the directory
+    // TODO(jmp): At some point, likely in C++-17, we will just have the
+    //            filesystem path, and we can clean this up
+#ifdef QUICKSTEP_OS_WINDOWS
+    std::filesystem::create_directories(fixed_storage_path);
+    LOG(FATAL) << "Failed when attempting to create the directory: " << fixed_storage_path << "\n";
+    LOG(FATAL) << "Check if the directory already exists. If so, delete it or move it before initializing \n";
+#else
+    {
+      string path_name = "mkdir " + fixed_storage_path;
+      if (std::system(path_name.c_str())) {
+        LOG(FATAL) << "Failed when attempting to create the directory: " << fixed_storage_path << "\n";
+      }
+    }
+#endif
+
+    // Create the default catalog file.
+    std::ofstream catalog_file(catalog_path);
+    if (!catalog_file.good()) {
+      LOG(FATAL) << "ERROR: Unable to open catalog.pb.bin for writing.\n";
+    }
+
+    quickstep::Catalog catalog;
+    catalog.addDatabase(new quickstep::CatalogDatabase(nullptr, "default"));
+
+    if (!catalog.getProto().SerializeToOstream(&catalog_file)) {
+      LOG(FATAL) << "ERROR: Unable to serialize catalog proto to file catalog.pb.bin\n";
+      return 1;
+    }
+
+    // Close the catalog file - it will be reopened below by the QueryProcessor.
+    catalog_file.close();
+  }
 
   // Setup QueryProcessor, including CatalogDatabase and StorageManager.
   std::unique_ptr<QueryProcessor> query_processor;
   try {
     query_processor.reset(new QueryProcessor(catalog_path, fixed_storage_path));
   } catch (const std::exception &e) {
-    LOG(FATAL) << "FATAL ERROR DURING STARTUP: " << e.what();
+    LOG(FATAL) << "FATAL ERROR DURING STARTUP: "
+               << e.what()
+               << "\nIf you intended to create a new database, "
+               << "please use the \"-initialize_db=true\" command line option.";
   } catch (...) {
     LOG(FATAL) << "NON-STANDARD EXCEPTION DURING STARTUP";
   }
@@ -239,18 +288,18 @@
   // Initialize the worker threads.
   DCHECK_EQ(static_cast<std::size_t>(real_num_workers),
             worker_cpu_affinities.size());
-  for (std::size_t worker_idx = 0;
-       worker_idx < worker_cpu_affinities.size();
-       ++worker_idx) {
+  for (std::size_t worker_thread_index = 0;
+       worker_thread_index < worker_cpu_affinities.size();
+       ++worker_thread_index) {
     int numa_node_id = -1;
-    if (worker_cpu_affinities[worker_idx] >= 0) {
+    if (worker_cpu_affinities[worker_thread_index] >= 0) {
       // This worker can be NUMA affinitized.
-      numa_node_id = cpu_numa_nodes[worker_cpu_affinities[worker_idx]];
+      numa_node_id = cpu_numa_nodes[worker_cpu_affinities[worker_thread_index]];
     }
     worker_numa_nodes.push_back(numa_node_id);
 
     workers.push_back(
-        new Worker(worker_idx, &bus, worker_cpu_affinities[worker_idx]));
+        new Worker(worker_thread_index, &bus, worker_cpu_affinities[worker_thread_index]));
     worker_client_ids.push_back(workers.back().getBusClientID());
   }
 
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
new file mode 100644
index 0000000..05ca58d
--- /dev/null
+++ b/expressions/aggregation/AggregationConcreteHandle.cpp
@@ -0,0 +1,71 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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 "expressions/aggregation/AggregationConcreteHandle.hpp"
+
+#include <cstddef>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/HashTable.hpp"
+#include "storage/HashTableFactory.hpp"
+
+namespace quickstep {
+
+class StorageManager;
+class Type;
+class ValueAccessor;
+
+AggregationStateHashTableBase* AggregationConcreteHandle::createDistinctifyHashTable(
+    const HashTableImplType hash_table_impl,
+    const std::vector<const Type*> &key_types,
+    const std::size_t estimated_num_distinct_keys,
+    StorageManager *storage_manager) const {
+  // Create a hash table with key types as key_types and value type as bool.
+  return AggregationStateHashTableFactory<bool>::CreateResizable(
+      hash_table_impl,
+      key_types,
+      estimated_num_distinct_keys,
+      storage_manager);
+}
+
+void AggregationConcreteHandle::insertValueAccessorIntoDistinctifyHashTable(
+    ValueAccessor *accessor,
+    const std::vector<attribute_id> &key_ids,
+    AggregationStateHashTableBase *distinctify_hash_table) const {
+  // If the key-value pair is already there, we don't need to update the value,
+  // which should always be "true". I.e. the value is just a placeholder.
+  const auto noop_upserter = [](const auto &accessor, const bool *value) -> void {};
+
+  AggregationStateHashTable<bool> *hash_table =
+      static_cast<AggregationStateHashTable<bool>*>(distinctify_hash_table);
+  if (key_ids.size() == 1) {
+    hash_table->upsertValueAccessor(accessor,
+                                    key_ids[0],
+                                    true /* check_for_null_keys */,
+                                    true /* initial_value */,
+                                    &noop_upserter);
+  } else {
+    hash_table->upsertValueAccessorCompositeKey(accessor,
+                                                key_ids,
+                                                true /* check_for_null_keys */,
+                                                true /* initial_value */,
+                                                &noop_upserter);
+  }
+}
+
+}  // namespace quickstep
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
new file mode 100644
index 0000000..52249f7
--- /dev/null
+++ b/expressions/aggregation/AggregationConcreteHandle.hpp
@@ -0,0 +1,378 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_
+#define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_
+
+#include <cstddef>
+#include <vector>
+#include <utility>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "storage/HashTable.hpp"
+#include "storage/HashTableBase.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class StorageManager;
+class Type;
+class ValueAccessor;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+
+/**
+ * @brief The helper intermediate subclass of AggregationHandle that provides
+ *        virtual method implementations as well as helper methods that are
+ *        shared among all its subclasses.
+ *
+ * @note The reason that we have this intermediate class instead of putting
+ *       everything inside AggregationHandle is to avoid cyclic dependency, e.g.
+ *       when HashTable has to be used.
+ **/
+class AggregationConcreteHandle : public AggregationHandle {
+ public:
+  /**
+   * @brief Default implementaion for AggregationHandle::accumulateNullary().
+   */
+  AggregationState* accumulateNullary(
+      const std::size_t num_tuples) const override {
+    LOG(FATAL) << "Called accumulateNullary on an AggregationHandle that "
+               << "takes at least one argument.";
+  }
+
+  /**
+   * @brief Implementaion for AggregationHandle::createDistinctifyHashTable()
+   *        that creates a new HashTable for the distinctify step for
+   *        DISTINCT aggregation.
+   */
+  AggregationStateHashTableBase* createDistinctifyHashTable(
+      const HashTableImplType hash_table_impl,
+      const std::vector<const Type*> &key_types,
+      const std::size_t estimated_num_distinct_keys,
+      StorageManager *storage_manager) const override;
+
+  /**
+   * @brief Implementaion for AggregationHandle::insertValueAccessorIntoDistinctifyHashTable()
+   *        that inserts the GROUP BY expressions and aggregation arguments together
+   *        as keys into the distinctify hash table.
+   */
+  void insertValueAccessorIntoDistinctifyHashTable(
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &key_ids,
+      AggregationStateHashTableBase *distinctify_hash_table) const override;
+
+ protected:
+  AggregationConcreteHandle() {
+  }
+
+  template <typename HandleT,
+            typename StateT,
+            typename HashTableT>
+  void aggregateValueAccessorIntoHashTableNullaryHelper(
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &group_by_key_ids,
+      const StateT &default_state,
+      AggregationStateHashTableBase *hash_table) const;
+
+  template <typename HandleT,
+            typename StateT,
+            typename HashTableT>
+  void aggregateValueAccessorIntoHashTableUnaryHelper(
+      ValueAccessor *accessor,
+      const attribute_id argument_id,
+      const std::vector<attribute_id> &group_by_key_ids,
+      const StateT &default_state,
+      AggregationStateHashTableBase *hash_table) const;
+
+  template <typename HandleT,
+            typename StateT>
+  StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelper(
+      const AggregationStateHashTableBase &distinctify_hash_table) const;
+
+  template <typename HandleT,
+            typename StateT,
+            typename HashTableT>
+  void aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
+      const AggregationStateHashTableBase &distinctify_hash_table,
+      const StateT &default_state,
+      AggregationStateHashTableBase *hash_table) const;
+
+  template <typename HandleT,
+            typename HashTableT>
+  ColumnVector* finalizeHashTableHelper(
+      const Type &result_type,
+      const AggregationStateHashTableBase &hash_table,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const;
+
+  template <typename HandleT, typename HashTableT>
+  inline TypedValue finalizeGroupInHashTable(
+      const AggregationStateHashTableBase &hash_table,
+      const std::vector<TypedValue> &group_key) const {
+    const AggregationState *group_state
+        = static_cast<const HashTableT&>(hash_table).getSingleCompositeKey(group_key);
+    DCHECK(group_state != nullptr)
+        << "Could not find entry for specified group_key in HashTable";
+    return static_cast<const HandleT*>(this)->finalizeHashTableEntry(*group_state);
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(AggregationConcreteHandle);
+};
+
+/**
+ * @brief Templated class to implement value-accessor-based upserter for each
+ *        aggregation state payload type. This version is for nullary
+ *        aggregates (those that take no arguments).
+ **/
+template <typename HandleT, typename StateT>
+class NullaryAggregationStateValueAccessorUpserter {
+ public:
+  explicit NullaryAggregationStateValueAccessorUpserter(const HandleT &handle)
+      : handle_(handle) {
+  }
+
+  template <typename ValueAccessorT>
+  inline void operator()(const ValueAccessorT &accessor, StateT *state) {
+    handle_.iterateNullaryInl(state);
+  }
+
+ private:
+  const HandleT &handle_;
+};
+
+/**
+ * @brief Templated class to implement value-accessor-based upserter for each
+ *        aggregation state payload type. This version is for unary aggregates
+ *        (those that take a single argument).
+ **/
+template <typename HandleT, typename StateT>
+class UnaryAggregationStateValueAccessorUpserter {
+ public:
+  UnaryAggregationStateValueAccessorUpserter(const HandleT &handle,
+                                             attribute_id value_id)
+    : handle_(handle),
+      value_id_(value_id) {
+  }
+
+  template <typename ValueAccessorT>
+  inline void operator()(const ValueAccessorT &accessor, StateT *state) {
+    handle_.iterateUnaryInl(state, accessor.getTypedValue(value_id_));
+  }
+
+ private:
+  const HandleT &handle_;
+  const attribute_id value_id_;
+};
+
+/**
+ * @brief Templated helper class used to implement
+ *        AggregationHandle::finalizeHashTable() by visiting each entry (i.e.
+ *        GROUP) in a HashTable, finalizing the aggregation for the GROUP, and
+ *        collecting the GROUP BY key values and the final aggregate values in
+ *        a ColumnVector.
+ **/
+template <typename HandleT, typename ColumnVectorT>
+class HashTableAggregateFinalizer {
+ public:
+  HashTableAggregateFinalizer(const HandleT &handle,
+                              std::vector<std::vector<TypedValue>> *group_by_keys,
+                              ColumnVectorT *output_column_vector)
+      : handle_(handle),
+        group_by_keys_(group_by_keys),
+        output_column_vector_(output_column_vector) {
+  }
+
+  inline void operator()(const std::vector<TypedValue> &group_by_key,
+                         const AggregationState &group_state) {
+    group_by_keys_->emplace_back(group_by_key);
+    output_column_vector_->appendTypedValue(handle_.finalizeHashTableEntry(group_state));
+  }
+
+ private:
+  const HandleT &handle_;
+  std::vector<std::vector<TypedValue>> *group_by_keys_;
+  ColumnVectorT *output_column_vector_;
+};
+
+/** @} */
+
+// ----------------------------------------------------------------------------
+// Implementations of templated methods follow:
+
+template <typename HandleT,
+          typename StateT,
+          typename HashTableT>
+void AggregationConcreteHandle::aggregateValueAccessorIntoHashTableNullaryHelper(
+    ValueAccessor *accessor,
+    const std::vector<attribute_id> &group_by_key_ids,
+    const StateT &default_state,
+    AggregationStateHashTableBase *hash_table) const {
+  NullaryAggregationStateValueAccessorUpserter<HandleT, StateT>
+      upserter(static_cast<const HandleT&>(*this));
+  static_cast<HashTableT*>(hash_table)->upsertValueAccessorCompositeKey(
+      accessor,
+      group_by_key_ids,
+      true,
+      default_state,
+      &upserter);
+}
+
+template <typename HandleT,
+          typename StateT,
+          typename HashTableT>
+void AggregationConcreteHandle::aggregateValueAccessorIntoHashTableUnaryHelper(
+    ValueAccessor *accessor,
+    const attribute_id argument_id,
+    const std::vector<attribute_id> &group_by_key_ids,
+    const StateT &default_state,
+    AggregationStateHashTableBase *hash_table) const {
+  UnaryAggregationStateValueAccessorUpserter<HandleT, StateT>
+      upserter(static_cast<const HandleT&>(*this), argument_id);
+  static_cast<HashTableT*>(hash_table)->upsertValueAccessorCompositeKey(
+      accessor,
+      group_by_key_ids,
+      true,
+      default_state,
+      &upserter);
+}
+
+template <typename HandleT,
+          typename StateT>
+StateT* AggregationConcreteHandle::aggregateOnDistinctifyHashTableForSingleUnaryHelper(
+    const AggregationStateHashTableBase &distinctify_hash_table) const {
+  const HandleT& handle = static_cast<const HandleT&>(*this);
+  StateT *state = static_cast<StateT*>(createInitialState());
+
+  // A lambda function which will be called on each key from the distinctify
+  // hash table.
+  const auto aggregate_functor = [&handle, &state](const TypedValue &key,
+                                                   const bool &dumb_placeholder) {
+    // For each (unary) key in the distinctify hash table, aggregate the key
+    // into "state".
+    handle.iterateUnaryInl(state, key);
+  };
+
+  const AggregationStateHashTable<bool> &hash_table =
+      static_cast<const AggregationStateHashTable<bool>&>(distinctify_hash_table);
+  // Invoke the lambda function "aggregate_functor" on each key from the distinctify
+  // hash table.
+  hash_table.forEach(&aggregate_functor);
+
+  return state;
+}
+
+template <typename HandleT,
+          typename StateT,
+          typename HashTableT>
+void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
+    const AggregationStateHashTableBase &distinctify_hash_table,
+    const StateT &default_state,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  const HandleT& handle = static_cast<const HandleT&>(*this);
+  HashTableT *target_hash_table = static_cast<HashTableT*>(aggregation_hash_table);
+
+  // A lambda function which will be called on each key-value pair from the
+  // distinctify hash table.
+  const auto aggregate_functor = [&handle, &target_hash_table, &default_state](
+      std::vector<TypedValue> &key,
+      const bool &dumb_placeholder) {
+    // For each (composite) key vector in the distinctify hash table with size N.
+    // The first N-1 entries are GROUP BY columns and the last entry is the argument
+    // to be aggregated on.
+    const TypedValue argument(std::move(key.back()));
+    key.pop_back();
+
+    // An upserter as lambda function for aggregating the argument into its
+    // GROUP BY group's entry inside aggregation_hash_table.
+    const auto upserter = [&handle, &argument](StateT *state) {
+      handle.iterateUnaryInl(state, argument);
+    };
+
+    target_hash_table->upsertCompositeKey(key, default_state, &upserter);
+  };
+
+  const AggregationStateHashTable<bool> &source_hash_table =
+      static_cast<const AggregationStateHashTable<bool>&>(distinctify_hash_table);
+  // Invoke the lambda function "aggregate_functor" on each composite key vector
+  // from the distinctify hash table.
+  source_hash_table.forEachCompositeKey(&aggregate_functor);
+}
+
+template <typename HandleT,
+          typename HashTableT>
+ColumnVector* AggregationConcreteHandle::finalizeHashTableHelper(
+    const Type &result_type,
+    const AggregationStateHashTableBase &hash_table,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  const HandleT &handle = static_cast<const HandleT&>(*this);
+  const HashTableT &hash_table_concrete = static_cast<const HashTableT&>(hash_table);
+
+  if (group_by_keys->empty()) {
+    if (NativeColumnVector::UsableForType(result_type)) {
+      NativeColumnVector *result = new NativeColumnVector(result_type,
+                                                          hash_table_concrete.numEntries());
+      HashTableAggregateFinalizer<HandleT, NativeColumnVector> finalizer(
+          handle,
+          group_by_keys,
+          result);
+      hash_table_concrete.forEachCompositeKey(&finalizer);
+      return result;
+    } else {
+      IndirectColumnVector *result = new IndirectColumnVector(result_type,
+                                                              hash_table_concrete.numEntries());
+      HashTableAggregateFinalizer<HandleT, IndirectColumnVector> finalizer(
+          handle,
+          group_by_keys,
+          result);
+      hash_table_concrete.forEachCompositeKey(&finalizer);
+      return result;
+    }
+  } else {
+    if (NativeColumnVector::UsableForType(result_type)) {
+      NativeColumnVector *result = new NativeColumnVector(result_type,
+                                                          group_by_keys->size());
+      for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
+        result->appendTypedValue(finalizeGroupInHashTable<HandleT, HashTableT>(hash_table,
+                                                                               group_by_key));
+      }
+      return result;
+    } else {
+      IndirectColumnVector *result = new IndirectColumnVector(result_type,
+                                                              hash_table_concrete.numEntries());
+      for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
+        result->appendTypedValue(finalizeGroupInHashTable<HandleT, HashTableT>(hash_table,
+                                                                               group_by_key));
+      }
+      return result;
+    }
+  }
+}
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index d2b667b..625f334 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -25,13 +27,11 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/HashTableBase.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
 #include "utility/Macros.hpp"
 
-#include "glog/logging.h"
-
 namespace quickstep {
 
+class ColumnVector;
 class StorageManager;
 class Type;
 class ValueAccessor;
@@ -72,7 +72,7 @@
  * methods that are used to actually compute the aggregate, storing
  * intermediate results in AggregationState objects.
  *
- * The work-flow for computing an aggregate without GROUP BY is as follows:
+ * I. The work-flow for computing an aggregate without GROUP BY is as follows:
  *     1. Create a global state for the aggregate with createInitialState().
  *     2. For each block in a relation (parallelizable):
  *        a. Call StorageBlock::aggregate() to accumulate results from the
@@ -83,7 +83,7 @@
  *           mergeStates() (this is threadsafe).
  *     3. Generate the final result by calling finalize() on the global state.
  *
- * The work-flow for computing an aggregate with GROUP BY is as follows:
+ * II. The work-flow for computing an aggregate with GROUP BY is as follows:
  *     1. Create a HashTable to hold per-group states by calling
  *        createGroupByHashTable().
  *     2. For each block in a relation (parallelizable):
@@ -152,10 +152,7 @@
    *         tuples.
    **/
   virtual AggregationState* accumulateNullary(
-      const std::size_t num_tuples) const {
-    LOG(FATAL) << "Called accumulateNullary on an AggregationHandle that "
-               << "takes at least one argument.";
-  }
+      const std::size_t num_tuples) const = 0;
 
   /**
    * @brief Accumulate (iterate over) all values in one or more ColumnVectors
@@ -270,218 +267,96 @@
       const AggregationStateHashTableBase &hash_table,
       std::vector<std::vector<TypedValue>> *group_by_keys) const = 0;
 
+  /**
+   * @brief Create a new HashTable for the distinctify step for DISTINCT aggregation.
+   *
+   * Distinctify is the first step for DISTINCT aggregation. This step inserts
+   * the GROUP BY expression values and aggregation arguments together as keys
+   * into the distinctify hash table, so that arguments are distinctified within
+   * each GROUP BY group. Later, a second-round aggregation on the distinctify
+   * hash table will be performed to actually compute the aggregated result for
+   * each GROUP BY group.
+   * 
+   * In the case of single aggregation where there is no GROUP BY expressions,
+   * we simply treat it as a special GROUP BY case that the GROUP BY expression
+   * vector is empty.
+   *
+   * @param hash_table_impl The choice of which concrete HashTable implementation
+   *        to use.
+   * @param key_types The types of the GROUP BY expressions together with the
+   *        types of the aggregation arguments.
+   * @param estimated_num_distinct_keys The estimated number of distinct keys
+   *        (i.e. GROUP BY expressions together with aggregation arguments) for
+   *        the distinctify step. This is used to size the initial HashTable.
+   *        This is an estimate only, and the HashTable will be resized if it
+   *        becomes over-full.
+   * @param storage_manager The StorageManager to use to create the HashTable.
+   *        A StorageBlob will be allocated to serve as the HashTable's in-memory
+   *        storage.
+   * @return A new HashTable instance with the appropriate state type for this
+   *         aggregate as the ValueT.
+   */
+  virtual AggregationStateHashTableBase* createDistinctifyHashTable(
+      const HashTableImplType hash_table_impl,
+      const std::vector<const Type*> &key_types,
+      const std::size_t estimated_num_distinct_keys,
+      StorageManager *storage_manager) const = 0;
+
+  /**
+   * @brief Inserts the GROUP BY expressions and aggregation arguments together
+   * as keys into the distinctify hash table.
+   *
+   * @param accessor The ValueAccessor that will be iterated over to read tuples.
+   * @param key_ids The attribute_ids of the GROUP BY expressions in accessor
+   *        together with the attribute_ids of the arguments to this aggregate
+   *        in accessor, in order.
+   * @param distinctify_hash_table The HashTable to store the GROUP BY expressions
+   *        and the aggregation arguments together as hash table keys and a bool
+   *        constant \c true as hash table value (So the hash table actually
+   *        serves as a hash set). This should have been created by calling
+   *        createDistinctifyHashTable();
+   */
+  virtual void insertValueAccessorIntoDistinctifyHashTable(
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &key_ids,
+      AggregationStateHashTableBase *distinctify_hash_table) const = 0;
+
+  /**
+   * @brief Perform single (i.e. without GROUP BY) aggregation on the keys from
+   * the distinctify hash table to actually compute the aggregated results.
+   *
+   * @param distinctify_hash_table Hash table which stores the distinctified
+   *        aggregation arguments as hash table keys. This should have been
+   *        created by calling createDistinctifyHashTable();
+   * @return A new AggregationState which contains the aggregated results from
+   *         applying the aggregate to the distinctify hash table.
+   *         Caller is responsible for deleting the returned AggregationState.
+   */
+  virtual AggregationState* aggregateOnDistinctifyHashTableForSingle(
+      const AggregationStateHashTableBase &distinctify_hash_table) const = 0;
+
+  /**
+   * @brief Perform GROUP BY aggregation on the keys from the distinctify hash
+   * table and upserts states into the aggregation hash table.
+   *
+   * @param distinctify_hash_table Hash table which stores the GROUP BY expression
+   *        values and aggregation arguments together as hash table keys.
+   * @param aggregation_hash_table The HashTable to upsert AggregationStates in.
+   *        This should have been created by calling createGroupByHashTable() on
+   *        this same AggregationHandle.
+   */
+  virtual void aggregateOnDistinctifyHashTableForGroupBy(
+      const AggregationStateHashTableBase &distinctify_hash_table,
+      AggregationStateHashTableBase *aggregation_hash_table) const = 0;
+
  protected:
   AggregationHandle() {
   }
 
-  template <typename HandleT,
-            typename StateT,
-            typename HashTableT>
-  void aggregateValueAccessorIntoHashTableNullaryHelper(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &group_by_key_ids,
-      const StateT &default_state,
-      AggregationStateHashTableBase *hash_table) const;
-
-  template <typename HandleT,
-            typename StateT,
-            typename HashTableT>
-  void aggregateValueAccessorIntoHashTableUnaryHelper(
-      ValueAccessor *accessor,
-      const attribute_id argument_id,
-      const std::vector<attribute_id> &group_by_key_ids,
-      const StateT &default_state,
-      AggregationStateHashTableBase *hash_table) const;
-
-  template <typename HandleT,
-            typename HashTableT>
-  ColumnVector* finalizeHashTableHelper(
-      const Type &result_type,
-      const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys) const;
-
-  template <typename HandleT, typename HashTableT>
-  inline TypedValue finalizeGroupInHashTable(
-      const AggregationStateHashTableBase &hash_table,
-      const std::vector<TypedValue> &group_key) const {
-    const AggregationState *group_state
-        = static_cast<const HashTableT&>(hash_table).getSingleCompositeKey(group_key);
-    DCHECK(group_state != nullptr)
-        << "Could not find entry for specified group_key in HashTable";
-    return static_cast<const HandleT*>(this)->finalizeHashTableEntry(*group_state);
-  }
-
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationHandle);
 };
 
-/**
- * @brief Templated class to implement value-accessor-based upserter for each
- *        aggregation state payload type. This version is for nullary
- *        aggregates (those that take no arguments).
- **/
-template <typename HandleT, typename StateT>
-class NullaryAggregationStateValueAccessorUpserter {
- public:
-  explicit NullaryAggregationStateValueAccessorUpserter(const HandleT &handle)
-      : handle_(handle) {
-  }
-
-  template <typename ValueAccessorT>
-  inline void operator()(const ValueAccessorT &accessor, StateT *state) {
-    handle_.iterateNullaryInl(state);
-  }
-
- private:
-  const HandleT &handle_;
-};
-
-/**
- * @brief Templated class to implement value-accessor-based upserter for each
- *        aggregation state payload type. This version is for unary aggregates
- *        (those that take a single argument).
- **/
-template <typename HandleT, typename StateT>
-class UnaryAggregationStateValueAccessorUpserter {
- public:
-  UnaryAggregationStateValueAccessorUpserter(const HandleT &handle,
-                                             attribute_id value_id)
-    : handle_(handle),
-      value_id_(value_id) {
-  }
-
-  template <typename ValueAccessorT>
-  inline void operator()(const ValueAccessorT &accessor, StateT *state) {
-    handle_.iterateUnaryInl(state, accessor.getTypedValue(value_id_));
-  }
-
- private:
-  const HandleT &handle_;
-  const attribute_id value_id_;
-};
-
-/**
- * @brief Templated helper class used to implement
- *        AggregationHandle::finalizeHashTable() by visiting each entry (i.e.
- *        GROUP) in a HashTable, finalizing the aggregation for the GROUP, and
- *        collecting the GROUP BY key values and the final aggregate values in
- *        a ColumnVector.
- **/
-template <typename HandleT, typename ColumnVectorT>
-class HashTableAggregateFinalizer {
- public:
-  HashTableAggregateFinalizer(const HandleT &handle,
-                              std::vector<std::vector<TypedValue>> *group_by_keys,
-                              ColumnVectorT *output_column_vector)
-      : handle_(handle),
-        group_by_keys_(group_by_keys),
-        output_column_vector_(output_column_vector) {
-  }
-
-  inline void operator()(const std::vector<TypedValue> &group_by_key,
-                         const AggregationState &group_state) {
-    group_by_keys_->emplace_back(group_by_key);
-    output_column_vector_->appendTypedValue(handle_.finalizeHashTableEntry(group_state));
-  }
-
- private:
-  const HandleT &handle_;
-  std::vector<std::vector<TypedValue>> *group_by_keys_;
-  ColumnVectorT *output_column_vector_;
-};
-
-/** @} */
-
-// ----------------------------------------------------------------------------
-// Implementations of templated methods follow:
-
-template <typename HandleT,
-          typename StateT,
-          typename HashTableT>
-void AggregationHandle::aggregateValueAccessorIntoHashTableNullaryHelper(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &group_by_key_ids,
-    const StateT &default_state,
-    AggregationStateHashTableBase *hash_table) const {
-  NullaryAggregationStateValueAccessorUpserter<HandleT, StateT>
-      upserter(static_cast<const HandleT&>(*this));
-  static_cast<HashTableT*>(hash_table)->upsertValueAccessorCompositeKey(
-      accessor,
-      group_by_key_ids,
-      true,
-      default_state,
-      &upserter);
-}
-
-template <typename HandleT,
-          typename StateT,
-          typename HashTableT>
-void AggregationHandle::aggregateValueAccessorIntoHashTableUnaryHelper(
-    ValueAccessor *accessor,
-    const attribute_id argument_id,
-    const std::vector<attribute_id> &group_by_key_ids,
-    const StateT &default_state,
-    AggregationStateHashTableBase *hash_table) const {
-  UnaryAggregationStateValueAccessorUpserter<HandleT, StateT>
-      upserter(static_cast<const HandleT&>(*this), argument_id);
-  static_cast<HashTableT*>(hash_table)->upsertValueAccessorCompositeKey(
-      accessor,
-      group_by_key_ids,
-      true,
-      default_state,
-      &upserter);
-}
-
-template <typename HandleT,
-          typename HashTableT>
-ColumnVector* AggregationHandle::finalizeHashTableHelper(
-    const Type &result_type,
-    const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys) const {
-  const HandleT &handle = static_cast<const HandleT&>(*this);
-  const HashTableT &hash_table_concrete = static_cast<const HashTableT&>(hash_table);
-
-  if (group_by_keys->empty()) {
-    if (NativeColumnVector::UsableForType(result_type)) {
-      NativeColumnVector *result = new NativeColumnVector(result_type,
-                                                          hash_table_concrete.numEntries());
-      HashTableAggregateFinalizer<HandleT, NativeColumnVector> finalizer(
-          handle,
-          group_by_keys,
-          result);
-      hash_table_concrete.forEachCompositeKey(&finalizer);
-      return result;
-    } else {
-      IndirectColumnVector *result = new IndirectColumnVector(result_type,
-                                                              hash_table_concrete.numEntries());
-      HashTableAggregateFinalizer<HandleT, IndirectColumnVector> finalizer(
-          handle,
-          group_by_keys,
-          result);
-      hash_table_concrete.forEachCompositeKey(&finalizer);
-      return result;
-    }
-  } else {
-    if (NativeColumnVector::UsableForType(result_type)) {
-      NativeColumnVector *result = new NativeColumnVector(result_type,
-                                                          group_by_keys->size());
-      for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
-        result->appendTypedValue(finalizeGroupInHashTable<HandleT, HashTableT>(hash_table,
-                                                                               group_by_key));
-      }
-      return result;
-    } else {
-      IndirectColumnVector *result = new IndirectColumnVector(result_type,
-                                                              hash_table_concrete.numEntries());
-      for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
-        result->appendTypedValue(finalizeGroupInHashTable<HandleT, HashTableT>(hash_table,
-                                                                               group_by_key));
-      }
-      return result;
-    }
-  }
-}
-
 }  // namespace quickstep
 
 #endif  // QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_HPP_
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index 9fc847d..cb0d63d 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -181,4 +183,24 @@
       group_by_keys);
 }
 
+AggregationState* AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
+    const AggregationStateHashTableBase &distinctify_hash_table) const {
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleAvg,
+      AggregationStateAvg>(
+          distinctify_hash_table);
+}
+
+void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
+    const AggregationStateHashTableBase &distinctify_hash_table,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+      AggregationHandleAvg,
+      AggregationStateAvg,
+      AggregationStateHashTable<AggregationStateAvg>>(
+          distinctify_hash_table,
+          blank_state_,
+          aggregation_hash_table);
+}
+
 }  // namespace quickstep
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 105540c..6a94ee6 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -24,6 +26,7 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
@@ -79,7 +82,7 @@
 /**
  * @brief An aggregationhandle for avg.
  **/
-class AggregationHandleAvg : public AggregationHandle {
+class AggregationHandleAvg : public AggregationConcreteHandle {
  public:
   ~AggregationHandleAvg() override {
   }
@@ -140,6 +143,21 @@
       const AggregationStateHashTableBase &hash_table,
       std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   *        for AVG aggregation.
+   */
+  AggregationState* aggregateOnDistinctifyHashTableForSingle(
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
+
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   *        for AVG aggregation.
+   */
+  void aggregateOnDistinctifyHashTableForGroupBy(
+      const AggregationStateHashTableBase &distinctify_hash_table,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
+
  private:
   friend class AggregateFunctionAvg;
 
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index ceb8e9f..9fb9fd4 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -178,6 +180,32 @@
       group_by_keys);
 }
 
+template <bool count_star, bool nullable_type>
+AggregationState* AggregationHandleCount<count_star, nullable_type>
+    ::aggregateOnDistinctifyHashTableForSingle(
+        const AggregationStateHashTableBase &distinctify_hash_table) const {
+  DCHECK_EQ(count_star, false);
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleCount<count_star, nullable_type>,
+      AggregationStateCount>(
+          distinctify_hash_table);
+}
+
+template <bool count_star, bool nullable_type>
+void AggregationHandleCount<count_star, nullable_type>
+    ::aggregateOnDistinctifyHashTableForGroupBy(
+        const AggregationStateHashTableBase &distinctify_hash_table,
+        AggregationStateHashTableBase *aggregation_hash_table) const {
+  DCHECK_EQ(count_star, false);
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+      AggregationHandleCount<count_star, nullable_type>,
+      AggregationStateCount,
+      AggregationStateHashTable<AggregationStateCount>>(
+          distinctify_hash_table,
+          AggregationStateCount(),
+          aggregation_hash_table);
+}
+
 // Explicitly instantiate and compile in the different versions of
 // AggregationHandleCount we need. Note that we do not compile a version with
 // 'count_star == true' and 'nullable_type == true', as that combination is
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index 28ea796..6bb4e65 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -25,6 +27,7 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
 #include "types/TypedValue.hpp"
@@ -86,7 +89,7 @@
  *        not nullable and NULL-checks can safely be skipped.
  **/
 template <bool count_star, bool nullable_type>
-class AggregationHandleCount : public AggregationHandle {
+class AggregationHandleCount : public AggregationConcreteHandle {
  public:
   ~AggregationHandleCount() override {
   }
@@ -148,6 +151,21 @@
       const AggregationStateHashTableBase &hash_table,
       std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   *        for SUM aggregation.
+   */
+  AggregationState* aggregateOnDistinctifyHashTableForSingle(
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
+
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   *        for SUM aggregation.
+   */
+  void aggregateOnDistinctifyHashTableForGroupBy(
+      const AggregationStateHashTableBase &distinctify_hash_table,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
+
  private:
   friend class AggregateFunctionCount;
 
diff --git a/expressions/aggregation/AggregationHandleDistinct.cpp b/expressions/aggregation/AggregationHandleDistinct.cpp
index a585c61..fe8ffcf 100644
--- a/expressions/aggregation/AggregationHandleDistinct.cpp
+++ b/expressions/aggregation/AggregationHandleDistinct.cpp
@@ -24,10 +24,11 @@
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
 
 #include "types/TypedValue.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 class ColumnVector;
@@ -40,7 +41,7 @@
     const std::vector<const Type*> &group_by_types,
     const std::size_t estimated_num_groups,
     StorageManager *storage_manager) const {
-  return AggregationStateHashTableFactory<bool>::CreateResizable(
+  return createDistinctifyHashTable(
       hash_table_impl,
       group_by_types,
       estimated_num_groups,
@@ -54,13 +55,10 @@
     AggregationStateHashTableBase *hash_table) const {
   DCHECK_EQ(argument_ids.size(), 0u);
 
-  const auto noop_upserter = [](const auto &accessor, const bool *value) -> void {};
-  static_cast<AggregationStateHashTable<bool>*>(hash_table)->upsertValueAccessorCompositeKey(
+  insertValueAccessorIntoDistinctifyHashTable(
       accessor,
       group_by_key_ids,
-      true,
-      true, /* Initial value */
-      &noop_upserter);
+      hash_table);
 }
 
 ColumnVector* AggregationHandleDistinct::finalizeHashTable(
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
index 7a3a6d0..918fdf8 100644
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ b/expressions/aggregation/AggregationHandleDistinct.hpp
@@ -23,7 +23,7 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationHandle.hpp"
+#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "storage/HashTableBase.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
@@ -32,6 +32,7 @@
 
 namespace quickstep {
 
+class AggregationState;
 class ColumnVector;
 class StorageManager;
 class Type;
@@ -41,7 +42,7 @@
  *  @{
  */
 
-class AggregationHandleDistinct : public AggregationHandle{
+class AggregationHandleDistinct : public AggregationConcreteHandle {
  public:
   /**
    * @brief Constructor.
@@ -79,6 +80,19 @@
     LOG(FATAL) << "AggregationHandleDistinct does not support finalize().";
   }
 
+  AggregationState* aggregateOnDistinctifyHashTableForSingle(
+      const AggregationStateHashTableBase &distinctify_hash_table) const override {
+    LOG(FATAL) << "AggregationHandleDistinct does not support "
+               << "aggregateOnDistinctifyHashTableForSingle().";
+  }
+
+  void aggregateOnDistinctifyHashTableForGroupBy(
+      const AggregationStateHashTableBase &distinctify_hash_table,
+      AggregationStateHashTableBase *groupby_hash_table) const override {
+    LOG(FATAL) << "AggregationHandleDistinct does not support "
+               << "aggregateOnDistinctifyHashTableForGroupBy().";
+  }
+
   AggregationStateHashTableBase* createGroupByHashTable(
       const HashTableImplType hash_table_impl,
       const std::vector<const Type*> &group_by_types,
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index 156142e..4703657 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.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.
@@ -119,4 +119,24 @@
       group_by_keys);
 }
 
+AggregationState* AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
+    const AggregationStateHashTableBase &distinctify_hash_table) const {
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleMax,
+      AggregationStateMax>(
+          distinctify_hash_table);
+}
+
+void AggregationHandleMax::aggregateOnDistinctifyHashTableForGroupBy(
+    const AggregationStateHashTableBase &distinctify_hash_table,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+      AggregationHandleMax,
+      AggregationStateMax,
+      AggregationStateHashTable<AggregationStateMax>>(
+          distinctify_hash_table,
+          AggregationStateMax(type_),
+          aggregation_hash_table);
+}
+
 }  // namespace quickstep
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index d803332..8932ef8 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -24,6 +26,7 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
@@ -79,7 +82,7 @@
 /**
  * @brief An aggregationhandle for max.
  **/
-class AggregationHandleMax : public AggregationHandle {
+class AggregationHandleMax : public AggregationConcreteHandle {
  public:
   ~AggregationHandleMax() override {
   }
@@ -132,6 +135,22 @@
       const AggregationStateHashTableBase &hash_table,
       std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   *        for MAX aggregation.
+   */
+  AggregationState* aggregateOnDistinctifyHashTableForSingle(
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
+
+
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   *        for MAX aggregation.
+   */
+  void aggregateOnDistinctifyHashTableForGroupBy(
+      const AggregationStateHashTableBase &distinctify_hash_table,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
+
  private:
   friend class AggregateFunctionMax;
 
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index dbedddc..de2709a 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -119,4 +121,24 @@
       group_by_keys);
 }
 
+AggregationState* AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
+    const AggregationStateHashTableBase &distinctify_hash_table) const {
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleMin,
+      AggregationStateMin>(
+          distinctify_hash_table);
+}
+
+void AggregationHandleMin::aggregateOnDistinctifyHashTableForGroupBy(
+    const AggregationStateHashTableBase &distinctify_hash_table,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+      AggregationHandleMin,
+      AggregationStateMin,
+      AggregationStateHashTable<AggregationStateMin>>(
+          distinctify_hash_table,
+          AggregationStateMin(type_),
+          aggregation_hash_table);
+}
+
 }  // namespace quickstep
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index 30a5ecb..4e4c05d 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -24,6 +26,7 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
@@ -78,7 +81,7 @@
 /**
  * @brief An aggregationhandle for min.
  **/
-class AggregationHandleMin : public AggregationHandle {
+class AggregationHandleMin : public AggregationConcreteHandle {
  public:
   ~AggregationHandleMin() override {
   }
@@ -131,6 +134,21 @@
       const AggregationStateHashTableBase &hash_table,
       std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   *        for MIN aggregation.
+   */
+  AggregationState* aggregateOnDistinctifyHashTableForSingle(
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
+
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   *        for MIN aggregation.
+   */
+  void aggregateOnDistinctifyHashTableForGroupBy(
+      const AggregationStateHashTableBase &distinctify_hash_table,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
+
  private:
   friend class AggregateFunctionMin;
 
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 171445f..410fecb 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -168,4 +170,24 @@
       group_by_keys);
 }
 
+AggregationState* AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
+    const AggregationStateHashTableBase &distinctify_hash_table) const {
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleSum,
+      AggregationStateSum>(
+          distinctify_hash_table);
+}
+
+void AggregationHandleSum::aggregateOnDistinctifyHashTableForGroupBy(
+    const AggregationStateHashTableBase &distinctify_hash_table,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+      AggregationHandleSum,
+      AggregationStateSum,
+      AggregationStateHashTable<AggregationStateSum>>(
+          distinctify_hash_table,
+          blank_state_,
+          aggregation_hash_table);
+}
+
 }  // namespace quickstep
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index 58fd54e..b765243 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -24,6 +26,7 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
@@ -78,7 +81,7 @@
 /**
  * @brief An aggregationhandle for sum.
  **/
-class AggregationHandleSum : public AggregationHandle {
+class AggregationHandleSum : public AggregationConcreteHandle {
  public:
   ~AggregationHandleSum() override {
   }
@@ -130,6 +133,21 @@
       const AggregationStateHashTableBase &hash_table,
       std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   *        for SUM aggregation.
+   */
+  AggregationState* aggregateOnDistinctifyHashTableForSingle(
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
+
+  /**
+   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   *        for SUM aggregation.
+   */
+  void aggregateOnDistinctifyHashTableForGroupBy(
+      const AggregationStateHashTableBase &distinctify_hash_table,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
+
  private:
   friend class AggregateFunctionSum;
 
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index 4d92b45..26cec7f 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -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.
@@ -41,6 +41,9 @@
 add_library(quickstep_expressions_aggregation_AggregateFunctionSum
             AggregateFunctionSum.cpp
             AggregateFunctionSum.hpp)
+add_library(quickstep_expressions_aggregation_AggregationConcreteHandle
+            AggregationConcreteHandle.cpp
+            AggregationConcreteHandle.hpp)
 add_library(quickstep_expressions_aggregation_AggregationHandle
             ../../empty_src.cpp
             AggregationHandle.hpp)
@@ -137,16 +140,26 @@
                       quickstep_types_operations_binaryoperations_BinaryOperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_aggregation_AggregationConcreteHandle
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_HashTable
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableFactory
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandle
                       glog
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_HashTableBase
                       quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
@@ -163,6 +176,7 @@
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleCount
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
@@ -178,15 +192,15 @@
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleDistinct
                       glog
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
                       quickstep_types_TypedValue
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMax
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
@@ -202,6 +216,7 @@
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMin
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
@@ -217,6 +232,7 @@
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleSum
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
@@ -242,6 +258,7 @@
                       quickstep_expressions_aggregation_AggregateFunctionMax
                       quickstep_expressions_aggregation_AggregateFunctionMin
                       quickstep_expressions_aggregation_AggregateFunctionSum
+                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_expressions_aggregation_AggregationHandleCount
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index 3d9f726..ccce7b3 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -95,6 +95,7 @@
 add_library(quickstep_parser_ParseLimit ParseLimit.cpp ParseLimit.hpp)
 add_library(quickstep_parser_ParseLiteralValue ParseLiteralValue.cpp ParseLiteralValue.hpp)
 add_library(quickstep_parser_ParseOrderBy ParseOrderBy.cpp ParseOrderBy.hpp)
+add_library(quickstep_parser_ParsePartitionClause ../empty_src.cpp ParsePartitionClause.hpp)
 add_library(quickstep_parser_ParsePredicate ParsePredicate.cpp ParsePredicate.hpp)
 add_library(quickstep_parser_ParseSample ParseSample.cpp ParseSample.hpp)
 add_library(quickstep_parser_ParseSelect ../empty_src.cpp ParseSelect.hpp)
@@ -188,13 +189,18 @@
                       quickstep_parser_ParseTreeNode
                       quickstep_utility_Macros
                       quickstep_utility_PtrList)
+target_link_libraries(quickstep_parser_ParsePartitionClause
+                      quickstep_parser_ParseLiteralValue
+                      quickstep_parser_ParseString
+                      quickstep_parser_ParseTreeNode
+                      quickstep_utility_Macros
+                      quickstep_utility_PtrList)
 target_link_libraries(quickstep_parser_ParsePredicate
                       quickstep_parser_ParseExpression
                       quickstep_parser_ParseTreeNode
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_utility_Macros
                       quickstep_utility_PtrList)
-
 target_link_libraries(quickstep_parser_ParseSample
                       quickstep_parser_ParseLiteralValue
                       quickstep_parser_ParseTreeNode
@@ -230,6 +236,7 @@
                       quickstep_parser_ParseBlockProperties
                       quickstep_parser_ParseIndexProperties
                       quickstep_parser_ParseKeyValue
+                      quickstep_parser_ParsePartitionClause
                       quickstep_parser_ParsePredicate
                       quickstep_parser_ParseSelect
                       quickstep_parser_ParseString
@@ -275,6 +282,7 @@
                       quickstep_utility_PtrList
                       quickstep_utility_PtrVector)
 target_link_libraries(quickstep_parser_SqlParser
+                      quickstep_catalog_PartitionSchemeHeader
                       quickstep_parser_ParseAssignment
                       quickstep_parser_ParseAttributeDefinition
                       quickstep_parser_ParseBasicExpressions
@@ -287,6 +295,7 @@
                       quickstep_parser_ParseLimit
                       quickstep_parser_ParseLiteralValue
                       quickstep_parser_ParseOrderBy
+                      quickstep_parser_ParsePartitionClause
                       quickstep_parser_ParsePredicate
                       quickstep_parser_ParseSample
                       quickstep_parser_ParseSelect
@@ -357,6 +366,7 @@
                       quickstep_parser_ParseLimit
                       quickstep_parser_ParseLiteralValue
                       quickstep_parser_ParseOrderBy
+                      quickstep_parser_ParsePartitionClause
                       quickstep_parser_ParsePredicate
                       quickstep_parser_ParserUtil
                       quickstep_parser_ParseSample
diff --git a/parser/ParseBasicExpressions.cpp b/parser/ParseBasicExpressions.cpp
index dd80abd..2b1d7e0 100644
--- a/parser/ParseBasicExpressions.cpp
+++ b/parser/ParseBasicExpressions.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -120,6 +122,9 @@
   if (star_ != nullptr) {
     name.push_back('*');
   } else {
+    if (is_distinct_) {
+      name.append("DISTINCT ");
+    }
     bool first = true;
     for (const ParseExpression &argument : *arguments_) {
       if (!first) {
@@ -144,6 +149,11 @@
   inline_field_names->push_back("name");
   inline_field_values->push_back(name_->value());
 
+  if (is_distinct_) {
+    inline_field_names->push_back("is_distinct");
+    inline_field_values->push_back("true");
+  }
+
   if (star_ != nullptr) {
     inline_field_names->push_back("is_star");
     inline_field_values->push_back("true");
@@ -155,4 +165,28 @@
   }
 }
 
+std::string ParseExtractFunction::generateName() const {
+  std::string name;
+  name.append("EXTRACT(");
+  name.append(extract_field_->value());
+  name.append(" FROM ");
+  name.append(date_expression_->generateName());
+  name.push_back(')');
+  return name;
+}
+
+void ParseExtractFunction::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<const ParseTreeNode*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const {
+  inline_field_names->push_back("unit");
+  inline_field_values->push_back(extract_field_->value());
+
+  non_container_child_field_names->push_back("date_expression");
+  non_container_child_fields->push_back(date_expression_.get());
+}
+
 }  // namespace quickstep
diff --git a/parser/ParseBasicExpressions.hpp b/parser/ParseBasicExpressions.hpp
index 2668489..1886c8b 100644
--- a/parser/ParseBasicExpressions.hpp
+++ b/parser/ParseBasicExpressions.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -352,14 +354,19 @@
    *
    * @param line_number The line number of the first token of the function call.
    * @param column_number The column number of the first token of the first call.
+   * @param is_distinct Whether this function call contains the DISTINCT keyword.
    * @param name The function name.
    * @param arguments The function argument list.
    */
   ParseFunctionCall(const int line_number,
                     const int column_number,
+                    const bool is_distinct,
                     ParseString *name,
                     PtrList<ParseExpression> *arguments)
-      : ParseExpression(line_number, column_number), name_(name), arguments_(arguments) {
+      : ParseExpression(line_number, column_number),
+        is_distinct_(is_distinct),
+        name_(name),
+        arguments_(arguments) {
   }
 
   /**
@@ -373,6 +380,7 @@
    */
   ParseFunctionCall(const int line_number, const int column_number, ParseString *name, ParseStar *star)
       : ParseExpression(line_number, column_number),
+        is_distinct_(false),
         name_(name),
         star_(star) {
   }
@@ -392,6 +400,13 @@
   }
 
   /**
+   * @return Whether this function call contains the DISTINCT keyword.
+   */
+  bool is_distinct() const {
+    return is_distinct_;
+  }
+
+  /**
    * @return The function name.
    */
   const ParseString* name() const {
@@ -424,6 +439,7 @@
       std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
 
  private:
+  const bool is_distinct_;
   std::unique_ptr<ParseString> name_;
   // Either <arguments_> or <star_> is NULL.
   std::unique_ptr<PtrList<ParseExpression>> arguments_;
@@ -432,6 +448,69 @@
   DISALLOW_COPY_AND_ASSIGN(ParseFunctionCall);
 };
 
+
+/**
+ * @brief Parsed representation of EXTRACT(unit FROM date).
+ */
+class ParseExtractFunction : public ParseExpression {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param line_number The line number of the token "extract" in the statement.
+   * @param column_number The column number of the token "extract in the statement.
+   * @param extract_field The field to extract.
+   * @param source_expression The expression to extract a field from.
+   */
+  ParseExtractFunction(const int line_number,
+                       const int column_number,
+                       ParseString *extract_field,
+                       ParseExpression *date_expression)
+      : ParseExpression(line_number, column_number),
+        extract_field_(extract_field),
+        date_expression_(date_expression) {
+  }
+
+  ExpressionType getExpressionType() const override {
+    return kExtract;
+  }
+
+  std::string getName() const override {
+    return "Extract";
+  }
+
+  /**
+   * @return The field to extract.
+   */
+  const ParseString* extract_field() const {
+    return extract_field_.get();
+  }
+
+  /**
+   * @return The expression to extract a field from.
+   */
+  const ParseExpression* date_expression() const {
+    return date_expression_.get();
+  }
+
+  std::string generateName() const override;
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<const ParseTreeNode*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
+
+ private:
+  std::unique_ptr<ParseString> extract_field_;
+  std::unique_ptr<ParseExpression> date_expression_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParseExtractFunction);
+};
+
 /** @} */
 
 }  // namespace quickstep
diff --git a/parser/ParseExpression.hpp b/parser/ParseExpression.hpp
index accad01..8e4efee 100644
--- a/parser/ParseExpression.hpp
+++ b/parser/ParseExpression.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -37,6 +39,7 @@
   enum ExpressionType {
     kAttribute,
     kBinaryExpression,
+    kExtract,
     kFunctionCall,
     kScalarLiteral,
     kSubqueryExpression,
diff --git a/parser/ParsePartitionClause.hpp b/parser/ParsePartitionClause.hpp
new file mode 100644
index 0000000..1b7096f
--- /dev/null
+++ b/parser/ParsePartitionClause.hpp
@@ -0,0 +1,132 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *   University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_PARSER_PARSE_PARTITION_CLAUSE_HPP_
+#define QUICKSTEP_PARSER_PARSE_PARTITION_CLAUSE_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parser/ParseLiteralValue.hpp"
+#include "parser/ParseString.hpp"
+#include "parser/ParseTreeNode.hpp"
+#include "utility/Macros.hpp"
+#include "utility/PtrList.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Parser
+ *  @{
+ */
+
+/**
+ * @brief A parsed representation of partition clause.
+ */
+class ParsePartitionClause : public ParseTreeNode {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param line_number The line number of "PARTITION CLAUSE" in the SQL statement.
+   * @param column_number The column number of "PARTITION CLAUSE" in the SQL statement.
+   * @param partition_type The type of partitioning to be made - Hash or Range
+   * @param attribute_name_list A list of attributes of the relation based on which the partitioning has to be done.
+   * @param num_partitions The number of partitions to be created.
+   */
+  ParsePartitionClause(const int line_number,
+                       const int column_number,
+                       ParseString *partition_type,
+                       PtrList<ParseString> *attribute_name_list,
+                       NumericParseLiteralValue *num_partitions)
+      : ParseTreeNode(line_number, column_number),
+        partition_type_(partition_type),
+        attribute_name_list_(attribute_name_list),
+        num_partitions_(num_partitions) {
+  }
+
+  /**
+   * @brief Destructor.
+   */
+  ~ParsePartitionClause() override {}
+
+  std::string getName() const override {
+    return "PartitionClause";
+  }
+
+  /**
+   * @brief Get the type of the partitioning to be created.
+   *
+   * @return The type of partitioning.
+   **/
+  const ParseString* partition_type() const {
+    return partition_type_.get();
+  }
+
+  /**
+   * @brief Get the list of attributes on which partitioning is supposed to be defined.
+   *
+   * @return The list of attributes on which partitioning is to be based on.
+   **/
+  const PtrList<ParseString>& attribute_name_list() const {
+    return *attribute_name_list_;
+  }
+
+  /**
+   * @brief Get the number of partitions.
+   *
+   * @return The number of partitions.
+   */
+  const NumericParseLiteralValue* num_partitions() const {
+    return num_partitions_.get();
+  }
+
+ protected:
+  void getFieldStringItems(std::vector<std::string> *inline_field_names,
+                           std::vector<std::string> *inline_field_values,
+                           std::vector<std::string> *non_container_child_field_names,
+                           std::vector<const ParseTreeNode*> *non_container_child_fields,
+                           std::vector<std::string> *container_child_field_names,
+                           std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override {
+    inline_field_names->push_back("partition_type");
+    inline_field_values->push_back(partition_type_->value());
+
+    if (attribute_name_list_.get() != nullptr) {
+      container_child_field_names->push_back("attribute_name_list");
+      container_child_fields->emplace_back();
+      for (const ParseString& attribute_name : *attribute_name_list_) {
+        container_child_fields->back().push_back(&attribute_name);
+      }
+    }
+
+    non_container_child_field_names->push_back("Number of Partitions");
+    non_container_child_fields->push_back(num_partitions_.get());
+  }
+
+ private:
+  std::unique_ptr<ParseString> partition_type_;
+  std::unique_ptr<PtrList<ParseString>> attribute_name_list_;
+  std::unique_ptr<NumericParseLiteralValue> num_partitions_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParsePartitionClause);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_PARSER_PARSE_PARTITION_CLAUSE_HPP_
diff --git a/parser/ParseStatement.hpp b/parser/ParseStatement.hpp
index 2400002..9807e0a 100644
--- a/parser/ParseStatement.hpp
+++ b/parser/ParseStatement.hpp
@@ -30,6 +30,7 @@
 #include "parser/ParseBlockProperties.hpp"
 #include "parser/ParseIndexProperties.hpp"
 #include "parser/ParseKeyValue.hpp"
+#include "parser/ParsePartitionClause.hpp"
 #include "parser/ParsePredicate.hpp"
 #include "parser/ParseSelect.hpp"
 #include "parser/ParseString.hpp"
@@ -109,11 +110,13 @@
                             const int column_number,
                             ParseString *relation_name,
                             PtrList<ParseAttributeDefinition> *attribute_definition_list,
-                            ParseBlockProperties *opt_block_properties)
+                            ParseBlockProperties *opt_block_properties,
+                            ParsePartitionClause *opt_partition_clause)
       : ParseStatement(line_number, column_number),
         relation_name_(relation_name),
         attribute_definition_list_(attribute_definition_list),
-        opt_block_properties_(opt_block_properties) {
+        opt_block_properties_(opt_block_properties),
+        opt_partition_clause_(opt_partition_clause) {
   }
 
   ~ParseStatementCreateTable() override {
@@ -152,6 +155,15 @@
     return opt_block_properties_.get();
   }
 
+  /**
+   * @brief Get a pointer to the PartitionClause.
+   *
+   * @return The PartitionClause or nullptr if not specified.
+   **/
+  const ParsePartitionClause* opt_partition_clause() const {
+    return opt_partition_clause_.get();
+  }
+
  protected:
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
@@ -174,12 +186,19 @@
       container_child_fields->emplace_back();
       container_child_fields->back().push_back(opt_block_properties_.get());
     }
+
+    if (opt_partition_clause_) {
+      container_child_field_names->push_back("partition_clause");
+      container_child_fields->emplace_back();
+      container_child_fields->back().push_back(opt_partition_clause_.get());
+    }
   }
 
  private:
   std::unique_ptr<ParseString> relation_name_;
   std::unique_ptr<PtrList<ParseAttributeDefinition> > attribute_definition_list_;
   std::unique_ptr<ParseBlockProperties> opt_block_properties_;
+  std::unique_ptr<ParsePartitionClause> opt_partition_clause_;
 
   DISALLOW_COPY_AND_ASSIGN(ParseStatementCreateTable);
 };
@@ -519,33 +538,37 @@
 
 /**
  * @brief The parsed representation of an INSERT statement.
+ *
+ * This is an abstract class where each of its subclass represents a concrete
+ * type of insert operation.
  **/
 class ParseStatementInsert : public ParseStatement {
  public:
+  enum class InsertType {
+    kTuple = 0,
+    kSelection
+  };
+
   /**
    * @brief Constructor.
    *
    * @param line_number Line number of the first token of this node in the SQL statement.
    * @param column_number Column number of the first token of this node in the SQL statement.
    * @param relation_name The name of the relation to insert into.
-   * @param literal_values A list of literal values (in attribute-definition
-   *        order) to insert into the specified relation as a new tuple.
-   *        Becomes owned by this ParseStatementInsert.
    **/
   ParseStatementInsert(const int line_number,
                        const int column_number,
-                       ParseString *relation_name,
-                       PtrList<ParseScalarLiteral> *literal_values)
+                       const ParseString *relation_name)
       : ParseStatement(line_number, column_number),
-        relation_name_(relation_name),
-        literal_values_(literal_values) {
+        relation_name_(relation_name) {
   }
 
   /**
-   * @brief Destructor.
+   * @brief Get the insert type of this insert statement.
+   *
+   * @return The insert type of this insert statement.
    */
-  ~ParseStatementInsert() override {
-  }
+  virtual InsertType getInsertType() const = 0;
 
   std::string getName() const override { return "InsertStatement"; }
 
@@ -562,6 +585,43 @@
     return relation_name_.get();
   }
 
+ private:
+  std::unique_ptr<const ParseString> relation_name_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParseStatementInsert);
+};
+
+
+/**
+ * @brief The parsed representation of an INSERT ... VALUES ... statement.
+ **/
+class ParseStatementInsertTuple : public ParseStatementInsert {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param line_number Line number of the first token of this node in the SQL statement.
+   * @param column_number Column number of the first token of this node in the SQL statement.
+   * @param relation_name The name of the relation to insert into.
+   * @param literal_values A list of literal values (in attribute-definition
+   *        order) to insert into the specified relation as a new tuple.
+   *        Becomes owned by this ParseStatementInsert.
+   **/
+  ParseStatementInsertTuple(const int line_number,
+                            const int column_number,
+                            const ParseString *relation_name,
+                            PtrList<ParseScalarLiteral> *literal_values)
+      : ParseStatementInsert(line_number, column_number, relation_name),
+        literal_values_(literal_values) {
+  }
+
+  ~ParseStatementInsertTuple() override {
+  }
+
+  InsertType getInsertType() const override {
+    return InsertType::kTuple;
+  }
+
   /**
    * @brief Get the parsed literal attribute values to insert.
    *
@@ -580,7 +640,7 @@
       std::vector<std::string> *container_child_field_names,
       std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override {
     inline_field_names->push_back("relation_name");
-    inline_field_values->push_back(relation_name_->value());
+    inline_field_values->push_back(relation_name()->value());
 
     container_child_field_names->push_back("tuple");
     container_child_fields->emplace_back();
@@ -590,10 +650,86 @@
   }
 
  private:
-  std::unique_ptr<ParseString> relation_name_;
   std::unique_ptr<PtrList<ParseScalarLiteral> > literal_values_;
 
-  DISALLOW_COPY_AND_ASSIGN(ParseStatementInsert);
+  DISALLOW_COPY_AND_ASSIGN(ParseStatementInsertTuple);
+};
+
+/**
+ * @brief The parsed representation of an INSERT ... SELECT ... statement.
+ **/
+class ParseStatementInsertSelection : public ParseStatementInsert {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param line_number Line number of the first token of this node in the SQL statement.
+   * @param column_number Column number of the first token of this node in the SQL statement.
+   * @param relation_name The name of the relation to insert into.
+   * @param select_query The SELECT query for generating insertion tuples.
+   * @param with_clause The WITH clause of common table query expressions.
+   **/
+  ParseStatementInsertSelection(const int line_number,
+                                const int column_number,
+                                const ParseString *relation_name,
+                                ParseSelect *select_query,
+                                PtrVector<ParseSubqueryTableReference> *with_clause)
+      : ParseStatementInsert(line_number, column_number, relation_name),
+        select_query_(select_query),
+        with_clause_(with_clause) {
+  }
+
+  ~ParseStatementInsertSelection() override {
+  }
+
+  InsertType getInsertType() const override {
+    return InsertType::kSelection;
+  }
+
+  /**
+   * @return Gets the SELECT query.
+   */
+  const ParseSelect* select_query() const {
+    return select_query_.get();
+  }
+
+  /**
+   * @brief Gets the WITH table queries.
+   *
+   * @return The parsed WITH table list.
+   */
+  const PtrVector<ParseSubqueryTableReference>* with_clause() const {
+    return with_clause_.get();
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<const ParseTreeNode*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override {
+    inline_field_names->push_back("relation_name");
+    inline_field_values->push_back(relation_name()->value());
+
+    non_container_child_field_names->push_back("select_query");
+    non_container_child_fields->push_back(select_query_.get());
+
+    if (with_clause_ != nullptr && !with_clause_->empty()) {
+      container_child_field_names->push_back("with_clause");
+      container_child_fields->emplace_back();
+      for (const ParseSubqueryTableReference &common_subquery : *with_clause_) {
+        container_child_fields->back().push_back(&common_subquery);
+      }
+    }
+  }
+
+ private:
+  std::unique_ptr<ParseSelect> select_query_;
+  std::unique_ptr<PtrVector<ParseSubqueryTableReference>> with_clause_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParseStatementInsertSelection);
 };
 
 /**
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index 4c13c28..e5708a0 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -55,6 +55,7 @@
 class ParseLimit;
 class ParseOrderBy;
 class ParseOrderByItem;
+class ParsePartitionClause;
 class ParsePredicate;
 class ParseSample;
 class ParseScalarLiteral;
@@ -71,6 +72,8 @@
 class ParseStatementDelete;
 class ParseStatementDropTable;
 class ParseStatementInsert;
+class ParseStatementInsertTuple;
+class ParseStatementInsertSelection;
 class ParseStatementSelect;
 class ParseStatementQuit;
 class ParseStatementUpdate;
@@ -191,6 +194,7 @@
   "double"           return TOKEN_DOUBLE;
   "drop"             return TOKEN_DROP;
   "escape_strings"   return TOKEN_ESCAPE_STRINGS;
+  "extract"          return TOKEN_EXTRACT;
   "false"            return TOKEN_FALSE;
   "first"            return TOKEN_FIRST;
   "float"            return TOKEN_FLOAT;
@@ -198,6 +202,7 @@
   "from"             return TOKEN_FROM;
   "full"             return TOKEN_FULL;
   "group"            return TOKEN_GROUP;
+  "hash"             return TOKEN_HASH;
   "having"           return TOKEN_HAVING;
   "index"            return TOKEN_INDEX;
   "inner"            return TOKEN_INNER;
@@ -222,9 +227,12 @@
   "or"               return TOKEN_OR;
   "order"            return TOKEN_ORDER;
   "outer"            return TOKEN_OUTER;
+  "partition"        return TOKEN_PARTITION;
+  "partitions"       return TOKEN_PARTITIONS;
   "percent"          return TOKEN_PERCENT;
   "primary"          return TOKEN_PRIMARY;
   "quit"             return TOKEN_QUIT;
+  "range"            return TOKEN_RANGE;
   "real"             return TOKEN_REAL;
   "references"       return TOKEN_REFERENCES;
   "regexp"           return TOKEN_REGEXP;
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index eada5e1..0673ae3 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -66,6 +66,7 @@
 #include <string>
 #include <utility>
 
+#include "catalog/PartitionSchemeHeader.hpp"
 #include "parser/ParseAssignment.hpp"
 #include "parser/ParseAttributeDefinition.hpp"
 #include "parser/ParseBasicExpressions.hpp"
@@ -78,6 +79,7 @@
 #include "parser/ParseLimit.hpp"
 #include "parser/ParseLiteralValue.hpp"
 #include "parser/ParseOrderBy.hpp"
+#include "parser/ParsePartitionClause.hpp"
 #include "parser/ParsePredicate.hpp"
 #include "parser/ParserUtil.hpp"
 #include "parser/ParseSample.hpp"
@@ -165,6 +167,7 @@
   quickstep::ParseStatementDelete *delete_statement_;
   quickstep::ParseStatementCopyFrom *copy_from_statement_;
   quickstep::ParseStatementCreateTable *create_table_statement_;
+  quickstep::ParsePartitionClause *partition_clause_;
   quickstep::ParseBlockProperties *block_properties_;
   quickstep::ParseStatementDropTable *drop_table_statement_;
   quickstep::ParseStatementQuit *quit_statement_;
@@ -248,6 +251,7 @@
 %token TOKEN_DOUBLE;
 %token TOKEN_DROP;
 %token TOKEN_ESCAPE_STRINGS;
+%token TOKEN_EXTRACT;
 %token TOKEN_FALSE;
 %token TOKEN_FIRST;
 %token TOKEN_FLOAT;
@@ -255,6 +259,7 @@
 %token TOKEN_FROM;
 %token TOKEN_FULL;
 %token TOKEN_GROUP;
+%token TOKEN_HASH;
 %token TOKEN_HAVING;
 %token TOKEN_INDEX;
 %token TOKEN_INNER;
@@ -276,9 +281,12 @@
 %token TOKEN_OR;
 %token TOKEN_ORDER;
 %token TOKEN_OUTER;
+%token TOKEN_PARTITION;
+%token TOKEN_PARTITIONS;
 %token TOKEN_PERCENT;
 %token TOKEN_PRIMARY;
 %token TOKEN_QUIT;
+%token TOKEN_RANGE;
 %token TOKEN_REAL;
 %token TOKEN_REFERENCES;
 %token TOKEN_REGEXP;
@@ -306,6 +314,7 @@
 %type <string_value_>
   any_name
   index_type
+  partition_type
 
 %type <boolean_value_>
   boolean_value
@@ -321,6 +330,7 @@
   unary_expression
   multiply_expression
   add_expression
+  extract_function
 
 %type <attribute_>
   attribute_ref
@@ -430,6 +440,9 @@
 %type <block_properties_>
   opt_block_properties
 
+%type <partition_clause_>
+  opt_partition_clause
+
 %type <statement_>
   create_index_statement
 
@@ -485,7 +498,7 @@
   opt_sample_clause
 
 %type <with_list_>
-  opt_with_clause
+  with_clause
   with_list
 
 %type <with_list_element_>
@@ -616,8 +629,8 @@
   };
 
 create_table_statement:
-  TOKEN_CREATE TOKEN_TABLE any_name '(' column_def_commalist ')' opt_table_constraint_def_commalist opt_block_properties {
-    $$ = new quickstep::ParseStatementCreateTable(@1.first_line, @1.first_column, $3, $5, $8);
+  TOKEN_CREATE TOKEN_TABLE any_name '(' column_def_commalist ')' opt_table_constraint_def_commalist opt_block_properties opt_partition_clause {
+    $$ = new quickstep::ParseStatementCreateTable(@1.first_line, @1.first_column, $3, $5, $8, $9);
   };
 
 create_index_statement:
@@ -861,6 +874,38 @@
     $$ = new quickstep::ParseBlockProperties(@2.first_line, @2.first_column, $4);
   }
 
+opt_partition_clause:
+  {
+    $$ = nullptr;
+  }
+  | TOKEN_PARTITION TOKEN_BY partition_type '(' name_commalist ')' TOKEN_PARTITIONS TOKEN_UNSIGNED_NUMVAL {
+    if ($8->float_like()) {
+      delete $8;
+      $$ = NULL;
+      quickstep_yyerror(&@8, yyscanner, NULL, "NUMBER OF PARTITIONS must be an integer");
+      YYERROR;
+    } else {
+      if ($8->long_value() <= 0 || $8->long_value() > 64) {
+        delete $8;
+        $$ = NULL;
+        quickstep_yyerror(&@8, yyscanner, NULL, "NUMBER OF PARITIONS must be between 1 and 64");
+        YYERROR;
+      } else {
+        $$ = new quickstep::ParsePartitionClause(@1.first_line, @1.first_column, $3, $5, $8);
+      }
+    }
+  }
+
+partition_type:
+  TOKEN_HASH {
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column,
+           std::to_string(quickstep::PartitionSchemeHeader::PartitionType::kHash));
+  }
+  | TOKEN_RANGE{
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column,
+           std::to_string(quickstep::PartitionSchemeHeader::PartitionType::kRange));
+  };
+
 key_value_list:
   key_value {
     $$ = new quickstep::PtrList<quickstep::ParseKeyValue>();
@@ -937,8 +982,15 @@
     YYERROR;
   }
   | TOKEN_INSERT TOKEN_INTO any_name TOKEN_VALUES '(' literal_value_commalist ')' {
-    $$ = new quickstep::ParseStatementInsert(@1.first_line, @1.first_column, $3, $6);
-  };
+    $$ = new quickstep::ParseStatementInsertTuple(@1.first_line, @1.first_column, $3, $6);
+  }
+  | TOKEN_INSERT TOKEN_INTO any_name select_query {
+    $$ = new quickstep::ParseStatementInsertSelection(@1.first_line, @2.first_column, $3, $4, nullptr);
+  }
+  | with_clause TOKEN_INSERT TOKEN_INTO any_name select_query {
+    $$ = new quickstep::ParseStatementInsertSelection(@1.first_line, @2.first_column, $4, $5, $1);
+  }
+  ;
 
 copy_from_statement:
   TOKEN_COPY any_name TOKEN_FROM TOKEN_STRING_SINGLE_QUOTED opt_copy_from_params {
@@ -998,15 +1050,15 @@
 
 /* Select Queries */
 select_statement:
-  opt_with_clause select_query {
+  select_query {
+    $$ = new quickstep::ParseStatementSelect(@1.first_line, @1.first_column, $1, nullptr);
+  }
+  | with_clause select_query {
     $$ = new quickstep::ParseStatementSelect(@1.first_line, @1.first_column, $2, $1);
   };
 
-opt_with_clause:
-  {
-    $$ = nullptr;
-  }
-  | TOKEN_WITH with_list {
+with_clause:
+  TOKEN_WITH with_list {
     $$ = $2;
   }
 
@@ -1408,6 +1460,9 @@
   | function_call {
     $$ = $1;
   }
+  | extract_function {
+    $$ = $1;
+  }
   | '(' add_expression ')' {
     $$ = $2;
   };
@@ -1415,14 +1470,22 @@
 function_call:
   any_name '(' ')' {
     $$ = new quickstep::ParseFunctionCall(
-        @1.first_line, @1.first_column, $1, new quickstep::PtrList<quickstep::ParseExpression>());
+        @1.first_line, @1.first_column, false, $1, new quickstep::PtrList<quickstep::ParseExpression>());
   }
   | any_name '(' '*' ')' {
     $$ = new quickstep::ParseFunctionCall(
         @1.first_line, @1.first_column, $1, new quickstep::ParseStar(@3.first_line, @3.first_column));
   }
   | any_name '(' expression_list ')' {
-    $$ = new quickstep::ParseFunctionCall(@1.first_line, @1.first_column, $1, $3);
+    $$ = new quickstep::ParseFunctionCall(@1.first_line, @1.first_column, false, $1, $3);
+  };
+  | any_name '(' TOKEN_DISTINCT expression_list ')' {
+    $$ = new quickstep::ParseFunctionCall(@1.first_line, @1.first_column, true, $1, $4);
+  };
+
+extract_function:
+  TOKEN_EXTRACT '(' any_name TOKEN_FROM add_expression ')' {
+    $$ = new quickstep::ParseExtractFunction(@1.first_line, @1.first_column, $3, $5);
   };
 
 expression_list:
diff --git a/parser/SqlParserWrapper.cpp b/parser/SqlParserWrapper.cpp
index 41abdec..e590291 100644
--- a/parser/SqlParserWrapper.cpp
+++ b/parser/SqlParserWrapper.cpp
@@ -43,6 +43,7 @@
 class ParseKeyStringList;
 class ParseKeyLiteralValue;
 class ParseLiteralValue;
+class ParsePartitionClause;
 class ParsePredicate;
 class ParseSample;
 class ParseScalarLiteral;
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index d8c6382..454ed0f 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -381,8 +381,8 @@
 	*yy_cp = '\0'; \
 	yyg->yy_c_buf_p = yy_cp;
 
-#define YY_NUM_RULES 130
-#define YY_END_OF_BUFFER 131
+#define YY_NUM_RULES 135
+#define YY_END_OF_BUFFER 136
 /* This struct is not used in this scanner,
    but its presence is necessary. */
 struct yy_trans_info
@@ -390,61 +390,64 @@
 	flex_int32_t yy_verify;
 	flex_int32_t yy_nxt;
 	};
-static yyconst flex_int16_t yy_accept[483] =
+static yyconst flex_int16_t yy_accept[504] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
-        0,    0,  131,    2,    2,  129,  129,  128,  127,  129,
-      106,  102,  105,  102,  102,  125,   98,   95,   99,  124,
-      124,  124,  124,  124,  124,  124,  124,  124,  124,  124,
-      124,  124,  124,  124,  124,  124,  124,  124,  124,  124,
-      124,  124,  124,  103,    4,    5,    5,    3,  121,  121,
-      118,  122,  122,  116,  123,  123,  120,    1,  128,   96,
-      126,  125,  125,  125,    0,  100,   97,  101,  124,  124,
-      124,  124,   10,  124,  124,  124,   20,  124,  124,  124,
-      124,  124,  124,  124,  124,  124,  104,  124,  124,  124,
+        0,    0,  136,    2,    2,  134,  134,  133,  132,  134,
+      111,  107,  110,  107,  107,  130,  103,  100,  104,  129,
+      129,  129,  129,  129,  129,  129,  129,  129,  129,  129,
+      129,  129,  129,  129,  129,  129,  129,  129,  129,  129,
+      129,  129,  129,  108,    4,    5,    5,    3,  126,  126,
+      123,  127,  127,  121,  128,  128,  125,    1,  133,  101,
+      131,  130,  130,  130,    0,  105,  102,  106,  129,  129,
+      129,  129,   10,  129,  129,  129,   20,  129,  129,  129,
+      129,  129,  129,  129,  129,  129,  109,  129,  129,  129,
 
-      124,  124,  124,  124,  124,  124,  124,   55,  124,  124,
-      124,  124,  124,  124,  124,  124,  124,   67,   68,  124,
-      124,  124,  124,  124,  124,  124,  124,  124,  124,  124,
-      124,  124,  124,  124,  124,  124,  124,  124,  124,    4,
-        5,    3,  121,  117,  122,  115,  115,  107,  109,  110,
-      111,  112,  113,  114,  115,  123,  119,  126,  125,    0,
-      125,    6,    7,  124,    9,   11,  124,  124,   15,  124,
-      124,  124,  124,  124,  124,  124,  124,  124,  124,  124,
-      124,  124,  124,  124,  124,  124,  124,  124,  124,  124,
-      124,  124,  124,  124,  124,  124,  124,   51,  124,   57,
+      129,  129,  129,  129,  129,  129,  129,  129,   57,  129,
+      129,  129,  129,  129,  129,  129,  129,  129,   69,   70,
+      129,  129,  129,  129,  129,  129,  129,  129,  129,  129,
+      129,  129,  129,  129,  129,  129,  129,  129,  129,  129,
+      129,  129,    4,    5,    3,  126,  122,  127,  120,  120,
+      112,  114,  115,  116,  117,  118,  119,  120,  128,  124,
+      131,  130,    0,  130,    6,    7,  129,    9,   11,  129,
+      129,   15,  129,  129,  129,  129,  129,  129,  129,  129,
+      129,  129,  129,  129,  129,  129,  129,  129,  129,  129,
+      129,  129,  129,  129,  129,  129,  129,  129,  129,  129,
 
-      124,  124,  124,  124,  124,   63,  124,   66,  124,  124,
-      124,  124,  124,  124,  124,  124,  124,  124,  124,   80,
-      124,  124,  124,  124,  124,  124,  124,  124,  124,  124,
-      124,  124,  124,  107,  109,  108,  124,  124,  124,  124,
-      124,  124,   21,  124,  124,  124,   26,  124,  124,   28,
-      124,  124,  124,  124,   34,  124,  124,   38,  124,  124,
-      124,  124,  124,   44,   45,  124,  124,  124,  124,  124,
-      124,   54,   56,   58,   59,   60,  124,   62,   64,  124,
-      124,  124,  124,   73,   74,  124,  124,  124,  124,  124,
-      124,  124,   83,   85,  124,  124,  124,  124,  124,  124,
+      129,  129,   53,  129,   59,  129,  129,  129,  129,  129,
+       65,  129,   68,  129,  129,  129,  129,  129,  129,  129,
+      129,  129,  129,  129,  129,  129,   85,  129,  129,  129,
+      129,  129,  129,  129,  129,  129,  129,  129,  129,  129,
+      112,  114,  113,  129,  129,  129,  129,  129,  129,   21,
+      129,  129,  129,   26,  129,  129,   28,  129,  129,  129,
+      129,   34,  129,  129,   38,  129,  129,  129,  129,  129,
+      129,   45,   46,  129,   48,  129,  129,  129,  129,  129,
+       56,   58,   60,   61,   62,  129,   64,   66,  129,  129,
+      129,  129,  129,   77,  129,   79,  129,  129,  129,  129,
 
-      124,   93,  124,  107,  108,    8,  124,  124,  124,  124,
-      124,  124,   23,  124,  124,  124,  124,  124,  124,  124,
-      124,  124,  124,  124,  124,  124,   40,   41,   42,  124,
-       46,  124,   48,   49,  124,  124,  124,   61,   65,   69,
-       70,  124,  124,  124,  124,   77,  124,  124,  124,   82,
-      124,  124,  124,  124,   89,  124,  124,   92,  124,  124,
-      124,   14,  124,  124,  124,  124,   24,  124,   27,  124,
-      124,  124,  124,   32,  124,  124,  124,   37,  124,  124,
-       47,   50,  124,  124,  124,  124,  124,   76,  124,   79,
-      124,  124,  124,   87,   88,   90,  124,  124,  124,   13,
+      129,  129,  129,   88,   90,  129,  129,  129,  129,  129,
+      129,  129,   98,  129,  112,  113,    8,  129,  129,  129,
+      129,  129,  129,   23,  129,  129,  129,  129,  129,  129,
+      129,  129,  129,  129,  129,  129,  129,  129,   41,   42,
+       43,  129,   47,  129,   50,   51,  129,  129,  129,   63,
+       67,   71,   72,  129,  129,  129,   78,  129,  129,   82,
+      129,  129,  129,   87,  129,  129,  129,  129,   94,  129,
+      129,   97,  129,  129,  129,   14,  129,  129,  129,  129,
+       24,  129,   27,  129,  129,  129,  129,   32,  129,  129,
+      129,   37,  129,  129,  129,   49,   52,  129,  129,  129,
 
-      124,  124,  124,  124,  124,   19,  124,   30,   31,  124,
-      124,  124,  124,   43,   52,  124,   71,   72,  124,  124,
-      124,  124,  124,   91,  124,  124,  124,  124,  124,  124,
-      124,   29,  124,  124,   36,  124,   53,  124,  124,   81,
-      124,  124,  124,   12,  124,  124,  124,   22,  124,   33,
-      124,  124,  124,  124,   84,  124,   94,  124,  124,  124,
-       25,   35,  124,   75,  124,  124,  124,   17,   18,  124,
-      124,   86,  124,  124,  124,  124,  124,   78,  124,   39,
-       16,    0
+      129,  129,  129,   81,  129,   84,  129,  129,  129,   92,
+       93,   95,  129,  129,  129,   13,  129,  129,  129,  129,
+      129,   19,  129,   30,   31,  129,  129,  129,  129,   40,
+       44,   54,  129,  129,   75,   76,  129,  129,  129,  129,
+      129,   96,  129,  129,  129,  129,  129,  129,  129,   29,
+      129,  129,   36,  129,   55,  129,  129,  129,   86,  129,
+      129,  129,   12,  129,  129,  129,   22,  129,   33,  129,
+      129,   73,  129,  129,   89,  129,   99,  129,  129,  129,
+       25,   35,  129,   74,   80,  129,  129,  129,   17,   18,
+      129,  129,   91,  129,  129,  129,  129,  129,   83,  129,
+
+       39,   16,    0
     } ;
 
 static yyconst YY_CHAR yy_ec[256] =
@@ -491,131 +494,135 @@
         8
     } ;
 
-static yyconst flex_uint16_t yy_base[498] =
+static yyconst flex_uint16_t yy_base[519] =
     {   0,
         0,    1,   46,    0,  117,  163,    2,    3,  128,  132,
-        6,   10,  198, 1090, 1090,    0, 1090,   13, 1090,  181,
-     1090, 1090, 1090,  183,    6,  130,    4, 1090,  142,  124,
-      161,  160,  204,  121,  251,   92,  112,  163,  114,  125,
-      209,    0,  173,  214,  169,  150,  221,  216,  253,  254,
-      182,  267,  191, 1090,  152,    4,   19,    0,    0,    0,
-      143,    0,    0,  319,    0,    0,  144,    0,   22, 1090,
-        0,  282,  289,  313,   18, 1090, 1090, 1090,    0,  194,
-      270,  263,  285,  273,  306,  297,    0,  319,  315,  323,
-      328,  312,  332,  315,  315,  325, 1090,  342,  334,  330,
+        6,   10,  198, 1136, 1136,    0, 1136,   13, 1136,  181,
+     1136, 1136, 1136,  183,    6,  130,    4, 1136,  172,  124,
+      161,  160,  204,  150,  251,   92,  110,  163,   98,  124,
+      209,    0,  173,  214,  182,  109,  253,  231,  258,  261,
+      195,  218,  196, 1136,  152,    4,   19,    0,    0,    0,
+      143,    0,    0,  323,    0,    0,  144,    0,   22, 1136,
+        0,  289,  316,  319,   18, 1136, 1136, 1136,    0,  217,
+      258,  241,  268,  273,  296,  287,    0,  325,  322,  312,
+      318,  307,  345,  314,  318,  325, 1136,  339,  324,  340,
 
-      337,  352,  356,  364,  363,  357,  378,    0,  374,  359,
-      366,  380,  376,  374,  370,  379,  387,    0,  390,  375,
-      378,  392,  393,  403,  396,  383,  416,  428,  432,  424,
-      419,  425,  433,  439,  435,  433,  441,  427,  447,  142,
-       29,    0,    0, 1090,    0, 1090, 1090,   22,   24, 1090,
-     1090, 1090, 1090, 1090,    0,    0, 1090,    0,  454,   26,
-       28,    0,    0,  447,    0,  448,  431,  446,    0,  453,
-      442,  458,  441,  445,  442,  477,  459,  481,  478,  491,
-      489,  492,  479,  498,  485,  501,  484,  485,  504,  501,
-      496,  498,  490,  503,  508,  509,  512,  521,  504,    0,
+      342,  346,  356,  360,  366,  364,  363,  380,    0,  374,
+      362,  369,  385,  384,  378,  376,  386,  395,    0,  398,
+      385,  388,  391,  401,  406,  404,  435,  412,  399,  420,
+      432,  435,  430,  424,  430,  440,  446,  443,  441,  449,
+      435,  455,  142,   29,    0,    0, 1136,    0, 1136, 1136,
+       22,   24, 1136, 1136, 1136, 1136, 1136,    0,    0, 1136,
+        0,  462,   26,   28,    0,    0,  455,    0,  458,  441,
+      459,    0,  483,  455,  473,  456,  461,  462,  489,  471,
+      489,  486,  498,  495,  498,  483,  505,  492,  509,  493,
+      493,  494,  513,  512,  505,  509,  501,  517,  517,  526,
 
-      499,  500,  517,  516,  530,    0,  532,    0,  540,  545,
-      549,  540,  536,  545,  553,  554,  552,   93,  556,    0,
-      550,  551,  559,  562,  556,  552,  569,  557,  551,  572,
-      558,  569,  560,   30,  125,    0,  562,  569,  579,  580,
-      584,  589,  602,  597,  597,  591,    0,  594,  597,  596,
-      604,  597,  599,  607,  616,  613,  613,    0,  610,  622,
-      608,  609,  621,    0,    0,  617,  620,  611,  618,  620,
-      634,    0,    0,    0,    0,    0,  622,    0,  634,  642,
-      643,  661,  667,    0,    0,  651,  648,  653,  670,  672,
-      664,  672,  659,    0,  674,  659,  663,  677,  680,  678,
+      529,  530,  533,  530,    0,  525,  529,  547,  544,  549,
+        0,  545,    0,  553,  554,  541,  559,  552,  546,  561,
+      557,  565,  566,  564,   93,  570,    0,  564,  567,  575,
+      578,  572,  572,  591,  579,  575,  599,  585,  598,  589,
+       30,  125,    0,  592,  597,  609,  601,  605,  604,  618,
+      609,  610,  604,    0,  606,  609,  608,  616,  609,  613,
+      621,  632,  629,  629,    0,  626,  646,  645,  631,  634,
+      651,    0,    0,  645,    0,  650,  641,  650,  651,  666,
+        0,    0,    0,    0,    0,  652,    0,  655,  657,  659,
+      669,  676,  681,    0,  680,    0,  668,  663,  668,  687,
 
-      682,    0,  675,  136, 1090,    0,  687,  687,  673,  680,
-      688,  697,    0,  696,  693,  713,  714,  715,  725,  715,
-      725,  722,  718,  719,  729,  730,    0,    0,    0,  729,
-        0,  730,    0,    0,  718,  736,  720,    0,    0,    0,
-        0,  729,  726,  740,  730,    0,  744,  730,  742,    0,
-      732,  735,  752,  753,    0,  749,  768,    0,  761,  768,
-      768,    0,  766,  784,  779,  769,    0,  789,    0,  786,
-      779,  781,  774,    0,  775,  792,  796,    0,   82,  786,
-        0,    0,  783,  801,  783,  779,  793,    0,  796,    0,
-      795,  809,  811,    0,    0,    0,  797,  802,  812,    0,
+      689,  683,  691,  680,    0,  695,  684,  688,  702,  707,
+      710,  714,    0,  709,  136, 1136,    0,  719,  721,  707,
+      714,  725,  729,    0,  721,  718,  734,  735,  733,  742,
+      732,  740,  737,  735,  736,  748,  749,  754,    0,    0,
+        0,  751,    0,  756,    0,    0,  746,  762,  748,    0,
+        0,    0,    0,  756,  763,  762,    0,  776,  768,    0,
+      780,  768,  780,    0,  770,  772,  788,  789,    0,  778,
+      797,    0,  785,  792,  788,    0,  785,  803,  798,  788,
+        0,  810,    0,  807,  802,  804,  801,    0,  804,  821,
+      825,    0,   83,  814,  821,    0,    0,  820,  838,  833,
 
-      812,  821,  823,  835,  833,    0,  838,    0,    0,  841,
-      838,  828,  830,    0,    0,  838,    0,    0,  848,  843,
-      833,  841,  844,    0,  838,  852,  844,  845,  842,  847,
-      852,    0,  849,  854,    0,  850,    0,  868,  861,    0,
-      868,  869,  884,    0,  888,  886,  895,    0,  881,    0,
-      897,  887,  887,  898,    0,  896,    0,  891,  905,  893,
-        0,    0,  903,    0,  895,  911,  897,    0,    0,  904,
-      914,    0,  911,  916,  906,  920,  907,    0,  909,    0,
-        0, 1090,  974,  984,  994, 1004, 1014, 1018, 1021, 1027,
-     1035, 1045, 1055, 1065, 1075, 1080, 1082
+      823,  821,  833,    0,  836,    0,  835,  850,  851,    0,
+        0,    0,  837,  842,  844,    0,  844,  847,  849,  857,
+      856,    0,  861,    0,    0,  864,  861,  853,  855,    0,
+        0,    0,  867,  867,    0,    0,  880,  877,  872,  880,
+      883,    0,  877,  893,  885,  888,  885,  888,  893,    0,
+      891,  896,    0,  893,    0,  900,  911,  904,    0,  902,
+      903,  912,    0,  918,  912,  922,    0,  908,    0,  924,
+      914,  918,  921,  932,    0,  932,    0,  932,  946,  936,
+        0,    0,  946,    0,    0,  938,  954,  942,    0,    0,
+      949,  959,    0,  956,  960,  950,  966,  953,    0,  955,
 
+        0,    0, 1136, 1020, 1030, 1040, 1050, 1060, 1064, 1067,
+     1073, 1081, 1091, 1101, 1111, 1121, 1126, 1128
     } ;
 
-static yyconst flex_int16_t yy_def[498] =
+static yyconst flex_int16_t yy_def[519] =
     {   0,
-      483,  483,  482,    3,  484,  484,  485,  485,  486,  486,
-      487,  487,  482,  482,  482,  488,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  482,  482,  482,  482,  490,  491,  491,
-      482,  492,  492,  493,  494,  494,  482,  488,  482,  482,
-      495,  482,  482,  482,  482,  482,  482,  482,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  482,  489,  489,  489,
+      504,  504,  503,    3,  505,  505,  506,  506,  507,  507,
+      508,  508,  503,  503,  503,  509,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  503,  503,  503,  503,  511,  512,  512,
+      503,  513,  513,  514,  515,  515,  503,  509,  503,  503,
+      516,  503,  503,  503,  503,  503,  503,  503,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  503,  510,  510,  510,
 
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  482,
-      482,  490,  491,  482,  492,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  496,  494,  482,  495,  482,  482,
-      482,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  503,  503,  511,  512,  503,  513,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  517,  515,  503,
+      516,  503,  503,  503,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
 
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  482,  482,  497,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      503,  503,  518,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
 
-      489,  489,  489,  482,  482,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  503,  503,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
 
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,  489,  489,  489,  489,  489,  489,  489,  489,  489,
-      489,    0,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
+      510,  510,  510,  510,  510,  510,  510,  510,  510,  510,
 
+      510,  510,    0,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503
     } ;
 
-static yyconst flex_uint16_t yy_nxt[1162] =
+static yyconst flex_uint16_t yy_nxt[1208] =
     {   0,
-      482,  482,   15,   15,   60,   60,  141,  141,   66,   61,
-       61,   67,   66,  482,   69,   67,   69,   72,   72,   76,
-       77,  141,  141,   69,  482,   69,  160,  160,  482,  161,
-      161,  141,  141,  234,  235,  235,  235,  161,  161,  161,
-      161,  304,  235,  482,   16,   16,   17,   18,   19,   18,
+      503,  503,   15,   15,   60,   60,  144,  144,   66,   61,
+       61,   67,   66,  503,   69,   67,   69,   72,   72,   76,
+       77,  144,  144,   69,  503,   69,  163,  163,  503,  164,
+      164,  144,  144,  241,  242,  242,  242,  164,  164,  164,
+      164,  315,  242,  503,   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,
@@ -624,130 +631,135 @@
 
        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,  105,  413,   97,  106,
-       63,   17,   17,   17,   63,   61,  235,  235,  289,   61,
-       73,   74,   74,  140,   80,  109,  110,  235,  235,  157,
-      144,   75,   81,  140,   82,  105,   98,   78,  106,   83,
+       57,   17,   17,   17,   17,   17,  106,  107,  429,  110,
+       63,   17,   17,   17,   63,   61,  242,  242,  300,   61,
+       73,   74,   74,  143,   80,  111,  125,  242,  242,  160,
+      147,   75,   81,  143,   82,  106,  107,   97,  110,   83,
        17,   17,   17,   17,   55,   56,   57,   17,   17,   17,
-       17,   17,   64,   80,  109,  110,   64,   17,   17,   17,
-       75,   81,   84,   82,   88,   98,   85,  123,   83,   86,
-      121,   89,   71,  107,   90,   91,   70,  482,  108,  136,
+       17,   17,   64,   80,  111,  125,   64,   17,   17,   17,
+       75,   81,   84,   82,   88,   98,   85,   78,   83,   86,
+       99,   89,   71,  108,   90,   91,   70,  503,  109,  122,
 
-      482,  482,   87,  122,  115,  482,   17,   17,   17,  482,
-      116,   84,  139,   88,  162,   85,  123,  482,   86,  121,
-       89,   92,  107,   90,   91,   93,  111,  108,  136,   94,
-      112,   87,  122,  115,  113,   95,  117,  127,   96,  116,
-      114,  139,  124,  162,  118,  128,  125,  482,  119,  482,
-       92,  120,  126,  482,   93,  111,  482,  482,   94,  112,
-      482,  482,  482,  113,   95,  117,  127,   96,   99,  114,
-      129,  124,  482,  118,  128,  125,  100,  119,  130,  101,
-      120,  126,  102,  165,  133,  103,  134,  131,  104,  135,
-      132,  137,  138,   72,   72,  482,  482,   99,  163,  129,
+      503,  503,   87,  123,  116,  503,   17,   17,   17,  503,
+      117,   84,  139,   88,   98,   85,  124,  142,   86,   99,
+       89,   92,  108,   90,   91,   93,  112,  109,  122,   94,
+      113,   87,  123,  116,  114,   95,  118,  165,   96,  117,
+      115,  139,  140,  141,  119,  124,  142,  503,  120,  503,
+       92,  121,  130,  503,   93,  112,  503,  503,   94,  113,
+      131,  168,  503,  114,   95,  118,  165,   96,  100,  115,
+      126,  140,  141,  119,  127,  132,  101,  120,  128,  102,
+      121,  130,  103,  133,  129,  104,  166,  169,  105,  131,
+      168,  136,  134,  137,  167,  135,  138,  100,  503,  126,
 
-      159,  159,  482,   75,  166,  100,  164,  130,  101,  167,
-       75,  102,  165,  133,  103,  134,  131,  104,  135,  132,
-      137,  138,  147,   73,   74,   74,  482,  163,  170,  168,
-      148,  149,   75,  166,   75,  164,  171,  150,  167,   75,
-      172,  151,  169,  173,  176,  174,  177,  175,  178,  152,
-      183,  179,  184,  153,  180,  154,  185,  170,  168,  155,
-      181,  186,  187,   75,  188,  171,  150,  182,  189,  172,
-      151,  169,  173,  176,  174,  177,  175,  178,  152,  183,
-      179,  184,  153,  180,  154,  185,  190,  191,  155,  181,
-      186,  187,  192,  188,  193,  194,  182,  189,  195,  199,
+       72,   72,  503,  127,  132,  101,  503,  128,  102,  170,
+       75,  103,  133,  129,  104,  166,  169,  105,  173,  171,
+      136,  134,  137,  167,  135,  138,  150,  162,  162,   73,
+       74,   74,  172,  179,  151,  152,  180,   75,  170,   75,
+       75,  153,  174,  181,  503,  154,  175,  173,  171,  186,
+      176,  503,  177,  155,  178,  187,  188,  156,  189,  157,
+      190,  172,  179,  158,  182,  180,   75,  183,  191,   75,
+      153,  174,  181,  184,  154,  175,  192,  193,  186,  176,
+      185,  177,  155,  178,  187,  188,  156,  189,  157,  190,
+      194,  195,  158,  182,  196,  197,  183,  191,  198,  204,
 
-      200,  201,  202,  203,  205,  204,  206,  207,  196,  208,
-      209,  210,  211,  197,  198,  190,  191,  212,  213,  217,
-      214,  192,  218,  193,  194,  215,  216,  195,  199,  200,
-      201,  202,  203,  205,  204,  206,  207,  196,  208,  209,
-      210,  211,  197,  198,  219,  221,  212,  213,  217,  214,
-      222,  218,  220,  223,  215,  216,  224,  225,  226,  227,
-      228,  229,  231,  232,  233,  159,  159,  230,  237,  238,
-      239,  240,  241,  219,  221,   75,  243,  244,  245,  222,
-      246,  220,  223,  247,  242,  224,  225,  226,  227,  228,
-      229,  231,  232,  233,  248,  249,  230,  237,  238,  239,
+      200,  199,  184,  205,  206,  192,  193,  207,  210,  185,
+      201,  208,  211,  209,  212,  202,  203,  213,  214,  194,
+      195,  215,  216,  196,  197,  217,  218,  198,  204,  200,
+      199,  219,  205,  206,  220,  224,  207,  210,  225,  201,
+      208,  211,  209,  212,  202,  203,  213,  214,  226,  228,
+      215,  216,  221,  229,  217,  218,  227,  222,  223,  230,
+      219,  231,  232,  220,  224,  233,  234,  225,  235,  236,
+      238,  239,  240,  162,  162,  237,  244,  226,  228,  245,
+      246,  221,  229,   75,  247,  227,  222,  223,  230,  250,
+      231,  232,  251,  252,  233,  234,  253,  235,  236,  238,
 
-      240,  241,  250,  251,   75,  243,  244,  245,  252,  246,
-      253,  255,  247,  242,  254,  256,  257,  258,  259,  260,
-      261,  262,  263,  248,  249,  264,  265,  266,  267,  268,
-      269,  250,  251,  270,  273,  274,  275,  252,  276,  253,
-      255,  277,  271,  254,  256,  257,  258,  259,  260,  261,
-      262,  263,  272,  278,  264,  265,  266,  267,  268,  269,
-      279,  280,  270,  273,  274,  275,  281,  276,  282,  283,
-      277,  271,  284,  285,  286,  287,  288,  290,  291,  292,
-      293,  272,  278,  294,  295,  296,  297,  298,  299,  279,
-      280,  300,  301,  302,  303,  281,  306,  282,  283,  307,
+      239,  240,  248,  254,  237,  244,  255,  256,  245,  246,
+      257,  258,   75,  247,  249,  259,  260,  262,  250,  263,
+      261,  251,  252,  264,  265,  253,  266,  267,  268,  269,
+      270,  248,  254,  271,  272,  255,  256,  273,  274,  257,
+      258,  275,  276,  249,  259,  260,  262,  277,  263,  261,
+      278,  279,  264,  265,  280,  266,  267,  268,  269,  270,
+      282,  283,  271,  272,  281,  284,  273,  274,  285,  286,
+      275,  276,  287,  288,  289,  290,  277,  291,  292,  278,
+      279,  293,  294,  280,  295,  296,  297,  298,  299,  282,
+      283,  301,  302,  281,  284,  303,  304,  285,  286,  305,
 
-      308,  284,  285,  286,  287,  288,  290,  291,  292,  293,
-      309,  310,  294,  295,  296,  297,  298,  299,  311,  312,
-      300,  301,  302,  303,  313,  306,  314,  315,  307,  308,
-      316,  317,  318,  319,  320,  321,  322,  323,  324,  309,
-      310,  325,  326,  327,  328,  329,  330,  311,  312,  331,
-      332,  333,  334,  313,  335,  314,  315,  336,  338,  316,
-      317,  318,  319,  320,  321,  322,  323,  324,  337,  339,
-      325,  326,  327,  328,  329,  330,  340,  341,  331,  332,
-      333,  334,  342,  335,  343,  344,  336,  338,  345,  346,
-      347,  348,  349,  350,  351,  352,  353,  337,  339,  354,
+      306,  287,  288,  289,  290,  307,  291,  292,  308,  309,
+      293,  294,  310,  295,  296,  297,  298,  299,  311,  312,
+      301,  302,  313,  314,  303,  304,  317,  318,  305,  306,
+      319,  320,  321,  322,  307,  323,  324,  308,  309,  325,
+      326,  310,  327,  328,  329,  330,  331,  311,  312,  332,
+      333,  313,  314,  334,  335,  317,  318,  336,  337,  319,
+      320,  321,  322,  338,  323,  324,  339,  340,  325,  326,
+      341,  327,  328,  329,  330,  331,  342,  343,  332,  333,
+      344,  345,  334,  335,  346,  347,  336,  337,  350,  348,
+      351,  352,  338,  353,  354,  339,  340,  355,  356,  341,
 
-      355,  356,  357,  358,  359,  340,  341,  360,  361,  362,
-      365,  342,  363,  343,  344,  364,  366,  345,  346,  347,
-      348,  349,  350,  351,  352,  353,  367,  368,  354,  355,
-      356,  357,  358,  359,  369,  370,  360,  361,  362,  365,
-      371,  363,  372,  373,  364,  366,  374,  375,  376,  377,
-      378,  379,  380,  381,  382,  367,  368,  383,  384,  385,
-      386,  387,  388,  369,  370,  389,  390,  391,  392,  371,
-      393,  372,  373,  394,  395,  374,  375,  376,  377,  378,
-      379,  380,  381,  382,  396,  397,  383,  384,  385,  386,
-      387,  388,  398,  399,  389,  390,  391,  392,  400,  393,
+      349,  357,  358,  359,  360,  342,  343,  361,  362,  344,
+      345,  363,  364,  346,  347,  365,  366,  350,  348,  351,
+      352,  367,  353,  354,  368,  369,  355,  356,  370,  349,
+      357,  358,  359,  360,  371,  372,  361,  362,  373,  374,
+      363,  364,  375,  376,  365,  366,  377,  379,  380,  378,
+      367,  381,  382,  368,  369,  383,  384,  370,  385,  386,
+      387,  388,  389,  371,  372,  390,  391,  373,  374,  392,
+      393,  375,  376,  394,  395,  377,  379,  380,  378,  396,
+      381,  382,  397,  398,  383,  384,  399,  385,  386,  387,
+      388,  389,  400,  401,  390,  391,  402,  403,  392,  393,
 
-      401,  402,  394,  395,  403,  404,  405,  406,  407,  408,
-      409,  410,  411,  396,  397,  412,  414,  415,  416,  417,
-      418,  398,  399,  419,  420,  421,  422,  400,  423,  401,
-      402,  424,  425,  403,  404,  405,  406,  407,  408,  409,
-      410,  411,  426,  427,  412,  414,  415,  416,  417,  418,
-      428,  429,  419,  420,  421,  422,  430,  423,  431,  432,
-      424,  425,  433,  434,  435,  436,  437,  438,  439,  440,
-      441,  426,  427,  442,  443,  444,  445,  446,  447,  428,
-      429,  448,  449,  450,  451,  430,  452,  431,  432,  453,
-      454,  433,  434,  435,  436,  437,  438,  439,  440,  441,
+      404,  405,  394,  395,  406,  407,  408,  409,  396,  410,
+      411,  397,  398,  412,  413,  399,  414,  415,  416,  417,
+      418,  400,  401,  419,  420,  402,  403,  421,  422,  404,
+      405,  423,  424,  406,  407,  408,  409,  425,  410,  411,
+      426,  427,  412,  413,  428,  414,  415,  416,  417,  418,
+      430,  431,  419,  420,  432,  433,  421,  422,  434,  435,
+      423,  424,  436,  437,  438,  439,  425,  440,  441,  426,
+      427,  442,  443,  428,  444,  445,  446,  447,  448,  430,
+      431,  449,  450,  432,  433,  451,  452,  434,  435,  453,
+      454,  436,  437,  438,  439,  455,  440,  441,  456,  457,
 
-      455,  456,  442,  443,  444,  445,  446,  447,  457,  458,
-      448,  449,  450,  451,  459,  452,  460,  461,  453,  454,
-      462,  463,  464,  465,  466,  467,  468,  469,  470,  455,
-      456,  471,  472,  473,  474,  475,  476,  457,  458,  477,
-      478,  479,  480,  459,  481,  460,  461,  482,  482,  462,
-      463,  464,  465,  466,  467,  468,  469,  470,  482,  482,
-      471,  472,  473,  474,  475,  476,  482,  482,  477,  478,
-      479,  480,  482,  481,   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,
+      442,  443,  458,  444,  445,  446,  447,  448,  459,  460,
+      449,  450,  461,  462,  451,  452,  463,  464,  453,  454,
+      465,  466,  467,  468,  455,  469,  470,  456,  457,  471,
+      472,  458,  473,  474,  475,  476,  477,  459,  460,  478,
+      479,  461,  462,  480,  481,  463,  464,  482,  483,  465,
+      466,  467,  468,  484,  469,  470,  485,  486,  471,  472,
+      487,  473,  474,  475,  476,  477,  488,  489,  478,  479,
+      490,  491,  480,  481,  492,  493,  482,  483,  494,  495,
+      496,  497,  484,  498,  499,  485,  486,  500,  501,  487,
+      502,  503,  503,  503,  503,  488,  489,  503,  503,  490,
 
-       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,  482,
-       79,  142,  142,  142,  142,  143,  143,  143,  482,  143,
-      143,  143,  143,  143,  143,  145,  145,  145,  482,  145,
-      145,  145,  145,  482,  145,  146,  146,  146,  146,  146,
-      146,  146,  146,  146,  146,  156,  156,  482,  156,  156,
-      156,  156,  156,  156,  156,  158,  482,  158,  158,  158,
-      158,  158,  158,  158,  158,  236,  236,  305,  305,   13,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
+      491,  503,  503,  492,  493,  503,  503,  494,  495,  496,
+      497,  503,  498,  499,  503,  503,  500,  501,  503,  502,
+       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,  503,   79,  145,  145,  145,
+      145,  146,  146,  146,  503,  146,  146,  146,  146,  146,
+      146,  148,  148,  148,  503,  148,  148,  148,  148,  503,
 
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482
+      148,  149,  149,  149,  149,  149,  149,  149,  149,  149,
+      149,  159,  159,  503,  159,  159,  159,  159,  159,  159,
+      159,  161,  503,  161,  161,  161,  161,  161,  161,  161,
+      161,  243,  243,  316,  316,   13,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+
+      503,  503,  503,  503,  503,  503,  503
     } ;
 
-static yyconst flex_int16_t yy_chk[1162] =
+static yyconst flex_int16_t yy_chk[1208] =
     {   0,
         0,    0,    1,    2,    7,    8,   56,   56,   11,    7,
         8,   11,   12,    0,   18,   12,   18,   25,   25,   27,
        27,   57,   57,   69,    0,   69,   75,   75,    0,   75,
-       75,  141,  141,  148,  148,  149,  149,  160,  160,  161,
-      161,  234,  234,    0,    1,    2,    3,    3,    3,    3,
+       75,  144,  144,  151,  151,  152,  152,  163,  163,  164,
+      164,  241,  241,    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,
@@ -756,133 +768,138 @@
 
         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,  379,   34,   37,
-        9,    5,    5,    5,   10,    9,  235,  235,  218,   10,
-       26,   26,   26,  140,   30,   39,   40,  304,  304,   67,
-       61,   26,   30,   55,   30,   36,   34,   29,   37,   30,
+        5,    5,    5,    5,    5,    5,   36,   37,  393,   39,
+        9,    5,    5,    5,   10,    9,  242,  242,  225,   10,
+       26,   26,   26,  143,   30,   40,   46,  315,  315,   67,
+       61,   26,   30,   55,   30,   36,   37,   34,   39,   30,
         5,    5,    5,    6,    6,    6,    6,    6,    6,    6,
-        6,    6,    9,   30,   39,   40,   10,    6,    6,    6,
-       26,   30,   31,   30,   32,   34,   31,   46,   30,   31,
-       45,   32,   24,   38,   32,   32,   20,   13,   38,   51,
+        6,    6,    9,   30,   40,   46,   10,    6,    6,    6,
+       26,   30,   31,   30,   32,   34,   31,   29,   30,   31,
+       34,   32,   24,   38,   32,   32,   20,   13,   38,   45,
 
         0,    0,   31,   45,   43,    0,    6,    6,    6,    0,
-       43,   31,   53,   32,   80,   31,   46,    0,   31,   45,
-       32,   33,   38,   32,   32,   33,   41,   38,   51,   33,
-       41,   31,   45,   43,   41,   33,   44,   48,   33,   43,
-       41,   53,   47,   80,   44,   48,   47,    0,   44,    0,
-       33,   44,   47,    0,   33,   41,    0,    0,   33,   41,
-        0,    0,    0,   41,   33,   44,   48,   33,   35,   41,
-       49,   47,    0,   44,   48,   47,   35,   44,   49,   35,
-       44,   47,   35,   82,   50,   35,   50,   49,   35,   50,
-       49,   52,   52,   72,   72,    0,    0,   35,   81,   49,
+       43,   31,   51,   32,   34,   31,   45,   53,   31,   34,
+       32,   33,   38,   32,   32,   33,   41,   38,   45,   33,
+       41,   31,   45,   43,   41,   33,   44,   80,   33,   43,
+       41,   51,   52,   52,   44,   45,   53,    0,   44,    0,
+       33,   44,   48,    0,   33,   41,    0,    0,   33,   41,
+       48,   82,    0,   41,   33,   44,   80,   33,   35,   41,
+       47,   52,   52,   44,   47,   49,   35,   44,   47,   35,
+       44,   48,   35,   49,   47,   35,   81,   83,   35,   48,
+       82,   50,   49,   50,   81,   49,   50,   35,    0,   47,
 
-       73,   73,    0,   72,   83,   35,   81,   49,   35,   84,
-       73,   35,   82,   50,   35,   50,   49,   35,   50,   49,
-       52,   52,   64,   74,   74,   74,    0,   81,   86,   85,
-       64,   64,   72,   83,   74,   81,   88,   64,   84,   73,
-       88,   64,   85,   89,   90,   89,   91,   89,   92,   64,
-       94,   93,   95,   64,   93,   64,   96,   86,   85,   64,
-       93,   98,   99,   74,  100,   88,   64,   93,  101,   88,
-       64,   85,   89,   90,   89,   91,   89,   92,   64,   94,
-       93,   95,   64,   93,   64,   96,  102,  103,   64,   93,
-       98,   99,  104,  100,  105,  106,   93,  101,  107,  109,
+       72,   72,    0,   47,   49,   35,    0,   47,   35,   84,
+       72,   35,   49,   47,   35,   81,   83,   35,   86,   85,
+       50,   49,   50,   81,   49,   50,   64,   73,   73,   74,
+       74,   74,   85,   90,   64,   64,   91,   73,   84,   72,
+       74,   64,   88,   92,    0,   64,   88,   86,   85,   94,
+       89,    0,   89,   64,   89,   95,   96,   64,   98,   64,
+       99,   85,   90,   64,   93,   91,   73,   93,  100,   74,
+       64,   88,   92,   93,   64,   88,  101,  102,   94,   89,
+       93,   89,   64,   89,   95,   96,   64,   98,   64,   99,
+      103,  104,   64,   93,  105,  106,   93,  100,  107,  110,
 
-      110,  111,  112,  113,  114,  113,  115,  116,  107,  117,
-      119,  120,  121,  107,  107,  102,  103,  122,  123,  125,
-      124,  104,  126,  105,  106,  124,  124,  107,  109,  110,
-      111,  112,  113,  114,  113,  115,  116,  107,  117,  119,
-      120,  121,  107,  107,  127,  128,  122,  123,  125,  124,
-      129,  126,  127,  130,  124,  124,  131,  132,  133,  134,
-      135,  136,  137,  138,  139,  159,  159,  136,  164,  166,
-      167,  168,  170,  127,  128,  159,  171,  172,  173,  129,
-      174,  127,  130,  175,  170,  131,  132,  133,  134,  135,
-      136,  137,  138,  139,  176,  177,  136,  164,  166,  167,
+      108,  107,   93,  111,  112,  101,  102,  113,  115,   93,
+      108,  114,  116,  114,  117,  108,  108,  118,  120,  103,
+      104,  121,  122,  105,  106,  123,  124,  107,  110,  108,
+      107,  125,  111,  112,  126,  128,  113,  115,  129,  108,
+      114,  116,  114,  117,  108,  108,  118,  120,  130,  131,
+      121,  122,  127,  132,  123,  124,  130,  127,  127,  133,
+      125,  134,  135,  126,  128,  136,  137,  129,  138,  139,
+      140,  141,  142,  162,  162,  139,  167,  130,  131,  169,
+      170,  127,  132,  162,  171,  130,  127,  127,  133,  174,
+      134,  135,  175,  176,  136,  137,  177,  138,  139,  140,
 
-      168,  170,  178,  179,  159,  171,  172,  173,  180,  174,
-      181,  182,  175,  170,  181,  183,  184,  185,  186,  187,
-      188,  189,  190,  176,  177,  191,  192,  193,  194,  195,
-      196,  178,  179,  197,  199,  201,  202,  180,  203,  181,
-      182,  204,  198,  181,  183,  184,  185,  186,  187,  188,
-      189,  190,  198,  205,  191,  192,  193,  194,  195,  196,
-      207,  209,  197,  199,  201,  202,  210,  203,  211,  212,
-      204,  198,  213,  214,  215,  216,  217,  219,  221,  222,
-      223,  198,  205,  224,  225,  226,  227,  228,  229,  207,
-      209,  230,  231,  232,  233,  210,  237,  211,  212,  238,
+      141,  142,  173,  178,  139,  167,  179,  180,  169,  170,
+      181,  182,  162,  171,  173,  183,  184,  185,  174,  186,
+      184,  175,  176,  187,  188,  177,  189,  190,  191,  192,
+      193,  173,  178,  194,  195,  179,  180,  196,  197,  181,
+      182,  198,  199,  173,  183,  184,  185,  200,  186,  184,
+      201,  202,  187,  188,  203,  189,  190,  191,  192,  193,
+      204,  206,  194,  195,  203,  207,  196,  197,  208,  209,
+      198,  199,  210,  212,  214,  215,  200,  216,  217,  201,
+      202,  218,  219,  203,  220,  221,  222,  223,  224,  204,
+      206,  226,  228,  203,  207,  229,  230,  208,  209,  231,
 
-      239,  213,  214,  215,  216,  217,  219,  221,  222,  223,
-      240,  241,  224,  225,  226,  227,  228,  229,  242,  243,
-      230,  231,  232,  233,  244,  237,  245,  246,  238,  239,
-      248,  249,  250,  251,  252,  253,  254,  255,  256,  240,
-      241,  257,  259,  260,  261,  262,  263,  242,  243,  266,
-      267,  268,  269,  244,  270,  245,  246,  271,  277,  248,
-      249,  250,  251,  252,  253,  254,  255,  256,  271,  279,
-      257,  259,  260,  261,  262,  263,  280,  281,  266,  267,
-      268,  269,  282,  270,  283,  286,  271,  277,  287,  288,
-      289,  290,  291,  292,  293,  295,  296,  271,  279,  297,
+      232,  210,  212,  214,  215,  233,  216,  217,  234,  235,
+      218,  219,  236,  220,  221,  222,  223,  224,  237,  238,
+      226,  228,  239,  240,  229,  230,  244,  245,  231,  232,
+      246,  247,  248,  249,  233,  250,  251,  234,  235,  252,
+      253,  236,  255,  256,  257,  258,  259,  237,  238,  260,
+      261,  239,  240,  262,  263,  244,  245,  264,  266,  246,
+      247,  248,  249,  267,  250,  251,  268,  269,  252,  253,
+      270,  255,  256,  257,  258,  259,  271,  274,  260,  261,
+      276,  277,  262,  263,  278,  279,  264,  266,  286,  280,
+      288,  289,  267,  290,  291,  268,  269,  292,  293,  270,
 
-      298,  299,  300,  301,  303,  280,  281,  307,  308,  309,
-      311,  282,  310,  283,  286,  310,  312,  287,  288,  289,
-      290,  291,  292,  293,  295,  296,  314,  315,  297,  298,
-      299,  300,  301,  303,  316,  317,  307,  308,  309,  311,
-      318,  310,  319,  320,  310,  312,  321,  322,  323,  324,
-      325,  326,  330,  332,  335,  314,  315,  336,  337,  342,
-      343,  344,  345,  316,  317,  347,  348,  349,  351,  318,
-      352,  319,  320,  353,  354,  321,  322,  323,  324,  325,
-      326,  330,  332,  335,  356,  357,  336,  337,  342,  343,
-      344,  345,  359,  360,  347,  348,  349,  351,  361,  352,
+      280,  295,  297,  298,  299,  271,  274,  300,  301,  276,
+      277,  302,  303,  278,  279,  304,  306,  286,  280,  288,
+      289,  307,  290,  291,  308,  309,  292,  293,  310,  280,
+      295,  297,  298,  299,  311,  312,  300,  301,  314,  318,
+      302,  303,  319,  320,  304,  306,  321,  322,  323,  321,
+      307,  325,  326,  308,  309,  327,  328,  310,  329,  330,
+      331,  332,  333,  311,  312,  334,  335,  314,  318,  336,
+      337,  319,  320,  338,  342,  321,  322,  323,  321,  344,
+      325,  326,  347,  348,  327,  328,  349,  329,  330,  331,
+      332,  333,  354,  355,  334,  335,  356,  358,  336,  337,
 
-      363,  364,  353,  354,  365,  366,  368,  370,  371,  372,
-      373,  375,  376,  356,  357,  377,  380,  383,  384,  385,
-      386,  359,  360,  387,  389,  391,  392,  361,  393,  363,
-      364,  397,  398,  365,  366,  368,  370,  371,  372,  373,
-      375,  376,  399,  401,  377,  380,  383,  384,  385,  386,
-      402,  403,  387,  389,  391,  392,  404,  393,  405,  407,
-      397,  398,  410,  411,  412,  413,  416,  419,  420,  421,
-      422,  399,  401,  423,  425,  426,  427,  428,  429,  402,
-      403,  430,  431,  433,  434,  404,  436,  405,  407,  438,
-      439,  410,  411,  412,  413,  416,  419,  420,  421,  422,
+      359,  361,  338,  342,  362,  363,  365,  366,  344,  367,
+      368,  347,  348,  370,  371,  349,  373,  374,  375,  377,
+      378,  354,  355,  379,  380,  356,  358,  382,  384,  359,
+      361,  385,  386,  362,  363,  365,  366,  387,  367,  368,
+      389,  390,  370,  371,  391,  373,  374,  375,  377,  378,
+      394,  395,  379,  380,  398,  399,  382,  384,  400,  401,
+      385,  386,  402,  403,  405,  407,  387,  408,  409,  389,
+      390,  413,  414,  391,  415,  417,  418,  419,  420,  394,
+      395,  421,  423,  398,  399,  426,  427,  400,  401,  428,
+      429,  402,  403,  405,  407,  433,  408,  409,  434,  437,
 
-      441,  442,  423,  425,  426,  427,  428,  429,  443,  445,
-      430,  431,  433,  434,  446,  436,  447,  449,  438,  439,
-      451,  452,  453,  454,  456,  458,  459,  460,  463,  441,
-      442,  465,  466,  467,  470,  471,  473,  443,  445,  474,
-      475,  476,  477,  446,  479,  447,  449,    0,    0,  451,
-      452,  453,  454,  456,  458,  459,  460,  463,    0,    0,
-      465,  466,  467,  470,  471,  473,    0,    0,  474,  475,
-      476,  477,    0,  479,  483,  483,  483,  483,  483,  483,
-      483,  483,  483,  483,  484,  484,  484,  484,  484,  484,
-      484,  484,  484,  484,  485,  485,  485,  485,  485,  485,
+      413,  414,  438,  415,  417,  418,  419,  420,  439,  440,
+      421,  423,  441,  443,  426,  427,  444,  445,  428,  429,
+      446,  447,  448,  449,  433,  451,  452,  434,  437,  454,
+      456,  438,  457,  458,  460,  461,  462,  439,  440,  464,
+      465,  441,  443,  466,  468,  444,  445,  470,  471,  446,
+      447,  448,  449,  472,  451,  452,  473,  474,  454,  456,
+      476,  457,  458,  460,  461,  462,  478,  479,  464,  465,
+      480,  483,  466,  468,  486,  487,  470,  471,  488,  491,
+      492,  494,  472,  495,  496,  473,  474,  497,  498,  476,
+      500,    0,    0,    0,    0,  478,  479,    0,    0,  480,
 
-      485,  485,  485,  485,  486,  486,  486,  486,  486,  486,
-      486,  486,  486,  486,  487,  487,  487,  487,  487,  487,
-      487,  487,  487,  487,  488,  488,  489,  489,  489,    0,
-      489,  490,  490,  490,  490,  491,  491,  491,    0,  491,
-      491,  491,  491,  491,  491,  492,  492,  492,    0,  492,
-      492,  492,  492,    0,  492,  493,  493,  493,  493,  493,
-      493,  493,  493,  493,  493,  494,  494,    0,  494,  494,
-      494,  494,  494,  494,  494,  495,    0,  495,  495,  495,
-      495,  495,  495,  495,  495,  496,  496,  497,  497,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
+      483,    0,    0,  486,  487,    0,    0,  488,  491,  492,
+      494,    0,  495,  496,    0,    0,  497,  498,    0,  500,
+      504,  504,  504,  504,  504,  504,  504,  504,  504,  504,
+      505,  505,  505,  505,  505,  505,  505,  505,  505,  505,
+      506,  506,  506,  506,  506,  506,  506,  506,  506,  506,
+      507,  507,  507,  507,  507,  507,  507,  507,  507,  507,
+      508,  508,  508,  508,  508,  508,  508,  508,  508,  508,
+      509,  509,  510,  510,  510,    0,  510,  511,  511,  511,
+      511,  512,  512,  512,    0,  512,  512,  512,  512,  512,
+      512,  513,  513,  513,    0,  513,  513,  513,  513,    0,
 
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482,  482,  482,  482,  482,  482,  482,  482,  482,  482,
-      482
+      513,  514,  514,  514,  514,  514,  514,  514,  514,  514,
+      514,  515,  515,    0,  515,  515,  515,  515,  515,  515,
+      515,  516,    0,  516,  516,  516,  516,  516,  516,  516,
+      516,  517,  517,  518,  518,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+      503,  503,  503,  503,  503,  503,  503,  503,  503,  503,
+
+      503,  503,  503,  503,  503,  503,  503
     } ;
 
 /* Table of booleans, true if rule could match eol. */
-static yyconst flex_int32_t yy_rule_can_match_eol[131] =
+static yyconst flex_int32_t yy_rule_can_match_eol[136] =
     {   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, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
-    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 
-    0, 1, 1, 1, 0, 0, 0, 1, 0, 0, 0,     };
+    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
+    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.
@@ -942,6 +959,7 @@
 class ParseLimit;
 class ParseOrderBy;
 class ParseOrderByItem;
+class ParsePartitionClause;
 class ParsePredicate;
 class ParseSample;
 class ParseScalarLiteral;
@@ -958,6 +976,8 @@
 class ParseStatementDelete;
 class ParseStatementDropTable;
 class ParseStatementInsert;
+class ParseStatementInsertTuple;
+class ParseStatementInsertSelection;
 class ParseStatementSelect;
 class ParseStatementQuit;
 class ParseStatementUpdate;
@@ -991,7 +1011,7 @@
 
 
 
-#line 995 "SqlLexer_gen.cpp"
+#line 1015 "SqlLexer_gen.cpp"
 
 #define INITIAL 0
 #define CONDITION_SQL 1
@@ -1279,10 +1299,10 @@
 		}
 
 	{
-#line 120 "../SqlLexer.lpp"
+#line 123 "../SqlLexer.lpp"
 
 
-#line 1286 "SqlLexer_gen.cpp"
+#line 1306 "SqlLexer_gen.cpp"
 
 	while ( /*CONSTCOND*/1 )		/* loops until end-of-file is reached */
 		{
@@ -1309,13 +1329,13 @@
 			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 >= 483 )
+				if ( yy_current_state >= 504 )
 					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 != 482 );
+		while ( yy_current_state != 503 );
 		yy_cp = yyg->yy_last_accepting_cpos;
 		yy_current_state = yyg->yy_last_accepting_state;
 
@@ -1349,7 +1369,7 @@
 
 case 1:
 YY_RULE_SETUP
-#line 123 "../SqlLexer.lpp"
+#line 126 "../SqlLexer.lpp"
 {
     /* A forward slash character represents a system command. */
     BEGIN(CONDITION_COMMAND);
@@ -1361,7 +1381,7 @@
 case 2:
 /* rule 2 can match eol */
 YY_RULE_SETUP
-#line 131 "../SqlLexer.lpp"
+#line 134 "../SqlLexer.lpp"
 {
     /* This is a SQL command. Place the char back and process normally. */
     yyless(0);
@@ -1373,7 +1393,7 @@
 
 case 3:
 YY_RULE_SETUP
-#line 140 "../SqlLexer.lpp"
+#line 143 "../SqlLexer.lpp"
 {
     /* This is a command argument. */
     yylval->string_value_ = new quickstep::ParseString(
@@ -1383,7 +1403,7 @@
 	YY_BREAK
 case 4:
 YY_RULE_SETUP
-#line 147 "../SqlLexer.lpp"
+#line 150 "../SqlLexer.lpp"
 {
     /* Ignore whitespace. */
   }
@@ -1391,7 +1411,7 @@
 case 5:
 /* rule 5 can match eol */
 YY_RULE_SETUP
-#line 151 "../SqlLexer.lpp"
+#line 154 "../SqlLexer.lpp"
 {
     /* Newline reverts the lexer to the initial state. */
     yycolumn = 0;
@@ -1403,517 +1423,542 @@
 
 case 6:
 YY_RULE_SETUP
-#line 160 "../SqlLexer.lpp"
+#line 163 "../SqlLexer.lpp"
 return TOKEN_ADD;
 	YY_BREAK
 case 7:
 YY_RULE_SETUP
-#line 161 "../SqlLexer.lpp"
+#line 164 "../SqlLexer.lpp"
 return TOKEN_ALL;
 	YY_BREAK
 case 8:
 YY_RULE_SETUP
-#line 162 "../SqlLexer.lpp"
+#line 165 "../SqlLexer.lpp"
 return TOKEN_ALTER;
 	YY_BREAK
 case 9:
 YY_RULE_SETUP
-#line 163 "../SqlLexer.lpp"
+#line 166 "../SqlLexer.lpp"
 return TOKEN_AND;
 	YY_BREAK
 case 10:
 YY_RULE_SETUP
-#line 164 "../SqlLexer.lpp"
+#line 167 "../SqlLexer.lpp"
 return TOKEN_AS;
 	YY_BREAK
 case 11:
 YY_RULE_SETUP
-#line 165 "../SqlLexer.lpp"
+#line 168 "../SqlLexer.lpp"
 return TOKEN_ASC;
 	YY_BREAK
 case 12:
 YY_RULE_SETUP
-#line 166 "../SqlLexer.lpp"
+#line 169 "../SqlLexer.lpp"
 return TOKEN_ASC;
 	YY_BREAK
 case 13:
 YY_RULE_SETUP
-#line 167 "../SqlLexer.lpp"
+#line 170 "../SqlLexer.lpp"
 return TOKEN_BETWEEN;
 	YY_BREAK
 case 14:
 YY_RULE_SETUP
-#line 168 "../SqlLexer.lpp"
+#line 171 "../SqlLexer.lpp"
 return TOKEN_BIGINT;
 	YY_BREAK
 case 15:
 YY_RULE_SETUP
-#line 169 "../SqlLexer.lpp"
+#line 172 "../SqlLexer.lpp"
 return TOKEN_BIT;
 	YY_BREAK
 case 16:
 YY_RULE_SETUP
-#line 170 "../SqlLexer.lpp"
+#line 173 "../SqlLexer.lpp"
 return TOKEN_BLOCKPROPERTIES;
 	YY_BREAK
 case 17:
 YY_RULE_SETUP
-#line 171 "../SqlLexer.lpp"
+#line 174 "../SqlLexer.lpp"
 return TOKEN_BLOCKSAMPLE;
 	YY_BREAK
 case 18:
 YY_RULE_SETUP
-#line 172 "../SqlLexer.lpp"
+#line 175 "../SqlLexer.lpp"
 return TOKEN_BLOOM_FILTER;
 	YY_BREAK
 case 19:
 YY_RULE_SETUP
-#line 173 "../SqlLexer.lpp"
+#line 176 "../SqlLexer.lpp"
 return TOKEN_CSB_TREE;
 	YY_BREAK
 case 20:
 YY_RULE_SETUP
-#line 174 "../SqlLexer.lpp"
+#line 177 "../SqlLexer.lpp"
 return TOKEN_BY;
 	YY_BREAK
 case 21:
 YY_RULE_SETUP
-#line 175 "../SqlLexer.lpp"
+#line 178 "../SqlLexer.lpp"
 return TOKEN_CHARACTER;
 	YY_BREAK
 case 22:
 YY_RULE_SETUP
-#line 176 "../SqlLexer.lpp"
+#line 179 "../SqlLexer.lpp"
 return TOKEN_CHARACTER;
 	YY_BREAK
 case 23:
 YY_RULE_SETUP
-#line 177 "../SqlLexer.lpp"
+#line 180 "../SqlLexer.lpp"
 return TOKEN_CHECK;
 	YY_BREAK
 case 24:
 YY_RULE_SETUP
-#line 178 "../SqlLexer.lpp"
+#line 181 "../SqlLexer.lpp"
 return TOKEN_COLUMN;
 	YY_BREAK
 case 25:
 YY_RULE_SETUP
-#line 179 "../SqlLexer.lpp"
+#line 182 "../SqlLexer.lpp"
 return TOKEN_CONSTRAINT;
 	YY_BREAK
 case 26:
 YY_RULE_SETUP
-#line 180 "../SqlLexer.lpp"
+#line 183 "../SqlLexer.lpp"
 return TOKEN_COPY;
 	YY_BREAK
 case 27:
 YY_RULE_SETUP
-#line 181 "../SqlLexer.lpp"
+#line 184 "../SqlLexer.lpp"
 return TOKEN_CREATE;
 	YY_BREAK
 case 28:
 YY_RULE_SETUP
-#line 182 "../SqlLexer.lpp"
+#line 185 "../SqlLexer.lpp"
 return TOKEN_DATE;
 	YY_BREAK
 case 29:
 YY_RULE_SETUP
-#line 183 "../SqlLexer.lpp"
+#line 186 "../SqlLexer.lpp"
 return TOKEN_DATETIME;
 	YY_BREAK
 case 30:
 YY_RULE_SETUP
-#line 184 "../SqlLexer.lpp"
+#line 187 "../SqlLexer.lpp"
 return TOKEN_DECIMAL;
 	YY_BREAK
 case 31:
 YY_RULE_SETUP
-#line 185 "../SqlLexer.lpp"
+#line 188 "../SqlLexer.lpp"
 return TOKEN_DEFAULT;
 	YY_BREAK
 case 32:
 YY_RULE_SETUP
-#line 186 "../SqlLexer.lpp"
+#line 189 "../SqlLexer.lpp"
 return TOKEN_DELETE;
 	YY_BREAK
 case 33:
 YY_RULE_SETUP
-#line 187 "../SqlLexer.lpp"
+#line 190 "../SqlLexer.lpp"
 return TOKEN_DELIMITER;
 	YY_BREAK
 case 34:
 YY_RULE_SETUP
-#line 188 "../SqlLexer.lpp"
+#line 191 "../SqlLexer.lpp"
 return TOKEN_DESC;
 	YY_BREAK
 case 35:
 YY_RULE_SETUP
-#line 189 "../SqlLexer.lpp"
+#line 192 "../SqlLexer.lpp"
 return TOKEN_DESC;
 	YY_BREAK
 case 36:
 YY_RULE_SETUP
-#line 190 "../SqlLexer.lpp"
+#line 193 "../SqlLexer.lpp"
 return TOKEN_DISTINCT;
 	YY_BREAK
 case 37:
 YY_RULE_SETUP
-#line 191 "../SqlLexer.lpp"
+#line 194 "../SqlLexer.lpp"
 return TOKEN_DOUBLE;
 	YY_BREAK
 case 38:
 YY_RULE_SETUP
-#line 192 "../SqlLexer.lpp"
+#line 195 "../SqlLexer.lpp"
 return TOKEN_DROP;
 	YY_BREAK
 case 39:
 YY_RULE_SETUP
-#line 193 "../SqlLexer.lpp"
+#line 196 "../SqlLexer.lpp"
 return TOKEN_ESCAPE_STRINGS;
 	YY_BREAK
 case 40:
 YY_RULE_SETUP
-#line 194 "../SqlLexer.lpp"
-return TOKEN_FALSE;
+#line 197 "../SqlLexer.lpp"
+return TOKEN_EXTRACT;
 	YY_BREAK
 case 41:
 YY_RULE_SETUP
-#line 195 "../SqlLexer.lpp"
-return TOKEN_FIRST;
+#line 198 "../SqlLexer.lpp"
+return TOKEN_FALSE;
 	YY_BREAK
 case 42:
 YY_RULE_SETUP
-#line 196 "../SqlLexer.lpp"
-return TOKEN_FLOAT;
+#line 199 "../SqlLexer.lpp"
+return TOKEN_FIRST;
 	YY_BREAK
 case 43:
 YY_RULE_SETUP
-#line 197 "../SqlLexer.lpp"
-return TOKEN_FOREIGN;
+#line 200 "../SqlLexer.lpp"
+return TOKEN_FLOAT;
 	YY_BREAK
 case 44:
 YY_RULE_SETUP
-#line 198 "../SqlLexer.lpp"
-return TOKEN_FROM;
+#line 201 "../SqlLexer.lpp"
+return TOKEN_FOREIGN;
 	YY_BREAK
 case 45:
 YY_RULE_SETUP
-#line 199 "../SqlLexer.lpp"
-return TOKEN_FULL;
+#line 202 "../SqlLexer.lpp"
+return TOKEN_FROM;
 	YY_BREAK
 case 46:
 YY_RULE_SETUP
-#line 200 "../SqlLexer.lpp"
-return TOKEN_GROUP;
+#line 203 "../SqlLexer.lpp"
+return TOKEN_FULL;
 	YY_BREAK
 case 47:
 YY_RULE_SETUP
-#line 201 "../SqlLexer.lpp"
-return TOKEN_HAVING;
+#line 204 "../SqlLexer.lpp"
+return TOKEN_GROUP;
 	YY_BREAK
 case 48:
 YY_RULE_SETUP
-#line 202 "../SqlLexer.lpp"
-return TOKEN_INDEX;
+#line 205 "../SqlLexer.lpp"
+return TOKEN_HASH;
 	YY_BREAK
 case 49:
 YY_RULE_SETUP
-#line 203 "../SqlLexer.lpp"
-return TOKEN_INNER;
+#line 206 "../SqlLexer.lpp"
+return TOKEN_HAVING;
 	YY_BREAK
 case 50:
 YY_RULE_SETUP
-#line 204 "../SqlLexer.lpp"
-return TOKEN_INSERT;
+#line 207 "../SqlLexer.lpp"
+return TOKEN_INDEX;
 	YY_BREAK
 case 51:
 YY_RULE_SETUP
-#line 205 "../SqlLexer.lpp"
-return TOKEN_INTEGER;
+#line 208 "../SqlLexer.lpp"
+return TOKEN_INNER;
 	YY_BREAK
 case 52:
 YY_RULE_SETUP
-#line 206 "../SqlLexer.lpp"
-return TOKEN_INTEGER;
+#line 209 "../SqlLexer.lpp"
+return TOKEN_INSERT;
 	YY_BREAK
 case 53:
 YY_RULE_SETUP
-#line 207 "../SqlLexer.lpp"
-return TOKEN_INTERVAL;
+#line 210 "../SqlLexer.lpp"
+return TOKEN_INTEGER;
 	YY_BREAK
 case 54:
 YY_RULE_SETUP
-#line 208 "../SqlLexer.lpp"
-return TOKEN_INTO;
+#line 211 "../SqlLexer.lpp"
+return TOKEN_INTEGER;
 	YY_BREAK
 case 55:
 YY_RULE_SETUP
-#line 209 "../SqlLexer.lpp"
-return TOKEN_IS;
+#line 212 "../SqlLexer.lpp"
+return TOKEN_INTERVAL;
 	YY_BREAK
 case 56:
 YY_RULE_SETUP
-#line 210 "../SqlLexer.lpp"
-return TOKEN_JOIN;
+#line 213 "../SqlLexer.lpp"
+return TOKEN_INTO;
 	YY_BREAK
 case 57:
 YY_RULE_SETUP
-#line 211 "../SqlLexer.lpp"
-return TOKEN_KEY;
+#line 214 "../SqlLexer.lpp"
+return TOKEN_IS;
 	YY_BREAK
 case 58:
 YY_RULE_SETUP
-#line 212 "../SqlLexer.lpp"
-return TOKEN_LAST;
+#line 215 "../SqlLexer.lpp"
+return TOKEN_JOIN;
 	YY_BREAK
 case 59:
 YY_RULE_SETUP
-#line 213 "../SqlLexer.lpp"
-return TOKEN_LEFT;
+#line 216 "../SqlLexer.lpp"
+return TOKEN_KEY;
 	YY_BREAK
 case 60:
 YY_RULE_SETUP
-#line 214 "../SqlLexer.lpp"
-return TOKEN_LIKE;
+#line 217 "../SqlLexer.lpp"
+return TOKEN_LAST;
 	YY_BREAK
 case 61:
 YY_RULE_SETUP
-#line 215 "../SqlLexer.lpp"
-return TOKEN_LIMIT;
+#line 218 "../SqlLexer.lpp"
+return TOKEN_LEFT;
 	YY_BREAK
 case 62:
 YY_RULE_SETUP
-#line 216 "../SqlLexer.lpp"
-return TOKEN_LONG;
+#line 219 "../SqlLexer.lpp"
+return TOKEN_LIKE;
 	YY_BREAK
 case 63:
 YY_RULE_SETUP
-#line 217 "../SqlLexer.lpp"
-return TOKEN_NOT;
+#line 220 "../SqlLexer.lpp"
+return TOKEN_LIMIT;
 	YY_BREAK
 case 64:
 YY_RULE_SETUP
-#line 218 "../SqlLexer.lpp"
-return TOKEN_NULL;
+#line 221 "../SqlLexer.lpp"
+return TOKEN_LONG;
 	YY_BREAK
 case 65:
 YY_RULE_SETUP
-#line 219 "../SqlLexer.lpp"
-return TOKEN_NULLS;
+#line 222 "../SqlLexer.lpp"
+return TOKEN_NOT;
 	YY_BREAK
 case 66:
 YY_RULE_SETUP
-#line 220 "../SqlLexer.lpp"
-return TOKEN_OFF;
+#line 223 "../SqlLexer.lpp"
+return TOKEN_NULL;
 	YY_BREAK
 case 67:
 YY_RULE_SETUP
-#line 221 "../SqlLexer.lpp"
-return TOKEN_ON;
+#line 224 "../SqlLexer.lpp"
+return TOKEN_NULLS;
 	YY_BREAK
 case 68:
 YY_RULE_SETUP
-#line 222 "../SqlLexer.lpp"
-return TOKEN_OR;
+#line 225 "../SqlLexer.lpp"
+return TOKEN_OFF;
 	YY_BREAK
 case 69:
 YY_RULE_SETUP
-#line 223 "../SqlLexer.lpp"
-return TOKEN_ORDER;
+#line 226 "../SqlLexer.lpp"
+return TOKEN_ON;
 	YY_BREAK
 case 70:
 YY_RULE_SETUP
-#line 224 "../SqlLexer.lpp"
-return TOKEN_OUTER;
+#line 227 "../SqlLexer.lpp"
+return TOKEN_OR;
 	YY_BREAK
 case 71:
 YY_RULE_SETUP
-#line 225 "../SqlLexer.lpp"
-return TOKEN_PERCENT;
+#line 228 "../SqlLexer.lpp"
+return TOKEN_ORDER;
 	YY_BREAK
 case 72:
 YY_RULE_SETUP
-#line 226 "../SqlLexer.lpp"
-return TOKEN_PRIMARY;
+#line 229 "../SqlLexer.lpp"
+return TOKEN_OUTER;
 	YY_BREAK
 case 73:
 YY_RULE_SETUP
-#line 227 "../SqlLexer.lpp"
-return TOKEN_QUIT;
+#line 230 "../SqlLexer.lpp"
+return TOKEN_PARTITION;
 	YY_BREAK
 case 74:
 YY_RULE_SETUP
-#line 228 "../SqlLexer.lpp"
-return TOKEN_REAL;
+#line 231 "../SqlLexer.lpp"
+return TOKEN_PARTITIONS;
 	YY_BREAK
 case 75:
 YY_RULE_SETUP
-#line 229 "../SqlLexer.lpp"
-return TOKEN_REFERENCES;
+#line 232 "../SqlLexer.lpp"
+return TOKEN_PERCENT;
 	YY_BREAK
 case 76:
 YY_RULE_SETUP
-#line 230 "../SqlLexer.lpp"
-return TOKEN_REGEXP;
+#line 233 "../SqlLexer.lpp"
+return TOKEN_PRIMARY;
 	YY_BREAK
 case 77:
 YY_RULE_SETUP
-#line 231 "../SqlLexer.lpp"
-return TOKEN_RIGHT;
+#line 234 "../SqlLexer.lpp"
+return TOKEN_QUIT;
 	YY_BREAK
 case 78:
 YY_RULE_SETUP
-#line 232 "../SqlLexer.lpp"
-return TOKEN_ROW_DELIMITER;
+#line 235 "../SqlLexer.lpp"
+return TOKEN_RANGE;
 	YY_BREAK
 case 79:
 YY_RULE_SETUP
-#line 233 "../SqlLexer.lpp"
-return TOKEN_SELECT;
+#line 236 "../SqlLexer.lpp"
+return TOKEN_REAL;
 	YY_BREAK
 case 80:
 YY_RULE_SETUP
-#line 234 "../SqlLexer.lpp"
-return TOKEN_SET;
+#line 237 "../SqlLexer.lpp"
+return TOKEN_REFERENCES;
 	YY_BREAK
 case 81:
 YY_RULE_SETUP
-#line 235 "../SqlLexer.lpp"
-return TOKEN_SMALLINT;
+#line 238 "../SqlLexer.lpp"
+return TOKEN_REGEXP;
 	YY_BREAK
 case 82:
 YY_RULE_SETUP
-#line 236 "../SqlLexer.lpp"
-return TOKEN_TABLE;
+#line 239 "../SqlLexer.lpp"
+return TOKEN_RIGHT;
 	YY_BREAK
 case 83:
 YY_RULE_SETUP
-#line 237 "../SqlLexer.lpp"
-return TOKEN_TIME;
+#line 240 "../SqlLexer.lpp"
+return TOKEN_ROW_DELIMITER;
 	YY_BREAK
 case 84:
 YY_RULE_SETUP
-#line 238 "../SqlLexer.lpp"
-return TOKEN_TIMESTAMP;
+#line 241 "../SqlLexer.lpp"
+return TOKEN_SELECT;
 	YY_BREAK
 case 85:
 YY_RULE_SETUP
-#line 239 "../SqlLexer.lpp"
-return TOKEN_TRUE;
+#line 242 "../SqlLexer.lpp"
+return TOKEN_SET;
 	YY_BREAK
 case 86:
 YY_RULE_SETUP
-#line 240 "../SqlLexer.lpp"
-return TOKEN_TUPLESAMPLE;
+#line 243 "../SqlLexer.lpp"
+return TOKEN_SMALLINT;
 	YY_BREAK
 case 87:
 YY_RULE_SETUP
-#line 241 "../SqlLexer.lpp"
-return TOKEN_UNIQUE;
+#line 244 "../SqlLexer.lpp"
+return TOKEN_TABLE;
 	YY_BREAK
 case 88:
 YY_RULE_SETUP
-#line 242 "../SqlLexer.lpp"
-return TOKEN_UPDATE;
+#line 245 "../SqlLexer.lpp"
+return TOKEN_TIME;
 	YY_BREAK
 case 89:
 YY_RULE_SETUP
-#line 243 "../SqlLexer.lpp"
-return TOKEN_USING;
+#line 246 "../SqlLexer.lpp"
+return TOKEN_TIMESTAMP;
 	YY_BREAK
 case 90:
 YY_RULE_SETUP
-#line 244 "../SqlLexer.lpp"
-return TOKEN_VALUES;
+#line 247 "../SqlLexer.lpp"
+return TOKEN_TRUE;
 	YY_BREAK
 case 91:
 YY_RULE_SETUP
-#line 245 "../SqlLexer.lpp"
-return TOKEN_VARCHAR;
+#line 248 "../SqlLexer.lpp"
+return TOKEN_TUPLESAMPLE;
 	YY_BREAK
 case 92:
 YY_RULE_SETUP
-#line 246 "../SqlLexer.lpp"
-return TOKEN_WHERE;
+#line 249 "../SqlLexer.lpp"
+return TOKEN_UNIQUE;
 	YY_BREAK
 case 93:
 YY_RULE_SETUP
-#line 247 "../SqlLexer.lpp"
-return TOKEN_WITH;
+#line 250 "../SqlLexer.lpp"
+return TOKEN_UPDATE;
 	YY_BREAK
 case 94:
 YY_RULE_SETUP
-#line 248 "../SqlLexer.lpp"
-return TOKEN_YEARMONTH;
+#line 251 "../SqlLexer.lpp"
+return TOKEN_USING;
 	YY_BREAK
 case 95:
 YY_RULE_SETUP
-#line 250 "../SqlLexer.lpp"
-return TOKEN_EQ;
+#line 252 "../SqlLexer.lpp"
+return TOKEN_VALUES;
 	YY_BREAK
 case 96:
 YY_RULE_SETUP
-#line 251 "../SqlLexer.lpp"
-return TOKEN_NEQ;
+#line 253 "../SqlLexer.lpp"
+return TOKEN_VARCHAR;
 	YY_BREAK
 case 97:
 YY_RULE_SETUP
-#line 252 "../SqlLexer.lpp"
-return TOKEN_NEQ;
+#line 254 "../SqlLexer.lpp"
+return TOKEN_WHERE;
 	YY_BREAK
 case 98:
 YY_RULE_SETUP
-#line 253 "../SqlLexer.lpp"
-return TOKEN_LT;
+#line 255 "../SqlLexer.lpp"
+return TOKEN_WITH;
 	YY_BREAK
 case 99:
 YY_RULE_SETUP
-#line 254 "../SqlLexer.lpp"
-return TOKEN_GT;
+#line 256 "../SqlLexer.lpp"
+return TOKEN_YEARMONTH;
 	YY_BREAK
 case 100:
 YY_RULE_SETUP
-#line 255 "../SqlLexer.lpp"
-return TOKEN_LEQ;
+#line 258 "../SqlLexer.lpp"
+return TOKEN_EQ;
 	YY_BREAK
 case 101:
 YY_RULE_SETUP
-#line 256 "../SqlLexer.lpp"
-return TOKEN_GEQ;
+#line 259 "../SqlLexer.lpp"
+return TOKEN_NEQ;
 	YY_BREAK
 case 102:
 YY_RULE_SETUP
-#line 258 "../SqlLexer.lpp"
-return yytext[0];
+#line 260 "../SqlLexer.lpp"
+return TOKEN_NEQ;
 	YY_BREAK
 case 103:
 YY_RULE_SETUP
-#line 259 "../SqlLexer.lpp"
+#line 261 "../SqlLexer.lpp"
+return TOKEN_LT;
+	YY_BREAK
+case 104:
+YY_RULE_SETUP
+#line 262 "../SqlLexer.lpp"
+return TOKEN_GT;
+	YY_BREAK
+case 105:
+YY_RULE_SETUP
+#line 263 "../SqlLexer.lpp"
+return TOKEN_LEQ;
+	YY_BREAK
+case 106:
+YY_RULE_SETUP
+#line 264 "../SqlLexer.lpp"
+return TOKEN_GEQ;
+	YY_BREAK
+case 107:
+YY_RULE_SETUP
+#line 266 "../SqlLexer.lpp"
+return yytext[0];
+	YY_BREAK
+case 108:
+YY_RULE_SETUP
+#line 267 "../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 104:
+case 109:
 YY_RULE_SETUP
-#line 265 "../SqlLexer.lpp"
+#line 273 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED_ESCAPED);
   }
 	YY_BREAK
-case 105:
+case 110:
 YY_RULE_SETUP
-#line 270 "../SqlLexer.lpp"
+#line 278 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED);
   }
 	YY_BREAK
-case 106:
+case 111:
 YY_RULE_SETUP
-#line 275 "../SqlLexer.lpp"
+#line 283 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_DOUBLE_QUOTED);
@@ -1925,7 +1970,7 @@
 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 284 "../SqlLexer.lpp"
+#line 292 "../SqlLexer.lpp"
 {
     delete yylval->string_value_;
     BEGIN(INITIAL);
@@ -1936,9 +1981,9 @@
 
 /* Process escape sequences. */
 
-case 107:
+case 112:
 YY_RULE_SETUP
-#line 294 "../SqlLexer.lpp"
+#line 302 "../SqlLexer.lpp"
 {
     /* Octal code */
     unsigned int code;
@@ -1952,9 +1997,9 @@
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 108:
+case 113:
 YY_RULE_SETUP
-#line 306 "../SqlLexer.lpp"
+#line 314 "../SqlLexer.lpp"
 {
     /* Hexadecimal code */
     unsigned int code;
@@ -1962,9 +2007,9 @@
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 109:
+case 114:
 YY_RULE_SETUP
-#line 312 "../SqlLexer.lpp"
+#line 320 "../SqlLexer.lpp"
 {
     /* A numeric escape sequence that isn't correctly specified. */
     delete yylval->string_value_;
@@ -1973,58 +2018,58 @@
     return TOKEN_LEX_ERROR;
   }
 	YY_BREAK
-case 110:
+case 115:
 YY_RULE_SETUP
-#line 319 "../SqlLexer.lpp"
+#line 327 "../SqlLexer.lpp"
 {
     /* Backspace */
     yylval->string_value_->push_back('\b');
   }
 	YY_BREAK
-case 111:
+case 116:
 YY_RULE_SETUP
-#line 323 "../SqlLexer.lpp"
+#line 331 "../SqlLexer.lpp"
 {
     /* Form-feed */
     yylval->string_value_->push_back('\f');
   }
 	YY_BREAK
-case 112:
+case 117:
 YY_RULE_SETUP
-#line 327 "../SqlLexer.lpp"
+#line 335 "../SqlLexer.lpp"
 {
     /* Newline */
     yylval->string_value_->push_back('\n');
   }
 	YY_BREAK
-case 113:
+case 118:
 YY_RULE_SETUP
-#line 331 "../SqlLexer.lpp"
+#line 339 "../SqlLexer.lpp"
 {
     /* Carriage-return */
     yylval->string_value_->push_back('\r');
   }
 	YY_BREAK
-case 114:
+case 119:
 YY_RULE_SETUP
-#line 335 "../SqlLexer.lpp"
+#line 343 "../SqlLexer.lpp"
 {
     /* Horizontal Tab */
     yylval->string_value_->push_back('\t');
   }
 	YY_BREAK
-case 115:
-/* rule 115 can match eol */
+case 120:
+/* rule 120 can match eol */
 YY_RULE_SETUP
-#line 339 "../SqlLexer.lpp"
+#line 347 "../SqlLexer.lpp"
 {
     /* Any other character (including actual newline or carriage return) */
     yylval->string_value_->push_back(yytext[1]);
   }
 	YY_BREAK
-case 116:
+case 121:
 YY_RULE_SETUP
-#line 343 "../SqlLexer.lpp"
+#line 351 "../SqlLexer.lpp"
 {
     /* This should only be encountered right before an EOF. */
     delete yylval->string_value_;
@@ -2035,17 +2080,17 @@
 	YY_BREAK
 
 
-case 117:
+case 122:
 YY_RULE_SETUP
-#line 353 "../SqlLexer.lpp"
+#line 361 "../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 118:
+case 123:
 YY_RULE_SETUP
-#line 357 "../SqlLexer.lpp"
+#line 365 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2054,17 +2099,17 @@
 	YY_BREAK
 
 
-case 119:
+case 124:
 YY_RULE_SETUP
-#line 365 "../SqlLexer.lpp"
+#line 373 "../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 120:
+case 125:
 YY_RULE_SETUP
-#line 369 "../SqlLexer.lpp"
+#line 377 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2072,94 +2117,94 @@
   }
 	YY_BREAK
 
-case 121:
-/* rule 121 can match eol */
+case 126:
+/* rule 126 can match eol */
 YY_RULE_SETUP
-#line 376 "../SqlLexer.lpp"
+#line 384 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 122:
-/* rule 122 can match eol */
+case 127:
+/* rule 127 can match eol */
 YY_RULE_SETUP
-#line 381 "../SqlLexer.lpp"
+#line 389 "../SqlLexer.lpp"
 {
   /* Scan up to a quote or escape sequence. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 123:
-/* rule 123 can match eol */
+case 128:
+/* rule 128 can match eol */
 YY_RULE_SETUP
-#line 386 "../SqlLexer.lpp"
+#line 394 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
 
-case 124:
+case 129:
 YY_RULE_SETUP
-#line 392 "../SqlLexer.lpp"
+#line 400 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(
         yylloc->first_line, yylloc->first_column, std::string(yytext, yyleng));
     return TOKEN_NAME;
   }
 	YY_BREAK
-case 125:
+case 130:
 YY_RULE_SETUP
-#line 398 "../SqlLexer.lpp"
+#line 406 "../SqlLexer.lpp"
 {
     yylval->numeric_literal_value_ = new quickstep::NumericParseLiteralValue(
         yylloc->first_line, yylloc->first_column, yytext);
     return TOKEN_UNSIGNED_NUMVAL;
   }
 	YY_BREAK
-case 126:
+case 131:
 YY_RULE_SETUP
-#line 404 "../SqlLexer.lpp"
+#line 412 "../SqlLexer.lpp"
 /* comment */
 	YY_BREAK
-case 127:
-/* rule 127 can match eol */
+case 132:
+/* rule 132 can match eol */
 YY_RULE_SETUP
-#line 406 "../SqlLexer.lpp"
+#line 414 "../SqlLexer.lpp"
 { yycolumn = 0; }
 	YY_BREAK
-case 128:
+case 133:
 YY_RULE_SETUP
-#line 408 "../SqlLexer.lpp"
+#line 416 "../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 412 "../SqlLexer.lpp"
+#line 420 "../SqlLexer.lpp"
 {
   /* All conditions except for mutli-state string extracting conditions. */
   BEGIN(INITIAL);
   return TOKEN_EOF;
 }
 	YY_BREAK
-case 129:
+case 134:
 YY_RULE_SETUP
-#line 418 "../SqlLexer.lpp"
+#line 426 "../SqlLexer.lpp"
 {
   BEGIN(INITIAL);
   quickstep_yyerror(NULL, yyscanner, NULL, "illegal character");
   return TOKEN_LEX_ERROR;
 }
 	YY_BREAK
-case 130:
+case 135:
 YY_RULE_SETUP
-#line 424 "../SqlLexer.lpp"
+#line 432 "../SqlLexer.lpp"
 YY_FATAL_ERROR( "flex scanner jammed" );
 	YY_BREAK
-#line 2163 "SqlLexer_gen.cpp"
+#line 2208 "SqlLexer_gen.cpp"
 
 	case YY_END_OF_BUFFER:
 		{
@@ -2453,7 +2498,7 @@
 		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 >= 483 )
+			if ( yy_current_state >= 504 )
 				yy_c = yy_meta[(unsigned int) yy_c];
 			}
 		yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
@@ -2482,11 +2527,11 @@
 	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 >= 483 )
+		if ( yy_current_state >= 504 )
 			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 == 482);
+	yy_is_jam = (yy_current_state == 503);
 
 	(void)yyg;
 	return yy_is_jam ? 0 : yy_current_state;
@@ -3320,7 +3365,7 @@
 
 #define YYTABLES_NAME "yytables"
 
-#line 424 "../SqlLexer.lpp"
+#line 432 "../SqlLexer.lpp"
 
 
 
diff --git a/parser/preprocessed/SqlLexer_gen.hpp b/parser/preprocessed/SqlLexer_gen.hpp
index dccb471..aa184f3 100644
--- a/parser/preprocessed/SqlLexer_gen.hpp
+++ b/parser/preprocessed/SqlLexer_gen.hpp
@@ -360,7 +360,7 @@
 #undef YY_DECL
 #endif
 
-#line 424 "../SqlLexer.lpp"
+#line 432 "../SqlLexer.lpp"
 
 
 #line 367 "SqlLexer_gen.hpp"
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 3de3d51..f7a5765 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -68,7 +68,7 @@
 
 
 /* Copy the first part of user declarations.  */
-#line 33 "../SqlParser.ypp" /* yacc.c:339  */
+#line 35 "../SqlParser.ypp" /* yacc.c:339  */
 
 
 /* Override the default definition, as we only need <first_line> and <first_column>. */
@@ -96,12 +96,13 @@
     }                                                           \
   } while (0)
 
-#line 62 "../SqlParser.ypp" /* yacc.c:339  */
+#line 64 "../SqlParser.ypp" /* yacc.c:339  */
 
 #include <cstdlib>
 #include <string>
 #include <utility>
 
+#include "catalog/PartitionSchemeHeader.hpp"
 #include "parser/ParseAssignment.hpp"
 #include "parser/ParseAttributeDefinition.hpp"
 #include "parser/ParseBasicExpressions.hpp"
@@ -114,6 +115,7 @@
 #include "parser/ParseLimit.hpp"
 #include "parser/ParseLiteralValue.hpp"
 #include "parser/ParseOrderBy.hpp"
+#include "parser/ParsePartitionClause.hpp"
 #include "parser/ParsePredicate.hpp"
 #include "parser/ParserUtil.hpp"
 #include "parser/ParseSample.hpp"
@@ -144,7 +146,7 @@
 // Needed for Bison 2.6 and higher, which do not automatically provide this typedef.
 typedef void* yyscan_t;
 
-#line 148 "SqlParser_gen.cpp" /* yacc.c:339  */
+#line 150 "SqlParser_gen.cpp" /* yacc.c:339  */
 
 # ifndef YY_NULLPTR
 #  if defined __cplusplus && 201103L <= __cplusplus
@@ -228,57 +230,62 @@
     TOKEN_DOUBLE = 304,
     TOKEN_DROP = 305,
     TOKEN_ESCAPE_STRINGS = 306,
-    TOKEN_FALSE = 307,
-    TOKEN_FIRST = 308,
-    TOKEN_FLOAT = 309,
-    TOKEN_FOREIGN = 310,
-    TOKEN_FROM = 311,
-    TOKEN_FULL = 312,
-    TOKEN_GROUP = 313,
-    TOKEN_HAVING = 314,
-    TOKEN_INDEX = 315,
-    TOKEN_INNER = 316,
-    TOKEN_INSERT = 317,
-    TOKEN_INTEGER = 318,
-    TOKEN_INTERVAL = 319,
-    TOKEN_INTO = 320,
-    TOKEN_JOIN = 321,
-    TOKEN_KEY = 322,
-    TOKEN_LAST = 323,
-    TOKEN_LEFT = 324,
-    TOKEN_LIMIT = 325,
-    TOKEN_LONG = 326,
-    TOKEN_NULL = 327,
-    TOKEN_NULLS = 328,
-    TOKEN_OFF = 329,
-    TOKEN_ON = 330,
-    TOKEN_ORDER = 331,
-    TOKEN_OUTER = 332,
-    TOKEN_PERCENT = 333,
-    TOKEN_PRIMARY = 334,
-    TOKEN_QUIT = 335,
-    TOKEN_REAL = 336,
-    TOKEN_REFERENCES = 337,
-    TOKEN_RIGHT = 338,
-    TOKEN_ROW_DELIMITER = 339,
-    TOKEN_SELECT = 340,
-    TOKEN_SET = 341,
-    TOKEN_SMALLINT = 342,
-    TOKEN_TABLE = 343,
-    TOKEN_TIME = 344,
-    TOKEN_TIMESTAMP = 345,
-    TOKEN_TRUE = 346,
-    TOKEN_TUPLESAMPLE = 347,
-    TOKEN_UNIQUE = 348,
-    TOKEN_UPDATE = 349,
-    TOKEN_USING = 350,
-    TOKEN_VALUES = 351,
-    TOKEN_VARCHAR = 352,
-    TOKEN_WHERE = 353,
-    TOKEN_WITH = 354,
-    TOKEN_YEARMONTH = 355,
-    TOKEN_EOF = 356,
-    TOKEN_LEX_ERROR = 357
+    TOKEN_EXTRACT = 307,
+    TOKEN_FALSE = 308,
+    TOKEN_FIRST = 309,
+    TOKEN_FLOAT = 310,
+    TOKEN_FOREIGN = 311,
+    TOKEN_FROM = 312,
+    TOKEN_FULL = 313,
+    TOKEN_GROUP = 314,
+    TOKEN_HASH = 315,
+    TOKEN_HAVING = 316,
+    TOKEN_INDEX = 317,
+    TOKEN_INNER = 318,
+    TOKEN_INSERT = 319,
+    TOKEN_INTEGER = 320,
+    TOKEN_INTERVAL = 321,
+    TOKEN_INTO = 322,
+    TOKEN_JOIN = 323,
+    TOKEN_KEY = 324,
+    TOKEN_LAST = 325,
+    TOKEN_LEFT = 326,
+    TOKEN_LIMIT = 327,
+    TOKEN_LONG = 328,
+    TOKEN_NULL = 329,
+    TOKEN_NULLS = 330,
+    TOKEN_OFF = 331,
+    TOKEN_ON = 332,
+    TOKEN_ORDER = 333,
+    TOKEN_OUTER = 334,
+    TOKEN_PARTITION = 335,
+    TOKEN_PARTITIONS = 336,
+    TOKEN_PERCENT = 337,
+    TOKEN_PRIMARY = 338,
+    TOKEN_QUIT = 339,
+    TOKEN_RANGE = 340,
+    TOKEN_REAL = 341,
+    TOKEN_REFERENCES = 342,
+    TOKEN_RIGHT = 343,
+    TOKEN_ROW_DELIMITER = 344,
+    TOKEN_SELECT = 345,
+    TOKEN_SET = 346,
+    TOKEN_SMALLINT = 347,
+    TOKEN_TABLE = 348,
+    TOKEN_TIME = 349,
+    TOKEN_TIMESTAMP = 350,
+    TOKEN_TRUE = 351,
+    TOKEN_TUPLESAMPLE = 352,
+    TOKEN_UNIQUE = 353,
+    TOKEN_UPDATE = 354,
+    TOKEN_USING = 355,
+    TOKEN_VALUES = 356,
+    TOKEN_VARCHAR = 357,
+    TOKEN_WHERE = 358,
+    TOKEN_WITH = 359,
+    TOKEN_YEARMONTH = 360,
+    TOKEN_EOF = 361,
+    TOKEN_LEX_ERROR = 362
   };
 #endif
 
@@ -287,7 +294,7 @@
 
 union YYSTYPE
 {
-#line 110 "../SqlParser.ypp" /* yacc.c:355  */
+#line 114 "../SqlParser.ypp" /* yacc.c:355  */
 
   quickstep::ParseString *string_value_;
 
@@ -344,6 +351,7 @@
   quickstep::ParseStatementDelete *delete_statement_;
   quickstep::ParseStatementCopyFrom *copy_from_statement_;
   quickstep::ParseStatementCreateTable *create_table_statement_;
+  quickstep::ParsePartitionClause *partition_clause_;
   quickstep::ParseBlockProperties *block_properties_;
   quickstep::ParseStatementDropTable *drop_table_statement_;
   quickstep::ParseStatementQuit *quit_statement_;
@@ -370,7 +378,7 @@
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
 
-#line 374 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 382 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -399,13 +407,13 @@
 #endif /* !YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED  */
 
 /* Copy the second part of user declarations.  */
-#line 193 "../SqlParser.ypp" /* yacc.c:358  */
+#line 198 "../SqlParser.ypp" /* yacc.c:358  */
 
 /* This header needs YYSTYPE, which is defined by the %union directive above */
 #include "SqlLexer_gen.hpp"
 void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string &feature);
 
-#line 409 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 417 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -647,23 +655,23 @@
 #endif /* !YYCOPY_NEEDED */
 
 /* YYFINAL -- State number of the termination state.  */
-#define YYFINAL  42
+#define YYFINAL  47
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   919
+#define YYLAST   982
 
 /* YYNTOKENS -- Number of terminals.  */
-#define YYNTOKENS  114
+#define YYNTOKENS  119
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  86
+#define YYNNTS  89
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  228
+#define YYNRULES  237
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  433
+#define YYNSTATES  460
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
 #define YYUNDEFTOK  2
-#define YYMAXUTOK   357
+#define YYMAXUTOK   362
 
 #define YYTRANSLATE(YYX)                                                \
   ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK)
@@ -673,11 +681,11 @@
 static const yytype_uint8 yytranslate[] =
 {
        0,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-     109,     2,     2,     2,     2,     2,     2,     2,     2,     2,
+     114,     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,   113,     2,     2,
-     110,   111,    23,    21,   112,    22,    27,    24,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,     2,     2,   108,
+       2,     2,     2,     2,     2,     2,     2,   118,     2,     2,
+     115,   116,    23,    21,   117,    22,    27,    24,     2,     2,
+       2,     2,     2,     2,     2,     2,     2,     2,     2,   113,
        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,
@@ -707,36 +715,38 @@
       70,    71,    72,    73,    74,    75,    76,    77,    78,    79,
       80,    81,    82,    83,    84,    85,    86,    87,    88,    89,
       90,    91,    92,    93,    94,    95,    96,    97,    98,    99,
-     100,   101,   102,   103,   104,   105,   106,   107
+     100,   101,   102,   103,   104,   105,   106,   107,   108,   109,
+     110,   111,   112
 };
 
 #if YYDEBUG
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   523,   523,   527,   531,   535,   539,   542,   549,   552,
-     555,   558,   561,   564,   567,   570,   573,   576,   582,   588,
-     595,   601,   608,   617,   622,   631,   636,   641,   645,   651,
-     656,   659,   662,   667,   670,   673,   676,   679,   682,   685,
-     688,   691,   694,   706,   709,   712,   730,   750,   753,   756,
-     761,   766,   772,   778,   787,   791,   797,   800,   805,   810,
-     815,   822,   829,   833,   839,   842,   847,   850,   855,   858,
-     863,   867,   873,   876,   879,   884,   887,   894,   899,   910,
-     914,   920,   923,   929,   937,   942,   947,   950,   955,   959,
-     963,   967,   973,   978,   983,   987,   993,   999,  1004,  1007,
-    1012,  1016,  1022,  1028,  1034,  1037,  1041,  1047,  1050,  1055,
-    1059,  1065,  1068,  1071,  1076,  1081,  1084,  1090,  1094,  1100,
-    1106,  1112,  1118,  1124,  1130,  1136,  1142,  1150,  1155,  1158,
-    1161,  1166,  1170,  1174,  1177,  1181,  1186,  1189,  1194,  1197,
-    1202,  1206,  1212,  1215,  1220,  1223,  1228,  1231,  1236,  1239,
-    1258,  1262,  1268,  1275,  1278,  1281,  1286,  1289,  1292,  1298,
-    1301,  1306,  1311,  1320,  1325,  1334,  1339,  1342,  1347,  1350,
-    1355,  1361,  1367,  1370,  1376,  1379,  1384,  1387,  1392,  1395,
-    1400,  1403,  1406,  1409,  1414,  1418,  1422,  1427,  1431,  1437,
-    1440,  1443,  1446,  1458,  1462,  1481,  1496,  1500,  1506,  1509,
-    1514,  1518,  1525,  1528,  1531,  1534,  1537,  1540,  1543,  1546,
-    1549,  1552,  1557,  1568,  1571,  1576,  1579,  1582,  1588,  1592,
-    1598,  1601,  1609,  1612,  1615,  1618,  1624,  1629,  1634
+       0,   538,   538,   542,   546,   550,   554,   557,   564,   567,
+     570,   573,   576,   579,   582,   585,   588,   591,   597,   603,
+     610,   616,   623,   632,   637,   646,   651,   656,   660,   666,
+     671,   674,   677,   682,   685,   688,   691,   694,   697,   700,
+     703,   706,   709,   721,   724,   727,   745,   765,   768,   771,
+     776,   781,   787,   793,   802,   806,   812,   815,   820,   825,
+     830,   837,   844,   848,   854,   857,   862,   865,   870,   873,
+     878,   881,   900,   904,   910,   914,   920,   923,   926,   931,
+     934,   941,   946,   957,   961,   967,   970,   976,   984,   987,
+     990,   996,  1001,  1004,  1009,  1013,  1017,  1021,  1027,  1032,
+    1037,  1041,  1047,  1053,  1056,  1061,  1066,  1070,  1076,  1082,
+    1088,  1091,  1095,  1101,  1104,  1109,  1113,  1119,  1122,  1125,
+    1130,  1135,  1138,  1144,  1148,  1154,  1160,  1166,  1172,  1178,
+    1184,  1190,  1196,  1204,  1209,  1212,  1215,  1220,  1224,  1228,
+    1231,  1235,  1240,  1243,  1248,  1251,  1256,  1260,  1266,  1269,
+    1274,  1277,  1282,  1285,  1290,  1293,  1312,  1316,  1322,  1329,
+    1332,  1335,  1340,  1343,  1346,  1352,  1355,  1360,  1365,  1374,
+    1379,  1388,  1393,  1396,  1401,  1404,  1409,  1415,  1421,  1424,
+    1430,  1433,  1438,  1441,  1446,  1449,  1454,  1457,  1460,  1463,
+    1466,  1471,  1475,  1479,  1482,  1487,  1492,  1496,  1502,  1505,
+    1508,  1511,  1523,  1527,  1546,  1561,  1565,  1571,  1574,  1579,
+    1583,  1590,  1593,  1596,  1599,  1602,  1605,  1608,  1611,  1614,
+    1617,  1622,  1633,  1636,  1641,  1644,  1647,  1653,  1657,  1663,
+    1666,  1674,  1677,  1680,  1683,  1689,  1694,  1699
 };
 #endif
 
@@ -758,34 +768,35 @@
   "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_ESCAPE_STRINGS", "TOKEN_FALSE", "TOKEN_FIRST", "TOKEN_FLOAT",
-  "TOKEN_FOREIGN", "TOKEN_FROM", "TOKEN_FULL", "TOKEN_GROUP",
-  "TOKEN_HAVING", "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_ON", "TOKEN_ORDER",
-  "TOKEN_OUTER", "TOKEN_PERCENT", "TOKEN_PRIMARY", "TOKEN_QUIT",
-  "TOKEN_REAL", "TOKEN_REFERENCES", "TOKEN_RIGHT", "TOKEN_ROW_DELIMITER",
-  "TOKEN_SELECT", "TOKEN_SET", "TOKEN_SMALLINT", "TOKEN_TABLE",
-  "TOKEN_TIME", "TOKEN_TIMESTAMP", "TOKEN_TRUE", "TOKEN_TUPLESAMPLE",
-  "TOKEN_UNIQUE", "TOKEN_UPDATE", "TOKEN_USING", "TOKEN_VALUES",
-  "TOKEN_VARCHAR", "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_ESCAPE_STRINGS", "TOKEN_EXTRACT", "TOKEN_FALSE", "TOKEN_FIRST",
+  "TOKEN_FLOAT", "TOKEN_FOREIGN", "TOKEN_FROM", "TOKEN_FULL",
+  "TOKEN_GROUP", "TOKEN_HASH", "TOKEN_HAVING", "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_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_SMALLINT",
+  "TOKEN_TABLE", "TOKEN_TIME", "TOKEN_TIMESTAMP", "TOKEN_TRUE",
+  "TOKEN_TUPLESAMPLE", "TOKEN_UNIQUE", "TOKEN_UPDATE", "TOKEN_USING",
+  "TOKEN_VALUES", "TOKEN_VARCHAR", "TOKEN_WHERE", "TOKEN_WITH",
+  "TOKEN_YEARMONTH", "TOKEN_EOF", "TOKEN_LEX_ERROR", "';'", "'\\n'", "'('",
+  "')'", "','", "'%'", "$accept", "start", "sql_statement",
+  "quit_statement", "alter_table_statement", "create_table_statement",
   "create_index_statement", "drop_table_statement", "column_def",
   "column_def_commalist", "data_type", "column_constraint_def",
   "column_constraint_def_list", "opt_column_constraint_def_list",
   "table_constraint_def", "table_constraint_def_commalist",
   "opt_table_constraint_def_commalist", "opt_column_list",
-  "opt_block_properties", "key_value_list", "key_value",
-  "key_string_value", "key_string_list", "key_integer_value", "index_type",
-  "opt_index_properties", "insert_statement", "copy_from_statement",
-  "opt_copy_from_params", "copy_from_params", "update_statement",
-  "delete_statement", "assignment_list", "assignment_item",
-  "select_statement", "opt_with_clause", "with_list", "with_list_element",
-  "select_query", "opt_all_distinct", "selection",
-  "selection_item_commalist", "selection_item", "from_clause",
+  "opt_block_properties", "opt_partition_clause", "partition_type",
+  "key_value_list", "key_value", "key_string_value", "key_string_list",
+  "key_integer_value", "index_type", "opt_index_properties",
+  "insert_statement", "copy_from_statement", "opt_copy_from_params",
+  "copy_from_params", "update_statement", "delete_statement",
+  "assignment_list", "assignment_item", "select_statement", "with_clause",
+  "with_list", "with_list_element", "select_query", "opt_all_distinct",
+  "selection", "selection_item_commalist", "selection_item", "from_clause",
   "opt_join_chain", "join_chain", "join", "subquery_expression",
   "opt_sample_clause", "table_reference", "table_reference_signature",
   "table_reference_signature_primary", "table_reference_commalist",
@@ -795,11 +806,11 @@
   "where_clause", "or_expression", "and_expression", "not_expression",
   "predicate_expression_base", "add_expression", "multiply_expression",
   "unary_expression", "expression_base", "function_call",
-  "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
+  "extract_function", "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
 };
 #endif
 
@@ -818,17 +829,17 @@
      320,   321,   322,   323,   324,   325,   326,   327,   328,   329,
      330,   331,   332,   333,   334,   335,   336,   337,   338,   339,
      340,   341,   342,   343,   344,   345,   346,   347,   348,   349,
-     350,   351,   352,   353,   354,   355,   356,   357,    59,    10,
-      40,    41,    44,    37
+     350,   351,   352,   353,   354,   355,   356,   357,   358,   359,
+     360,   361,   362,    59,    10,    40,    41,    44,    37
 };
 # endif
 
-#define YYPACT_NINF -194
+#define YYPACT_NINF -209
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-194)))
+  (!!((Yystate) == (-209)))
 
-#define YYTABLE_NINF -99
+#define YYTABLE_NINF -1
 
 #define yytable_value_is_error(Yytable_value) \
   0
@@ -837,50 +848,52 @@
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-      40,  -194,  -194,   -55,   206,    -5,    30,    20,    35,  -194,
-     206,   206,  -194,   147,   108,  -194,  -194,  -194,  -194,  -194,
-    -194,  -194,  -194,  -194,  -194,    19,   -69,   157,   206,  -194,
-    -194,   137,   206,   206,   206,   206,   206,   114,    79,  -194,
-     171,    99,  -194,  -194,  -194,    -2,  -194,  -194,  -194,  -194,
-      66,   212,   170,   142,   145,  -194,   -49,   206,   206,   144,
-     206,  -194,  -194,   396,   181,   185,   152,   206,   206,   503,
-    -194,  -194,   148,   206,   -46,  -194,   251,  -194,    19,  -194,
-      42,  -194,  -194,  -194,   260,   274,  -194,  -194,  -194,   174,
-    -194,   219,  -194,  -194,  -194,  -194,   288,  -194,  -194,  -194,
-    -194,  -194,  -194,   186,   229,   569,   294,   239,   189,  -194,
-     182,    -3,  -194,  -194,  -194,  -194,  -194,   635,   -10,   206,
-     -16,   206,   206,   192,  -194,   193,  -194,   109,   801,   701,
-     503,   296,   297,  -194,  -194,   897,   285,   751,   121,   206,
-    -194,   569,   197,  -194,   206,  -194,  -194,   302,  -194,  -194,
-     303,  -194,    12,  -194,    10,   145,   569,  -194,  -194,   206,
-     569,  -194,  -194,  -194,  -194,   569,   274,  -194,   206,   392,
-    -194,   201,   240,   242,   205,  -194,  -194,  -194,    75,   206,
-     217,   -16,   206,  -194,   138,  -194,     1,   155,   503,   503,
-     178,  -194,  -194,  -194,  -194,  -194,  -194,  -194,  -194,   569,
-     569,    16,  -194,   123,   218,  -194,   216,  -194,  -194,   210,
-     213,  -194,    70,  -194,   118,    70,   -18,   262,  -194,  -194,
-      -3,  -194,  -194,   215,  -194,   216,   128,   503,   220,   221,
-     206,   318,   -21,   130,  -194,   133,   301,   209,  -194,   222,
-     231,  -194,   256,   226,   751,  -194,   265,   206,  -194,  -194,
-     138,  -194,  -194,   297,  -194,  -194,  -194,   569,   115,   216,
-     264,  -194,  -194,   751,   232,  -194,  -194,   206,  -194,  -194,
-      -9,   268,   206,    67,   112,    10,  -194,   119,  -194,  -194,
-     336,   337,    70,   307,   283,  -194,  -194,   569,     3,   206,
-     206,   149,  -194,  -194,  -194,  -194,  -194,  -194,  -194,   100,
-    -194,   206,  -194,  -194,   238,   -16,   314,  -194,  -194,   503,
-    -194,  -194,   241,  -194,   136,   569,  -194,  -194,   751,  -194,
-     206,   281,   206,   -41,   206,   287,   206,   292,  -194,  -194,
-     282,   284,  -194,   569,   503,   289,   216,  -194,   153,   158,
-    -194,   359,   -21,  -194,   206,  -194,  -194,   258,    11,   206,
-     569,   216,   161,   -35,   206,   -34,   503,   -32,   206,   -31,
-     206,  -194,  -194,   254,   296,   330,   299,   290,  -194,  -194,
-    -194,   165,  -194,  -194,  -194,  -194,     6,   206,  -194,   267,
-     216,  -194,   503,   -25,   503,   296,   503,   -15,   503,   -13,
-     569,   364,  -194,   206,  -194,   206,  -194,  -194,   206,  -194,
-     168,  -194,   296,   503,   296,   296,   503,   296,   503,   263,
-    -194,   111,  -194,   270,  -194,   179,  -194,   296,   296,   296,
-     569,  -194,  -194,   304,   206,  -194,  -194,    46,  -194,   183,
-    -194,  -194,  -194
+     303,  -209,  -209,    -9,   150,   -10,    65,    46,    37,  -209,
+       7,   150,   150,  -209,   176,    47,  -209,  -209,  -209,  -209,
+    -209,  -209,  -209,  -209,  -209,  -209,    76,  -209,   -21,   178,
+     150,  -209,  -209,   122,   150,   150,   150,   150,   150,  -209,
+    -209,   463,   103,    86,  -209,   204,    94,  -209,  -209,  -209,
+     153,  -209,  -209,  -209,  -209,    84,   222,   147,   135,   144,
+    -209,   145,  -209,  -209,   254,   257,  -209,  -209,  -209,   163,
+    -209,   213,  -209,  -209,   173,  -209,  -209,   285,  -209,  -209,
+    -209,  -209,  -209,  -209,   177,   224,   623,   295,   240,   203,
+    -209,   215,     1,  -209,  -209,  -209,  -209,  -209,  -209,   703,
+      -8,   150,   150,   201,   150,   150,    10,    53,   217,   150,
+     150,   543,  -209,  -209,   212,   150,  -209,  -209,  -209,   321,
+    -209,   150,  -209,   322,  -209,     6,  -209,     5,   144,   623,
+    -209,  -209,   150,   623,  -209,  -209,  -209,  -209,   623,   257,
+    -209,   150,   383,   -14,  -209,   320,  -209,   237,  -209,    73,
+    -209,   237,   150,    22,   150,   150,   219,  -209,   226,  -209,
+     106,   872,   783,   543,   329,   334,  -209,  -209,   943,   324,
+     817,   138,   229,   284,   233,  -209,   143,  -209,    89,   143,
+     -16,   287,  -209,  -209,     1,  -209,  -209,   236,   623,  -209,
+     235,   142,   150,  -209,   623,   241,  -209,   150,  -209,  -209,
+     244,   286,   288,   246,  -209,  -209,  -209,    -1,   150,   258,
+      22,   150,  -209,   136,  -209,     2,    54,   543,   543,   199,
+    -209,  -209,  -209,  -209,  -209,  -209,  -209,  -209,   623,   623,
+      25,  -209,   154,   259,  -209,   623,  -209,   150,  -209,  -209,
+     107,   298,   150,   113,   169,     5,  -209,   137,  -209,  -209,
+     361,   366,   143,   335,   309,  -209,   156,  -209,   623,  -209,
+     235,  -209,  -209,   543,   261,   263,   150,   374,    82,   158,
+    -209,   160,   353,   243,  -209,   264,   273,  -209,   304,   269,
+     817,  -209,   311,   150,  -209,  -209,   136,  -209,  -209,   334,
+    -209,  -209,  -209,   623,   157,   235,   307,  -209,  -209,   817,
+     276,    16,  -209,   150,   323,   150,   -59,   150,   326,   150,
+     327,  -209,  -209,   306,   308,  -209,   623,   543,   314,  -209,
+     235,     8,   150,   150,   166,  -209,  -209,  -209,  -209,  -209,
+    -209,  -209,     0,  -209,   150,  -209,  -209,   279,    22,   368,
+     325,  -209,   543,  -209,  -209,   293,  -209,   223,   623,  -209,
+    -209,   817,  -209,   -43,   150,   -40,   543,   -39,   150,   -38,
+     150,  -209,  -209,   292,   329,   376,   336,  -209,   170,   181,
+    -209,   413,    82,  -209,   150,  -209,  -209,   305,   386,  -209,
+      13,   150,   623,   235,   192,   543,   -37,   543,   329,   543,
+     -35,   543,   -33,   623,   415,  -209,   340,  -209,  -209,  -209,
+     194,  -209,  -209,  -209,  -209,    11,   150,   -36,  -209,   310,
+     235,  -209,   329,   543,   329,   329,   543,   329,   543,   316,
+    -209,   121,  -209,   150,  -209,   150,  -209,  -209,   150,  -209,
+     196,  -209,  -209,   312,  -209,   329,   329,   329,   623,  -209,
+    -209,   348,   319,  -209,   198,  -209,   150,  -209,    33,  -209,
+     150,  -209,   206,  -209,  -209,   208,   349,  -209,   431,  -209
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -888,351 +901,370 @@
      means the default is an error.  */
 static const yytype_uint8 yydefact[] =
 {
-       0,     6,   228,     0,     0,     0,     0,     0,     0,    18,
-       0,     0,     7,     0,     0,    15,     8,    10,    11,    13,
-      14,     9,    17,    12,    16,     0,     0,   226,     0,   220,
-     221,     0,     0,     0,     0,     0,     0,     0,    99,   100,
-       0,   138,     1,     3,     2,   104,    97,     5,     4,   227,
-       0,     0,     0,     0,   159,    25,     0,     0,     0,     0,
-       0,   105,   106,     0,     0,     0,    86,     0,     0,     0,
-      93,   160,     0,     0,   159,    95,     0,   101,     0,   102,
-       0,   218,   193,   190,     0,   212,   107,    40,    29,     0,
-      30,    31,    34,    36,    37,    39,     0,    41,   189,    35,
-      38,    32,    33,     0,     0,     0,     0,     0,   108,   109,
-     113,   175,   177,   179,   182,   181,   180,     0,   198,     0,
-       0,     0,     0,     0,    85,    66,    27,     0,     0,     0,
-       0,   161,   163,   165,   167,     0,   180,     0,     0,     0,
-      92,     0,     0,   139,     0,   191,   192,     0,    43,   194,
-       0,    44,     0,   195,     0,   159,     0,   213,   214,     0,
-       0,   112,   216,   217,   215,     0,     0,   178,     0,     0,
-      19,     0,     0,     0,     0,    20,    21,    22,     0,     0,
-       0,    64,     0,    42,    56,   166,     0,     0,     0,     0,
-       0,   202,   204,   205,   206,   207,   203,   208,   210,     0,
-       0,     0,   196,     0,     0,    94,    96,   127,   219,     0,
-       0,   183,     0,   140,   115,   135,   128,   142,   110,   111,
-     174,   176,   199,     0,   184,   187,     0,     0,     0,     0,
-       0,     0,     0,     0,   200,     0,   198,     0,    63,    65,
-      68,    28,     0,     0,     0,    47,     0,     0,    49,    55,
-      57,    26,   173,   162,   164,   209,   211,     0,     0,   172,
-       0,   171,    84,     0,     0,    45,    46,     0,   131,   136,
-       0,     0,     0,     0,     0,     0,   114,   116,   118,   134,
-       0,     0,   133,     0,   144,   185,   186,     0,     0,     0,
-       0,     0,    88,   224,   225,   223,   222,    89,    87,     0,
-      67,     0,    79,    80,    81,     0,     0,    23,    48,     0,
-      51,    50,     0,    54,     0,     0,   170,   197,     0,   137,
-       0,     0,     0,     0,     0,     0,     0,     0,   141,   117,
-       0,     0,   132,     0,     0,   146,   188,    61,     0,     0,
-      58,     0,     0,   201,     0,    24,    62,     0,     0,     0,
-       0,   168,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,   129,   130,   143,   145,     0,   148,     0,    59,    90,
-      91,     0,    70,    72,    73,    74,     0,     0,    52,     0,
-     169,    83,     0,     0,     0,   120,     0,     0,     0,     0,
-       0,     0,   103,     0,    82,     0,    78,    76,     0,    75,
-       0,    53,   126,     0,   119,   122,     0,   124,     0,   147,
-     150,   153,   149,     0,    71,     0,    69,   125,   121,   123,
-       0,   154,   155,   156,     0,    77,   151,     0,   152,     0,
-     157,   158,    60
+       0,     6,   237,     0,     0,     0,     0,     0,     0,    18,
+     110,     0,     0,     7,     0,     0,    15,     8,    10,    11,
+      13,    14,     9,    17,    12,    16,     0,   103,     0,   235,
+       0,   229,   230,     0,     0,     0,     0,     0,     0,   111,
+     112,     0,     0,   105,   106,     0,   144,     1,     3,     2,
+       0,   104,     5,     4,   236,     0,     0,     0,     0,   165,
+      25,     0,   202,   199,     0,   221,   113,    40,    29,     0,
+      30,    31,    34,    36,     0,    37,    39,     0,    41,   198,
+      35,    38,    32,    33,     0,     0,     0,     0,     0,   114,
+     115,   119,   181,   183,   185,   188,   189,   187,   186,     0,
+     207,     0,     0,     0,     0,     0,     0,     0,    92,     0,
+       0,     0,    99,   166,     0,     0,    89,   200,   201,     0,
+      43,     0,   203,     0,    44,     0,   204,     0,   165,     0,
+     222,   223,     0,     0,   118,   225,   226,   224,     0,     0,
+     184,     0,     0,   165,   101,     0,   107,     0,   108,     0,
+     227,     0,     0,     0,     0,     0,     0,    91,    66,    27,
+       0,     0,     0,     0,   167,   169,   171,   173,     0,   186,
+       0,     0,     0,     0,     0,   190,     0,   146,   121,   141,
+     134,   148,   116,   117,   180,   182,   208,     0,     0,   191,
+     196,     0,     0,    98,     0,     0,   145,     0,    90,    19,
+       0,     0,     0,     0,    20,    21,    22,     0,     0,     0,
+      64,     0,    42,    56,   172,     0,     0,     0,     0,     0,
+     211,   213,   214,   215,   216,   212,   217,   219,     0,     0,
+       0,   205,     0,     0,    45,     0,    46,     0,   137,   142,
+       0,     0,     0,     0,     0,     0,   120,   122,   124,   140,
+       0,     0,   139,     0,   150,   192,     0,   193,     0,   100,
+     102,   133,   228,     0,     0,     0,     0,     0,     0,     0,
+     209,     0,   207,     0,    63,    65,    68,    28,     0,     0,
+       0,    47,     0,     0,    49,    55,    57,    26,   179,   168,
+     170,   218,   220,     0,     0,   178,     0,   177,    88,     0,
+       0,     0,   143,     0,     0,     0,     0,     0,     0,     0,
+       0,   147,   123,     0,     0,   138,     0,     0,   152,   194,
+     197,     0,     0,     0,     0,    94,   233,   234,   232,   231,
+      95,    93,     0,    67,     0,    83,    84,    85,     0,     0,
+      70,    48,     0,    51,    50,     0,    54,     0,     0,   176,
+     206,     0,   195,     0,     0,     0,     0,     0,     0,     0,
+       0,   135,   136,   149,   151,     0,   154,    61,     0,     0,
+      58,     0,     0,   210,     0,    24,    62,     0,     0,    23,
+       0,     0,     0,   174,     0,     0,     0,     0,   126,     0,
+       0,     0,     0,     0,     0,   109,     0,    59,    96,    97,
+       0,    74,    76,    77,    78,     0,     0,     0,    52,     0,
+     175,    87,   132,     0,   125,   128,     0,   130,     0,   153,
+     156,   159,   155,     0,    86,     0,    82,    80,     0,    79,
+       0,    72,    73,     0,    53,   131,   127,   129,     0,   160,
+     161,   162,     0,    75,     0,    69,     0,   157,     0,   158,
+       0,    81,     0,   163,   164,     0,     0,    60,     0,    71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -194,  -194,  -194,  -194,  -194,  -194,  -194,  -194,   -96,  -194,
-     255,   134,  -194,  -194,  -177,  -194,  -194,  -194,  -194,     8,
-       0,  -194,  -194,  -194,  -194,  -194,  -194,  -194,  -194,  -194,
-    -194,  -194,  -194,   248,  -194,  -194,  -194,   335,   326,  -194,
-    -194,  -194,   249,  -194,  -194,  -194,   129,   349,  -194,   141,
-    -193,    -8,  -194,  -194,  -194,  -194,  -194,  -194,   -11,  -194,
-    -194,   -59,  -194,     4,   223,   233,   295,   -58,   261,   266,
-     306,  -146,    94,  -136,   110,   -27,  -194,  -194,  -194,  -194,
-    -194,   -71,    -4,    91,  -194,  -194
+    -209,  -209,  -209,  -209,  -209,  -209,  -209,  -209,  -130,  -209,
+     278,   155,  -209,  -209,  -208,  -209,  -209,  -209,  -209,  -209,
+    -209,    38,    20,  -209,  -209,  -209,  -209,  -209,  -209,  -209,
+    -209,  -209,  -209,  -209,  -209,   255,  -209,  -209,  -209,   346,
+     -13,  -209,  -209,  -209,   328,  -209,  -209,  -209,   202,   347,
+    -209,   207,  -167,   -11,  -209,  -209,  -209,  -209,  -209,  -209,
+      17,  -209,  -209,   -82,  -209,  -122,   234,   238,   296,   -27,
+     330,   333,   360,  -124,  -209,  -183,  -164,   109,   -49,  -209,
+    -209,  -209,  -209,  -209,  -111,    -4,    92,  -209,  -209
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
-      -1,    13,    14,    15,    16,    17,    18,    19,   126,   127,
-     106,   249,   250,   251,   175,   239,   240,   180,   307,   371,
-     372,   373,   374,   375,   304,   345,    20,    21,   124,   233,
-      22,    23,    74,    75,    24,    25,    38,    39,    46,    63,
-     107,   108,   109,   155,   276,   277,   278,   212,   282,   213,
-     268,   269,   214,   284,   335,   366,   392,   409,   410,   423,
-     428,    70,    71,   131,   132,   133,   134,   135,   111,   112,
-     113,   114,   226,   115,   203,   116,   235,   200,   117,   160,
-     165,    80,   118,   297,    26,    27
+      -1,    14,    15,    16,    17,    18,    19,    20,   159,   160,
+      87,   285,   286,   287,   204,   275,   276,   209,   340,   379,
+     433,   400,   401,   402,   403,   404,   337,   375,    21,    22,
+     157,   269,    23,    24,   143,   144,    25,    26,    43,    44,
+      27,    41,    88,    89,    90,   128,   246,   247,   248,   176,
+     252,   177,   238,   239,   178,   254,   318,   366,   395,   419,
+     420,   441,   449,   112,   113,   164,   165,   166,   167,   168,
+      92,    93,    94,    95,    96,   191,    97,   232,    98,   271,
+     229,    99,   133,   138,   149,   100,   330,    28,    29
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
      positive, shift that token.  If negative, reduce the rule whose
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
-static const yytype_int16 yytable[] =
+static const yytype_uint16 yytable[] =
 {
-      31,   202,   138,    40,   238,   110,    37,    41,   215,   188,
-      29,   188,    30,   396,    29,   140,    30,   168,   280,   188,
-     162,   163,   279,   170,    50,   171,   260,    61,    52,    53,
-      54,    55,    56,   157,   158,   397,   293,    47,    28,   356,
-      48,     1,   136,     2,   172,   382,   384,   152,   386,   388,
-      40,    62,    72,    76,    41,   403,    81,    69,   294,   295,
-      32,    73,   320,   125,   128,   406,   139,   408,   173,    81,
-       3,   144,   187,   321,    29,   296,    30,   144,   144,   281,
-     144,   144,   174,   206,     4,     5,   241,   144,    33,   332,
-       6,    34,   169,   261,    64,     7,   217,   144,   110,   144,
-     169,   267,   136,   136,   430,    36,   161,     8,   310,    45,
-     164,   225,   252,    35,   337,   128,   398,   176,   177,   431,
-      78,    65,   378,   211,   315,     9,   231,   317,   346,   215,
-     -98,   232,   157,   158,   186,    76,   157,   158,   324,    10,
-     208,   258,   259,   421,    11,   350,    12,    42,   242,   325,
-     216,   341,   234,   143,   144,   219,   342,   157,   158,   291,
-      49,   136,   136,   422,   222,   190,   191,   192,   193,   194,
-     195,   196,   197,   198,   199,   236,   157,   158,   128,   243,
-     270,   270,   202,   326,   271,   271,    29,   244,    30,   272,
-     272,    58,   273,   273,   327,   255,   256,   257,    51,   314,
-     136,   323,    59,   157,   158,    57,   274,   274,    41,    60,
-      29,    41,    30,   159,    43,   245,    44,    66,   338,   339,
-     181,   182,   246,   119,   120,   247,    81,   121,   122,   336,
-     275,   288,   204,   144,   262,   263,   248,   157,   158,   286,
-     287,   298,   299,   312,   300,   301,   302,   303,    69,   353,
-      67,   355,    68,   357,    78,   359,   123,   351,   137,   319,
-     340,   144,   141,    41,   367,   144,   211,   145,    81,   368,
-     144,   216,   381,   263,   343,   225,   394,   395,    41,   416,
-     395,   146,   136,   383,   147,    81,    81,   387,   148,   389,
-     425,   144,   380,   149,   432,   144,   150,   236,   151,   153,
-     154,   156,   178,   179,   188,   201,   189,   136,   207,   209,
-     210,   227,   228,   348,   229,   230,    81,   237,    81,   264,
-      81,   265,    81,   292,   266,   283,   285,   415,   168,   136,
-     289,   290,   411,   308,   305,   306,   309,   311,   364,   322,
-     376,   316,   318,   330,   331,   379,   333,   334,   344,   347,
-      81,   349,   354,   429,    81,   136,    81,   136,   358,   136,
-     385,   136,   411,   360,   369,   361,   287,   362,   377,   390,
-     365,   412,   399,   376,   391,   420,   136,   393,   401,   136,
-     424,   136,   427,   184,   313,   400,   402,   205,   404,   413,
-     405,   376,   407,    77,    81,   414,    29,    82,    30,    83,
-      29,    82,    30,    83,   142,   218,   329,   417,    79,   426,
-     418,   253,   419,    84,    85,   223,   328,    84,    85,    86,
-      81,   220,   254,   167,   185,    87,    88,   363,   352,    87,
-      88,   221,    89,   370,     0,     0,    89,     0,    90,    91,
-      92,     0,    90,    91,    92,     0,    93,     0,     0,     0,
-      93,    94,     0,     0,     0,    94,     0,     0,     0,     0,
-      95,    96,     0,     0,    95,    96,     0,     0,    97,    98,
-       0,     0,    97,    98,     0,     0,     0,     0,    99,     0,
-       0,     0,    99,     0,   100,     0,   101,   102,   100,     0,
-     101,   102,     0,     0,   103,     0,     0,   104,   103,     0,
-       0,   104,   105,   224,     0,     0,   105,    29,    82,    30,
-      83,     0,     0,   129,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    84,    85,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,    87,    88,     0,     0,
-       0,     0,     0,    89,     0,     0,     0,     0,     0,    90,
-      91,    92,     0,     0,     0,     0,     0,    93,     0,     0,
-       0,     0,    94,     0,     0,     0,     0,     0,     0,     0,
-       0,    95,    96,    29,    82,    30,    83,     0,     0,    97,
-      98,     0,     0,     0,     0,     0,     0,     0,     0,    99,
-      84,    85,     0,     0,     0,   100,     0,   101,   102,     0,
-       0,     0,    87,    88,     0,   103,     0,     0,   104,    89,
-       0,     0,     0,   130,     0,    90,    91,    92,     0,     0,
-       0,     0,     0,    93,     0,     0,     0,     0,    94,     0,
-       0,     0,     0,     0,     0,     0,     0,    95,    96,    29,
-      82,    30,    83,     0,     0,    97,    98,     0,     0,     0,
-       0,     0,     0,     0,     0,    99,    84,   166,     0,     0,
-       0,   100,     0,   101,   102,     0,     0,     0,    87,    88,
-       0,   103,     0,     0,   104,    89,     0,     0,     0,   105,
-       0,    90,    91,    92,     0,     0,     0,     0,     0,    93,
-       0,     0,     0,     0,    94,     0,     0,     0,     0,     0,
-       0,     0,     0,    95,    96,    29,    82,    30,    83,     0,
-       0,    97,    98,     0,     0,     0,     0,     0,     0,     0,
-       0,    99,    84,    85,     0,     0,     0,   100,     0,   101,
-     102,     0,     0,     0,    87,    88,     0,   103,     0,     0,
-     104,    89,     0,     0,     0,   105,     0,    90,    91,    92,
-       0,     0,     0,     0,     0,    93,    82,     0,    83,     0,
-      94,     0,     0,     0,     0,     0,     0,     0,     0,    95,
-      96,     0,    84,   166,     0,     0,     0,    97,    98,     0,
-       0,     0,     0,     0,    87,    88,     0,    99,     0,     0,
-       0,    89,     0,   100,     0,   101,   102,    90,    91,    92,
-       0,     0,     0,   103,     0,    93,   104,     0,     0,     0,
-      94,   130,     0,     0,     0,     0,     0,     0,     0,    95,
-      96,     0,     0,     0,     0,     0,     0,    97,    98,     0,
-       0,     0,     0,     0,    87,    88,     0,    99,     0,     0,
-       0,    89,     0,   100,     0,   101,   102,    90,    91,    92,
-       0,     0,     0,   103,     0,    93,   104,     0,     0,     0,
-      94,     0,     0,     0,     0,     0,     0,     0,     0,    95,
-     183,     0,     0,     0,     0,     0,     0,    97,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    99,     0,     0,
-       0,     0,     0,   100,     0,   101,   102,     0,     0,     0,
-       0,     0,     0,   103,     0,     0,   104,   190,   191,   192,
-     193,   194,   195,   196,   197,   198,   199,     0,   157,   158
+      33,    45,   274,   179,   171,   256,   231,    42,    46,    31,
+     217,    32,   249,    51,    91,    31,   217,    32,   426,   141,
+     250,   217,   199,   356,   135,   136,    55,   130,   131,   431,
+      57,    58,    59,    60,    61,   296,    39,   130,   131,   385,
+     427,   215,   387,   389,   391,   413,   181,   416,   116,   418,
+     267,   371,   152,   153,   432,   268,   372,    34,   197,   125,
+      40,   193,   169,   200,   219,   220,   221,   222,   223,   224,
+     225,   226,   227,   228,   197,   130,   131,   197,   197,   197,
+     197,   277,   197,   201,   197,   315,   251,   134,    35,    30,
+      52,    45,   453,    53,   111,   154,   155,   145,    46,   142,
+     150,   151,    91,   192,   297,   158,   161,   142,   454,    38,
+     202,   150,   106,   169,   169,   190,   343,   173,   288,   137,
+     147,   179,   175,   180,   367,   203,   428,    36,   183,   408,
+     376,   306,   352,   363,   195,   350,   216,   186,   198,   107,
+     326,   321,   130,   131,    37,    50,   278,    31,   161,    32,
+     205,   206,   240,   439,    31,   324,    32,   241,    48,   270,
+      49,   190,   242,   327,   328,   243,   348,   260,   169,   169,
+     175,    10,    46,   440,   237,    46,    47,   279,   130,   131,
+     303,    54,   244,   329,    56,   280,   307,   231,   145,   196,
+     197,   304,   353,   262,   355,   364,   357,   308,   359,   101,
+     240,   294,   295,   102,   272,   241,   245,   161,   301,   104,
+     242,   368,   369,   243,   169,   281,   291,   292,   293,    31,
+     380,    32,   210,   211,   282,   105,   302,   108,   283,   109,
+     244,   320,   382,    46,   388,   103,   130,   131,   150,   284,
+      10,   180,   309,   386,   130,   131,   132,   390,    46,   392,
+     110,   114,   111,   310,   233,   197,   130,   131,   257,   258,
+     115,   117,   150,   412,   118,   414,   347,   415,   169,   417,
+     298,   299,   319,   258,   331,   332,   333,   334,   119,   345,
+     335,   336,   370,   197,   120,   373,   396,   197,   121,   190,
+     122,   435,   123,   169,   436,   124,   437,   397,   197,   150,
+     126,   150,   127,   150,     1,   150,     2,   169,   411,   299,
+     424,   425,   445,   425,   451,   197,   147,   444,   150,   150,
+     129,   383,   456,   197,   457,   197,   156,   170,   172,   174,
+     272,   194,    10,     3,   207,   452,   169,   217,   169,   455,
+     169,   208,   169,   218,   230,   234,   235,     4,     5,   236,
+     150,   253,   255,     6,   150,   410,   150,   261,     7,   263,
+     264,   266,   265,   273,   169,   300,   421,   169,   313,   169,
+     405,   305,     8,   314,   316,   317,   322,   409,   323,   325,
+     141,   338,   339,   341,   342,   344,   349,    31,    62,    32,
+      63,   351,     9,   361,   374,   362,   354,   365,    10,   358,
+     360,   429,   405,   377,    64,    65,   187,    11,   381,   258,
+     378,   421,    12,   394,    13,   393,    67,    68,   398,   442,
+     406,   405,   422,    69,   150,   407,   434,   446,   448,    70,
+      71,    72,   423,   438,   450,   458,   188,    73,   459,   213,
+      74,   346,   150,    75,   430,   443,   150,   259,   146,   312,
+     148,   289,   311,    76,    77,   447,   290,   182,   214,   140,
+     384,    78,    79,   184,   399,     0,     0,    31,    62,    32,
+      63,   185,     0,     0,    80,     0,     0,     0,     0,     0,
+      81,     0,    82,    83,    64,    65,    66,     0,     0,     0,
+      84,     0,     0,    85,     0,     0,    67,    68,    86,   189,
+       0,     0,     0,    69,     0,     0,     0,     0,     0,    70,
+      71,    72,     0,     0,     0,     0,     0,    73,     0,     0,
+      74,     0,     0,    75,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    76,    77,     0,     0,     0,     0,     0,
+       0,    78,    79,     0,     0,     0,     0,    31,    62,    32,
+      63,     0,     0,   162,    80,     0,     0,     0,     0,     0,
+      81,     0,    82,    83,    64,    65,     0,     0,     0,     0,
+      84,     0,     0,    85,     0,     0,    67,    68,    86,     0,
+       0,     0,     0,    69,     0,     0,     0,     0,     0,    70,
+      71,    72,     0,     0,     0,     0,     0,    73,     0,     0,
+      74,     0,     0,    75,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    76,    77,     0,     0,     0,     0,     0,
+       0,    78,    79,     0,     0,     0,     0,    31,    62,    32,
+      63,     0,     0,     0,    80,     0,     0,     0,     0,     0,
+      81,     0,    82,    83,    64,    65,     0,     0,     0,     0,
+      84,     0,     0,    85,     0,     0,    67,    68,   163,     0,
+       0,     0,     0,    69,     0,     0,     0,     0,     0,    70,
+      71,    72,     0,     0,     0,     0,     0,    73,     0,     0,
+      74,     0,     0,    75,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    76,    77,     0,     0,     0,     0,     0,
+       0,    78,    79,     0,     0,     0,     0,    31,    62,    32,
+      63,     0,     0,     0,    80,     0,     0,     0,     0,     0,
+      81,     0,    82,    83,    64,   139,     0,     0,     0,     0,
+      84,     0,     0,    85,     0,     0,    67,    68,    86,     0,
+       0,     0,     0,    69,     0,     0,     0,     0,     0,    70,
+      71,    72,     0,     0,     0,     0,     0,    73,     0,     0,
+      74,     0,     0,    75,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    76,    77,     0,     0,     0,     0,     0,
+       0,    78,    79,     0,     0,     0,     0,    31,    62,    32,
+      63,     0,     0,     0,    80,     0,     0,     0,     0,     0,
+      81,     0,    82,    83,    64,    65,     0,     0,     0,     0,
+      84,     0,     0,    85,     0,     0,    67,    68,    86,     0,
+       0,     0,    62,    69,    63,     0,     0,     0,     0,    70,
+      71,    72,     0,     0,     0,     0,     0,    73,    64,   139,
+      74,     0,     0,    75,     0,     0,     0,     0,     0,     0,
+      67,    68,     0,    76,    77,     0,     0,    69,     0,     0,
+       0,    78,    79,    70,    71,    72,     0,     0,     0,     0,
+       0,    73,     0,     0,    80,     0,     0,    75,     0,     0,
+      81,     0,    82,    83,     0,     0,     0,    76,    77,     0,
+      84,     0,     0,    85,     0,    78,    79,     0,   163,     0,
+       0,     0,     0,     0,     0,    67,    68,     0,    80,     0,
+       0,     0,    69,     0,    81,     0,    82,    83,    70,    71,
+      72,     0,     0,     0,    84,     0,    73,    85,     0,     0,
+       0,     0,    75,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,    76,   212,     0,     0,     0,     0,     0,     0,
+      78,     0,     0,   219,   220,   221,   222,   223,   224,   225,
+     226,   227,   228,    80,   130,   131,     0,     0,     0,    81,
+       0,    82,    83,     0,     0,     0,     0,     0,     0,    84,
+       0,     0,    85
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   137,    73,    11,   181,    63,    10,    11,   154,     8,
-       4,     8,     6,     7,     4,    74,     6,    27,    36,     8,
-      23,    24,   215,   119,    28,    41,    10,    29,    32,    33,
-      34,    35,    36,    21,    22,    29,    57,   106,    93,    80,
-     109,     1,    69,     3,    60,    80,    80,   105,    80,    80,
-      58,    53,   101,    57,    58,    80,    60,   103,    79,    80,
-      65,   110,    71,    67,    68,    80,   112,    80,    84,    73,
-      30,   112,   130,    82,     4,    96,     6,   112,   112,    97,
-     112,   112,    98,   141,    44,    45,   182,   112,    93,   282,
-      50,    61,   110,    77,    28,    55,   155,   112,   156,   112,
-     110,    31,   129,   130,    58,    70,   110,    67,   244,    90,
-     113,   169,   111,    93,   111,   119,   110,   121,   122,    73,
-     110,    55,   111,   111,     9,    85,    51,   263,   305,   275,
-      90,    56,    21,    22,   130,   139,    21,    22,    71,    99,
-     144,   199,   200,    32,   104,     9,   106,     0,    10,    82,
-     154,    51,   179,   111,   112,   159,    56,    21,    22,   230,
-       3,   188,   189,    52,   168,    10,    11,    12,    13,    14,
-      15,    16,    17,    18,    19,   179,    21,    22,   182,    41,
-      62,    62,   318,    71,    66,    66,     4,    49,     6,    71,
-      71,   112,    74,    74,    82,    17,    18,    19,    61,   257,
-     227,   272,    31,    21,    22,    91,    88,    88,   212,   110,
-       4,   215,     6,    31,   106,    77,   108,     5,   289,   290,
-     111,   112,    84,    42,    43,    87,   230,    42,    43,   287,
-     112,   227,   111,   112,   111,   112,    98,    21,    22,   111,
-     112,   111,   112,   247,   111,   112,    37,    38,   103,   320,
-      80,   322,   110,   324,   110,   326,   104,   315,   110,   267,
-     111,   112,    11,   267,   111,   112,   111,     7,   272,   111,
-     112,   275,   111,   112,   301,   333,   111,   112,   282,   111,
-     112,     7,   309,   354,   110,   289,   290,   358,    69,   360,
-     111,   112,   350,     5,   111,   112,   110,   301,    69,     5,
-      61,   112,   110,   110,     8,    20,     9,   334,   111,     7,
-       7,   110,    72,   309,    72,   110,   320,   100,   322,   101,
-     324,   111,   326,     5,   111,    63,   111,   398,    27,   356,
-     110,   110,   390,    77,   112,   104,   110,    72,   334,    71,
-     344,    77,   110,     7,     7,   349,    39,    64,   110,    35,
-     354,   110,    71,   424,   358,   382,   360,   384,    71,   386,
-     356,   388,   420,    71,     5,    83,   112,    83,   110,    39,
-      81,     7,   376,   377,    75,   112,   403,    87,   111,   406,
-     110,   408,    78,   128,   250,   377,   382,   139,   384,   393,
-     386,   395,   388,    58,   398,   395,     4,     5,     6,     7,
-       4,     5,     6,     7,    78,   156,   277,   403,    59,   420,
-     406,   188,   408,    21,    22,    23,   275,    21,    22,    23,
-     424,   160,   189,   117,   129,    33,    34,   333,   318,    33,
-      34,   165,    40,   342,    -1,    -1,    40,    -1,    46,    47,
-      48,    -1,    46,    47,    48,    -1,    54,    -1,    -1,    -1,
-      54,    59,    -1,    -1,    -1,    59,    -1,    -1,    -1,    -1,
-      68,    69,    -1,    -1,    68,    69,    -1,    -1,    76,    77,
-      -1,    -1,    76,    77,    -1,    -1,    -1,    -1,    86,    -1,
-      -1,    -1,    86,    -1,    92,    -1,    94,    95,    92,    -1,
-      94,    95,    -1,    -1,   102,    -1,    -1,   105,   102,    -1,
-      -1,   105,   110,   111,    -1,    -1,   110,     4,     5,     6,
-       7,    -1,    -1,    10,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    21,    22,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    33,    34,    -1,    -1,
+       4,    12,   210,   127,   115,   188,   170,    11,    12,     4,
+       8,     6,   179,    26,    41,     4,     8,     6,     7,    27,
+      36,     8,   152,    82,    23,    24,    30,    21,    22,    65,
+      34,    35,    36,    37,    38,    10,    29,    21,    22,    82,
+      29,   163,    82,    82,    82,    82,   128,    82,    61,    82,
+      51,    51,    42,    43,    90,    56,    56,    67,   117,    86,
+      53,   143,   111,    41,    10,    11,    12,    13,    14,    15,
+      16,    17,    18,    19,   117,    21,    22,   117,   117,   117,
+     117,   211,   117,    61,   117,   252,   102,    91,    98,    98,
+     111,   102,    59,   114,   108,    42,    43,   101,   102,   115,
+     104,   105,   129,   117,    79,   109,   110,   115,    75,    72,
+      88,   115,    28,   162,   163,   142,   280,   121,   116,   118,
+     115,   245,   116,   127,   116,   103,   115,    62,   132,   116,
+     338,   242,   116,   316,   147,   299,   163,   141,   151,    55,
+      58,   263,    21,    22,    98,    69,    10,     4,   152,     6,
+     154,   155,    63,    32,     4,   266,     6,    68,   111,   208,
+     113,   188,    73,    81,    82,    76,     9,   194,   217,   218,
+     116,    95,   176,    52,    31,   179,     0,    41,    21,    22,
+      73,     3,    93,   101,    62,    49,    73,   351,   192,   116,
+     117,    84,   303,   197,   305,   317,   307,    84,   309,    96,
+      63,   228,   229,   117,   208,    68,   117,   211,   235,   115,
+      73,   322,   323,    76,   263,    79,    17,    18,    19,     4,
+     342,     6,   116,   117,    88,    72,   237,     5,    92,    82,
+      93,   258,     9,   237,   356,    31,    21,    22,   242,   103,
+      95,   245,    73,   354,    21,    22,    31,   358,   252,   360,
+     115,   106,   108,    84,   116,   117,    21,    22,   116,   117,
+     115,     7,   266,   385,     7,   387,   293,   389,   317,   391,
+     116,   117,   116,   117,   116,   117,   116,   117,   115,   283,
+      37,    38,   116,   117,    71,   334,   116,   117,   115,   316,
+       5,   413,   115,   342,   416,    71,   418,   116,   117,   303,
+       5,   305,    62,   307,     1,   309,     3,   356,   116,   117,
+     116,   117,   116,   117,   116,   117,   115,   428,   322,   323,
+     117,   348,   116,   117,   116,   117,   109,   115,     7,     7,
+     334,    11,    95,    30,   115,   446,   385,     8,   387,   450,
+     389,   115,   391,     9,    20,   116,    62,    44,    45,   116,
+     354,    64,   116,    50,   358,   382,   360,   116,    55,   115,
+      74,   115,    74,   105,   413,   106,   393,   416,     7,   418,
+     374,    73,    69,     7,    39,    66,   115,   381,   115,     5,
+      27,   117,   109,    79,   115,    74,    79,     4,     5,     6,
+       7,   115,    89,    87,   115,    87,    73,    83,    95,    73,
+      73,   405,   406,    35,    21,    22,    23,   104,   115,   117,
+      85,   438,   109,    77,   111,    39,    33,    34,     5,   423,
+     115,   425,     7,    40,   428,    39,   116,   115,    80,    46,
+      47,    48,    92,   117,   115,    86,    53,    54,     7,   161,
+      57,   286,   446,    60,   406,   425,   450,   192,   102,   247,
+     103,   217,   245,    70,    71,   438,   218,   129,   162,    99,
+     351,    78,    79,   133,   372,    -1,    -1,     4,     5,     6,
+       7,   138,    -1,    -1,    91,    -1,    -1,    -1,    -1,    -1,
+      97,    -1,    99,   100,    21,    22,    23,    -1,    -1,    -1,
+     107,    -1,    -1,   110,    -1,    -1,    33,    34,   115,   116,
       -1,    -1,    -1,    40,    -1,    -1,    -1,    -1,    -1,    46,
       47,    48,    -1,    -1,    -1,    -1,    -1,    54,    -1,    -1,
-      -1,    -1,    59,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    68,    69,     4,     5,     6,     7,    -1,    -1,    76,
-      77,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    86,
-      21,    22,    -1,    -1,    -1,    92,    -1,    94,    95,    -1,
-      -1,    -1,    33,    34,    -1,   102,    -1,    -1,   105,    40,
-      -1,    -1,    -1,   110,    -1,    46,    47,    48,    -1,    -1,
-      -1,    -1,    -1,    54,    -1,    -1,    -1,    -1,    59,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    68,    69,     4,
-       5,     6,     7,    -1,    -1,    76,    77,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    86,    21,    22,    -1,    -1,
-      -1,    92,    -1,    94,    95,    -1,    -1,    -1,    33,    34,
-      -1,   102,    -1,    -1,   105,    40,    -1,    -1,    -1,   110,
-      -1,    46,    47,    48,    -1,    -1,    -1,    -1,    -1,    54,
-      -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    68,    69,     4,     5,     6,     7,    -1,
-      -1,    76,    77,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    86,    21,    22,    -1,    -1,    -1,    92,    -1,    94,
-      95,    -1,    -1,    -1,    33,    34,    -1,   102,    -1,    -1,
-     105,    40,    -1,    -1,    -1,   110,    -1,    46,    47,    48,
-      -1,    -1,    -1,    -1,    -1,    54,     5,    -1,     7,    -1,
-      59,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    68,
-      69,    -1,    21,    22,    -1,    -1,    -1,    76,    77,    -1,
-      -1,    -1,    -1,    -1,    33,    34,    -1,    86,    -1,    -1,
-      -1,    40,    -1,    92,    -1,    94,    95,    46,    47,    48,
-      -1,    -1,    -1,   102,    -1,    54,   105,    -1,    -1,    -1,
-      59,   110,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    68,
-      69,    -1,    -1,    -1,    -1,    -1,    -1,    76,    77,    -1,
-      -1,    -1,    -1,    -1,    33,    34,    -1,    86,    -1,    -1,
-      -1,    40,    -1,    92,    -1,    94,    95,    46,    47,    48,
-      -1,    -1,    -1,   102,    -1,    54,   105,    -1,    -1,    -1,
-      59,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    68,
-      69,    -1,    -1,    -1,    -1,    -1,    -1,    76,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    86,    -1,    -1,
-      -1,    -1,    -1,    92,    -1,    94,    95,    -1,    -1,    -1,
-      -1,    -1,    -1,   102,    -1,    -1,   105,    10,    11,    12,
-      13,    14,    15,    16,    17,    18,    19,    -1,    21,    22
+      57,    -1,    -1,    60,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    70,    71,    -1,    -1,    -1,    -1,    -1,
+      -1,    78,    79,    -1,    -1,    -1,    -1,     4,     5,     6,
+       7,    -1,    -1,    10,    91,    -1,    -1,    -1,    -1,    -1,
+      97,    -1,    99,   100,    21,    22,    -1,    -1,    -1,    -1,
+     107,    -1,    -1,   110,    -1,    -1,    33,    34,   115,    -1,
+      -1,    -1,    -1,    40,    -1,    -1,    -1,    -1,    -1,    46,
+      47,    48,    -1,    -1,    -1,    -1,    -1,    54,    -1,    -1,
+      57,    -1,    -1,    60,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    70,    71,    -1,    -1,    -1,    -1,    -1,
+      -1,    78,    79,    -1,    -1,    -1,    -1,     4,     5,     6,
+       7,    -1,    -1,    -1,    91,    -1,    -1,    -1,    -1,    -1,
+      97,    -1,    99,   100,    21,    22,    -1,    -1,    -1,    -1,
+     107,    -1,    -1,   110,    -1,    -1,    33,    34,   115,    -1,
+      -1,    -1,    -1,    40,    -1,    -1,    -1,    -1,    -1,    46,
+      47,    48,    -1,    -1,    -1,    -1,    -1,    54,    -1,    -1,
+      57,    -1,    -1,    60,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    70,    71,    -1,    -1,    -1,    -1,    -1,
+      -1,    78,    79,    -1,    -1,    -1,    -1,     4,     5,     6,
+       7,    -1,    -1,    -1,    91,    -1,    -1,    -1,    -1,    -1,
+      97,    -1,    99,   100,    21,    22,    -1,    -1,    -1,    -1,
+     107,    -1,    -1,   110,    -1,    -1,    33,    34,   115,    -1,
+      -1,    -1,    -1,    40,    -1,    -1,    -1,    -1,    -1,    46,
+      47,    48,    -1,    -1,    -1,    -1,    -1,    54,    -1,    -1,
+      57,    -1,    -1,    60,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    70,    71,    -1,    -1,    -1,    -1,    -1,
+      -1,    78,    79,    -1,    -1,    -1,    -1,     4,     5,     6,
+       7,    -1,    -1,    -1,    91,    -1,    -1,    -1,    -1,    -1,
+      97,    -1,    99,   100,    21,    22,    -1,    -1,    -1,    -1,
+     107,    -1,    -1,   110,    -1,    -1,    33,    34,   115,    -1,
+      -1,    -1,     5,    40,     7,    -1,    -1,    -1,    -1,    46,
+      47,    48,    -1,    -1,    -1,    -1,    -1,    54,    21,    22,
+      57,    -1,    -1,    60,    -1,    -1,    -1,    -1,    -1,    -1,
+      33,    34,    -1,    70,    71,    -1,    -1,    40,    -1,    -1,
+      -1,    78,    79,    46,    47,    48,    -1,    -1,    -1,    -1,
+      -1,    54,    -1,    -1,    91,    -1,    -1,    60,    -1,    -1,
+      97,    -1,    99,   100,    -1,    -1,    -1,    70,    71,    -1,
+     107,    -1,    -1,   110,    -1,    78,    79,    -1,   115,    -1,
+      -1,    -1,    -1,    -1,    -1,    33,    34,    -1,    91,    -1,
+      -1,    -1,    40,    -1,    97,    -1,    99,   100,    46,    47,
+      48,    -1,    -1,    -1,   107,    -1,    54,   110,    -1,    -1,
+      -1,    -1,    60,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    70,    71,    -1,    -1,    -1,    -1,    -1,    -1,
+      78,    -1,    -1,    10,    11,    12,    13,    14,    15,    16,
+      17,    18,    19,    91,    21,    22,    -1,    -1,    -1,    97,
+      -1,    99,   100,    -1,    -1,    -1,    -1,    -1,    -1,   107,
+      -1,    -1,   110
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
      symbol of state STATE-NUM.  */
 static const yytype_uint8 yystos[] =
 {
-       0,     1,     3,    30,    44,    45,    50,    55,    67,    85,
-      99,   104,   106,   115,   116,   117,   118,   119,   120,   121,
-     140,   141,   144,   145,   148,   149,   198,   199,    93,     4,
-       6,   196,    65,    93,    61,    93,    70,   196,   150,   151,
-     165,   196,     0,   106,   108,    90,   152,   106,   109,     3,
-     196,    61,   196,   196,   196,   196,   196,    91,   112,    31,
-     110,    29,    53,   153,    28,    55,     5,    80,   110,   103,
-     175,   176,   101,   110,   146,   147,   196,   151,   110,   161,
-     195,   196,     5,     7,    21,    22,    23,    33,    34,    40,
-      46,    47,    48,    54,    59,    68,    69,    76,    77,    86,
-      92,    94,    95,   102,   105,   110,   124,   154,   155,   156,
-     181,   182,   183,   184,   185,   187,   189,   192,   196,    42,
-      43,    42,    43,   104,   142,   196,   122,   123,   196,    10,
-     110,   177,   178,   179,   180,   181,   189,   110,   195,   112,
-     175,    11,   152,   111,   112,     7,     7,   110,    69,     5,
-     110,    69,   181,     5,    61,   157,   112,    21,    22,    31,
-     193,   196,    23,    24,   113,   194,    22,   184,    27,   110,
-     122,    41,    60,    84,    98,   128,   196,   196,   110,   110,
-     131,   111,   112,    69,   124,   180,   177,   181,     8,     9,
-      10,    11,    12,    13,    14,    15,    16,    17,    18,    19,
-     191,    20,   187,   188,   111,   147,   181,   111,   196,     7,
-       7,   111,   161,   163,   166,   185,   196,   175,   156,   196,
-     182,   183,   196,    23,   111,   181,   186,   110,    72,    72,
-     110,    51,    56,   143,   189,   190,   196,   100,   128,   129,
-     130,   122,    10,    41,    49,    77,    84,    87,    98,   125,
-     126,   127,   111,   178,   179,    17,    18,    19,   181,   181,
-      10,    77,   111,   112,   101,   111,   111,    31,   164,   165,
-      62,    66,    71,    74,    88,   112,   158,   159,   160,   164,
-      36,    97,   162,    63,   167,   111,   111,   112,   177,   110,
-     110,   195,     5,    57,    79,    80,    96,   197,   111,   112,
-     111,   112,    37,    38,   138,   112,   104,   132,    77,   110,
-     187,    72,   196,   125,   181,     9,    77,   187,   110,   165,
-      71,    82,    71,   195,    71,    82,    71,    82,   163,   160,
-       7,     7,   164,    39,    64,   168,   181,   111,   195,   195,
-     111,    51,    56,   189,   110,   139,   128,    35,   177,   110,
-       9,   181,   188,   195,    71,   195,    80,   195,    71,   195,
-      71,    83,    83,   186,   177,    81,   169,   111,   111,     5,
-     197,   133,   134,   135,   136,   137,   196,   110,   111,   196,
-     181,   111,    80,   195,    80,   177,    80,   195,    80,   195,
-      39,    75,   170,    87,   111,   112,     7,    29,   110,   196,
-     133,   111,   177,    80,   177,   177,    80,   177,    80,   171,
-     172,   181,     7,   196,   134,   195,   111,   177,   177,   177,
-     112,    32,    52,   173,   110,   111,   172,    78,   174,   195,
-      58,    73,   111
+       0,     1,     3,    30,    44,    45,    50,    55,    69,    89,
+      95,   104,   109,   111,   120,   121,   122,   123,   124,   125,
+     126,   147,   148,   151,   152,   155,   156,   159,   206,   207,
+      98,     4,     6,   204,    67,    98,    62,    98,    72,    29,
+      53,   160,   204,   157,   158,   172,   204,     0,   111,   113,
+      69,   159,   111,   114,     3,   204,    62,   204,   204,   204,
+     204,   204,     5,     7,    21,    22,    23,    33,    34,    40,
+      46,    47,    48,    54,    57,    60,    70,    71,    78,    79,
+      91,    97,    99,   100,   107,   110,   115,   129,   161,   162,
+     163,   188,   189,   190,   191,   192,   193,   195,   197,   200,
+     204,    96,   117,    31,   115,    72,    28,    55,     5,    82,
+     115,   108,   182,   183,   106,   115,   159,     7,     7,   115,
+      71,   115,     5,   115,    71,   188,     5,    62,   164,   117,
+      21,    22,    31,   201,   204,    23,    24,   118,   202,    22,
+     191,    27,   115,   153,   154,   204,   158,   115,   168,   203,
+     204,   204,    42,    43,    42,    43,   109,   149,   204,   127,
+     128,   204,    10,   115,   184,   185,   186,   187,   188,   197,
+     115,   203,     7,   204,     7,   116,   168,   170,   173,   192,
+     204,   182,   163,   204,   189,   190,   204,    23,    53,   116,
+     188,   194,   117,   182,    11,   159,   116,   117,   159,   127,
+      41,    61,    88,   103,   133,   204,   204,   115,   115,   136,
+     116,   117,    71,   129,   187,   184,   188,     8,     9,    10,
+      11,    12,    13,    14,    15,    16,    17,    18,    19,   199,
+      20,   195,   196,   116,   116,    62,   116,    31,   171,   172,
+      63,    68,    73,    76,    93,   117,   165,   166,   167,   171,
+      36,   102,   169,    64,   174,   116,   194,   116,   117,   154,
+     188,   116,   204,   115,    74,    74,   115,    51,    56,   150,
+     197,   198,   204,   105,   133,   134,   135,   127,    10,    41,
+      49,    79,    88,    92,   103,   130,   131,   132,   116,   185,
+     186,    17,    18,    19,   188,   188,    10,    79,   116,   117,
+     106,   188,   172,    73,    84,    73,   203,    73,    84,    73,
+      84,   170,   167,     7,     7,   171,    39,    66,   175,   116,
+     188,   184,   115,   115,   203,     5,    58,    81,    82,   101,
+     205,   116,   117,   116,   117,    37,    38,   145,   117,   109,
+     137,    79,   115,   195,    74,   204,   130,   188,     9,    79,
+     195,   115,   116,   203,    73,   203,    82,   203,    73,   203,
+      73,    87,    87,   194,   184,    83,   176,   116,   203,   203,
+     116,    51,    56,   197,   115,   146,   133,    35,    85,   138,
+     184,   115,     9,   188,   196,    82,   203,    82,   184,    82,
+     203,    82,   203,    39,    77,   177,   116,   116,     5,   205,
+     140,   141,   142,   143,   144,   204,   115,    39,   116,   204,
+     188,   116,   184,    82,   184,   184,    82,   184,    82,   178,
+     179,   188,     7,    92,   116,   117,     7,    29,   115,   204,
+     140,    65,    90,   139,   116,   184,   184,   184,   117,    32,
+      52,   180,   204,   141,   203,   116,   115,   179,    80,   181,
+     115,   116,   203,    59,    75,   203,   116,   116,    86,     7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
 static const yytype_uint8 yyr1[] =
 {
-       0,   114,   115,   115,   115,   115,   115,   115,   116,   116,
-     116,   116,   116,   116,   116,   116,   116,   116,   117,   118,
-     118,   118,   118,   119,   120,   121,   122,   123,   123,   124,
-     124,   124,   124,   124,   124,   124,   124,   124,   124,   124,
-     124,   124,   124,   124,   124,   124,   124,   125,   125,   125,
-     125,   125,   125,   125,   126,   126,   127,   127,   128,   128,
-     128,   128,   129,   129,   130,   130,   131,   131,   132,   132,
-     133,   133,   134,   134,   134,   135,   135,   136,   137,   138,
-     138,   139,   139,   140,   140,   141,   142,   142,   143,   143,
-     143,   143,   144,   145,   146,   146,   147,   148,   149,   149,
-     150,   150,   151,   152,   153,   153,   153,   154,   154,   155,
-     155,   156,   156,   156,   157,   158,   158,   159,   159,   160,
-     160,   160,   160,   160,   160,   160,   160,   161,   162,   162,
-     162,   163,   163,   163,   163,   163,   164,   164,   165,   165,
-     166,   166,   167,   167,   168,   168,   169,   169,   170,   170,
-     171,   171,   172,   173,   173,   173,   174,   174,   174,   175,
-     175,   176,   177,   177,   178,   178,   179,   179,   180,   180,
-     180,   180,   180,   180,   181,   181,   182,   182,   183,   183,
-     184,   184,   184,   184,   185,   185,   185,   186,   186,   187,
-     187,   187,   187,   187,   187,   187,   188,   188,   189,   189,
-     190,   190,   191,   191,   191,   191,   191,   191,   191,   191,
-     191,   191,   192,   193,   193,   194,   194,   194,   195,   195,
-     196,   196,   197,   197,   197,   197,   198,   199,   199
+       0,   119,   120,   120,   120,   120,   120,   120,   121,   121,
+     121,   121,   121,   121,   121,   121,   121,   121,   122,   123,
+     123,   123,   123,   124,   125,   126,   127,   128,   128,   129,
+     129,   129,   129,   129,   129,   129,   129,   129,   129,   129,
+     129,   129,   129,   129,   129,   129,   129,   130,   130,   130,
+     130,   130,   130,   130,   131,   131,   132,   132,   133,   133,
+     133,   133,   134,   134,   135,   135,   136,   136,   137,   137,
+     138,   138,   139,   139,   140,   140,   141,   141,   141,   142,
+     142,   143,   144,   145,   145,   146,   146,   147,   147,   147,
+     147,   148,   149,   149,   150,   150,   150,   150,   151,   152,
+     153,   153,   154,   155,   155,   156,   157,   157,   158,   159,
+     160,   160,   160,   161,   161,   162,   162,   163,   163,   163,
+     164,   165,   165,   166,   166,   167,   167,   167,   167,   167,
+     167,   167,   167,   168,   169,   169,   169,   170,   170,   170,
+     170,   170,   171,   171,   172,   172,   173,   173,   174,   174,
+     175,   175,   176,   176,   177,   177,   178,   178,   179,   180,
+     180,   180,   181,   181,   181,   182,   182,   183,   184,   184,
+     185,   185,   186,   186,   187,   187,   187,   187,   187,   187,
+     188,   188,   189,   189,   190,   190,   191,   191,   191,   191,
+     191,   192,   192,   192,   192,   193,   194,   194,   195,   195,
+     195,   195,   195,   195,   195,   196,   196,   197,   197,   198,
+     198,   199,   199,   199,   199,   199,   199,   199,   199,   199,
+     199,   200,   201,   201,   202,   202,   202,   203,   203,   204,
+     204,   205,   205,   205,   205,   206,   207,   207
 };
 
   /* YYR2[YYN] -- Number of symbols on the right hand side of rule YYN.  */
@@ -1240,27 +1272,28 @@
 {
        0,     2,     2,     2,     2,     2,     1,     1,     1,     1,
        1,     1,     1,     1,     1,     1,     1,     1,     1,     6,
-       6,     6,     6,     8,     9,     3,     3,     1,     3,     1,
+       6,     6,     6,     9,     9,     3,     3,     1,     3,     1,
        1,     1,     1,     1,     1,     1,     1,     1,     1,     1,
        1,     1,     1,     2,     2,     4,     4,     1,     2,     1,
        2,     2,     4,     5,     2,     1,     0,     1,     4,     5,
       10,     4,     3,     1,     0,     1,     0,     3,     0,     5,
-       1,     3,     1,     1,     1,     2,     2,     4,     2,     1,
-       1,     0,     3,    10,     7,     5,     0,     4,     2,     2,
-       4,     4,     5,     4,     3,     1,     3,     2,     0,     2,
-       1,     3,     3,     9,     0,     1,     1,     1,     1,     1,
-       3,     3,     2,     1,     3,     0,     1,     2,     1,     5,
-       4,     6,     5,     6,     5,     6,     5,     3,     0,     3,
-       3,     2,     3,     2,     2,     1,     1,     2,     1,     4,
-       1,     3,     0,     3,     0,     2,     0,     3,     0,     2,
-       1,     3,     3,     0,     1,     1,     0,     2,     2,     0,
-       1,     2,     3,     1,     3,     1,     2,     1,     5,     6,
-       4,     3,     3,     3,     3,     1,     3,     1,     2,     1,
-       1,     1,     1,     3,     3,     4,     4,     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
+       0,     8,     1,     1,     1,     3,     1,     1,     1,     2,
+       2,     4,     2,     1,     1,     0,     3,    10,     7,     4,
+       5,     5,     0,     4,     2,     2,     4,     4,     5,     4,
+       3,     1,     3,     1,     2,     2,     1,     3,     3,     9,
+       0,     1,     1,     1,     1,     1,     3,     3,     2,     1,
+       3,     0,     1,     2,     1,     5,     4,     6,     5,     6,
+       5,     6,     5,     3,     0,     3,     3,     2,     3,     2,
+       2,     1,     1,     2,     1,     4,     1,     3,     0,     3,
+       0,     2,     0,     3,     0,     2,     1,     3,     3,     0,
+       1,     1,     0,     2,     2,     0,     1,     2,     3,     1,
+       3,     1,     2,     1,     5,     6,     4,     3,     3,     3,
+       3,     1,     3,     1,     2,     1,     1,     1,     1,     1,
+       3,     3,     4,     4,     5,     6,     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
 };
 
 
@@ -1757,803 +1790,833 @@
   switch (yytype)
     {
           case 3: /* TOKEN_COMMAND  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1767 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1800 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 4: /* TOKEN_NAME  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1777 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1810 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 5: /* TOKEN_STRING_SINGLE_QUOTED  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1787 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1820 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 6: /* TOKEN_STRING_DOUBLE_QUOTED  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1797 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1830 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 7: /* TOKEN_UNSIGNED_NUMVAL  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).numeric_literal_value_) != nullptr) {
     delete ((*yyvaluep).numeric_literal_value_);
   }
 }
-#line 1807 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1840 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 116: /* sql_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 121: /* sql_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 1817 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1850 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 117: /* quit_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 122: /* quit_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).quit_statement_) != nullptr) {
     delete ((*yyvaluep).quit_statement_);
   }
 }
-#line 1827 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1860 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 118: /* alter_table_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 123: /* alter_table_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 1837 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1870 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 119: /* create_table_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 124: /* create_table_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).create_table_statement_) != nullptr) {
     delete ((*yyvaluep).create_table_statement_);
   }
 }
-#line 1847 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1880 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 120: /* create_index_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 125: /* create_index_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).statement_) != nullptr) {
     delete ((*yyvaluep).statement_);
   }
 }
-#line 1857 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1890 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 121: /* drop_table_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 126: /* drop_table_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).drop_table_statement_) != nullptr) {
     delete ((*yyvaluep).drop_table_statement_);
   }
 }
-#line 1867 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1900 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 122: /* column_def  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 127: /* column_def  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_definition_) != nullptr) {
     delete ((*yyvaluep).attribute_definition_);
   }
 }
-#line 1877 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1910 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 123: /* column_def_commalist  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 128: /* column_def_commalist  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_definition_list_) != nullptr) {
     delete ((*yyvaluep).attribute_definition_list_);
   }
 }
-#line 1887 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1920 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 124: /* data_type  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 129: /* data_type  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).data_type_) != nullptr) {
     delete ((*yyvaluep).data_type_);
   }
 }
-#line 1897 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1930 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 125: /* column_constraint_def  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 130: /* column_constraint_def  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_) != nullptr) {
     delete ((*yyvaluep).column_constraint_);
   }
 }
-#line 1907 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1940 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 126: /* column_constraint_def_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 131: /* column_constraint_def_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_list_) != nullptr) {
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 1917 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1950 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 127: /* opt_column_constraint_def_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 132: /* opt_column_constraint_def_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).column_constraint_list_) != nullptr) {
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 1927 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1960 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 131: /* opt_column_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 136: /* opt_column_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_list_) != nullptr) {
     delete ((*yyvaluep).attribute_list_);
   }
 }
-#line 1937 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1970 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 132: /* opt_block_properties  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 137: /* opt_block_properties  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).block_properties_) != nullptr) {
     delete ((*yyvaluep).block_properties_);
   }
 }
-#line 1947 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1980 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 133: /* key_value_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 138: /* opt_partition_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
+      {
+  if (((*yyvaluep).partition_clause_) != nullptr) {
+    delete ((*yyvaluep).partition_clause_);
+  }
+}
+#line 1990 "SqlParser_gen.cpp" /* yacc.c:1257  */
+        break;
+
+    case 139: /* partition_type  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
+      {
+  if (((*yyvaluep).string_value_) != nullptr) {
+    delete ((*yyvaluep).string_value_);
+  }
+}
+#line 2000 "SqlParser_gen.cpp" /* yacc.c:1257  */
+        break;
+
+    case 140: /* key_value_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 1957 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2010 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 134: /* key_value  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 141: /* key_value  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_) != nullptr) {
     delete ((*yyvaluep).key_value_);
   }
 }
-#line 1967 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2020 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 135: /* key_string_value  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 142: /* key_string_value  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_string_value_) != nullptr) {
     delete ((*yyvaluep).key_string_value_);
   }
 }
-#line 1977 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2030 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 136: /* key_string_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 143: /* key_string_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_string_list_) != nullptr) {
     delete ((*yyvaluep).key_string_list_);
   }
 }
-#line 1987 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2040 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 137: /* key_integer_value  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 144: /* key_integer_value  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_integer_value_) != nullptr) {
     delete ((*yyvaluep).key_integer_value_);
   }
 }
-#line 1997 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2050 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 138: /* index_type  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 145: /* index_type  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2007 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2060 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 139: /* opt_index_properties  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 146: /* opt_index_properties  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).key_value_list_) != nullptr) {
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2017 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2070 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 140: /* insert_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 147: /* insert_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).insert_statement_) != nullptr) {
     delete ((*yyvaluep).insert_statement_);
   }
 }
-#line 2027 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2080 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 141: /* copy_from_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 148: /* copy_from_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).copy_from_statement_) != nullptr) {
     delete ((*yyvaluep).copy_from_statement_);
   }
 }
-#line 2037 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2090 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 142: /* opt_copy_from_params  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 149: /* opt_copy_from_params  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).copy_from_params_) != nullptr) {
     delete ((*yyvaluep).copy_from_params_);
   }
 }
-#line 2047 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2100 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 143: /* copy_from_params  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 150: /* copy_from_params  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).copy_from_params_) != nullptr) {
     delete ((*yyvaluep).copy_from_params_);
   }
 }
-#line 2057 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2110 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 144: /* update_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 151: /* update_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).update_statement_) != nullptr) {
     delete ((*yyvaluep).update_statement_);
   }
 }
-#line 2067 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2120 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 145: /* delete_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 152: /* delete_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).delete_statement_) != nullptr) {
     delete ((*yyvaluep).delete_statement_);
   }
 }
-#line 2077 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2130 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 146: /* assignment_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 153: /* assignment_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).assignment_list_) != nullptr) {
     delete ((*yyvaluep).assignment_list_);
   }
 }
-#line 2087 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2140 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 147: /* assignment_item  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 154: /* assignment_item  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).assignment_) != nullptr) {
     delete ((*yyvaluep).assignment_);
   }
 }
-#line 2097 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2150 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 148: /* select_statement  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 155: /* select_statement  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).select_statement_) != nullptr) {
     delete ((*yyvaluep).select_statement_);
   }
 }
-#line 2107 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2160 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 149: /* opt_with_clause  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 156: /* with_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).with_list_) != nullptr) {
     delete ((*yyvaluep).with_list_);
   }
 }
-#line 2117 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2170 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 150: /* with_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 157: /* with_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).with_list_) != nullptr) {
     delete ((*yyvaluep).with_list_);
   }
 }
-#line 2127 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2180 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 151: /* with_list_element  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 158: /* with_list_element  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).with_list_element_) != nullptr) {
     delete ((*yyvaluep).with_list_element_);
   }
 }
-#line 2137 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2190 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 152: /* select_query  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 159: /* select_query  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).select_query_) != nullptr) {
     delete ((*yyvaluep).select_query_);
   }
 }
-#line 2147 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2200 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 154: /* selection  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 161: /* selection  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).selection_) != nullptr) {
     delete ((*yyvaluep).selection_);
   }
 }
-#line 2157 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2210 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 155: /* selection_item_commalist  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 162: /* selection_item_commalist  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).selection_list_) != nullptr) {
     delete ((*yyvaluep).selection_list_);
   }
 }
-#line 2167 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2220 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 156: /* selection_item  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 163: /* selection_item  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).selection_item_) != nullptr) {
     delete ((*yyvaluep).selection_item_);
   }
 }
-#line 2177 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2230 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 157: /* from_clause  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 164: /* from_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).table_reference_list_) != nullptr) {
     delete ((*yyvaluep).table_reference_list_);
   }
 }
-#line 2187 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2240 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 161: /* subquery_expression  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 168: /* subquery_expression  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).subquery_expression_) != nullptr) {
     delete ((*yyvaluep).subquery_expression_);
   }
 }
-#line 2197 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2250 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 162: /* opt_sample_clause  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 169: /* opt_sample_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).opt_sample_clause_) != nullptr) {
     delete ((*yyvaluep).opt_sample_clause_);
   }
 }
-#line 2207 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2260 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 163: /* table_reference  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 170: /* table_reference  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).table_reference_) != nullptr) {
     delete ((*yyvaluep).table_reference_);
   }
 }
-#line 2217 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2270 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 164: /* table_reference_signature  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 171: /* table_reference_signature  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).table_reference_signature_) != nullptr) {
     delete ((*yyvaluep).table_reference_signature_);
   }
 }
-#line 2227 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2280 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 165: /* table_reference_signature_primary  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 172: /* table_reference_signature_primary  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).table_reference_signature_) != nullptr) {
     delete ((*yyvaluep).table_reference_signature_);
   }
 }
-#line 2237 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2290 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 166: /* table_reference_commalist  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 173: /* table_reference_commalist  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).table_reference_list_) != nullptr) {
     delete ((*yyvaluep).table_reference_list_);
   }
 }
-#line 2247 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2300 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 167: /* opt_group_by_clause  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 174: /* opt_group_by_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).opt_group_by_clause_) != nullptr) {
     delete ((*yyvaluep).opt_group_by_clause_);
   }
 }
-#line 2257 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2310 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 168: /* opt_having_clause  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 175: /* opt_having_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).opt_having_clause_) != nullptr) {
     delete ((*yyvaluep).opt_having_clause_);
   }
 }
-#line 2267 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2320 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 169: /* opt_order_by_clause  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 176: /* opt_order_by_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).opt_order_by_clause_) != nullptr) {
     delete ((*yyvaluep).opt_order_by_clause_);
   }
 }
-#line 2277 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2330 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 170: /* opt_limit_clause  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 177: /* opt_limit_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).opt_limit_clause_) != nullptr) {
     delete ((*yyvaluep).opt_limit_clause_);
   }
 }
-#line 2287 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2340 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 171: /* order_commalist  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 178: /* order_commalist  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).order_commalist_) != nullptr) {
     delete ((*yyvaluep).order_commalist_);
   }
 }
-#line 2297 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2350 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 172: /* order_item  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 179: /* order_item  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).order_item_) != nullptr) {
     delete ((*yyvaluep).order_item_);
   }
 }
-#line 2307 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2360 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 173: /* opt_order_direction  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 180: /* opt_order_direction  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).order_direction_) != nullptr) {
     delete ((*yyvaluep).order_direction_);
   }
 }
-#line 2317 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2370 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 174: /* opt_nulls_first  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 181: /* opt_nulls_first  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).order_direction_) != nullptr) {
     delete ((*yyvaluep).order_direction_);
   }
 }
-#line 2327 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2380 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 175: /* opt_where_clause  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 182: /* opt_where_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).predicate_) != nullptr) {
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2337 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2390 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 176: /* where_clause  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 183: /* where_clause  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).predicate_) != nullptr) {
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2347 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2400 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 177: /* or_expression  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 184: /* or_expression  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).predicate_) != nullptr) {
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2357 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2410 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 178: /* and_expression  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 185: /* and_expression  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).predicate_) != nullptr) {
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2367 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2420 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 179: /* not_expression  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 186: /* not_expression  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).predicate_) != nullptr) {
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2377 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2430 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 180: /* predicate_expression_base  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 187: /* predicate_expression_base  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).predicate_) != nullptr) {
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2387 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2440 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 181: /* add_expression  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 188: /* add_expression  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).expression_) != nullptr) {
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2397 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2450 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 182: /* multiply_expression  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 189: /* multiply_expression  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).expression_) != nullptr) {
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2407 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2460 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 183: /* unary_expression  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 190: /* unary_expression  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).expression_) != nullptr) {
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2417 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2470 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 184: /* expression_base  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 191: /* expression_base  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).expression_) != nullptr) {
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2427 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2480 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 185: /* function_call  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 192: /* function_call  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).function_call_) != nullptr) {
     delete ((*yyvaluep).function_call_);
   }
 }
-#line 2437 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2490 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 186: /* expression_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 193: /* extract_function  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
+      {
+  if (((*yyvaluep).expression_) != nullptr) {
+    delete ((*yyvaluep).expression_);
+  }
+}
+#line 2500 "SqlParser_gen.cpp" /* yacc.c:1257  */
+        break;
+
+    case 194: /* expression_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).expression_list_) != nullptr) {
     delete ((*yyvaluep).expression_list_);
   }
 }
-#line 2447 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2510 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 187: /* literal_value  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 195: /* literal_value  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).literal_value_) != nullptr) {
     delete ((*yyvaluep).literal_value_);
   }
 }
-#line 2457 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2520 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 188: /* literal_value_commalist  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 196: /* literal_value_commalist  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).literal_value_list_) != nullptr) {
     delete ((*yyvaluep).literal_value_list_);
   }
 }
-#line 2467 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2530 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 189: /* attribute_ref  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 197: /* attribute_ref  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_) != nullptr) {
     delete ((*yyvaluep).attribute_);
   }
 }
-#line 2477 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2540 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 190: /* attribute_ref_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 198: /* attribute_ref_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).attribute_list_) != nullptr) {
     delete ((*yyvaluep).attribute_list_);
   }
 }
-#line 2487 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2550 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 191: /* comparison_operation  */
-#line 510 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 199: /* comparison_operation  */
+#line 525 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2493 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2556 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 192: /* unary_operation  */
-#line 511 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 200: /* unary_operation  */
+#line 526 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2499 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2562 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 193: /* add_operation  */
-#line 512 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 201: /* add_operation  */
+#line 527 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2505 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2568 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 194: /* multiply_operation  */
-#line 512 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 202: /* multiply_operation  */
+#line 527 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2511 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2574 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 195: /* name_commalist  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 203: /* name_commalist  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_list_) != nullptr) {
     delete ((*yyvaluep).string_list_);
   }
 }
-#line 2521 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2584 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 196: /* any_name  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 204: /* any_name  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).string_value_) != nullptr) {
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2531 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2594 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 197: /* boolean_value  */
-#line 509 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 205: /* boolean_value  */
+#line 524 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2537 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2600 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 198: /* command  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 206: /* command  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).command_) != nullptr) {
     delete ((*yyvaluep).command_);
   }
 }
-#line 2547 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2610 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
-    case 199: /* command_argument_list  */
-#line 514 "../SqlParser.ypp" /* yacc.c:1257  */
+    case 207: /* command_argument_list  */
+#line 529 "../SqlParser.ypp" /* yacc.c:1257  */
       {
   if (((*yyvaluep).command_argument_list_) != nullptr) {
     delete ((*yyvaluep).command_argument_list_);
   }
 }
-#line 2557 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2620 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
 
@@ -2845,148 +2908,148 @@
   switch (yyn)
     {
         case 2:
-#line 523 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 538 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].statement_);
     YYACCEPT;
   }
-#line 2854 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2917 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 3:
-#line 527 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 542 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].statement_);
     YYACCEPT;
   }
-#line 2863 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2926 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 4:
-#line 531 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 546 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].command_);
     YYACCEPT;
   }
-#line 2872 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2935 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 5:
-#line 535 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 550 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].command_);
     YYACCEPT;
   }
-#line 2881 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2944 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 6:
-#line 539 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 554 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     YYABORT;
   }
-#line 2889 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2952 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 7:
-#line 542 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 557 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     // Regular yyparse() return codes are non-negative, so use a negative one here.
     return -1;
   }
-#line 2898 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2961 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 8:
-#line 549 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 564 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].statement_);
   }
-#line 2906 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2969 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 9:
-#line 552 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 567 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].copy_from_statement_);
   }
-#line 2914 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2977 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 10:
-#line 555 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 570 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].create_table_statement_);
   }
-#line 2922 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2985 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 11:
-#line 558 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 573 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].statement_);
   }
-#line 2930 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 2993 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 12:
-#line 561 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 576 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].delete_statement_);
   }
-#line 2938 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3001 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 13:
-#line 564 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 579 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].drop_table_statement_);
   }
-#line 2946 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3009 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 14:
-#line 567 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 582 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].insert_statement_);
   }
-#line 2954 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3017 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 15:
-#line 570 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 585 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].quit_statement_);
   }
-#line 2962 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3025 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 16:
-#line 573 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 588 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].select_statement_);
   }
-#line 2970 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3033 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 17:
-#line 576 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 591 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].update_statement_);
   }
-#line 2978 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3041 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 18:
-#line 582 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 597 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.quit_statement_) = new quickstep::ParseStatementQuit((yylsp[0]).first_line, (yylsp[0]).first_column);
   }
-#line 2986 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3049 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 19:
-#line 588 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 603 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     delete (yyvsp[0].attribute_definition_);
@@ -2994,22 +3057,22 @@
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 2998 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3061 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 20:
-#line 595 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 610 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     (yyval.statement_) = nullptr;
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3009 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3072 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 21:
-#line 601 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 616 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     delete (yyvsp[0].string_value_);
@@ -3017,11 +3080,11 @@
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3021 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3084 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 22:
-#line 608 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 623 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     delete (yyvsp[0].string_value_);
@@ -3029,19 +3092,19 @@
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3033 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3096 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 23:
-#line 617 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 632 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.create_table_statement_) = new quickstep::ParseStatementCreateTable((yylsp[-7]).first_line, (yylsp[-7]).first_column, (yyvsp[-5].string_value_), (yyvsp[-3].attribute_definition_list_), (yyvsp[0].block_properties_));
+    (yyval.create_table_statement_) = new quickstep::ParseStatementCreateTable((yylsp[-8]).first_line, (yylsp[-8]).first_column, (yyvsp[-6].string_value_), (yyvsp[-4].attribute_definition_list_), (yyvsp[-1].block_properties_), (yyvsp[0].partition_clause_));
   }
-#line 3041 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3104 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 24:
-#line 622 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 637 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].key_value_list_)) {
       (yyval.statement_) = new quickstep::ParseStatementCreateIndex((yylsp[-8]).first_line, (yylsp[-8]).first_column, (yyvsp[-6].string_value_), (yyvsp[-4].string_value_), (yyvsp[-3].attribute_list_), (yyvsp[-1].string_value_), (yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].key_value_list_));
@@ -3049,153 +3112,153 @@
       (yyval.statement_) = new quickstep::ParseStatementCreateIndex((yylsp[-8]).first_line, (yylsp[-8]).first_column, (yyvsp[-6].string_value_), (yyvsp[-4].string_value_), (yyvsp[-3].attribute_list_), (yyvsp[-1].string_value_));
     }
   }
-#line 3053 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3116 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 25:
-#line 631 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 646 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.drop_table_statement_) = new quickstep::ParseStatementDropTable((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].string_value_));
   }
-#line 3061 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3124 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 26:
-#line 636 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 651 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_definition_) = new quickstep::ParseAttributeDefinition((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].string_value_), (yyvsp[-1].data_type_), (yyvsp[0].column_constraint_list_));
   }
-#line 3069 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3132 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 27:
-#line 641 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 656 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_definition_list_) = new quickstep::PtrList<quickstep::ParseAttributeDefinition>();
     (yyval.attribute_definition_list_)->push_back((yyvsp[0].attribute_definition_));
   }
-#line 3078 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3141 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 28:
-#line 645 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 660 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_definition_list_) = (yyvsp[-2].attribute_definition_list_);
     (yyval.attribute_definition_list_)->push_back((yyvsp[0].attribute_definition_));
   }
-#line 3087 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3150 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 29:
-#line 651 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 666 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = nullptr;
     NotSupported(&(yylsp[0]), yyscanner, "BIT data type");
     YYERROR;
   }
-#line 3097 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3160 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 30:
-#line 656 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 671 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetime));
   }
-#line 3105 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3168 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 31:
-#line 659 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 674 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetime));
   }
-#line 3113 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3176 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 32:
-#line 662 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 677 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = nullptr;
     NotSupported(&(yylsp[0]), yyscanner, "TIME data type");
     YYERROR;
   }
-#line 3123 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3186 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 33:
-#line 667 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 682 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetime));
   }
-#line 3131 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3194 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 34:
-#line 670 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 685 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
   }
-#line 3139 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3202 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 35:
-#line 673 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 688 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
   }
-#line 3147 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3210 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 36:
-#line 676 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 691 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
   }
-#line 3155 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3218 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 37:
-#line 679 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 694 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kFloat));
   }
-#line 3163 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3226 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 38:
-#line 682 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 697 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kInt));
   }
-#line 3171 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3234 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 39:
-#line 685 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 700 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kInt));
   }
-#line 3179 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3242 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 40:
-#line 688 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 703 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kLong));
   }
-#line 3187 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3250 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 41:
-#line 691 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 706 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kLong));
   }
-#line 3195 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3258 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 42:
-#line 694 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 709 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /**
      * NOTE(chasseur): This pattern exhibits a shift/reduce conflict with the
@@ -3208,27 +3271,27 @@
         "or YEARMONTH INTERVAL");
     YYERROR;
   }
-#line 3212 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3275 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 43:
-#line 706 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 721 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetimeInterval));
   }
-#line 3220 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3283 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 44:
-#line 709 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 724 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kYearMonthInterval));
   }
-#line 3228 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3291 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 45:
-#line 712 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 727 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-1].numeric_literal_value_)->float_like()) {
       delete (yyvsp[-1].numeric_literal_value_);
@@ -3247,11 +3310,11 @@
       }
     }
   }
-#line 3251 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3314 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 46:
-#line 730 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 745 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-1].numeric_literal_value_)->float_like()) {
       delete (yyvsp[-1].numeric_literal_value_);
@@ -3270,69 +3333,69 @@
       }
     }
   }
-#line 3274 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3337 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 47:
-#line 750 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 765 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = new quickstep::ParseColumnConstraintNull((yylsp[0]).first_line, (yylsp[0]).first_column);
   }
-#line 3282 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3345 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 48:
-#line 753 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 768 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = new quickstep::ParseColumnConstraintNotNull((yylsp[-1]).first_line, (yylsp[-1]).first_column);
   }
-#line 3290 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3353 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 49:
-#line 756 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 771 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     NotSupported(&(yylsp[0]), yyscanner, "Column Constraints (UNIQUE)");
     YYERROR;
   }
-#line 3300 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3363 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 50:
-#line 761 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 776 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     NotSupported(&(yylsp[-1]), yyscanner, "Column Constraints (PRIMARY KEY)");
     YYERROR;
   }
-#line 3310 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3373 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 51:
-#line 766 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 781 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     delete (yyvsp[0].literal_value_);
     NotSupported(&(yylsp[-1]), yyscanner, "Column Constraints (DEFAULT)");
     YYERROR;
   }
-#line 3321 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3384 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 52:
-#line 772 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 787 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     delete (yyvsp[-1].predicate_);
     NotSupported(&(yylsp[-3]), yyscanner, "Column Constraints (CHECK)");
     YYERROR;
   }
-#line 3332 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3395 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 53:
-#line 778 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 793 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     delete (yyvsp[-3].string_value_);
@@ -3340,65 +3403,65 @@
     NotSupported(&(yylsp[-4]), yyscanner, "Foreign Keys");
     YYERROR;
   }
-#line 3344 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3407 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 54:
-#line 787 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 802 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = (yyvsp[-1].column_constraint_list_);
     (yyval.column_constraint_list_)->push_back((yyvsp[0].column_constraint_));
   }
-#line 3353 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3416 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 55:
-#line 791 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 806 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = new quickstep::PtrList<quickstep::ParseColumnConstraint>();
     (yyval.column_constraint_list_)->push_back((yyvsp[0].column_constraint_));
   }
-#line 3362 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3425 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 56:
-#line 797 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 812 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = nullptr;
   }
-#line 3370 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3433 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 57:
-#line 800 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 815 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = (yyvsp[0].column_constraint_list_);
   }
-#line 3378 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3441 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 58:
-#line 805 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 820 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-1].string_list_);
     NotSupported(&(yylsp[-3]), yyscanner, "Table Constraints (UNIQUE)");
     YYERROR;
   }
-#line 3388 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3451 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 59:
-#line 810 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 825 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-1].string_list_);
     NotSupported(&(yylsp[-4]), yyscanner, "Table Constraints (PRIMARY KEY)");
     YYERROR;
   }
-#line 3398 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3461 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 60:
-#line 815 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 830 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-6].string_list_);
     delete (yyvsp[-3].string_value_);
@@ -3406,155 +3469,203 @@
     NotSupported(&(yylsp[-9]), yyscanner, "Table Constraints (FOREIGN KEY)");
     YYERROR;
   }
-#line 3410 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3473 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 61:
-#line 822 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 837 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-1].predicate_);
     NotSupported(&(yylsp[-3]), yyscanner, "Table Constraints (CHECK)");
     YYERROR;
   }
-#line 3420 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3483 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 62:
-#line 829 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 844 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     NotSupported(&(yylsp[-2]), yyscanner, "Table Constraints");
     YYERROR;
   }
-#line 3429 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3492 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 63:
-#line 833 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 848 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     NotSupported(&(yylsp[0]), yyscanner, "Table Constraints");
     YYERROR;
   }
-#line 3438 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3501 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 64:
-#line 839 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 854 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /* $$ = nullptr; */
   }
-#line 3446 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3509 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 65:
-#line 842 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 857 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /* $$ = $1; */
   }
-#line 3454 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3517 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 66:
-#line 847 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 862 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_list_) = nullptr;
   }
-#line 3462 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3525 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 67:
-#line 850 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 865 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_list_) = (yyvsp[-1].attribute_list_);
   }
-#line 3470 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3533 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 68:
-#line 855 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 870 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.block_properties_) = nullptr;
   }
-#line 3478 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3541 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 69:
-#line 858 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 873 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.block_properties_) = new quickstep::ParseBlockProperties((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-1].key_value_list_));
   }
-#line 3486 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3549 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 70:
-#line 863 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 878 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.partition_clause_) = nullptr;
+  }
+#line 3557 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 71:
+#line 881 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    if ((yyvsp[0].numeric_literal_value_)->float_like()) {
+      delete (yyvsp[0].numeric_literal_value_);
+      (yyval.partition_clause_) = NULL;
+      quickstep_yyerror(&(yylsp[0]), yyscanner, NULL, "NUMBER OF PARTITIONS must be an integer");
+      YYERROR;
+    } else {
+      if ((yyvsp[0].numeric_literal_value_)->long_value() <= 0 || (yyvsp[0].numeric_literal_value_)->long_value() > 64) {
+        delete (yyvsp[0].numeric_literal_value_);
+        (yyval.partition_clause_) = NULL;
+        quickstep_yyerror(&(yylsp[0]), yyscanner, NULL, "NUMBER OF PARITIONS must be between 1 and 64");
+        YYERROR;
+      } else {
+        (yyval.partition_clause_) = new quickstep::ParsePartitionClause((yylsp[-7]).first_line, (yylsp[-7]).first_column, (yyvsp[-5].string_value_), (yyvsp[-3].string_list_), (yyvsp[0].numeric_literal_value_));
+      }
+    }
+  }
+#line 3579 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 72:
+#line 900 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
+           std::to_string(quickstep::PartitionSchemeHeader::PartitionType::kHash));
+  }
+#line 3588 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 73:
+#line 904 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
+           std::to_string(quickstep::PartitionSchemeHeader::PartitionType::kRange));
+  }
+#line 3597 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 74:
+#line 910 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = new quickstep::PtrList<quickstep::ParseKeyValue>();
     (yyval.key_value_list_)->push_back((yyvsp[0].key_value_));
   }
-#line 3495 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3606 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 71:
-#line 867 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 75:
+#line 914 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = (yyvsp[-2].key_value_list_);
     (yyval.key_value_list_)->push_back((yyvsp[0].key_value_));
   }
-#line 3504 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 72:
-#line 873 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.key_value_) = (yyvsp[0].key_string_value_);
-  }
-#line 3512 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 73:
-#line 876 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.key_value_) = (yyvsp[0].key_string_list_);
-  }
-#line 3520 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 74:
-#line 879 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.key_value_) = (yyvsp[0].key_integer_value_);
-  }
-#line 3528 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 75:
-#line 884 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.key_string_value_) = new quickstep::ParseKeyStringValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].string_value_));
-  }
-#line 3536 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3615 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 76:
-#line 887 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 920 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.key_value_) = (yyvsp[0].key_string_value_);
+  }
+#line 3623 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 77:
+#line 923 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.key_value_) = (yyvsp[0].key_string_list_);
+  }
+#line 3631 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 78:
+#line 926 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.key_value_) = (yyvsp[0].key_integer_value_);
+  }
+#line 3639 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 79:
+#line 931 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.key_string_value_) = new quickstep::ParseKeyStringValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].string_value_));
+  }
+#line 3647 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 80:
+#line 934 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     // This is a special case to handle the COMPRESS ALL option of the BLOCK PROPERTIES.
     (yyval.key_string_value_) = new quickstep::ParseKeyStringValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_),
         new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column, "ALL"));
   }
-#line 3546 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3657 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 77:
-#line 894 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 81:
+#line 941 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_string_list_) = new quickstep::ParseKeyStringList((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].string_list_));
   }
-#line 3554 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3665 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 78:
-#line 899 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 82:
+#line 946 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].numeric_literal_value_)->float_like()) {
       delete (yyvsp[0].numeric_literal_value_);
@@ -3564,45 +3675,45 @@
     }
     (yyval.key_integer_value_) = new quickstep::ParseKeyIntegerValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].numeric_literal_value_));
   }
-#line 3568 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3679 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 79:
-#line 910 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 83:
+#line 957 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
            std::to_string(quickstep::IndexSubBlockType::kBloomFilter));
   }
-#line 3577 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3688 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 80:
-#line 914 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 84:
+#line 961 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
            std::to_string(quickstep::IndexSubBlockType::kCSBTree));
   }
-#line 3586 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3697 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 81:
-#line 920 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 85:
+#line 967 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = nullptr;
   }
-#line 3594 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3705 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 82:
-#line 923 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 86:
+#line 970 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = (yyvsp[-1].key_value_list_);
   }
-#line 3602 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3713 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 83:
-#line 929 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 87:
+#line 976 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-7].string_value_);
     delete (yyvsp[-5].string_list_);
@@ -3611,576 +3722,592 @@
     NotSupported(&(yylsp[-6]), yyscanner, "list of column names in INSERT statement");
     YYERROR;
   }
-#line 3615 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 84:
-#line 937 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.insert_statement_) = new quickstep::ParseStatementInsert((yylsp[-6]).first_line, (yylsp[-6]).first_column, (yyvsp[-4].string_value_), (yyvsp[-1].literal_value_list_));
-  }
-#line 3623 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 85:
-#line 942 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.copy_from_statement_) = new quickstep::ParseStatementCopyFrom((yylsp[-4]).first_line, (yylsp[-4]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].string_value_), (yyvsp[0].copy_from_params_));
-  }
-#line 3631 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 86:
-#line 947 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.copy_from_params_) = nullptr;
-  }
-#line 3639 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 87:
-#line 950 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.copy_from_params_) = (yyvsp[-1].copy_from_params_);
-  }
-#line 3647 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3726 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 88:
-#line 955 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 984 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.copy_from_params_) = new quickstep::ParseCopyFromParams((yylsp[-1]).first_line, (yylsp[-1]).first_column);
-    (yyval.copy_from_params_)->set_delimiter((yyvsp[0].string_value_));
+    (yyval.insert_statement_) = new quickstep::ParseStatementInsertTuple((yylsp[-6]).first_line, (yylsp[-6]).first_column, (yyvsp[-4].string_value_), (yyvsp[-1].literal_value_list_));
   }
-#line 3656 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3734 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 89:
-#line 959 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 987 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.copy_from_params_) = new quickstep::ParseCopyFromParams((yylsp[-1]).first_line, (yylsp[-1]).first_column);
-    (yyval.copy_from_params_)->escape_strings = (yyvsp[0].boolean_value_);
+    (yyval.insert_statement_) = new quickstep::ParseStatementInsertSelection((yylsp[-3]).first_line, (yylsp[-2]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].select_query_), nullptr);
   }
-#line 3665 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3742 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 90:
-#line 963 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 990 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.copy_from_params_) = (yyvsp[-3].copy_from_params_);
-    (yyval.copy_from_params_)->set_delimiter((yyvsp[0].string_value_));
+    (yyval.insert_statement_) = new quickstep::ParseStatementInsertSelection((yylsp[-4]).first_line, (yylsp[-3]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].select_query_), (yyvsp[-4].with_list_));
   }
-#line 3674 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3750 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 91:
-#line 967 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 996 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.copy_from_params_) = (yyvsp[-3].copy_from_params_);
-    (yyval.copy_from_params_)->escape_strings = (yyvsp[0].boolean_value_);
-  }
-#line 3683 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 92:
-#line 973 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.update_statement_) = new quickstep::ParseStatementUpdate((yylsp[-4]).first_line, (yylsp[-4]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].assignment_list_), (yyvsp[0].predicate_));
-  }
-#line 3691 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 93:
-#line 978 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.delete_statement_) = new quickstep::ParseStatementDelete((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].predicate_));
-  }
-#line 3699 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 94:
-#line 983 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.assignment_list_) = (yyvsp[-2].assignment_list_);
-    (yyval.assignment_list_)->push_back((yyvsp[0].assignment_));
-  }
-#line 3708 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 95:
-#line 987 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.assignment_list_) = new quickstep::PtrList<quickstep::ParseAssignment>();
-    (yyval.assignment_list_)->push_back((yyvsp[0].assignment_));
-  }
-#line 3717 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 96:
-#line 993 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.assignment_) = new quickstep::ParseAssignment((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].string_value_), (yyvsp[0].expression_));
-  }
-#line 3725 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 97:
-#line 999 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.select_statement_) = new quickstep::ParseStatementSelect((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[0].select_query_), (yyvsp[-1].with_list_));
-  }
-#line 3733 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 98:
-#line 1004 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.with_list_) = nullptr;
-  }
-#line 3741 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 99:
-#line 1007 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.with_list_) = (yyvsp[0].with_list_);
-  }
-#line 3749 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 100:
-#line 1012 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.with_list_) = new quickstep::PtrVector<quickstep::ParseSubqueryTableReference>();
-    (yyval.with_list_)->push_back((yyvsp[0].with_list_element_));
+    (yyval.copy_from_statement_) = new quickstep::ParseStatementCopyFrom((yylsp[-4]).first_line, (yylsp[-4]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].string_value_), (yyvsp[0].copy_from_params_));
   }
 #line 3758 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 101:
-#line 1016 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 92:
+#line 1001 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.with_list_) = (yyvsp[-2].with_list_);
-    (yyval.with_list_)->push_back((yyvsp[0].with_list_element_));
+    (yyval.copy_from_params_) = nullptr;
   }
-#line 3767 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3766 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 102:
-#line 1022 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 93:
+#line 1004 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.with_list_element_) = new quickstep::ParseSubqueryTableReference((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].subquery_expression_));
-    (yyval.with_list_element_)->set_table_reference_signature((yyvsp[-2].table_reference_signature_));
+    (yyval.copy_from_params_) = (yyvsp[-1].copy_from_params_);
   }
-#line 3776 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3774 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 103:
-#line 1029 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 94:
+#line 1009 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.select_query_) = new quickstep::ParseSelect((yylsp[-8]).first_line, (yylsp[-8]).first_column, (yyvsp[-6].selection_), (yyvsp[-5].table_reference_list_), (yyvsp[-4].predicate_), (yyvsp[-3].opt_group_by_clause_), (yyvsp[-2].opt_having_clause_), (yyvsp[-1].opt_order_by_clause_), (yyvsp[0].opt_limit_clause_));
+    (yyval.copy_from_params_) = new quickstep::ParseCopyFromParams((yylsp[-1]).first_line, (yylsp[-1]).first_column);
+    (yyval.copy_from_params_)->set_delimiter((yyvsp[0].string_value_));
   }
-#line 3784 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3783 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 104:
-#line 1034 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 95:
+#line 1013 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    /* $$ = nullptr; */
+    (yyval.copy_from_params_) = new quickstep::ParseCopyFromParams((yylsp[-1]).first_line, (yylsp[-1]).first_column);
+    (yyval.copy_from_params_)->escape_strings = (yyvsp[0].boolean_value_);
   }
 #line 3792 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 105:
-#line 1037 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 96:
+#line 1017 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    NotSupported(&(yylsp[0]), yyscanner, "ALL in selection");
-    YYERROR;
+    (yyval.copy_from_params_) = (yyvsp[-3].copy_from_params_);
+    (yyval.copy_from_params_)->set_delimiter((yyvsp[0].string_value_));
   }
 #line 3801 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 106:
-#line 1041 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 97:
+#line 1021 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    NotSupported(&(yylsp[0]), yyscanner, "DISTINCT in selection");
-    YYERROR;
+    (yyval.copy_from_params_) = (yyvsp[-3].copy_from_params_);
+    (yyval.copy_from_params_)->escape_strings = (yyvsp[0].boolean_value_);
   }
 #line 3810 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 107:
-#line 1047 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 98:
+#line 1027 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.selection_) = new quickstep::ParseSelectionStar((yylsp[0]).first_line, (yylsp[0]).first_column);
+    (yyval.update_statement_) = new quickstep::ParseStatementUpdate((yylsp[-4]).first_line, (yylsp[-4]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].assignment_list_), (yyvsp[0].predicate_));
   }
 #line 3818 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 108:
-#line 1050 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 99:
+#line 1032 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.selection_) = (yyvsp[0].selection_list_);
+    (yyval.delete_statement_) = new quickstep::ParseStatementDelete((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].predicate_));
   }
 #line 3826 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 109:
-#line 1055 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 100:
+#line 1037 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.selection_list_) = new quickstep::ParseSelectionList((yylsp[0]).first_line, (yylsp[0]).first_column);
-    (yyval.selection_list_)->add((yyvsp[0].selection_item_));
+    (yyval.assignment_list_) = (yyvsp[-2].assignment_list_);
+    (yyval.assignment_list_)->push_back((yyvsp[0].assignment_));
   }
 #line 3835 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 110:
-#line 1059 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 101:
+#line 1041 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.selection_list_) = (yyvsp[-2].selection_list_);
-    (yyval.selection_list_)->add((yyvsp[0].selection_item_));
+    (yyval.assignment_list_) = new quickstep::PtrList<quickstep::ParseAssignment>();
+    (yyval.assignment_list_)->push_back((yyvsp[0].assignment_));
   }
 #line 3844 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 111:
-#line 1065 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 102:
+#line 1047 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].expression_), (yyvsp[0].string_value_));
+    (yyval.assignment_) = new quickstep::ParseAssignment((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].string_value_), (yyvsp[0].expression_));
   }
 #line 3852 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 112:
-#line 1068 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 103:
+#line 1053 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].expression_), (yyvsp[0].string_value_));
+    (yyval.select_statement_) = new quickstep::ParseStatementSelect((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].select_query_), nullptr);
   }
 #line 3860 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 113:
-#line 1071 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 104:
+#line 1056 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].expression_));
+    (yyval.select_statement_) = new quickstep::ParseStatementSelect((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[0].select_query_), (yyvsp[-1].with_list_));
   }
 #line 3868 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 114:
-#line 1076 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 105:
+#line 1061 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.table_reference_list_) = (yyvsp[-1].table_reference_list_);
+    (yyval.with_list_) = (yyvsp[0].with_list_);
   }
 #line 3876 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 115:
-#line 1081 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 106:
+#line 1066 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    /* $$ = nullptr; */
+    (yyval.with_list_) = new quickstep::PtrVector<quickstep::ParseSubqueryTableReference>();
+    (yyval.with_list_)->push_back((yyvsp[0].with_list_element_));
   }
-#line 3884 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3885 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 116:
-#line 1084 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 107:
+#line 1070 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    NotSupported(&(yylsp[0]), yyscanner, "alternate JOIN syntax (specify in WHERE clause instead)");
-    YYERROR;
+    (yyval.with_list_) = (yyvsp[-2].with_list_);
+    (yyval.with_list_)->push_back((yyvsp[0].with_list_element_));
   }
-#line 3893 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3894 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 117:
-#line 1090 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 108:
+#line 1076 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    NotSupported(&(yylsp[-1]), yyscanner, "alternate JOIN syntax (specify in WHERE clause instead)");
-    YYERROR;
+    (yyval.with_list_element_) = new quickstep::ParseSubqueryTableReference((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].subquery_expression_));
+    (yyval.with_list_element_)->set_table_reference_signature((yyvsp[-2].table_reference_signature_));
   }
-#line 3902 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 3903 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 118:
-#line 1094 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 109:
+#line 1083 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    NotSupported(&(yylsp[0]), yyscanner, "alternate JOIN syntax (specify in WHERE clause instead)");
-    YYERROR;
+    (yyval.select_query_) = new quickstep::ParseSelect((yylsp[-8]).first_line, (yylsp[-8]).first_column, (yyvsp[-6].selection_), (yyvsp[-5].table_reference_list_), (yyvsp[-4].predicate_), (yyvsp[-3].opt_group_by_clause_), (yyvsp[-2].opt_having_clause_), (yyvsp[-1].opt_order_by_clause_), (yyvsp[0].opt_limit_clause_));
   }
 #line 3911 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
+  case 110:
+#line 1088 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    /* $$ = nullptr; */
+  }
+#line 3919 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 111:
+#line 1091 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    NotSupported(&(yylsp[0]), yyscanner, "ALL in selection");
+    YYERROR;
+  }
+#line 3928 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 112:
+#line 1095 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    NotSupported(&(yylsp[0]), yyscanner, "DISTINCT in selection");
+    YYERROR;
+  }
+#line 3937 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 113:
+#line 1101 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.selection_) = new quickstep::ParseSelectionStar((yylsp[0]).first_line, (yylsp[0]).first_column);
+  }
+#line 3945 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 114:
+#line 1104 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.selection_) = (yyvsp[0].selection_list_);
+  }
+#line 3953 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 115:
+#line 1109 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.selection_list_) = new quickstep::ParseSelectionList((yylsp[0]).first_line, (yylsp[0]).first_column);
+    (yyval.selection_list_)->add((yyvsp[0].selection_item_));
+  }
+#line 3962 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 116:
+#line 1113 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.selection_list_) = (yyvsp[-2].selection_list_);
+    (yyval.selection_list_)->add((yyvsp[0].selection_item_));
+  }
+#line 3971 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 117:
+#line 1119 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].expression_), (yyvsp[0].string_value_));
+  }
+#line 3979 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 118:
+#line 1122 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].expression_), (yyvsp[0].string_value_));
+  }
+#line 3987 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
   case 119:
-#line 1100 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1125 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].expression_));
+  }
+#line 3995 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 120:
+#line 1130 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_list_) = (yyvsp[-1].table_reference_list_);
+  }
+#line 4003 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 121:
+#line 1135 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    /* $$ = nullptr; */
+  }
+#line 4011 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 122:
+#line 1138 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    NotSupported(&(yylsp[0]), yyscanner, "alternate JOIN syntax (specify in WHERE clause instead)");
+    YYERROR;
+  }
+#line 4020 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 123:
+#line 1144 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    NotSupported(&(yylsp[-1]), yyscanner, "alternate JOIN syntax (specify in WHERE clause instead)");
+    YYERROR;
+  }
+#line 4029 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 124:
+#line 1148 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    NotSupported(&(yylsp[0]), yyscanner, "alternate JOIN syntax (specify in WHERE clause instead)");
+    YYERROR;
+  }
+#line 4038 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 125:
+#line 1154 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-2].string_list_);
     delete (yyvsp[0].predicate_);
     NotSupported(&(yylsp[-4]), yyscanner, "alternate JOIN syntax (specify in WHERE clause instead)");
     YYERROR;
   }
-#line 3922 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4049 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 120:
-#line 1106 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 126:
+#line 1160 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-2].string_list_);
     delete (yyvsp[0].predicate_);
     NotSupported(&(yylsp[-3]), yyscanner, "alternate JOIN syntax (specify in WHERE clause instead)");
     YYERROR;
   }
-#line 3933 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 121:
-#line 1112 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    delete (yyvsp[-2].string_list_);
-    delete (yyvsp[0].predicate_);
-    NotSupported(&(yylsp[-5]), yyscanner, "OUTER JOIN");
-    YYERROR;
-  }
-#line 3944 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 122:
-#line 1118 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    delete (yyvsp[-2].string_list_);
-    delete (yyvsp[0].predicate_);
-    NotSupported(&(yylsp[-4]), yyscanner, "OUTER JOIN");
-    YYERROR;
-  }
-#line 3955 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 123:
-#line 1124 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    delete (yyvsp[-2].string_list_);
-    delete (yyvsp[0].predicate_);
-    NotSupported(&(yylsp[-5]), yyscanner, "OUTER JOIN");
-    YYERROR;
-  }
-#line 3966 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 124:
-#line 1130 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    delete (yyvsp[-2].string_list_);
-    delete (yyvsp[0].predicate_);
-    NotSupported(&(yylsp[-4]), yyscanner, "OUTER JOIN");
-    YYERROR;
-  }
-#line 3977 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 125:
-#line 1136 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    delete (yyvsp[-2].string_list_);
-    delete (yyvsp[0].predicate_);
-    NotSupported(&(yylsp[-5]), yyscanner, "OUTER JOIN");
-    YYERROR;
-  }
-#line 3988 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 126:
-#line 1142 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    delete (yyvsp[-2].string_list_);
-    delete (yyvsp[0].predicate_);
-    NotSupported(&(yylsp[-4]), yyscanner, "OUTER JOIN");
-    YYERROR;
-  }
-#line 3999 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4060 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 127:
-#line 1150 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1166 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.subquery_expression_) = new quickstep::ParseSubqueryExpression((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-1].select_query_));
+    delete (yyvsp[-2].string_list_);
+    delete (yyvsp[0].predicate_);
+    NotSupported(&(yylsp[-5]), yyscanner, "OUTER JOIN");
+    YYERROR;
   }
-#line 4007 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4071 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 128:
-#line 1155 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1172 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.opt_sample_clause_) = NULL;
-  }
-#line 4015 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 129:
-#line 1158 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.opt_sample_clause_) = new quickstep::ParseSample((yylsp[-2]).first_line, (yylsp[-2]).first_column, true, (yyvsp[-1].numeric_literal_value_));
-  }
-#line 4023 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 130:
-#line 1161 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.opt_sample_clause_) = new quickstep::ParseSample((yylsp[-2]).first_line, (yylsp[-2]).first_column, false, (yyvsp[-1].numeric_literal_value_));
-  }
-#line 4031 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 131:
-#line 1166 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.table_reference_) = new quickstep::ParseSubqueryTableReference((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].subquery_expression_));
-    (yyval.table_reference_)->set_table_reference_signature((yyvsp[0].table_reference_signature_));
-  }
-#line 4040 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 132:
-#line 1170 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.table_reference_) = new quickstep::ParseSimpleTableReference((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].string_value_), (yyvsp[-1].opt_sample_clause_));
-    (yyval.table_reference_)->set_table_reference_signature((yyvsp[0].table_reference_signature_));
-  }
-#line 4049 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 133:
-#line 1174 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.table_reference_) = new quickstep::ParseSimpleTableReference((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].opt_sample_clause_));
-  }
-#line 4057 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 134:
-#line 1177 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.table_reference_) = new quickstep::ParseGeneratorTableReference((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].function_call_));
-    (yyval.table_reference_)->set_table_reference_signature((yyvsp[0].table_reference_signature_));
-  }
-#line 4066 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 135:
-#line 1181 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.table_reference_) = new quickstep::ParseGeneratorTableReference((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].function_call_));
-  }
-#line 4074 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 136:
-#line 1186 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.table_reference_signature_) = (yyvsp[0].table_reference_signature_);
+    delete (yyvsp[-2].string_list_);
+    delete (yyvsp[0].predicate_);
+    NotSupported(&(yylsp[-4]), yyscanner, "OUTER JOIN");
+    YYERROR;
   }
 #line 4082 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 137:
-#line 1189 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 129:
+#line 1178 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.table_reference_signature_) = (yyvsp[0].table_reference_signature_);
+    delete (yyvsp[-2].string_list_);
+    delete (yyvsp[0].predicate_);
+    NotSupported(&(yylsp[-5]), yyscanner, "OUTER JOIN");
+    YYERROR;
   }
-#line 4090 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4093 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 138:
-#line 1194 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 130:
+#line 1184 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.table_reference_signature_) = new ::quickstep::ParseTableReferenceSignature((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].string_value_));
+    delete (yyvsp[-2].string_list_);
+    delete (yyvsp[0].predicate_);
+    NotSupported(&(yylsp[-4]), yyscanner, "OUTER JOIN");
+    YYERROR;
   }
-#line 4098 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4104 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 139:
-#line 1197 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 131:
+#line 1190 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.table_reference_signature_) = new ::quickstep::ParseTableReferenceSignature((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].string_list_));
-  }
-#line 4106 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 140:
-#line 1202 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.table_reference_list_) = new quickstep::PtrList<quickstep::ParseTableReference>();
-    (yyval.table_reference_list_)->push_back((yyvsp[0].table_reference_));
+    delete (yyvsp[-2].string_list_);
+    delete (yyvsp[0].predicate_);
+    NotSupported(&(yylsp[-5]), yyscanner, "OUTER JOIN");
+    YYERROR;
   }
 #line 4115 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
+  case 132:
+#line 1196 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    delete (yyvsp[-2].string_list_);
+    delete (yyvsp[0].predicate_);
+    NotSupported(&(yylsp[-4]), yyscanner, "OUTER JOIN");
+    YYERROR;
+  }
+#line 4126 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 133:
+#line 1204 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.subquery_expression_) = new quickstep::ParseSubqueryExpression((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-1].select_query_));
+  }
+#line 4134 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 134:
+#line 1209 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.opt_sample_clause_) = NULL;
+  }
+#line 4142 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 135:
+#line 1212 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.opt_sample_clause_) = new quickstep::ParseSample((yylsp[-2]).first_line, (yylsp[-2]).first_column, true, (yyvsp[-1].numeric_literal_value_));
+  }
+#line 4150 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 136:
+#line 1215 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.opt_sample_clause_) = new quickstep::ParseSample((yylsp[-2]).first_line, (yylsp[-2]).first_column, false, (yyvsp[-1].numeric_literal_value_));
+  }
+#line 4158 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 137:
+#line 1220 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_) = new quickstep::ParseSubqueryTableReference((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].subquery_expression_));
+    (yyval.table_reference_)->set_table_reference_signature((yyvsp[0].table_reference_signature_));
+  }
+#line 4167 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 138:
+#line 1224 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_) = new quickstep::ParseSimpleTableReference((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].string_value_), (yyvsp[-1].opt_sample_clause_));
+    (yyval.table_reference_)->set_table_reference_signature((yyvsp[0].table_reference_signature_));
+  }
+#line 4176 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 139:
+#line 1228 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_) = new quickstep::ParseSimpleTableReference((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].opt_sample_clause_));
+  }
+#line 4184 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 140:
+#line 1231 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_) = new quickstep::ParseGeneratorTableReference((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].function_call_));
+    (yyval.table_reference_)->set_table_reference_signature((yyvsp[0].table_reference_signature_));
+  }
+#line 4193 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
   case 141:
-#line 1206 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1235 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_) = new quickstep::ParseGeneratorTableReference((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].function_call_));
+  }
+#line 4201 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 142:
+#line 1240 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_signature_) = (yyvsp[0].table_reference_signature_);
+  }
+#line 4209 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 143:
+#line 1243 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_signature_) = (yyvsp[0].table_reference_signature_);
+  }
+#line 4217 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 144:
+#line 1248 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_signature_) = new ::quickstep::ParseTableReferenceSignature((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].string_value_));
+  }
+#line 4225 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 145:
+#line 1251 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_signature_) = new ::quickstep::ParseTableReferenceSignature((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].string_list_));
+  }
+#line 4233 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 146:
+#line 1256 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.table_reference_list_) = new quickstep::PtrList<quickstep::ParseTableReference>();
+    (yyval.table_reference_list_)->push_back((yyvsp[0].table_reference_));
+  }
+#line 4242 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 147:
+#line 1260 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_list_) = (yyvsp[-2].table_reference_list_);
     (yyval.table_reference_list_)->push_back((yyvsp[0].table_reference_));
   }
-#line 4124 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 142:
-#line 1212 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.opt_group_by_clause_) = nullptr;
-  }
-#line 4132 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 143:
-#line 1215 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.opt_group_by_clause_) = new quickstep::ParseGroupBy((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].expression_list_));
-  }
-#line 4140 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 144:
-#line 1220 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.opt_having_clause_) = nullptr;
-  }
-#line 4148 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 145:
-#line 1223 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.opt_having_clause_) = new quickstep::ParseHaving((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[0].predicate_));
-  }
-#line 4156 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 146:
-#line 1228 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.opt_order_by_clause_) = nullptr;
-  }
-#line 4164 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 147:
-#line 1231 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.opt_order_by_clause_) = new quickstep::ParseOrderBy((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].order_commalist_));
-  }
-#line 4172 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4251 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 148:
-#line 1236 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1266 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.opt_limit_clause_) = nullptr;
+    (yyval.opt_group_by_clause_) = nullptr;
   }
-#line 4180 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4259 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 149:
-#line 1239 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1269 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.opt_group_by_clause_) = new quickstep::ParseGroupBy((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].expression_list_));
+  }
+#line 4267 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 150:
+#line 1274 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.opt_having_clause_) = nullptr;
+  }
+#line 4275 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 151:
+#line 1277 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.opt_having_clause_) = new quickstep::ParseHaving((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[0].predicate_));
+  }
+#line 4283 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 152:
+#line 1282 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.opt_order_by_clause_) = nullptr;
+  }
+#line 4291 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 153:
+#line 1285 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.opt_order_by_clause_) = new quickstep::ParseOrderBy((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].order_commalist_));
+  }
+#line 4299 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 154:
+#line 1290 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.opt_limit_clause_) = nullptr;
+  }
+#line 4307 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 155:
+#line 1293 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].numeric_literal_value_)->float_like()) {
       delete (yyvsp[0].numeric_literal_value_);
@@ -4198,111 +4325,111 @@
       }
     }
   }
-#line 4202 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4329 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 150:
-#line 1258 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 156:
+#line 1312 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_commalist_) = new quickstep::PtrList<quickstep::ParseOrderByItem>();
     (yyval.order_commalist_)->push_back((yyvsp[0].order_item_));
   }
-#line 4211 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4338 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 151:
-#line 1262 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 157:
+#line 1316 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_commalist_) = (yyvsp[-2].order_commalist_);
     (yyval.order_commalist_)->push_back((yyvsp[0].order_item_));
   }
-#line 4220 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4347 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 152:
-#line 1268 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 158:
+#line 1322 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_item_) = new quickstep::ParseOrderByItem((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].expression_), (yyvsp[-1].order_direction_), (yyvsp[0].order_direction_));
     delete (yyvsp[-1].order_direction_);
     delete (yyvsp[0].order_direction_);
   }
-#line 4230 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 153:
-#line 1275 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.order_direction_) = nullptr;
-  }
-#line 4238 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 154:
-#line 1278 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.order_direction_) = new bool(true);
-  }
-#line 4246 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 155:
-#line 1281 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.order_direction_) = new bool(false);
-  }
-#line 4254 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 156:
-#line 1286 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.order_direction_) = nullptr;
-  }
-#line 4262 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 157:
-#line 1289 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.order_direction_) = new bool(true);
-  }
-#line 4270 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 158:
-#line 1292 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.order_direction_) = new bool(false);
-  }
-#line 4278 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4357 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 159:
-#line 1298 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1329 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.predicate_) = nullptr;
+    (yyval.order_direction_) = nullptr;
   }
-#line 4286 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4365 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 160:
-#line 1301 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1332 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.predicate_) = (yyvsp[0].predicate_);
+    (yyval.order_direction_) = new bool(true);
   }
-#line 4294 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4373 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 161:
-#line 1306 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1335 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.predicate_) = (yyvsp[0].predicate_);
+    (yyval.order_direction_) = new bool(false);
   }
-#line 4302 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4381 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 162:
-#line 1311 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1340 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.order_direction_) = nullptr;
+  }
+#line 4389 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 163:
+#line 1343 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.order_direction_) = new bool(true);
+  }
+#line 4397 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 164:
+#line 1346 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.order_direction_) = new bool(false);
+  }
+#line 4405 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 165:
+#line 1352 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.predicate_) = nullptr;
+  }
+#line 4413 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 166:
+#line 1355 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.predicate_) = (yyvsp[0].predicate_);
+  }
+#line 4421 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 167:
+#line 1360 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.predicate_) = (yyvsp[0].predicate_);
+  }
+#line 4429 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 168:
+#line 1365 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-2].predicate_)->getParsePredicateType() == quickstep::ParsePredicate::kDisjunction) {
       (yyval.predicate_) = (yyvsp[-2].predicate_);
@@ -4312,19 +4439,19 @@
     }
     static_cast<quickstep::ParsePredicateDisjunction *>((yyval.predicate_))->addPredicate((yyvsp[0].predicate_));
   }
-#line 4316 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4443 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 163:
-#line 1320 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 169:
+#line 1374 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = (yyvsp[0].predicate_);
   }
-#line 4324 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4451 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 164:
-#line 1325 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 170:
+#line 1379 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-2].predicate_)->getParsePredicateType() == quickstep::ParsePredicate::kConjunction) {
       (yyval.predicate_) = (yyvsp[-2].predicate_);
@@ -4334,239 +4461,263 @@
     }
     static_cast<quickstep::ParsePredicateConjunction *>((yyval.predicate_))->addPredicate((yyvsp[0].predicate_));
   }
-#line 4338 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4465 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 165:
-#line 1334 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 171:
+#line 1388 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = (yyvsp[0].predicate_);
   }
-#line 4346 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4473 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 166:
-#line 1339 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 172:
+#line 1393 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateNegation((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[0].predicate_));
   }
-#line 4354 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4481 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 167:
-#line 1342 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 173:
+#line 1396 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = (yyvsp[0].predicate_);
   }
-#line 4362 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4489 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 168:
-#line 1347 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 174:
+#line 1401 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateBetween((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-4].expression_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
   }
-#line 4370 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4497 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 169:
-#line 1350 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 175:
+#line 1404 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateNegation(
         (yylsp[-4]).first_line, (yylsp[-4]).first_column,
         new quickstep::ParsePredicateBetween((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-5].expression_), (yyvsp[-2].expression_), (yyvsp[0].expression_)));
   }
-#line 4380 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4507 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 170:
-#line 1355 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 176:
+#line 1409 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].attribute_);
     (yyval.predicate_) = nullptr;
     NotSupported(&(yylsp[-2]), yyscanner, "NULL comparison predicates");
     YYERROR;
   }
-#line 4391 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4518 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 171:
-#line 1361 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 177:
+#line 1415 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-2].attribute_);
     (yyval.predicate_) = nullptr;
     NotSupported(&(yylsp[-1]), yyscanner, "NULL comparison predicates");
     YYERROR;
   }
-#line 4402 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 172:
-#line 1367 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.predicate_) = new quickstep::ParsePredicateComparison((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].comparison_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
-  }
-#line 4410 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 173:
-#line 1370 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.predicate_) = (yyvsp[-1].predicate_);
-  }
-#line 4418 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 174:
-#line 1376 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.expression_) = new quickstep::ParseBinaryExpression((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].binary_operation_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
-  }
-#line 4426 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 175:
-#line 1379 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.expression_) = (yyvsp[0].expression_);
-  }
-#line 4434 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 176:
-#line 1384 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.expression_) = new quickstep::ParseBinaryExpression((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].binary_operation_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
-  }
-#line 4442 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 177:
-#line 1387 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.expression_) = (yyvsp[0].expression_);
-  }
-#line 4450 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4529 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 178:
-#line 1392 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1421 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.expression_) = new quickstep::ParseUnaryExpression((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].unary_operation_), (yyvsp[0].expression_));
+    (yyval.predicate_) = new quickstep::ParsePredicateComparison((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].comparison_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
   }
-#line 4458 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4537 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 179:
-#line 1395 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1424 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.expression_) = (yyvsp[0].expression_);
+    (yyval.predicate_) = (yyvsp[-1].predicate_);
   }
-#line 4466 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4545 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 180:
-#line 1400 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1430 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.expression_) = (yyvsp[0].attribute_);
+    (yyval.expression_) = new quickstep::ParseBinaryExpression((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].binary_operation_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
   }
-#line 4474 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4553 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 181:
-#line 1403 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1433 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.expression_) = new quickstep::ParseScalarLiteral((yyvsp[0].literal_value_));
+    (yyval.expression_) = (yyvsp[0].expression_);
   }
-#line 4482 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4561 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 182:
-#line 1406 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1438 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.expression_) = (yyvsp[0].function_call_);
+    (yyval.expression_) = new quickstep::ParseBinaryExpression((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].binary_operation_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
   }
-#line 4490 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4569 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 183:
-#line 1409 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1441 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.expression_) = (yyvsp[-1].expression_);
+    (yyval.expression_) = (yyvsp[0].expression_);
   }
-#line 4498 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4577 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 184:
-#line 1414 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1446 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.function_call_) = new quickstep::ParseFunctionCall(
-        (yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].string_value_), new quickstep::PtrList<quickstep::ParseExpression>());
+    (yyval.expression_) = new quickstep::ParseUnaryExpression((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].unary_operation_), (yyvsp[0].expression_));
   }
-#line 4507 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4585 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 185:
-#line 1418 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1449 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.expression_) = (yyvsp[0].expression_);
+  }
+#line 4593 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 186:
+#line 1454 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.expression_) = (yyvsp[0].attribute_);
+  }
+#line 4601 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 187:
+#line 1457 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.expression_) = new quickstep::ParseScalarLiteral((yyvsp[0].literal_value_));
+  }
+#line 4609 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 188:
+#line 1460 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.expression_) = (yyvsp[0].function_call_);
+  }
+#line 4617 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 189:
+#line 1463 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.expression_) = (yyvsp[0].expression_);
+  }
+#line 4625 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 190:
+#line 1466 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.expression_) = (yyvsp[-1].expression_);
+  }
+#line 4633 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 191:
+#line 1471 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.function_call_) = new quickstep::ParseFunctionCall(
+        (yylsp[-2]).first_line, (yylsp[-2]).first_column, false, (yyvsp[-2].string_value_), new quickstep::PtrList<quickstep::ParseExpression>());
+  }
+#line 4642 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 192:
+#line 1475 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.function_call_) = new quickstep::ParseFunctionCall(
         (yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-3].string_value_), new quickstep::ParseStar((yylsp[-1]).first_line, (yylsp[-1]).first_column));
   }
-#line 4516 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4651 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 186:
-#line 1422 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 193:
+#line 1479 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.function_call_) = new quickstep::ParseFunctionCall((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].expression_list_));
+    (yyval.function_call_) = new quickstep::ParseFunctionCall((yylsp[-3]).first_line, (yylsp[-3]).first_column, false, (yyvsp[-3].string_value_), (yyvsp[-1].expression_list_));
   }
-#line 4524 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4659 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 187:
-#line 1427 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 194:
+#line 1482 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.function_call_) = new quickstep::ParseFunctionCall((yylsp[-4]).first_line, (yylsp[-4]).first_column, true, (yyvsp[-4].string_value_), (yyvsp[-1].expression_list_));
+  }
+#line 4667 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 195:
+#line 1487 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.expression_) = new quickstep::ParseExtractFunction((yylsp[-5]).first_line, (yylsp[-5]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].expression_));
+  }
+#line 4675 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 196:
+#line 1492 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_list_) = new quickstep::PtrList<quickstep::ParseExpression>();
     (yyval.expression_list_)->push_back((yyvsp[0].expression_));
   }
-#line 4533 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4684 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 188:
-#line 1431 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 197:
+#line 1496 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_list_) = (yyvsp[-2].expression_list_);
     (yyval.expression_list_)->push_back((yyvsp[0].expression_));
   }
-#line 4542 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4693 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 189:
-#line 1437 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 198:
+#line 1502 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.literal_value_) = new quickstep::NullParseLiteralValue((yylsp[0]).first_line, (yylsp[0]).first_column);
   }
-#line 4550 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4701 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 190:
-#line 1440 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 199:
+#line 1505 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.literal_value_) = (yyvsp[0].numeric_literal_value_);
   }
-#line 4558 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4709 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 191:
-#line 1443 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 200:
+#line 1508 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.literal_value_) = (yyvsp[0].numeric_literal_value_);
   }
-#line 4566 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4717 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 192:
-#line 1446 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 201:
+#line 1511 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /**
      * NOTE(chasseur): This case exhibits a shift/reduce conflict with the
@@ -4579,20 +4730,20 @@
     (yyvsp[0].numeric_literal_value_)->prependMinus();
     (yyval.literal_value_) = (yyvsp[0].numeric_literal_value_);
   }
-#line 4583 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4734 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 193:
-#line 1458 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 202:
+#line 1523 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.literal_value_) = new quickstep::StringParseLiteralValue((yyvsp[0].string_value_),
                                                 nullptr);  // No explicit type.
   }
-#line 4592 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4743 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 194:
-#line 1462 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 203:
+#line 1527 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /**
      * NOTE(chasseur): This case exhibits a shift/reduce conflict with the
@@ -4612,11 +4763,11 @@
       YYERROR;
     }
   }
-#line 4616 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4767 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 195:
-#line 1481 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 204:
+#line 1546 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     quickstep::StringParseLiteralValue *parse_value
         = new quickstep::StringParseLiteralValue((yyvsp[0].string_value_), &((yyvsp[-1].data_type_)->getType()));
@@ -4630,143 +4781,143 @@
       (yyval.literal_value_) = parse_value;
     }
   }
-#line 4634 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4785 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 196:
-#line 1496 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 205:
+#line 1561 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.literal_value_list_) = new quickstep::PtrList<quickstep::ParseScalarLiteral>();
     (yyval.literal_value_list_)->push_back(new quickstep::ParseScalarLiteral((yyvsp[0].literal_value_)));
   }
-#line 4643 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4794 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 197:
-#line 1500 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 206:
+#line 1565 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.literal_value_list_) = (yyvsp[-2].literal_value_list_);
     (yyval.literal_value_list_)->push_back(new quickstep::ParseScalarLiteral((yyvsp[0].literal_value_)));
   }
-#line 4652 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4803 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 198:
-#line 1506 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 207:
+#line 1571 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_) = new quickstep::ParseAttribute((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].string_value_));
   }
-#line 4660 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4811 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 199:
-#line 1509 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 208:
+#line 1574 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_) = new quickstep::ParseAttribute((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].string_value_), (yyvsp[-2].string_value_));
   }
-#line 4668 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4819 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 200:
-#line 1514 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 209:
+#line 1579 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_list_) = new quickstep::PtrList<quickstep::ParseAttribute>();
     (yyval.attribute_list_)->push_back((yyvsp[0].attribute_));
   }
-#line 4677 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4828 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 201:
-#line 1518 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 210:
+#line 1583 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_list_) = (yyvsp[-2].attribute_list_);
     (yyval.attribute_list_)->push_back((yyvsp[0].attribute_));
   }
-#line 4686 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 202:
-#line 1525 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kEqual);
-  }
-#line 4694 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 203:
-#line 1528 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kNotEqual);
-  }
-#line 4702 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 204:
-#line 1531 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kLess);
-  }
-#line 4710 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 205:
-#line 1534 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kLessOrEqual);
-  }
-#line 4718 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 206:
-#line 1537 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kGreater);
-  }
-#line 4726 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 207:
-#line 1540 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kGreaterOrEqual);
-  }
-#line 4734 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 208:
-#line 1543 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.comparison_) =  &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kLike);
-  }
-#line 4742 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 209:
-#line 1546 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.comparison_) =  &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kNotLike);
-  }
-#line 4750 "SqlParser_gen.cpp" /* yacc.c:1661  */
-    break;
-
-  case 210:
-#line 1549 "../SqlParser.ypp" /* yacc.c:1661  */
-    {
-    (yyval.comparison_) =  &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kRegexMatch);
-  }
-#line 4758 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4837 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 211:
-#line 1552 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1590 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.comparison_) =  &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kNotRegexMatch);
+    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kEqual);
   }
-#line 4766 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4845 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 212:
-#line 1557 "../SqlParser.ypp" /* yacc.c:1661  */
+#line 1593 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kNotEqual);
+  }
+#line 4853 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 213:
+#line 1596 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kLess);
+  }
+#line 4861 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 214:
+#line 1599 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kLessOrEqual);
+  }
+#line 4869 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 215:
+#line 1602 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kGreater);
+  }
+#line 4877 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 216:
+#line 1605 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.comparison_) = &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kGreaterOrEqual);
+  }
+#line 4885 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 217:
+#line 1608 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.comparison_) =  &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kLike);
+  }
+#line 4893 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 218:
+#line 1611 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.comparison_) =  &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kNotLike);
+  }
+#line 4901 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 219:
+#line 1614 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.comparison_) =  &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kRegexMatch);
+  }
+#line 4909 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 220:
+#line 1617 "../SqlParser.ypp" /* yacc.c:1661  */
+    {
+    (yyval.comparison_) =  &quickstep::ComparisonFactory::GetComparison(quickstep::ComparisonID::kNotRegexMatch);
+  }
+#line 4917 "SqlParser_gen.cpp" /* yacc.c:1661  */
+    break;
+
+  case 221:
+#line 1622 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /**
      * NOTE(chasseur): This case exhibits a shift/reduce conflict with the
@@ -4776,146 +4927,146 @@
      **/
     (yyval.unary_operation_) = &quickstep::UnaryOperationFactory::GetUnaryOperation(quickstep::UnaryOperationID::kNegate);
   }
-#line 4780 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4931 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 213:
-#line 1568 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 222:
+#line 1633 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.binary_operation_) = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kAdd);
   }
-#line 4788 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4939 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 214:
-#line 1571 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 223:
+#line 1636 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.binary_operation_) = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kSubtract);
   }
-#line 4796 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4947 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 215:
-#line 1576 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 224:
+#line 1641 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.binary_operation_) = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kModulo);
   }
-#line 4804 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4955 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 216:
-#line 1579 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 225:
+#line 1644 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.binary_operation_) = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kMultiply);
   }
-#line 4812 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4963 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 217:
-#line 1582 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 226:
+#line 1647 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.binary_operation_) = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kDivide);
   }
-#line 4820 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4971 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 218:
-#line 1588 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 227:
+#line 1653 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_list_) = new quickstep::PtrList<quickstep::ParseString>();
     (yyval.string_list_)->push_back((yyvsp[0].string_value_));
   }
-#line 4829 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4980 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 219:
-#line 1592 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 228:
+#line 1657 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_list_) = (yyvsp[-2].string_list_);
     (yyval.string_list_)->push_back((yyvsp[0].string_value_));
   }
-#line 4838 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4989 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 220:
-#line 1598 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 229:
+#line 1663 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_value_) = (yyvsp[0].string_value_);
   }
-#line 4846 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 4997 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 221:
-#line 1601 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 230:
+#line 1666 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].string_value_)->value().empty()) {
       quickstep_yyerror(&(yylsp[0]), yyscanner, nullptr, "Zero-length identifier");
     }
     (yyval.string_value_) = (yyvsp[0].string_value_);
   }
-#line 4857 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 5008 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 222:
-#line 1609 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 231:
+#line 1674 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.boolean_value_) = true;
   }
-#line 4865 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 5016 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 223:
-#line 1612 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 232:
+#line 1677 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.boolean_value_) = true;
   }
-#line 4873 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 5024 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 224:
-#line 1615 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 233:
+#line 1680 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.boolean_value_) = false;
   }
-#line 4881 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 5032 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 225:
-#line 1618 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 234:
+#line 1683 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.boolean_value_) = false;
   }
-#line 4889 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 5040 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 226:
-#line 1624 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 235:
+#line 1689 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.command_) = new quickstep::ParseCommand((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].command_argument_list_));
   }
-#line 4897 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 5048 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 227:
-#line 1629 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 236:
+#line 1694 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     quickstep::PtrVector<quickstep::ParseString> *argument_list = (yyvsp[-1].command_argument_list_);
     argument_list->push_back((yyvsp[0].string_value_));
     (yyval.command_argument_list_) = argument_list;
   }
-#line 4907 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 5058 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
-  case 228:
-#line 1634 "../SqlParser.ypp" /* yacc.c:1661  */
+  case 237:
+#line 1699 "../SqlParser.ypp" /* yacc.c:1661  */
     { /* Epsilon, an empy match. */
     (yyval.command_argument_list_) = new quickstep::PtrVector<quickstep::ParseString>();
   }
-#line 4915 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 5066 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
 
-#line 4919 "SqlParser_gen.cpp" /* yacc.c:1661  */
+#line 5070 "SqlParser_gen.cpp" /* yacc.c:1661  */
       default: break;
     }
   /* User semantic actions sometimes alter yychar, and that requires
@@ -5150,7 +5301,7 @@
 #endif
   return yyresult;
 }
-#line 1638 "../SqlParser.ypp" /* yacc.c:1906  */
+#line 1703 "../SqlParser.ypp" /* yacc.c:1906  */
 
 
 void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string &feature) {
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index 7119cba..71fddb3 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -94,57 +94,62 @@
     TOKEN_DOUBLE = 304,
     TOKEN_DROP = 305,
     TOKEN_ESCAPE_STRINGS = 306,
-    TOKEN_FALSE = 307,
-    TOKEN_FIRST = 308,
-    TOKEN_FLOAT = 309,
-    TOKEN_FOREIGN = 310,
-    TOKEN_FROM = 311,
-    TOKEN_FULL = 312,
-    TOKEN_GROUP = 313,
-    TOKEN_HAVING = 314,
-    TOKEN_INDEX = 315,
-    TOKEN_INNER = 316,
-    TOKEN_INSERT = 317,
-    TOKEN_INTEGER = 318,
-    TOKEN_INTERVAL = 319,
-    TOKEN_INTO = 320,
-    TOKEN_JOIN = 321,
-    TOKEN_KEY = 322,
-    TOKEN_LAST = 323,
-    TOKEN_LEFT = 324,
-    TOKEN_LIMIT = 325,
-    TOKEN_LONG = 326,
-    TOKEN_NULL = 327,
-    TOKEN_NULLS = 328,
-    TOKEN_OFF = 329,
-    TOKEN_ON = 330,
-    TOKEN_ORDER = 331,
-    TOKEN_OUTER = 332,
-    TOKEN_PERCENT = 333,
-    TOKEN_PRIMARY = 334,
-    TOKEN_QUIT = 335,
-    TOKEN_REAL = 336,
-    TOKEN_REFERENCES = 337,
-    TOKEN_RIGHT = 338,
-    TOKEN_ROW_DELIMITER = 339,
-    TOKEN_SELECT = 340,
-    TOKEN_SET = 341,
-    TOKEN_SMALLINT = 342,
-    TOKEN_TABLE = 343,
-    TOKEN_TIME = 344,
-    TOKEN_TIMESTAMP = 345,
-    TOKEN_TRUE = 346,
-    TOKEN_TUPLESAMPLE = 347,
-    TOKEN_UNIQUE = 348,
-    TOKEN_UPDATE = 349,
-    TOKEN_USING = 350,
-    TOKEN_VALUES = 351,
-    TOKEN_VARCHAR = 352,
-    TOKEN_WHERE = 353,
-    TOKEN_WITH = 354,
-    TOKEN_YEARMONTH = 355,
-    TOKEN_EOF = 356,
-    TOKEN_LEX_ERROR = 357
+    TOKEN_EXTRACT = 307,
+    TOKEN_FALSE = 308,
+    TOKEN_FIRST = 309,
+    TOKEN_FLOAT = 310,
+    TOKEN_FOREIGN = 311,
+    TOKEN_FROM = 312,
+    TOKEN_FULL = 313,
+    TOKEN_GROUP = 314,
+    TOKEN_HASH = 315,
+    TOKEN_HAVING = 316,
+    TOKEN_INDEX = 317,
+    TOKEN_INNER = 318,
+    TOKEN_INSERT = 319,
+    TOKEN_INTEGER = 320,
+    TOKEN_INTERVAL = 321,
+    TOKEN_INTO = 322,
+    TOKEN_JOIN = 323,
+    TOKEN_KEY = 324,
+    TOKEN_LAST = 325,
+    TOKEN_LEFT = 326,
+    TOKEN_LIMIT = 327,
+    TOKEN_LONG = 328,
+    TOKEN_NULL = 329,
+    TOKEN_NULLS = 330,
+    TOKEN_OFF = 331,
+    TOKEN_ON = 332,
+    TOKEN_ORDER = 333,
+    TOKEN_OUTER = 334,
+    TOKEN_PARTITION = 335,
+    TOKEN_PARTITIONS = 336,
+    TOKEN_PERCENT = 337,
+    TOKEN_PRIMARY = 338,
+    TOKEN_QUIT = 339,
+    TOKEN_RANGE = 340,
+    TOKEN_REAL = 341,
+    TOKEN_REFERENCES = 342,
+    TOKEN_RIGHT = 343,
+    TOKEN_ROW_DELIMITER = 344,
+    TOKEN_SELECT = 345,
+    TOKEN_SET = 346,
+    TOKEN_SMALLINT = 347,
+    TOKEN_TABLE = 348,
+    TOKEN_TIME = 349,
+    TOKEN_TIMESTAMP = 350,
+    TOKEN_TRUE = 351,
+    TOKEN_TUPLESAMPLE = 352,
+    TOKEN_UNIQUE = 353,
+    TOKEN_UPDATE = 354,
+    TOKEN_USING = 355,
+    TOKEN_VALUES = 356,
+    TOKEN_VARCHAR = 357,
+    TOKEN_WHERE = 358,
+    TOKEN_WITH = 359,
+    TOKEN_YEARMONTH = 360,
+    TOKEN_EOF = 361,
+    TOKEN_LEX_ERROR = 362
   };
 #endif
 
@@ -153,7 +158,7 @@
 
 union YYSTYPE
 {
-#line 110 "../SqlParser.ypp" /* yacc.c:1915  */
+#line 114 "../SqlParser.ypp" /* yacc.c:1915  */
 
   quickstep::ParseString *string_value_;
 
@@ -210,6 +215,7 @@
   quickstep::ParseStatementDelete *delete_statement_;
   quickstep::ParseStatementCopyFrom *copy_from_statement_;
   quickstep::ParseStatementCreateTable *create_table_statement_;
+  quickstep::ParsePartitionClause *partition_clause_;
   quickstep::ParseBlockProperties *block_properties_;
   quickstep::ParseStatementDropTable *drop_table_statement_;
   quickstep::ParseStatementQuit *quit_statement_;
@@ -236,7 +242,7 @@
   quickstep::PtrVector<quickstep::ParseSubqueryTableReference> *with_list_;
   quickstep::ParseSubqueryTableReference *with_list_element_;
 
-#line 240 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 246 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;
diff --git a/parser/tests/Aggregate.test b/parser/tests/Aggregate.test
index 3fbf3f1..7962d3e 100644
--- a/parser/tests/Aggregate.test
+++ b/parser/tests/Aggregate.test
@@ -166,3 +166,53 @@
   |   +-right_operand=FunctionCall[name=AGG]
   +-from_clause=
     +-TableReference[table=test]
+==
+
+SELECT AGG(DISTINCT x), AGG(y) FROM test GROUP BY z
+--
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  | | +-FunctionCall[name=AGG,is_distinct=true]
+  | |   +-AttributeReference[attribute_name=x]
+  | +-SelectListItem
+  |   +-FunctionCall[name=AGG]
+  |     +-AttributeReference[attribute_name=y]
+  +-group_by=GroupBy
+  | +-AttributeReference[attribute_name=z]
+  +-from_clause=
+    +-TableReference[table=test]
+==
+
+SELECT AGG(DISTINCT FUN(x) + y) * AGG(z)
+FROM test
+GROUP BY z
+HAVING AGG(DISTINCT w) + AGG(s) > 1
+--
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  |   +-Multiply
+  |     +-left_operand=FunctionCall[name=AGG,is_distinct=true]
+  |     | +-Add
+  |     |   +-left_operand=FunctionCall[name=FUN]
+  |     |   | +-AttributeReference[attribute_name=x]
+  |     |   +-right_operand=AttributeReference[attribute_name=y]
+  |     +-right_operand=FunctionCall[name=AGG]
+  |       +-AttributeReference[attribute_name=z]
+  +-group_by=GroupBy
+  | +-AttributeReference[attribute_name=z]
+  +-having=HAVING
+  | +-Greater
+  |   +-left_operand=Add
+  |   | +-left_operand=FunctionCall[name=AGG,is_distinct=true]
+  |   | | +-AttributeReference[attribute_name=w]
+  |   | +-right_operand=FunctionCall[name=AGG]
+  |   |   +-AttributeReference[attribute_name=s]
+  |   +-right_operand=Literal
+  |     +-NumericLiteral[numeric_string=1,float_like=false]
+  +-from_clause=
+    +-TableReference[table=test]
+==
diff --git a/parser/tests/Create.test b/parser/tests/Create.test
index b156db8..2670c66 100644
--- a/parser/tests/Create.test
+++ b/parser/tests/Create.test
@@ -139,6 +139,74 @@
                              ^
 ==
 
+CREATE TABLE test (attr INT) PARTITION BY HASH(attr) PARTITIONS 4
+--
+CreateTableStatement[relation_name=test]
++-attribute_list=
+| +-AttributeDefinition[name=attr,type=Int]
++-partition_clause=
+  +-PartitionClause[partition_type=0]
+    +-Number of Partitions=NumericLiteral[numeric_string=4,float_like=false]
+    +-attribute_name_list=
+      +-String[value=attr]
+==
+
+# Empty partitioning attribute is not allowed
+CREATE TABLE test (attr INT) PARTITION BY HASH() PARTITIONS 4
+--
+ERROR: syntax error (1 : 48)
+...TABLE test (attr INT) PARTITION BY HASH() PARTITIONS 4
+                                           ^
+==
+
+# Zero partitions not allowed
+CREATE TABLE test (attr INT) PARTITION BY HASH(attr) PARTITIONS 0
+--
+ERROR: NUMBER OF PARITIONS must be between 1 and 64 (1 : 65)
+...INT) PARTITION BY HASH(attr) PARTITIONS 0
+                                           ^
+==
+
+# Missing number of partitions not allowed
+CREATE TABLE test (attr INT) PARTITION BY HASH(attr) PARTITIONS
+--
+ERROR: syntax error (1 : 64)
+... INT) PARTITION BY HASH(attr) PARTITIONS
+                                           ^
+==
+
+# Missing PARTITIONS attribute not allowed
+CREATE TABLE test (attr INT) PARTITION BY HASH(attr)
+--
+ERROR: syntax error (1 : 53)
+... test (attr INT) PARTITION BY HASH(attr)
+                                           ^
+==
+
+# Missing partitioning type not allowed
+CREATE TABLE test (attr INT) PARTITION BY (attr) PARTITIONS 4
+--
+ERROR: syntax error (1 : 43)
+CREATE TABLE test (attr INT) PARTITION BY (attr) PARTITIONS 4
+                                          ^
+==
+
+# Number of partitions greater than 64 not allowed
+CREATE TABLE test (attr INT) PARTITION BY HASH(attr) PARTITIONS 65
+--
+ERROR: NUMBER OF PARITIONS must be between 1 and 64 (1 : 65)
+...INT) PARTITION BY HASH(attr) PARTITIONS 65
+                                           ^
+==
+
+# Missing PARTITION BY clause not allowed
+CREATE TABLE test (attr INT) PARTITIONS 4
+--
+ERROR: syntax error (1 : 30)
+CREATE TABLE test (attr INT) PARTITIONS 4
+                             ^
+==
+
 CREATE TABLE test (attr INT) WITH BLOCKPROPERTIES
 (TYPE rowstore)
 --
diff --git a/parser/tests/TPCH.test b/parser/tests/TPCH.test
index fffd59e..f62c314 100644
--- a/parser/tests/TPCH.test
+++ b/parser/tests/TPCH.test
@@ -1,5 +1,7 @@
 #   Copyright 2011-2015 Quickstep Technologies LLC.
 #   Copyright 2015 Pivotal Software, Inc.
+#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+#     University of Wisconsin—Madison.
 #
 #   Licensed under the Apache License, Version 2.0 (the "License");
 #   you may not use this file except in compliance with the License.
@@ -436,9 +438,105 @@
   cust_nation,
   l_year
 --
-ERROR: syntax error (11 : 20)
-      EXTRACT(year FROM l_shipdate) AS l_year,
-                   ^
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=supp_nation]
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=cust_nation]
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=l_year]
+  | +-SelectListItem[alias=revenue]
+  |   +-FunctionCall[name=SUM]
+  |     +-AttributeReference[attribute_name=volume]
+  +-group_by=GroupBy
+  | +-AttributeReference[attribute_name=supp_nation]
+  | +-AttributeReference[attribute_name=cust_nation]
+  | +-AttributeReference[attribute_name=l_year]
+  +-order_by=OrderBy
+  | +-OrderByItem[is_asc=true,nulls_first=false]
+  | | +-AttributeReference[attribute_name=supp_nation]
+  | +-OrderByItem[is_asc=true,nulls_first=false]
+  | | +-AttributeReference[attribute_name=cust_nation]
+  | +-OrderByItem[is_asc=true,nulls_first=false]
+  |   +-AttributeReference[attribute_name=l_year]
+  +-from_clause=
+    +-SubqueryTable
+      +-table_signature=TableSignature[table_alias=shipping]
+      +-SubqueryExpression
+        +-Select
+          +-select_clause=SelectList
+          | +-SelectListItem[alias=supp_nation]
+          | | +-AttributeReference[attribute_name=n_name,relation_name=n1]
+          | +-SelectListItem[alias=cust_nation]
+          | | +-AttributeReference[attribute_name=n_name,relation_name=n2]
+          | +-SelectListItem[alias=l_year]
+          | | +-Extract[unit=year]
+          | |   +-date_expression=AttributeReference[attribute_name=l_shipdate]
+          | +-SelectListItem[alias=volume]
+          |   +-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
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+          | | +-right_operand=AttributeReference[attribute_name=l_suppkey]
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=o_orderkey]
+          | | +-right_operand=AttributeReference[attribute_name=l_orderkey]
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=c_custkey]
+          | | +-right_operand=AttributeReference[attribute_name=o_custkey]
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+          | | +-right_operand=AttributeReference[attribute_name=n_nationkey,
+          | |   relation_name=n1]
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=c_nationkey]
+          | | +-right_operand=AttributeReference[attribute_name=n_nationkey,
+          | |   relation_name=n2]
+          | +-Or
+          | | +-And
+          | | | +-Equal
+          | | | | +-left_operand=AttributeReference[attribute_name=n_name,
+          | | | | | relation_name=n1]
+          | | | | +-right_operand=Literal
+          | | | |   +-StringLiteral[value=ETHIOPIA]
+          | | | +-Equal
+          | | |   +-left_operand=AttributeReference[attribute_name=n_name,
+          | | |   | relation_name=n2]
+          | | |   +-right_operand=Literal
+          | | |     +-StringLiteral[value=UNITED STATES]
+          | | +-And
+          | |   +-Equal
+          | |   | +-left_operand=AttributeReference[attribute_name=n_name,
+          | |   | | relation_name=n1]
+          | |   | +-right_operand=Literal
+          | |   |   +-StringLiteral[value=UNITED STATES]
+          | |   +-Equal
+          | |     +-left_operand=AttributeReference[attribute_name=n_name,
+          | |     | relation_name=n2]
+          | |     +-right_operand=Literal
+          | |       +-StringLiteral[value=ETHIOPIA]
+          | +-Between
+          |   +-check_operand=AttributeReference[attribute_name=l_shipdate]
+          |   +-lower_bound_operand=Literal
+          |   | +-StringLiteral[value=1995-01-01,explicit_type=Datetime]
+          |   +-upper_bound_operand=Literal
+          |     +-StringLiteral[value=1996-12-31,explicit_type=Datetime]
+          +-from_clause=
+            +-TableReference[table=supplier]
+            +-TableReference[table=lineitem]
+            +-TableReference[table=orders]
+            +-TableReference[table=customer]
+            +-TableReference[table=nation]
+            | +-table_signature=TableSignature[table_alias=n1]
+            +-TableReference[table=nation]
+              +-table_signature=TableSignature[table_alias=n2]
 ==
 
 # Query 8
@@ -519,9 +617,77 @@
   nation,
   o_year DESC
 --
-ERROR: syntax error (9 : 20)
-      EXTRACT(year FROM o_orderdate) AS o_year,
-                   ^
+SelectStatement
++-select_query=Select
+  +-select_clause=SelectList
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=nation]
+  | +-SelectListItem
+  | | +-AttributeReference[attribute_name=o_year]
+  | +-SelectListItem[alias=sum_profit]
+  |   +-FunctionCall[name=SUM]
+  |     +-AttributeReference[attribute_name=amount]
+  +-group_by=GroupBy
+  | +-AttributeReference[attribute_name=nation]
+  | +-AttributeReference[attribute_name=o_year]
+  +-order_by=OrderBy
+  | +-OrderByItem[is_asc=true,nulls_first=false]
+  | | +-AttributeReference[attribute_name=nation]
+  | +-OrderByItem[is_asc=false,nulls_first=true]
+  |   +-AttributeReference[attribute_name=o_year]
+  +-from_clause=
+    +-SubqueryTable
+      +-table_signature=TableSignature[table_alias=profit]
+      +-SubqueryExpression
+        +-Select
+          +-select_clause=SelectList
+          | +-SelectListItem[alias=nation]
+          | | +-AttributeReference[attribute_name=n_name]
+          | +-SelectListItem[alias=o_year]
+          | | +-Extract[unit=year]
+          | |   +-date_expression=AttributeReference[attribute_name=o_orderdate]
+          | +-SelectListItem[alias=amount]
+          |   +-Subtract
+          |     +-left_operand=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]
+          |     +-right_operand=Multiply
+          |       +-left_operand=AttributeReference[attribute_name=ps_supplycost]
+          |       +-right_operand=AttributeReference[attribute_name=l_quantity]
+          +-where_clause=And
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+          | | +-right_operand=AttributeReference[attribute_name=l_suppkey]
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=ps_suppkey]
+          | | +-right_operand=AttributeReference[attribute_name=l_suppkey]
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=ps_partkey]
+          | | +-right_operand=AttributeReference[attribute_name=l_partkey]
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=p_partkey]
+          | | +-right_operand=AttributeReference[attribute_name=l_partkey]
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=o_orderkey]
+          | | +-right_operand=AttributeReference[attribute_name=l_orderkey]
+          | +-Equal
+          | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+          | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
+          | +-kLike
+          |   +-left_operand=AttributeReference[attribute_name=p_name]
+          |   +-right_operand=Literal
+          |     +-StringLiteral[value=%ghost%]
+          +-from_clause=
+            +-TableReference[table=part]
+            +-TableReference[table=supplier]
+            +-TableReference[table=lineitem]
+            +-TableReference[table=partsupp]
+            +-TableReference[table=orders]
+            +-TableReference[table=nation]
 ==
 
 # Query 10
@@ -812,9 +978,9 @@
   p_type,
   p_size
 --
-ERROR: syntax error (5 : 9)
-  COUNT(distinct ps_suppkey) AS supplie...
-        ^
+ERROR: syntax error (13 : 14)
+  AND p_size IN (32, 42, 9, 18, 50, 30, 12,...
+             ^
 ==
 
 # Query 17
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 7eca518..b682618 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -29,6 +29,7 @@
 add_library(quickstep_queryexecution_QueryExecutionMessages_proto
             ${queryexecution_QueryExecutionMessages_proto_srcs}
             ${queryexecution_QueryExecutionMessages_proto_hdrs})
+add_library(quickstep_queryexecution_QueryExecutionState ../empty_src.cpp QueryExecutionState.hpp) 
 add_library(quickstep_queryexecution_QueryExecutionTypedefs ../empty_src.cpp QueryExecutionTypedefs.hpp)
 add_library(quickstep_queryexecution_QueryExecutionUtil ../empty_src.cpp QueryExecutionUtil.hpp)
 add_library(quickstep_queryexecution_WorkOrdersContainer WorkOrdersContainer.cpp WorkOrdersContainer.hpp)
@@ -48,6 +49,7 @@
                       quickstep_queryexecution_ForemanLite
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryExecutionMessages_proto
+                      quickstep_queryexecution_QueryExecutionState
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_WorkOrdersContainer
@@ -100,6 +102,9 @@
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_queryexecution_QueryExecutionMessages_proto
                       ${PROTOBUF_LIBRARY})
+target_link_libraries(quickstep_queryexecution_QueryExecutionState
+                      glog
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_threading_ThreadIDBasedMap
                       tmb)
@@ -139,6 +144,7 @@
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryContext_proto
                       quickstep_queryexecution_QueryExecutionMessages_proto
+                      quickstep_queryexecution_QueryExecutionState
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_WorkOrdersContainer
@@ -159,6 +165,7 @@
                       quickstep_queryexecution_Foreman
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryContext_proto
+                      quickstep_queryexecution_QueryExecutionState
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_queryexecution_WorkerDirectory
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index fd4aea7..2b2581a 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -19,7 +19,6 @@
 
 #include <cstddef>
 #include <memory>
-#include <unordered_map>
 #include <utility>
 #include <vector>
 
@@ -77,12 +76,12 @@
 }
 
 void Foreman::processWorkOrderCompleteMessage(const dag_node_index op_index,
-                                              const size_t worker_id) {
-  --queued_workorders_per_op_[op_index];
+                                              const size_t worker_thread_index) {
+  query_exec_state_->decrementNumQueuedWorkOrders(op_index);
 
   // As the given worker finished executing a WorkOrder, decrement its number
   // of queued WorkOrders.
-  workers_->decrementNumQueuedWorkOrders(worker_id);
+  workers_->decrementNumQueuedWorkOrders(worker_thread_index);
 
   // Check if new work orders are available and fetch them if so.
   fetchNormalWorkOrders(op_index);
@@ -123,13 +122,13 @@
   // for the schedulable WorkOrders beginning from 'op_index'. The first
   // candidate worker to receive the next WorkOrder is the one that sent the
   // response message to Foreman.
-  dispatchWorkerMessages(worker_id, op_index);
+  dispatchWorkerMessages(worker_thread_index, op_index);
 }
 
 void Foreman::processRebuildWorkOrderCompleteMessage(const dag_node_index op_index,
-                                                     const size_t worker_id) {
-  --rebuild_status_[op_index].second;
-  workers_->decrementNumQueuedWorkOrders(worker_id);
+                                                     const size_t worker_thread_index) {
+  query_exec_state_->decrementNumRebuildWorkOrders(op_index);
+  workers_->decrementNumQueuedWorkOrders(worker_thread_index);
 
   if (checkRebuildOver(op_index)) {
     markOperatorFinished(op_index);
@@ -147,7 +146,7 @@
   // for the schedulable WorkOrders beginning from 'op_index'. The first
   // candidate worker to receive the next WorkOrder is the one that sent the
   // response message to Foreman.
-  dispatchWorkerMessages(worker_id, op_index);
+  dispatchWorkerMessages(worker_thread_index, op_index);
 }
 
 void Foreman::processDataPipelineMessage(const dag_node_index op_index,
@@ -183,7 +182,7 @@
   initialize();
 
   // Event loop
-  while (!checkQueryExecutionFinished()) {
+  while (!query_exec_state_->hasQueryExecutionFinished()) {
     // Receive() causes this thread to sleep until next message is received.
     AnnotatedMessage annotated_msg = bus_->Receive(foreman_client_id_, 0, true);
     const TaggedMessage &tagged_message = annotated_msg.tagged_message;
@@ -192,14 +191,14 @@
         serialization::WorkOrderCompletionMessage proto;
         CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
 
-        processWorkOrderCompleteMessage(proto.operator_index(), proto.worker_id());
+        processWorkOrderCompleteMessage(proto.operator_index(), proto.worker_thread_index());
         break;
       }
       case kRebuildWorkOrderCompleteMessage: {
         serialization::WorkOrderCompletionMessage proto;
         CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
 
-        processRebuildWorkOrderCompleteMessage(proto.operator_index(), proto.worker_id());
+        processRebuildWorkOrderCompleteMessage(proto.operator_index(), proto.worker_thread_index());
         break;
       }
       case kCatalogRelationNewBlockMessage: {
@@ -288,19 +287,11 @@
 
 void Foreman::initializeState() {
   const dag_node_index dag_size = query_dag_->size();
-  num_operators_finished_ = 0;
-
-  done_gen_.assign(dag_size, false);
-
-  execution_finished_.assign(dag_size, false);
 
   output_consumers_.resize(dag_size);
   blocking_dependencies_.resize(dag_size);
 
-  rebuild_required_.assign(dag_size, false);
-
-  queued_workorders_per_op_.assign(dag_size, 0);
-
+  query_exec_state_.reset(new QueryExecutionState(dag_size));
   workorders_container_.reset(new WorkOrdersContainer(dag_size, num_numa_nodes_));
 
   for (dag_node_index node_index = 0; node_index < dag_size; ++node_index) {
@@ -308,8 +299,8 @@
         query_dag_->getNodePayload(node_index).getInsertDestinationID();
     if (insert_destination_index != QueryContext::kInvalidInsertDestinationId) {
       // Rebuild is necessary whenever InsertDestination is present.
-      rebuild_required_[node_index] = true;
-      rebuild_status_[node_index] = std::make_pair(false, 0);
+      query_exec_state_->setRebuildRequired(node_index);
+      query_exec_state_->setRebuildStatus(node_index, 0, false);
     }
 
     for (const pair<dag_node_index, bool> &dependent_link :
@@ -340,7 +331,7 @@
   for (dag_node_index index = start_operator_index;
        num_operators_checked < query_dag_->size();
        index = (index + 1) % query_dag_->size(), ++num_operators_checked) {
-    if (execution_finished_[index]) {
+    if (query_exec_state_->hasExecutionFinished(index)) {
       continue;
     }
     if (numa_node != -1) {
@@ -348,7 +339,7 @@
       work_order = workorders_container_->getNormalWorkOrderForNUMANode(index, numa_node);
       if (work_order != nullptr) {
         // A WorkOrder found on the given NUMA node.
-        ++queued_workorders_per_op_[index];
+        query_exec_state_->incrementNumQueuedWorkOrders(index);
         return WorkerMessage::WorkOrderMessage(work_order, index);
       } else {
         // Normal workorder not found on this node. Look for a rebuild workorder
@@ -363,7 +354,7 @@
     // Try to get a normal WorkOrder from other NUMA nodes.
     work_order = workorders_container_->getNormalWorkOrder(index);
     if (work_order != nullptr) {
-      ++queued_workorders_per_op_[index];
+      query_exec_state_->incrementNumQueuedWorkOrders(index);
       return WorkerMessage::WorkOrderMessage(work_order, index);
     } else {
       // Normal WorkOrder not found, look for a RebuildWorkOrder.
@@ -377,7 +368,7 @@
   return nullptr;
 }
 
-void Foreman::sendWorkerMessage(const std::size_t worker_id,
+void Foreman::sendWorkerMessage(const std::size_t worker_thread_index,
                                 const WorkerMessage &message) {
   message_type_id type;
   if (message.getType() == WorkerMessage::kRebuildWorkOrder) {
@@ -392,17 +383,17 @@
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          foreman_client_id_,
-                                         workers_->getClientID(worker_id),
+                                         workers_->getClientID(worker_thread_index),
                                          move(worker_tagged_message));
   CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
       "Message could not be sent from Foreman with TMB client ID "
       << foreman_client_id_ << " to Foreman with TMB client ID "
-      << workers_->getClientID(worker_id);
+      << workers_->getClientID(worker_thread_index);
 }
 
 bool Foreman::fetchNormalWorkOrders(const dag_node_index index) {
   bool generated_new_workorders = false;
-  if (!done_gen_[index]) {
+  if (!query_exec_state_->hasDoneGenerationWorkOrders(index)) {
     // Do not fetch any work units until all blocking dependencies are met.
     // The releational operator is not aware of blocking dependencies for
     // uncorrelated scalar queries.
@@ -411,12 +402,15 @@
     }
     const size_t num_pending_workorders_before =
         workorders_container_->getNumNormalWorkOrders(index);
-    done_gen_[index] =
+    const bool done_generation =
         query_dag_->getNodePayloadMutable(index)->getAllWorkOrders(workorders_container_.get(),
                                                                    query_context_.get(),
                                                                    storage_manager_,
                                                                    foreman_client_id_,
                                                                    bus_);
+    if (done_generation) {
+      query_exec_state_->setDoneGenerationWorkOrders(index);
+    }
 
     // TODO(shoban): It would be a good check to see if operator is making
     // useful progress, i.e., the operator either generates work orders to
@@ -473,8 +467,7 @@
 }
 
 void Foreman::markOperatorFinished(const dag_node_index index) {
-  execution_finished_[index] = true;
-  ++num_operators_finished_;
+  query_exec_state_->setExecutionFinished(index);
 
   RelationalOperator *op = query_dag_->getNodePayloadMutable(index);
   op->updateCatalogOnCompletion();
@@ -500,10 +493,10 @@
 
   getRebuildWorkOrders(index, workorders_container_.get());
 
-  rebuild_status_[index] = std::make_pair(
-      true, workorders_container_->getNumRebuildWorkOrders(index));
+  query_exec_state_->setRebuildStatus(
+      index, workorders_container_->getNumRebuildWorkOrders(index), true);
 
-  return (rebuild_status_[index].second == 0);
+  return (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
 }
 
 void Foreman::getRebuildWorkOrders(const dag_node_index index, WorkOrdersContainer *container) {
diff --git a/query_execution/Foreman.hpp b/query_execution/Foreman.hpp
index 0b09f79..373529d 100644
--- a/query_execution/Foreman.hpp
+++ b/query_execution/Foreman.hpp
@@ -20,13 +20,12 @@
 
 #include <cstddef>
 #include <memory>
-#include <unordered_map>
-#include <utility>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/ForemanLite.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/QueryExecutionState.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
 #include "query_execution/WorkerMessage.hpp"
@@ -80,7 +79,6 @@
       : ForemanLite(bus, cpu_id),
         catalog_database_(DCHECK_NOTNULL(catalog_database)),
         storage_manager_(DCHECK_NOTNULL(storage_manager)),
-        num_operators_finished_(0),
         max_msgs_per_worker_(1),
         num_numa_nodes_(num_numa_nodes) {
     bus_->RegisterClientAsSender(foreman_client_id_, kWorkOrderMessage);
@@ -122,7 +120,7 @@
    **/
   inline void reconstructQueryContextFromProto(const serialization::QueryContext &proto) {
     query_context_.reset(
-        new QueryContext(proto, catalog_database_, storage_manager_, foreman_client_id_, bus_));
+        new QueryContext(proto, *catalog_database_, storage_manager_, foreman_client_id_, bus_));
   }
 
   /**
@@ -159,15 +157,6 @@
   typedef DAG<RelationalOperator, bool>::size_type_nodes dag_node_index;
 
   /**
-   * @brief Check if the current query has finished its execution.
-   *
-   * @return True if the query has finished. Otherwise false.
-   **/
-  bool checkQueryExecutionFinished() const {
-    return num_operators_finished_ == query_dag_->size();
-  }
-
-  /**
    * @brief Check if all the dependencies of the node at specified index have
    *        finished their execution.
    *
@@ -182,7 +171,7 @@
   inline bool checkAllDependenciesMet(const dag_node_index node_index) const {
     for (const dag_node_index dependency_index : query_dag_->getDependencies(node_index)) {
       // If at least one of the dependencies is not met, return false.
-      if (!execution_finished_[dependency_index]) {
+      if (!query_exec_state_->hasExecutionFinished(dependency_index)) {
         return false;
       }
     }
@@ -204,7 +193,7 @@
    **/
   inline bool checkAllBlockingDependenciesMet(const dag_node_index node_index) const {
     for (const dag_node_index blocking_dependency_index : blocking_dependencies_[node_index]) {
-      if (!execution_finished_[blocking_dependency_index]) {
+      if (!query_exec_state_->hasExecutionFinished(blocking_dependency_index)) {
         return false;
       }
     }
@@ -242,21 +231,22 @@
    *
    * @param node_index The index of the specified operator node in the query DAG
    *        for the completed WorkOrder.
-   * @param worker_id The logical ID of the worker for the completed WorkOrder.
+   * @param worker_thread_index The logical index of the worker thread in
+   *        WorkerDirectory for the completed WorkOrder.
    **/
   void processWorkOrderCompleteMessage(const dag_node_index op_index,
-                                       const std::size_t worker_id);
+                                       const std::size_t worker_thread_index);
 
   /**
    * @brief Process the received RebuildWorkOrder complete message.
    *
    * @param node_index The index of the specified operator node in the query DAG
    *        for the completed RebuildWorkOrder.
-   * @param worker_id The logical ID of the worker for the completed
-   *        RebuildWorkOrder.
+   * @param worker_thread_index The logical index of the worker thread in
+   *        WorkerDirectory for the completed RebuildWorkOrder.
    **/
   void processRebuildWorkOrderCompleteMessage(const dag_node_index op_index,
-                                              const std::size_t worker_id);
+                                              const std::size_t worker_thread_index);
 
   /**
    * @brief Process the received data pipeline message.
@@ -284,10 +274,6 @@
   void cleanUp() {
     output_consumers_.clear();
     blocking_dependencies_.clear();
-    done_gen_.clear();
-    execution_finished_.clear();
-    rebuild_required_.clear();
-    rebuild_status_.clear();
   }
 
   /**
@@ -319,10 +305,11 @@
   /**
    * @brief Send the given message to the specified worker.
    *
-   * @param worker_id The logical ID of the recipient worker.
+   * @param worker_thread_index The logical index of the recipient worker thread
+   *        in WorkerDirectory.
    * @param message The WorkerMessage to be sent.
    **/
-  void sendWorkerMessage(const std::size_t worker_id, const WorkerMessage &message);
+  void sendWorkerMessage(const std::size_t worker_thread_index, const WorkerMessage &message);
 
   /**
    * @brief Fetch all work orders currently available in relational operator and
@@ -382,8 +369,8 @@
   inline bool checkNormalExecutionOver(const dag_node_index index) const {
     return (checkAllDependenciesMet(index) &&
             !workorders_container_->hasNormalWorkOrder(index) &&
-            queued_workorders_per_op_[index] == 0 &&
-            done_gen_[index]);
+            query_exec_state_->getNumQueuedWorkOrders(index) == 0 &&
+            query_exec_state_->hasDoneGenerationWorkOrders(index));
   }
 
   /**
@@ -394,7 +381,7 @@
    * @return True if the rebuild operation is required, false otherwise.
    **/
   inline bool checkRebuildRequired(const dag_node_index index) const {
-    return rebuild_required_[index];
+    return query_exec_state_->isRebuildRequired(index);
   }
 
   /**
@@ -405,13 +392,9 @@
    * @return True if the rebuild operation is over, false otherwise.
    **/
   inline bool checkRebuildOver(const dag_node_index index) const {
-    std::unordered_map<dag_node_index,
-                       std::pair<bool, std::size_t>>::const_iterator
-        search_res = rebuild_status_.find(index);
-    DEBUG_ASSERT(search_res != rebuild_status_.end());
-    return checkRebuildInitiated(index) &&
+    return query_exec_state_->hasRebuildInitiated(index) &&
            !workorders_container_->hasRebuildWorkOrder(index) &&
-           (search_res->second.second == 0);
+           (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
   }
 
   /**
@@ -423,7 +406,7 @@
    * @return True if the rebuild operation has been initiated, false otherwise.
    **/
   inline bool checkRebuildInitiated(const dag_node_index index) const {
-    return rebuild_status_.at(index).first;
+    return query_exec_state_->hasRebuildInitiated(index);
   }
 
   /**
@@ -456,38 +439,17 @@
 
   std::unique_ptr<QueryContext> query_context_;
 
-  // Number of operators who've finished their execution.
-  std::size_t num_operators_finished_;
-
   // During a single round of WorkOrder dispatch, a Worker should be allocated
   // at most these many WorkOrders.
   std::size_t max_msgs_per_worker_;
 
-  // The ith bit denotes if the operator with ID = i has finished its execution.
-  std::vector<bool> execution_finished_;
-
   // For all nodes, store their receiving dependents.
   std::vector<std::vector<dag_node_index>> output_consumers_;
 
   // For all nodes, store their pipeline breaking dependencies (if any).
   std::vector<std::vector<dag_node_index>> blocking_dependencies_;
 
-  // The ith bit denotes if the operator with ID = i has finished generating
-  // work orders.
-  std::vector<bool> done_gen_;
-
-  // The ith bit denotes if the operator with ID = i requires generation of
-  // rebuild WorkOrders.
-  std::vector<bool> rebuild_required_;
-
-  // Key is dag_node_index of the operator for which rebuild is required. Value is
-  // a pair - first element has a bool (whether rebuild for operator at index i
-  // has been initiated) and if the boolean is true, the second element denotes
-  // the number of pending rebuild workorders for the operator.
-  std::unordered_map<dag_node_index, std::pair<bool, std::size_t>> rebuild_status_;
-
-  // A vector to track the number of workorders in execution, for each operator.
-  std::vector<int> queued_workorders_per_op_;
+  std::unique_ptr<QueryExecutionState> query_exec_state_;
 
   std::unique_ptr<WorkOrdersContainer> workorders_container_;
 
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 18e0b72..e9d6d57 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -50,18 +50,18 @@
 namespace quickstep {
 
 QueryContext::QueryContext(const serialization::QueryContext &proto,
-                           CatalogDatabaseLite *database,
+                           const CatalogDatabaseLite &database,
                            StorageManager *storage_manager,
                            const tmb::client_id foreman_client_id,
                            tmb::MessageBus *bus) {
-  DCHECK(ProtoIsValid(proto, *database))
+  DCHECK(ProtoIsValid(proto, database))
       << "Attempted to create QueryContext from an invalid proto description:\n"
       << proto.DebugString();
 
   for (int i = 0; i < proto.aggregation_states_size(); ++i) {
     aggregation_states_.emplace_back(
         AggregationOperationState::ReconstructFromProto(proto.aggregation_states(i),
-                                                        *database,
+                                                        database,
                                                         storage_manager));
   }
 
@@ -83,7 +83,7 @@
     const serialization::InsertDestination &insert_destination_proto = proto.insert_destinations(i);
     insert_destinations_.emplace_back(
         InsertDestination::ReconstructFromProto(insert_destination_proto,
-                                                database->getRelationSchemaById(
+                                                database.getRelationSchemaById(
                                                     insert_destination_proto.relation_id()),
                                                 storage_manager,
                                                 foreman_client_id,
@@ -92,7 +92,7 @@
 
   for (int i = 0; i < proto.predicates_size(); ++i) {
     predicates_.emplace_back(
-        PredicateFactory::ReconstructFromProto(proto.predicates(i), *database));
+        PredicateFactory::ReconstructFromProto(proto.predicates(i), database));
   }
 
   for (int i = 0; i < proto.scalar_groups_size(); ++i) {
@@ -101,7 +101,7 @@
     const serialization::QueryContext::ScalarGroup &scalar_group_proto = proto.scalar_groups(i);
     for (int j = 0; j < scalar_group_proto.scalars_size(); ++j) {
       scalar_group.emplace_back(
-          ScalarFactory::ReconstructFromProto(scalar_group_proto.scalars(j), *database));
+          ScalarFactory::ReconstructFromProto(scalar_group_proto.scalars(j), database));
     }
 
     scalar_groups_.push_back(move(scalar_group));
@@ -109,7 +109,7 @@
 
   for (int i = 0; i < proto.sort_configs_size(); ++i) {
     sort_configs_.emplace_back(
-        SortConfiguration::ReconstructFromProto(proto.sort_configs(i), *database));
+        SortConfiguration::ReconstructFromProto(proto.sort_configs(i), database));
   }
 
   for (int i = 0; i < proto.tuples_size(); ++i) {
@@ -125,7 +125,7 @@
           update_group_proto.update_assignments(j);
 
       unique_ptr<const Scalar> scalar(
-          ScalarFactory::ReconstructFromProto(update_assignment_proto.scalar(), *database));
+          ScalarFactory::ReconstructFromProto(update_assignment_proto.scalar(), database));
 
       update_group.emplace(update_assignment_proto.attribute_id(), move(scalar));
     }
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index 5cdfd47..c0c63cf 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -113,8 +113,6 @@
 
   /**
    * @brief Constructor.
-   * @note During some query execution, an InsertDestination could change
-   *       CatalogDatabase, more specifically, CatalogRelation's blocks.
    *
    * @param proto A serialized Protocol Buffer representation of a
    *        QueryContext, originally generated by the optimizer.
@@ -125,7 +123,7 @@
    * @param bus A pointer to the TMB.
    **/
   QueryContext(const serialization::QueryContext &proto,
-               CatalogDatabaseLite *database,
+               const CatalogDatabaseLite &database,
                StorageManager *storage_manager,
                const tmb::client_id foreman_client_id,
                tmb::MessageBus *bus);
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 43cd628..8d2efd0 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -22,7 +22,7 @@
 // WorkOrders.
 message WorkOrderCompletionMessage {
   required uint64 operator_index = 1;
-  required uint64 worker_id = 2;
+  required uint64 worker_thread_index = 2;
 }
 
 message CatalogRelationNewBlockMessage {
diff --git a/query_execution/QueryExecutionState.hpp b/query_execution/QueryExecutionState.hpp
new file mode 100644
index 0000000..7395461
--- /dev/null
+++ b/query_execution/QueryExecutionState.hpp
@@ -0,0 +1,293 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_STATE_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_STATE_HPP_
+
+#include <cstddef>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup QueryExecution
+ *  @{
+ */
+
+/**
+ * @brief A class that tracks the state of the execution of a query which
+ *        includes status of operators, number of dispatched work orders etc.
+ **/
+class QueryExecutionState {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param num_operators Number of relational operators in the query.
+   **/
+  explicit QueryExecutionState(const std::size_t num_operators)
+      : num_operators_(num_operators),
+        num_operators_finished_(0),
+        queued_workorders_per_op_(num_operators, 0),
+        rebuild_required_(num_operators, false),
+        done_gen_(num_operators, false),
+        execution_finished_(num_operators, false) {}
+
+  /**
+   * @brief Get the number of operators in the query.
+   **/
+  inline const std::size_t getNumOperators() const {
+    return num_operators_;
+  }
+
+  /**
+   * @brief Get the number of operators who have finished their execution.
+   **/
+  inline const std::size_t getNumOperatorsFinished() const {
+    return num_operators_finished_;
+  }
+
+  /**
+   * @brief Check if the query has finished its execution.
+   *
+   * @return True if the query has finished its execution, false otherwise.
+   **/
+  inline bool hasQueryExecutionFinished() const {
+    return num_operators_finished_ == num_operators_;
+  }
+
+  /**
+   * @brief Set the rebuild status of the given operator that includes the
+   *        flag for whether the rebuild has been initiated and if so, the
+   *        number of pending rebuild work orders.
+   *
+   * @param operator_index The index of the given operator.
+   * @param num_rebuild_workorders The number of rebuild workorders of the given
+   *        operator.
+   * @param rebuild_initiated True if the rebuild has been initiated, false
+   *        otherwise.
+   **/
+  inline void setRebuildStatus(const std::size_t operator_index,
+                               const std::size_t num_rebuild_workorders,
+                               const bool rebuild_initiated) {
+    DCHECK(operator_index < num_operators_);
+    rebuild_status_[operator_index].second = num_rebuild_workorders;
+    rebuild_status_[operator_index].first = rebuild_initiated;
+  }
+
+  /**
+   * @brief Check if the rebuild has been initiated for the given operator.
+   *
+   * @param operator_index The index of the given operator.
+   *
+   * @return True if the rebuild has been initiated, false otherwise.
+   **/
+  inline bool hasRebuildInitiated(const std::size_t operator_index) const {
+    DCHECK(operator_index < num_operators_);
+    const auto search_res = rebuild_status_.find(operator_index);
+    if (search_res != rebuild_status_.end()) {
+      return search_res->second.first;
+    }
+    return false;
+  }
+
+  /**
+   * @brief Get the number of pending rebuild workorders for the given operator.
+   *
+   * @param operator_index The index of the given operator.
+   *
+   * @return The number of pending rebuild workorders for the given operator.
+   **/
+  inline const std::size_t getNumRebuildWorkOrders(
+      const std::size_t operator_index) const {
+    DCHECK(operator_index < num_operators_);
+    const auto search_res = rebuild_status_.find(operator_index);
+    if (search_res != rebuild_status_.end()) {
+      return search_res->second.second;
+    }
+    LOG(WARNING) << "Called QueryExecutionState::getNumRebuildWorkOrders() "
+                    "for an operator whose rebuild entry doesn't exist.";
+    return 0;
+  }
+
+  /**
+   * @brief Decrement the number of rebuild WorkOrders for the given operator.
+   *
+   * @param operator_index The index of the given operator.
+   **/
+  inline void decrementNumRebuildWorkOrders(const std::size_t operator_index) {
+    DCHECK(operator_index < num_operators_);
+    const auto search_res = rebuild_status_.find(operator_index);
+    if (search_res != rebuild_status_.end()) {
+      DCHECK(search_res->second.first);
+      DCHECK_GE(search_res->second.second, 1u);
+      --rebuild_status_[operator_index].second;
+    } else {
+      LOG(FATAL) <<
+          "Called QueryExecutionState::decrementNumRebuildWorkOrders() for an "
+          "operator whose rebuild entry doesn't exist.";
+    }
+  }
+
+  /**
+   * @brief Increment the number of queued (normal) WorkOrders for the given
+   *        operator.
+   *
+   * @param operator_index The index of the given operator.
+   **/
+  inline void incrementNumQueuedWorkOrders(const std::size_t operator_index) {
+    DCHECK(operator_index < num_operators_);
+    ++queued_workorders_per_op_[operator_index];
+  }
+
+  /**
+   * @brief Decrement the number of queued (normal) WorkOrders for the given
+   *        operator.
+   *
+   * @param operator_index The index of the given operator.
+   **/
+  inline void decrementNumQueuedWorkOrders(const std::size_t operator_index) {
+    DCHECK(operator_index < num_operators_);
+    DCHECK_GT(queued_workorders_per_op_[operator_index], 0u);
+    --queued_workorders_per_op_[operator_index];
+  }
+
+  /**
+   * @brief Get the number of queued (normal) WorkOrders for the given operator.
+   *
+   * @note Queued WorkOrders mean those WorkOrders which have been dispatched
+   *       for execution by the Foreman and haven't yet completed. These are
+   *       different from pending WorkOrders which mean the WorkOrders that
+   *       haven't been dispatched for execution yet.
+   *
+   * @param operator_index The index of the given operator.
+   *
+   * @return The number of queued (normal) WorkOrders for the given operators.
+   **/
+  inline const std::size_t getNumQueuedWorkOrders(
+      const std::size_t operator_index) const {
+    DCHECK(operator_index < num_operators_);
+    return queued_workorders_per_op_[operator_index];
+  }
+
+  /**
+   * @brief Set the rebuild required flag as true for the given operator.
+   *
+   * @param operator_index The index of the given operator.
+   **/
+  inline void setRebuildRequired(const std::size_t operator_index) {
+    DCHECK(operator_index < num_operators_);
+    rebuild_required_[operator_index] = true;
+  }
+
+  /**
+   * @brief Get the rebuild required flag for the given operator.
+   *
+   * @param operator_index The index of the given operator.
+   **/
+  inline bool isRebuildRequired(const std::size_t operator_index) const {
+    DCHECK(operator_index < num_operators_);
+    return rebuild_required_[operator_index];
+  }
+
+  /**
+   * @brief Set the execution finished flag for the given operator as true.
+   *
+   * @note By default this flag is false.
+   *
+   * @param operator_index The index of the given operator.
+   **/
+  inline void setExecutionFinished(const std::size_t operator_index) {
+    DCHECK(operator_index < num_operators_);
+    execution_finished_[operator_index] = true;
+    ++num_operators_finished_;
+  }
+
+  /**
+   * @brief Get the execution finished flag for the given operator.
+   *
+   * @param operator_index The index of the given operator.
+   **/
+  inline bool hasExecutionFinished(const std::size_t operator_index) const {
+    DCHECK(operator_index < num_operators_);
+    return execution_finished_[operator_index];
+  }
+
+  /**
+   * @brief Set the "done generation of workorders" flag as true for the given
+   *        operator.
+   *
+   * @note By default this flag is false.
+   *
+   * @param operator_index The index of the given operator.
+   **/
+  inline void setDoneGenerationWorkOrders(const std::size_t operator_index) {
+    DCHECK(operator_index < num_operators_);
+    done_gen_[operator_index] = true;
+  }
+
+  /**
+   * @brief Get the "done generation of workorders" flag for the given operator.
+   *
+   * @param operator_index The index of the given operator.
+   **/
+  inline bool hasDoneGenerationWorkOrders(const std::size_t operator_index)
+      const {
+    DCHECK(operator_index < num_operators_);
+    return done_gen_[operator_index];
+  }
+
+ private:
+  // Total number of operators in the query.
+  const std::size_t num_operators_;
+
+  // Number of operators who've finished their execution.
+  std::size_t num_operators_finished_;
+
+  // A vector to track the number of normal WorkOrders in execution.
+  std::vector<std::size_t> queued_workorders_per_op_;
+
+  // The ith bit denotes if the operator with ID = i requires generation of
+  // rebuild WorkOrders.
+  std::vector<bool> rebuild_required_;
+
+  // The ith bit denotes if the operator with ID = i has finished generating
+  // work orders.
+  std::vector<bool> done_gen_;
+
+  // The ith bit denotes if the operator with ID = i has finished its execution.
+  std::vector<bool> execution_finished_;
+
+  // Key is dag_node_index for which rebuild is required. Value is a pair -
+  // first element is a bool (whether rebuild for operator at index i has been
+  // initiated) and if the boolean is true, the second element denotes the
+  // number of pending rebuild workorders for the operator.
+  std::unordered_map<std::size_t, std::pair<bool, std::size_t>> rebuild_status_;
+
+  DISALLOW_COPY_AND_ASSIGN(QueryExecutionState);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_STATE_HPP_
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index f049c18..caa2d9e 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -79,7 +79,7 @@
                                           const bool is_rebuild_work_order) {
   serialization::WorkOrderCompletionMessage proto;
   proto.set_operator_index(op_index);
-  proto.set_worker_id(worker_id_);
+  proto.set_worker_thread_index(worker_thread_index_);
 
   // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
   const size_t proto_length = proto.ByteSize();
diff --git a/query_execution/Worker.hpp b/query_execution/Worker.hpp
index 498fa32..90092d7 100644
--- a/query_execution/Worker.hpp
+++ b/query_execution/Worker.hpp
@@ -41,17 +41,18 @@
   /**
    * @brief Constructor
    *
-   * @param worker_id The unique ID of this worker thread.
+   * @param worker_thread_index The unique index of this worker thread in
+   *        WorkerDirectory.
    * @param bus A pointer to the TMB.
    * @param cpu_id The ID of the CPU to which this worker thread can be pinned.
    *
    * @note If cpu_id is not specified, worker thread can be possibly moved
    *       around on different CPUs by the OS.
    **/
-  Worker(std::size_t worker_id,
+  Worker(const std::size_t worker_thread_index,
          MessageBus *bus,
          int cpu_id = -1)
-      : worker_id_(worker_id),
+      : worker_thread_index_(worker_thread_index),
         bus_(bus),
         cpu_id_(cpu_id) {
     DEBUG_ASSERT(bus_ != nullptr);
@@ -102,7 +103,7 @@
                                     const std::size_t op_index,
                                     const bool is_rebuild_work_order);
 
-  const std::size_t worker_id_;
+  const std::size_t worker_thread_index_;
   MessageBus *bus_;
 
   const int cpu_id_;
diff --git a/query_execution/WorkerDirectory.hpp b/query_execution/WorkerDirectory.hpp
index 32164c6..a1f3b02 100644
--- a/query_execution/WorkerDirectory.hpp
+++ b/query_execution/WorkerDirectory.hpp
@@ -1,5 +1,5 @@
 /**
- *   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.
@@ -80,73 +80,73 @@
    *       execution and the workorder being executed by the specified worker
    *       at the time this function is called.
    *
-   * @param worker_id The logical ID of the given worker.
+   * @param worker_thread_index The logical ID of the given worker.
    *
    * @return The number of queued workorders.
    **/
   inline const std::size_t getNumQueuedWorkOrders(
-      const std::size_t worker_id) const {
-    DEBUG_ASSERT(worker_id < num_workers_);
-    return num_queued_workorders_[worker_id];
+      const std::size_t worker_thread_index) const {
+    DEBUG_ASSERT(worker_thread_index < num_workers_);
+    return num_queued_workorders_[worker_thread_index];
   }
 
   /**
    * @brief Increment the number of queued workorders for the given worker by 1.
    *
-   * @param worker_id The logical ID of the given worker.
+   * @param worker_thread_index The logical ID of the given worker.
    **/
-  inline void incrementNumQueuedWorkOrders(const std::size_t worker_id) {
-    DEBUG_ASSERT(worker_id < num_workers_);
-    ++num_queued_workorders_[worker_id];
+  inline void incrementNumQueuedWorkOrders(const std::size_t worker_thread_index) {
+    DEBUG_ASSERT(worker_thread_index < num_workers_);
+    ++num_queued_workorders_[worker_thread_index];
   }
 
   /**
    * @brief Decrement the number of queued workorders for the given worker by 1.
    *
-   * @param worker_id The logical ID of the given worker.
+   * @param worker_thread_index The logical ID of the given worker.
    **/
-  inline void decrementNumQueuedWorkOrders(const std::size_t worker_id) {
-    DEBUG_ASSERT(worker_id < num_workers_);
-    DEBUG_ASSERT(num_queued_workorders_[worker_id] >= 1);
-    --num_queued_workorders_[worker_id];
+  inline void decrementNumQueuedWorkOrders(const std::size_t worker_thread_index) {
+    DEBUG_ASSERT(worker_thread_index < num_workers_);
+    DEBUG_ASSERT(num_queued_workorders_[worker_thread_index] >= 1);
+    --num_queued_workorders_[worker_thread_index];
   }
 
   /**
    * @brief Get the NUMA node where the specified worker is pinned to.
    *
-   * @param worker_id The logical ID of the given worker.
+   * @param worker_thread_index The logical ID of the given worker.
    *
    * @return The NUMA node ID where the given worker is pinned. If the worker
    *         hasn't been pinned to any NUMA node, this value is -1.
    **/
-  inline int getNUMANode(const std::size_t worker_id) const {
-    DEBUG_ASSERT(worker_id < num_workers_);
-    return numa_node_ids_[worker_id];
+  inline int getNUMANode(const std::size_t worker_thread_index) const {
+    DEBUG_ASSERT(worker_thread_index < num_workers_);
+    return numa_node_ids_[worker_thread_index];
   }
 
   /**
    * @brief Get the TMB client ID of the specified worker.
    *
-   * @param worker_id The logical ID of the given worker.
+   * @param worker_thread_index The logical ID of the given worker.
    *
    * @return The TMB client ID of the given worker.
    **/
-  inline const client_id getClientID(const std::size_t worker_id) const {
-    DEBUG_ASSERT(worker_id < num_workers_);
-    return client_ids_[worker_id];
+  inline const client_id getClientID(const std::size_t worker_thread_index) const {
+    DEBUG_ASSERT(worker_thread_index < num_workers_);
+    return client_ids_[worker_thread_index];
   }
 
   /**
    * @brief Generate address of a worker.
    *
-   * @param worker_id The logical ID of the given worker.
+   * @param worker_thread_index The logical ID of the given worker.
    *
    * @return TMB Address of the given worker.
    **/
-  inline Address getWorkerAddress(std::size_t worker_id) const {
-    DEBUG_ASSERT(worker_id < num_workers_);
+  inline Address getWorkerAddress(std::size_t worker_thread_index) const {
+    DEBUG_ASSERT(worker_thread_index < num_workers_);
     Address worker_address;
-    worker_address.AddRecipient(client_ids_[worker_id]);
+    worker_address.AddRecipient(client_ids_[worker_thread_index]);
     return worker_address;
   }
 
@@ -185,9 +185,9 @@
         std::min_element(std::begin(num_queued_workorders_),
                          std::end(num_queued_workorders_));
     DEBUG_ASSERT(min_element_iter != num_queued_workorders_.end());
-    const std::size_t least_loaded_worker_id =
+    const std::size_t least_loaded_worker_thread_index =
         std::distance(num_queued_workorders_.begin(), min_element_iter);
-    return std::make_pair(least_loaded_worker_id, *min_element_iter);
+    return std::make_pair(least_loaded_worker_thread_index, *min_element_iter);
   }
 
   /**
@@ -212,9 +212,9 @@
         std::max_element(std::begin(num_queued_workorders_),
                          std::end(num_queued_workorders_));
     DEBUG_ASSERT(max_element_iter != num_queued_workorders_.end());
-    const std::size_t most_loaded_worker_id =
+    const std::size_t most_loaded_worker_thread_index =
         std::distance(num_queued_workorders_.begin(), max_element_iter);
-    return std::make_pair(most_loaded_worker_id, *max_element_iter);
+    return std::make_pair(most_loaded_worker_thread_index, *max_element_iter);
   }
 
  private:
diff --git a/query_execution/WorkerSelectionPolicy.hpp b/query_execution/WorkerSelectionPolicy.hpp
index 2ee18b8..289b1c4 100644
--- a/query_execution/WorkerSelectionPolicy.hpp
+++ b/query_execution/WorkerSelectionPolicy.hpp
@@ -1,5 +1,5 @@
 /**
- *   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.
@@ -70,7 +70,7 @@
    * @brief Constructor.
    *
    * @param worker_directory The worker directory.
-   * @param start_worker_id The ID of the worker which should be chosen
+   * @param start_worker_thread_index The ID of the worker which should be chosen
    *        initially.
    *
    * @warning If the number of workers change between the constructor and
@@ -78,23 +78,23 @@
    *          getNextWorkerID() initially may not be the same as desired.
    **/
   RoundRobinWorkerSelectionPolicy(const WorkerDirectory &worker_directory,
-                                  const std::size_t start_worker_id)
+                                  const std::size_t start_worker_thread_index)
       : WorkerSelectionPolicy(worker_directory) {
-    if (start_worker_id == 0) {
-      previous_worker_id_ = worker_directory.getNumWorkers() - 1;
+    if (start_worker_thread_index == 0) {
+      previous_worker_thread_index_ = worker_directory.getNumWorkers() - 1;
     } else {
-      previous_worker_id_ = start_worker_id - 1;
+      previous_worker_thread_index_ = start_worker_thread_index - 1;
     }
   }
 
   std::size_t getNextWorkerID() override {
-    previous_worker_id_ =
-        (previous_worker_id_ + 1) % worker_directory_.getNumWorkers();
-    return previous_worker_id_;
+    previous_worker_thread_index_ =
+        (previous_worker_thread_index_ + 1) % worker_directory_.getNumWorkers();
+    return previous_worker_thread_index_;
   }
 
  private:
-  std::size_t previous_worker_id_;
+  std::size_t previous_worker_thread_index_;
 
   DISALLOW_COPY_AND_ASSIGN(RoundRobinWorkerSelectionPolicy);
 };
diff --git a/query_execution/tests/Foreman_unittest.cpp b/query_execution/tests/Foreman_unittest.cpp
index 119a491..47cc641 100644
--- a/query_execution/tests/Foreman_unittest.cpp
+++ b/query_execution/tests/Foreman_unittest.cpp
@@ -26,6 +26,7 @@
 #include "query_execution/Foreman.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryContext.pb.h"
+#include "query_execution/QueryExecutionState.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
 #include "query_execution/WorkerDirectory.hpp"
@@ -251,15 +252,15 @@
   }
 
   inline const int getNumWorkOrdersInExecution(const QueryPlan::DAGNodeIndex index) const {
-    return foreman_->queued_workorders_per_op_[index];
+    return foreman_->query_exec_state_->getNumQueuedWorkOrders(index);
   }
 
   inline const int getNumOperatorsFinished() const {
-    return foreman_->num_operators_finished_;
+    return foreman_->query_exec_state_->getNumOperatorsFinished();
   }
 
   inline bool getOperatorFinishedStatus(const QueryPlan::DAGNodeIndex index) const {
-    return foreman_->execution_finished_[index];
+    return foreman_->query_exec_state_->hasExecutionFinished(index);
   }
 
   inline bool popWorkOrderIfAvailable(MockWorkOrder **workorder) {
@@ -279,19 +280,19 @@
   inline bool placeDataPipelineMessage(const QueryPlan::DAGNodeIndex source_operator_index) {
     VLOG(3) << "Place DataPipeline message for Op[" << source_operator_index << "]";
     foreman_->processDataPipelineMessage(source_operator_index, 0 /* block_id */, 0 /* relation_id */);
-    return foreman_->checkQueryExecutionFinished();
+    return foreman_->query_exec_state_->hasQueryExecutionFinished();
   }
 
   inline bool placeWorkOrderCompleteMessage(const QueryPlan::DAGNodeIndex index) {
     VLOG(3) << "Place WorkOrderComplete message for Op[" << index << "]";
     foreman_->processWorkOrderCompleteMessage(index, 0 /* worker id */);
-    return foreman_->checkQueryExecutionFinished();
+    return foreman_->query_exec_state_->hasQueryExecutionFinished();
   }
 
   inline bool placeRebuildWorkOrderCompleteMessage(const QueryPlan::DAGNodeIndex index) {
     VLOG(3) << "Place RebuildWorkOrderComplete message for Op[" << index << "]";
     foreman_->processRebuildWorkOrderCompleteMessage(index, 0 /* worker id */);
-    return foreman_->checkQueryExecutionFinished();
+    return foreman_->query_exec_state_->hasQueryExecutionFinished();
   }
 
   inline bool placeOutputBlockMessage(const QueryPlan::DAGNodeIndex index) {
@@ -299,12 +300,12 @@
     foreman_->processDataPipelineMessage(index,
                                          BlockIdUtil::GetBlockId(1 /* domain */, 1),
                                          0 /* relation_id */);
-    return foreman_->checkQueryExecutionFinished();
+    return foreman_->query_exec_state_->hasQueryExecutionFinished();
   }
 
   inline bool startForeman() {
     foreman_->initialize();
-    return foreman_->checkQueryExecutionFinished();
+    return foreman_->query_exec_state_->hasQueryExecutionFinished();
   }
 
   inline int getWorkerInputQueueSize() {
diff --git a/query_execution/tests/WorkerDirectory_unittest.cpp b/query_execution/tests/WorkerDirectory_unittest.cpp
index 90cdde5..eb83583 100644
--- a/query_execution/tests/WorkerDirectory_unittest.cpp
+++ b/query_execution/tests/WorkerDirectory_unittest.cpp
@@ -1,5 +1,5 @@
 /**
- *   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.
@@ -62,29 +62,29 @@
     numa_nodes.reserve(kNumWorkers);
     client_ids.reserve(kNumWorkers);
 
-    for (std::size_t worker_id = 0; worker_id < kNumWorkers; ++worker_id) {
-      // NUMA node id = worker_id % 4
-      // Client ID = worker_id * 2 + 1
-      const client_id cid = worker_id * 2 + 1;
-      const int numa_node_id = worker_id % 4;
+    for (std::size_t worker_thread_index = 0; worker_thread_index < kNumWorkers; ++worker_thread_index) {
+      // NUMA node id = worker_thread_index % 4
+      // Client ID = worker_thread_index * 2 + 1
+      const client_id cid = worker_thread_index * 2 + 1;
+      const int numa_node_id = worker_thread_index % 4;
 
       numa_nodes.push_back(numa_node_id);
       client_ids.push_back(cid);
 
-      worker_ids_.push_back(worker_id);
-      actual_workers_[worker_id] = std::make_pair(numa_node_id, cid);
+      worker_thread_indexs_.push_back(worker_thread_index);
+      actual_workers_[worker_thread_index] = std::make_pair(numa_node_id, cid);
     }
     wd_.reset(new WorkerDirectory(kNumWorkers, client_ids, numa_nodes));
 
     // Randomize the order of worker IDs.
-    std::random_shuffle(worker_ids_.begin(), worker_ids_.end());
+    std::random_shuffle(worker_thread_indexs_.begin(), worker_thread_indexs_.end());
   }
 
   // First element is NUMA Node. Second element is client ID.
   // Input is the logical worker ID.
   std::pair<int, client_id> getNUMANodeAndClientIDForWorker(
-      const std::size_t worker_id) {
-    return actual_workers_[worker_id];
+      const std::size_t worker_thread_index) {
+    return actual_workers_[worker_thread_index];
   }
 
   WorkerDirectory* getWorkerDirectory() {
@@ -96,8 +96,8 @@
   }
 
   std::vector<size_t>& getRandomizedWorkerIDs() {
-    std::random_shuffle(worker_ids_.begin(), worker_ids_.end());
-    return worker_ids_;
+    std::random_shuffle(worker_thread_indexs_.begin(), worker_thread_indexs_.end());
+    return worker_thread_indexs_;
   }
 
  private:
@@ -107,7 +107,7 @@
   // Value = pair <NUMA node ID, client ID> for the worker.
   std::unordered_map<std::size_t, std::pair<int, client_id>> actual_workers_;
 
-  std::vector<std::size_t> worker_ids_;
+  std::vector<std::size_t> worker_thread_indexs_;
 
   const std::size_t kNumWorkers = 100;
 };
@@ -119,12 +119,12 @@
 
   EXPECT_EQ(getActualNumWorkers(), wd->getNumWorkers());
 
-  for (std::size_t worker_id : getRandomizedWorkerIDs()) {
+  for (std::size_t worker_thread_index : getRandomizedWorkerIDs()) {
     const std::pair<int, client_id> observed_ids =
-        getNUMANodeAndClientIDForWorker(worker_id);
-    EXPECT_EQ(observed_ids.first, wd->getNUMANode(worker_id));
-    EXPECT_EQ(observed_ids.second, wd->getClientID(worker_id));
-    EXPECT_EQ(0u, wd->getNumQueuedWorkOrders(worker_id));
+        getNUMANodeAndClientIDForWorker(worker_thread_index);
+    EXPECT_EQ(observed_ids.first, wd->getNUMANode(worker_thread_index));
+    EXPECT_EQ(observed_ids.second, wd->getClientID(worker_thread_index));
+    EXPECT_EQ(0u, wd->getNumQueuedWorkOrders(worker_thread_index));
   }
 }
 
@@ -159,38 +159,38 @@
 
   // Perform the increment or decrement operation as determined by the sequence
   // and check the correctness.
-  std::stack<std::size_t> worker_ids_used;
+  std::stack<std::size_t> worker_thread_indexs_used;
   for (bool to_increment : increment_decrement_sequence) {
     if (to_increment) {
       // Pick a random worker ID and increment its number of workorders.
-      const std::size_t chosen_worker_id = dist(mt);
-      worker_ids_used.push(chosen_worker_id);
-      EXPECT_EQ(actual_num_workorders[chosen_worker_id],
-                wd->getNumQueuedWorkOrders(chosen_worker_id));
-      wd->incrementNumQueuedWorkOrders(chosen_worker_id);
-      ++actual_num_workorders[chosen_worker_id];
-      EXPECT_EQ(actual_num_workorders[chosen_worker_id],
-                wd->getNumQueuedWorkOrders(chosen_worker_id));
+      const std::size_t chosen_worker_thread_index = dist(mt);
+      worker_thread_indexs_used.push(chosen_worker_thread_index);
+      EXPECT_EQ(actual_num_workorders[chosen_worker_thread_index],
+                wd->getNumQueuedWorkOrders(chosen_worker_thread_index));
+      wd->incrementNumQueuedWorkOrders(chosen_worker_thread_index);
+      ++actual_num_workorders[chosen_worker_thread_index];
+      EXPECT_EQ(actual_num_workorders[chosen_worker_thread_index],
+                wd->getNumQueuedWorkOrders(chosen_worker_thread_index));
     } else {
       // For the worker with ID = top of stack, decrement a workorder.
-      const std::size_t chosen_worker_id = worker_ids_used.top();
-      worker_ids_used.pop();
-      EXPECT_EQ(actual_num_workorders[chosen_worker_id],
-                wd->getNumQueuedWorkOrders(chosen_worker_id));
-      wd->decrementNumQueuedWorkOrders(chosen_worker_id);
-      --actual_num_workorders[chosen_worker_id];
-      EXPECT_EQ(actual_num_workorders[chosen_worker_id],
-                wd->getNumQueuedWorkOrders(chosen_worker_id));
+      const std::size_t chosen_worker_thread_index = worker_thread_indexs_used.top();
+      worker_thread_indexs_used.pop();
+      EXPECT_EQ(actual_num_workorders[chosen_worker_thread_index],
+                wd->getNumQueuedWorkOrders(chosen_worker_thread_index));
+      wd->decrementNumQueuedWorkOrders(chosen_worker_thread_index);
+      --actual_num_workorders[chosen_worker_thread_index];
+      EXPECT_EQ(actual_num_workorders[chosen_worker_thread_index],
+                wd->getNumQueuedWorkOrders(chosen_worker_thread_index));
     }
   }
 
   // Stack should be empty.
-  EXPECT_TRUE(worker_ids_used.empty());
+  EXPECT_TRUE(worker_thread_indexs_used.empty());
   // Expect no queued up workorders for any worker.
-  for (const std::size_t random_worker_id : getRandomizedWorkerIDs()) {
-    EXPECT_EQ(0u, wd->getNumQueuedWorkOrders(random_worker_id));
-    EXPECT_EQ(actual_num_workorders[random_worker_id],
-              wd->getNumQueuedWorkOrders(random_worker_id));
+  for (const std::size_t random_worker_thread_index : getRandomizedWorkerIDs()) {
+    EXPECT_EQ(0u, wd->getNumQueuedWorkOrders(random_worker_thread_index));
+    EXPECT_EQ(actual_num_workorders[random_worker_thread_index],
+              wd->getNumQueuedWorkOrders(random_worker_thread_index));
   }
 }
 
@@ -204,21 +204,21 @@
   wd->addWorker(new_worker_client_id, new_worker_numa_node);
 
   // The logical ID of the new worker.
-  const std::size_t new_worker_id = getActualNumWorkers();
+  const std::size_t new_worker_thread_index = getActualNumWorkers();
 
   EXPECT_EQ(getActualNumWorkers() + 1, wd->getNumWorkers());
   // Check if the client ID is set correctly.
-  EXPECT_EQ(new_worker_client_id, wd->getClientID(new_worker_id));
+  EXPECT_EQ(new_worker_client_id, wd->getClientID(new_worker_thread_index));
   // Check if the NUMA node ID is set correctly.
-  EXPECT_EQ(new_worker_numa_node, wd->getNUMANode(new_worker_id));
+  EXPECT_EQ(new_worker_numa_node, wd->getNUMANode(new_worker_thread_index));
   // Check if the new worker has no queued up workorders.
-  EXPECT_EQ(0u, wd->getNumQueuedWorkOrders(new_worker_id));
+  EXPECT_EQ(0u, wd->getNumQueuedWorkOrders(new_worker_thread_index));
   // Increment a workorder for the new worker, check if the increment is
   // successful, then perform a decrement and check the correctness.
-  wd->incrementNumQueuedWorkOrders(new_worker_id);
-  EXPECT_EQ(1u, wd->getNumQueuedWorkOrders(new_worker_id));
-  wd->decrementNumQueuedWorkOrders(new_worker_id);
-  EXPECT_EQ(0u, wd->getNumQueuedWorkOrders(new_worker_id));
+  wd->incrementNumQueuedWorkOrders(new_worker_thread_index);
+  EXPECT_EQ(1u, wd->getNumQueuedWorkOrders(new_worker_thread_index));
+  wd->decrementNumQueuedWorkOrders(new_worker_thread_index);
+  EXPECT_EQ(0u, wd->getNumQueuedWorkOrders(new_worker_thread_index));
 }
 
 TEST_F(WorkerDirectoryTest, WorkerLoadTest) {
@@ -231,8 +231,8 @@
   actual_num_workorders.resize(getActualNumWorkers(), 0);
 
   // Loop over workers sequentially and increment workorder of all the workers.
-  for (std::size_t worker_id = 0; worker_id < getActualNumWorkers();
-       ++worker_id) {
+  for (std::size_t worker_thread_index = 0; worker_thread_index < getActualNumWorkers();
+       ++worker_thread_index) {
     const std::pair<std::size_t, std::size_t> &actual_min_loaded_worker =
         getMinElement(actual_num_workorders);
     const std::pair<std::size_t, std::size_t> &actual_max_loaded_worker =
@@ -241,15 +241,15 @@
     EXPECT_EQ(actual_min_loaded_worker, wd->getLeastLoadedWorker());
     EXPECT_EQ(actual_max_loaded_worker, wd->getMostLoadedWorker());
 
-    wd->incrementNumQueuedWorkOrders(worker_id);
-    ++actual_num_workorders[worker_id];
-    EXPECT_EQ(actual_num_workorders[worker_id],
-              wd->getNumQueuedWorkOrders(worker_id));
+    wd->incrementNumQueuedWorkOrders(worker_thread_index);
+    ++actual_num_workorders[worker_thread_index];
+    EXPECT_EQ(actual_num_workorders[worker_thread_index],
+              wd->getNumQueuedWorkOrders(worker_thread_index));
   }
 
   // At this time, every worker has exactly one workorder assigned to it.
   // Now increment workorders in a random order.
-  for (const std::size_t random_worker_id : getRandomizedWorkerIDs()) {
+  for (const std::size_t random_worker_thread_index : getRandomizedWorkerIDs()) {
     const std::pair<std::size_t, std::size_t> actual_min_loaded_worker =
         getMinElement(actual_num_workorders);
     const std::pair<std::size_t, std::size_t> actual_max_loaded_worker =
@@ -258,16 +258,16 @@
     EXPECT_EQ(actual_min_loaded_worker, wd->getLeastLoadedWorker());
     EXPECT_EQ(actual_max_loaded_worker, wd->getMostLoadedWorker());
 
-    wd->incrementNumQueuedWorkOrders(random_worker_id);
-    ++actual_num_workorders[random_worker_id];
-    EXPECT_EQ(actual_num_workorders[random_worker_id],
-              wd->getNumQueuedWorkOrders(random_worker_id));
+    wd->incrementNumQueuedWorkOrders(random_worker_thread_index);
+    ++actual_num_workorders[random_worker_thread_index];
+    EXPECT_EQ(actual_num_workorders[random_worker_thread_index],
+              wd->getNumQueuedWorkOrders(random_worker_thread_index));
   }
 
   // At this time, every worker has two workorders assigned to it.
   // Now decrement workorders in a random order twice.
   for (std::size_t iteration = 0; iteration < 2; ++iteration) {
-    for (const std::size_t random_worker_id : getRandomizedWorkerIDs()) {
+    for (const std::size_t random_worker_thread_index : getRandomizedWorkerIDs()) {
       const std::pair<std::size_t, std::size_t> actual_min_loaded_worker =
           getMinElement(actual_num_workorders);
       const std::pair<std::size_t, std::size_t> actual_max_loaded_worker =
@@ -276,10 +276,10 @@
       EXPECT_EQ(actual_min_loaded_worker, wd->getLeastLoadedWorker());
       EXPECT_EQ(actual_max_loaded_worker, wd->getMostLoadedWorker());
 
-      wd->decrementNumQueuedWorkOrders(random_worker_id);
-      --actual_num_workorders[random_worker_id];
-      EXPECT_EQ(actual_num_workorders[random_worker_id],
-                wd->getNumQueuedWorkOrders(random_worker_id));
+      wd->decrementNumQueuedWorkOrders(random_worker_thread_index);
+      --actual_num_workorders[random_worker_thread_index];
+      EXPECT_EQ(actual_num_workorders[random_worker_thread_index],
+                wd->getNumQueuedWorkOrders(random_worker_thread_index));
     }
   }
   const std::pair<std::size_t, std::size_t> actual_min_loaded_worker =
diff --git a/query_execution/tests/WorkerSelectionPolicy_unittest.cpp b/query_execution/tests/WorkerSelectionPolicy_unittest.cpp
index 32f161e..83d3c78 100644
--- a/query_execution/tests/WorkerSelectionPolicy_unittest.cpp
+++ b/query_execution/tests/WorkerSelectionPolicy_unittest.cpp
@@ -1,5 +1,5 @@
 /**
- *   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.
@@ -49,11 +49,11 @@
 
     std::random_device rd;
     mt_.reset(new std::mt19937_64(rd()));
-    for (std::size_t worker_id = 0; worker_id < kNumWorkers; ++worker_id) {
-      // NUMA node id = worker_id % 4
-      // Client ID = worker_id * 2 + 1
-      numa_nodes.push_back(worker_id % 4);
-      client_ids.push_back(worker_id * 2 + 1);
+    for (std::size_t worker_thread_index = 0; worker_thread_index < kNumWorkers; ++worker_thread_index) {
+      // NUMA node id = worker_thread_index % 4
+      // Client ID = worker_thread_index * 2 + 1
+      numa_nodes.push_back(worker_thread_index % 4);
+      client_ids.push_back(worker_thread_index * 2 + 1);
     }
     directory_.reset(new WorkerDirectory(kNumWorkers, client_ids, numa_nodes));
   }
@@ -72,10 +72,10 @@
 
   const std::size_t kNumIterations =
       (getWorkerDirectory()->getNumWorkers()) * 2;
-  std::size_t expected_next_worker_id = kStartWorkerID;
+  std::size_t expected_next_worker_thread_index = kStartWorkerID;
   for (std::size_t iteration = 0; iteration < kNumIterations; ++iteration) {
-    EXPECT_EQ(expected_next_worker_id, rr_policy.getNextWorkerID());
-    expected_next_worker_id = (expected_next_worker_id + 1) %
+    EXPECT_EQ(expected_next_worker_thread_index, rr_policy.getNextWorkerID());
+    expected_next_worker_thread_index = (expected_next_worker_thread_index + 1) %
                               (getWorkerDirectory()->getNumWorkers());
   }
 }
@@ -90,11 +90,11 @@
   EXPECT_EQ(kStartWorkerID, rr_policy.getNextWorkerID());
 
   // Add a new worker.
-  const std::size_t new_worker_id = getWorkerDirectory()->getNumWorkers();
-  getWorkerDirectory()->addWorker(new_worker_id * 2 + 1 /** client_id **/,
+  const std::size_t new_worker_thread_index = getWorkerDirectory()->getNumWorkers();
+  getWorkerDirectory()->addWorker(new_worker_thread_index * 2 + 1 /** client_id **/,
                                   0 /** NUMA node ID **/);
 
-  EXPECT_EQ(new_worker_id, rr_policy.getNextWorkerID());
+  EXPECT_EQ(new_worker_thread_index, rr_policy.getNextWorkerID());
 }
 
 TEST_F(WorkerSelectionPolicyTest, LoadBalancingTest) {
@@ -107,9 +107,9 @@
   std::mt19937_64 mt(rd());
   std::uniform_int_distribution<std::size_t> dist(0, kMaxLoad);
 
-  for (std::size_t worker_id = 0;
-       worker_id < getWorkerDirectory()->getNumWorkers();
-       ++worker_id) {
+  for (std::size_t worker_thread_index = 0;
+       worker_thread_index < getWorkerDirectory()->getNumWorkers();
+       ++worker_thread_index) {
     // For each worker ..
     const std::size_t worker_load = dist(mt);
     // assign a random load.
@@ -117,7 +117,7 @@
     for (std::size_t workorder_count = 0;
          workorder_count < worker_load;
          ++workorder_count) {
-      getWorkerDirectory()->incrementNumQueuedWorkOrders(worker_id);
+      getWorkerDirectory()->incrementNumQueuedWorkOrders(worker_thread_index);
     }
   }
 
@@ -148,18 +148,18 @@
   const std::size_t kNumIterations = 100;
 
   for (std::size_t iter = 0; iter < kNumIterations; ++iter) {
-    const std::size_t chosen_worker_id = rand_policy.getNextWorkerID();
-    EXPECT_GE(getWorkerDirectory()->getNumWorkers(), chosen_worker_id);
+    const std::size_t chosen_worker_thread_index = rand_policy.getNextWorkerID();
+    EXPECT_GE(getWorkerDirectory()->getNumWorkers(), chosen_worker_thread_index);
   }
 
   // Add a new worker and repeat the test above.
-  const std::size_t new_worker_id = getWorkerDirectory()->getNumWorkers();
-  getWorkerDirectory()->addWorker(new_worker_id * 2 + 1 /** client_id **/,
+  const std::size_t new_worker_thread_index = getWorkerDirectory()->getNumWorkers();
+  getWorkerDirectory()->addWorker(new_worker_thread_index * 2 + 1 /** client_id **/,
                                   0 /** NUMA node ID **/);
 
   for (std::size_t iter = 0; iter < kNumIterations; ++iter) {
-    const std::size_t chosen_worker_id = rand_policy.getNextWorkerID();
-    EXPECT_GE(getWorkerDirectory()->getNumWorkers(), chosen_worker_id);
+    const std::size_t chosen_worker_thread_index = rand_policy.getNextWorkerID();
+    EXPECT_GE(getWorkerDirectory()->getNumWorkers(), chosen_worker_thread_index);
   }
 }
 
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 6b7d097..a11a994 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -76,6 +76,7 @@
                       quickstep_queryoptimizer_physical_DeleteTuples
                       quickstep_queryoptimizer_physical_DropTable
                       quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_InsertSelection
                       quickstep_queryoptimizer_physical_InsertTuple
                       quickstep_queryoptimizer_physical_NestedLoopsJoin
                       quickstep_queryoptimizer_physical_PatternMatcher
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 68a07ea..f9d7d98 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -58,6 +58,7 @@
 #include "query_optimizer/physical/DeleteTuples.hpp"
 #include "query_optimizer/physical/DropTable.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/InsertSelection.hpp"
 #include "query_optimizer/physical/InsertTuple.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
 #include "query_optimizer/physical/PatternMatcher.hpp"
@@ -213,6 +214,9 @@
     case P::PhysicalType::kHashJoin:
       return convertHashJoin(
           std::static_pointer_cast<const P::HashJoin>(physical_plan));
+    case P::PhysicalType::kInsertSelection:
+      return convertInsertSelection(
+          std::static_pointer_cast<const P::InsertSelection>(physical_plan));
     case P::PhysicalType::kInsertTuple:
       return convertInsertTuple(
           std::static_pointer_cast<const P::InsertTuple>(physical_plan));
@@ -1020,6 +1024,87 @@
                                        false /* is_pipeline_breaker */);
 }
 
+void ExecutionGenerator::convertInsertSelection(
+    const P::InsertSelectionPtr &physical_plan) {
+  // InsertSelection is converted to a Select and a SaveBlocks.
+
+  const CatalogRelationInfo *destination_relation_info =
+      findRelationInfoOutputByPhysical(physical_plan->destination());
+  const CatalogRelation &destination_relation = *destination_relation_info->relation;
+
+  // FIXME(qzeng): A better way is using a traits struct to look up whether a storage
+  //               block supports ad-hoc insertion instead of hard-coding the block types.
+  const StorageBlockLayout &storage_block_layout =
+      destination_relation.getDefaultStorageBlockLayout();
+  if (storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
+          TupleStorageSubBlockDescription::COMPRESSED_COLUMN_STORE
+      || storage_block_layout.getDescription().tuple_store_description().sub_block_type() ==
+             TupleStorageSubBlockDescription::COMPRESSED_PACKED_ROW_STORE) {
+    THROW_SQL_ERROR() << "INSERT statement is not supported for the relation "
+                      << destination_relation.getName()
+                      << ", because its storage blocks do not support ad-hoc insertion";
+  }
+
+  // Create InsertDestination proto.
+  const QueryContext::insert_destination_id insert_destination_index =
+      query_context_proto_->insert_destinations_size();
+  S::InsertDestination *insert_destination_proto = query_context_proto_->add_insert_destinations();
+
+  insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
+  insert_destination_proto->set_relation_id(destination_relation.getID());
+  insert_destination_proto->mutable_layout()->MergeFrom(
+      destination_relation.getDefaultStorageBlockLayout().getDescription());
+
+  const vector<block_id> blocks(destination_relation.getBlocksSnapshot());
+  for (const block_id block : blocks) {
+    insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+  }
+
+  const CatalogRelationInfo *selection_relation_info =
+      findRelationInfoOutputByPhysical(physical_plan->selection());
+
+  // Prepare the attributes, which are output columns of the selection relation.
+  std::unique_ptr<std::vector<attribute_id>> attributes(new std::vector<attribute_id>());
+  for (E::AttributeReferencePtr attr_ref : physical_plan->selection()->getOutputAttributes()) {
+    unique_ptr<const Scalar> attribute(attr_ref->concretize(attribute_substitution_map_));
+
+    DCHECK_EQ(Scalar::kAttribute, attribute->getDataSource());
+    attributes->emplace_back(
+        static_cast<const ScalarAttribute*>(attribute.get())->getAttribute().getID());
+  }
+
+  // Create the select operator.
+  // TODO(jianqiao): This select operator is actually redundant. That is,
+  // we may directly set physical_plan_->selection()'s output relation to be
+  // destination_relation, instead of creating an intermediate selection_relation
+  // and then copy the data into destination_relation. One way to achieve this
+  // optimization is to enable specifying a specific output relation for each
+  // physical plan by modifying class Physical.
+  SelectOperator *insert_selection_op =
+      new SelectOperator(*selection_relation_info->relation,
+                         destination_relation,
+                         insert_destination_index,
+                         QueryContext::kInvalidPredicateId,
+                         attributes.release(),
+                         selection_relation_info->isStoredRelation());
+
+  const QueryPlan::DAGNodeIndex insert_selection_index =
+      execution_plan_->addRelationalOperator(insert_selection_op);
+  insert_destination_proto->set_relational_op_index(insert_selection_index);
+
+  const QueryPlan::DAGNodeIndex save_blocks_index =
+      execution_plan_->addRelationalOperator(new SaveBlocksOperator());
+
+  if (!selection_relation_info->isStoredRelation()) {
+    execution_plan_->addDirectDependency(insert_selection_index,
+                                         selection_relation_info->producer_operator_index,
+                                         false /* is_pipeline_breaker */);
+  }
+  execution_plan_->addDirectDependency(save_blocks_index,
+                                       insert_selection_index,
+                                       false /* is_pipeline_breaker */);
+}
+
 void ExecutionGenerator::convertUpdateTable(
     const P::UpdateTablePtr &physical_plan) {
   // UpdateTable is converted to an Update and a SaveBlocks.
@@ -1124,6 +1209,9 @@
       unique_ptr<const Scalar> concretized_argument(argument->concretize(attribute_substitution_map_));
       aggr_proto->add_argument()->CopyFrom(concretized_argument->getProto());
     }
+
+    // Set whether it is a DISTINCT aggregation.
+    aggr_proto->set_is_distinct(unnamed_aggregate_expression->is_distinct());
   }
 
   std::vector<const Type*> group_by_types;
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index 78cecc9..11de106 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -40,6 +40,7 @@
 #include "query_optimizer/physical/DeleteTuples.hpp"
 #include "query_optimizer/physical/DropTable.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/InsertSelection.hpp"
 #include "query_optimizer/physical/InsertTuple.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
@@ -292,6 +293,13 @@
   void convertDropTable(const physical::DropTablePtr &physical_plan);
 
   /**
+   * @brief Converts an InsertSelection to a Select and a SaveBlocks.
+   *
+   * @param physical_plan The InsertSelection to be converted.
+   */
+  void convertInsertSelection(const physical::InsertSelectionPtr &physical_plan);
+
+  /**
    * @brief Converts an InsertTuple to an Insert and a SaveBlocks.
    *
    * @param physical_plan The InsertTuple to be converted.
diff --git a/query_optimizer/expressions/AggregateFunction.cpp b/query_optimizer/expressions/AggregateFunction.cpp
index 1722450..e061f03 100644
--- a/query_optimizer/expressions/AggregateFunction.cpp
+++ b/query_optimizer/expressions/AggregateFunction.cpp
@@ -1,5 +1,7 @@
 /**
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -59,7 +61,8 @@
 AggregateFunctionPtr AggregateFunction::Create(
     const ::quickstep::AggregateFunction &aggregate,
     const std::vector<ScalarPtr> &arguments,
-    const bool is_vector_aggregate) {
+    const bool is_vector_aggregate,
+    const bool is_distinct) {
 #ifdef QUICKSTEP_DEBUG
   std::vector<const Type*> argument_types;
   for (const ScalarPtr &argument : arguments) {
@@ -69,7 +72,7 @@
 #endif  // QUICKSTEP_DEBUG
 
   return AggregateFunctionPtr(
-      new AggregateFunction(aggregate, arguments, is_vector_aggregate));
+      new AggregateFunction(aggregate, arguments, is_vector_aggregate, is_distinct));
 }
 
 ExpressionPtr AggregateFunction::copyWithNewChildren(
@@ -82,7 +85,7 @@
     new_arguments.emplace_back(std::move(expr_as_scalar));
   }
 
-  return Create(aggregate_, new_arguments, is_vector_aggregate_);
+  return Create(aggregate_, new_arguments, is_vector_aggregate_, is_distinct_);
 }
 
 std::vector<AttributeReferencePtr> AggregateFunction::getReferencedAttributes() const {
@@ -107,6 +110,11 @@
   inline_field_names->push_back("function");
   inline_field_values->push_back(aggregate_.getName());
 
+  if (is_distinct_) {
+    inline_field_names->push_back("is_distinct");
+    inline_field_values->push_back("true");
+  }
+
   container_child_field_names->push_back("");
   container_child_fields->emplace_back(CastSharedPtrVector<OptimizerTreeBase>(arguments_));
 }
diff --git a/query_optimizer/expressions/AggregateFunction.hpp b/query_optimizer/expressions/AggregateFunction.hpp
index dcc09ea..86698fd 100644
--- a/query_optimizer/expressions/AggregateFunction.hpp
+++ b/query_optimizer/expressions/AggregateFunction.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -104,6 +106,13 @@
     return arguments_;
   }
 
+  /*
+   * @return Whether this is a DISTINCT aggregation.
+   */
+  inline bool is_distinct() const {
+    return is_distinct_;
+  }
+
   /**
    * @brief Create a new AggregateFunction.
    *
@@ -114,11 +123,13 @@
    *        system.
    * @param arguments A list of arguments to the aggregate function.
    * @param is_vector_aggregate Whether the aggregate has a GROUP BY clause.
+   * @param is_distinct Whether this is a DISTINCT aggregation.
    * @return A new AggregateFunctionPtr.
    **/
   static AggregateFunctionPtr Create(const ::quickstep::AggregateFunction &aggregate,
                                      const std::vector<ScalarPtr> &arguments,
-                                     const bool is_vector_aggregate);
+                                     const bool is_vector_aggregate,
+                                     const bool is_distinct);
 
  protected:
   void getFieldStringItems(
@@ -137,13 +148,16 @@
    * @param arguments The scalar arguments to the aggregate function.
    * @param is_vector_aggregate Indicates whether this aggregation is a vector
    *                            aggregation (i.e. GROUP BY exists).
+   * @param is_distinct Indicates whether this is a DISTINCT aggregation.
    */
   AggregateFunction(const ::quickstep::AggregateFunction &aggregate,
                     const std::vector<ScalarPtr> &arguments,
-                    const bool is_vector_aggregate)
+                    const bool is_vector_aggregate,
+                    const bool is_distinct)
       : aggregate_(aggregate),
         arguments_(arguments),
-        is_vector_aggregate_(is_vector_aggregate) {
+        is_vector_aggregate_(is_vector_aggregate),
+        is_distinct_(is_distinct) {
     for (const ScalarPtr &child : arguments_) {
       addChild(child);
     }
@@ -151,7 +165,8 @@
 
   const ::quickstep::AggregateFunction &aggregate_;
   std::vector<ScalarPtr> arguments_;
-  bool is_vector_aggregate_;
+  const bool is_vector_aggregate_;
+  const bool is_distinct_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregateFunction);
 };
diff --git a/query_optimizer/expressions/UnaryExpression.cpp b/query_optimizer/expressions/UnaryExpression.cpp
index 508ed6c..ae88d57 100644
--- a/query_optimizer/expressions/UnaryExpression.cpp
+++ b/query_optimizer/expressions/UnaryExpression.cpp
@@ -37,8 +37,7 @@
 namespace expressions {
 
 std::string UnaryExpression::getName() const {
-  DCHECK(operation_.getUnaryOperationID() == UnaryOperationID::kNegate);
-  return "Negate";
+  return operation_.getName();
 }
 
 ExpressionPtr UnaryExpression::copyWithNewChildren(
diff --git a/query_optimizer/logical/CMakeLists.txt b/query_optimizer/logical/CMakeLists.txt
index de660a4..38cff8d 100644
--- a/query_optimizer/logical/CMakeLists.txt
+++ b/query_optimizer/logical/CMakeLists.txt
@@ -23,6 +23,7 @@
 add_library(quickstep_queryoptimizer_logical_DropTable DropTable.cpp DropTable.hpp)
 add_library(quickstep_queryoptimizer_logical_Filter Filter.cpp Filter.hpp)
 add_library(quickstep_queryoptimizer_logical_HashJoin ../../empty_src.cpp HashJoin.hpp)
+add_library(quickstep_queryoptimizer_logical_InsertSelection InsertSelection.cpp InsertSelection.hpp)
 add_library(quickstep_queryoptimizer_logical_InsertTuple InsertTuple.cpp InsertTuple.hpp)
 add_library(quickstep_queryoptimizer_logical_Join ../../empty_src.cpp Join.hpp)
 add_library(quickstep_queryoptimizer_logical_Logical ../../empty_src.cpp Logical.hpp)
@@ -123,6 +124,13 @@
                       quickstep_queryoptimizer_logical_LogicalType
                       quickstep_utility_Cast
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_logical_InsertSelection
+                      glog
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_logical_Logical
+                      quickstep_queryoptimizer_logical_LogicalType
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_logical_InsertTuple
                       glog
                       quickstep_queryoptimizer_OptimizerTree
@@ -249,6 +257,7 @@
                       quickstep_queryoptimizer_logical_DropTable
                       quickstep_queryoptimizer_logical_Filter
                       quickstep_queryoptimizer_logical_HashJoin
+                      quickstep_queryoptimizer_logical_InsertSelection
                       quickstep_queryoptimizer_logical_InsertTuple
                       quickstep_queryoptimizer_logical_Join
                       quickstep_queryoptimizer_logical_Logical
diff --git a/query_optimizer/logical/InsertSelection.cpp b/query_optimizer/logical/InsertSelection.cpp
new file mode 100644
index 0000000..2744b44
--- /dev/null
+++ b/query_optimizer/logical/InsertSelection.cpp
@@ -0,0 +1,45 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "query_optimizer/logical/InsertSelection.hpp"
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+
+namespace quickstep {
+namespace optimizer {
+namespace logical {
+
+void InsertSelection::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
+  non_container_child_field_names->push_back("destination");
+  non_container_child_fields->push_back(destination_);
+
+  non_container_child_field_names->push_back("selection");
+  non_container_child_fields->push_back(selection_);
+}
+
+}  // namespace logical
+}  // namespace optimizer
+}  // namespace quickstep
diff --git a/query_optimizer/logical/InsertSelection.hpp b/query_optimizer/logical/InsertSelection.hpp
new file mode 100644
index 0000000..c36ecd0
--- /dev/null
+++ b/query_optimizer/logical/InsertSelection.hpp
@@ -0,0 +1,120 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_INSERT_SELECTION_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_INSERT_SELECTION_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/logical/Logical.hpp"
+#include "query_optimizer/logical/LogicalType.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace logical {
+
+/** \addtogroup OptimizerLogical
+ *  @{
+ */
+
+class InsertSelection;
+typedef std::shared_ptr<const InsertSelection> InsertSelectionPtr;
+
+/**
+ * @brief Inserts the results from a SELECT query into a relation.
+ */
+class InsertSelection : public Logical {
+ public:
+  LogicalType getLogicalType() const override { return LogicalType::kInsertSelection; }
+
+  std::string getName() const override { return "InsertSelection"; }
+
+  /**
+   * @return The destination relation to insert the selection to.
+   */
+  const LogicalPtr& destination() const { return destination_; }
+
+  /**
+   * @return The selection to be inserted.
+   */
+  const LogicalPtr& selection() const { return selection_; }
+
+  LogicalPtr copyWithNewChildren(
+      const std::vector<LogicalPtr> &new_children) const override {
+    DCHECK_EQ(getNumChildren(), new_children.size());
+    return Create(new_children[0], new_children[1]);
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override {
+    return {};
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override {
+    return selection_->getOutputAttributes();
+  }
+
+  /**
+   * @brief Creates an InsertSelection logical node.
+   *
+   * @param destination The destination relation to insert the selection to.
+   * @param selection The selection to be inserted.
+   * @return An immutable InsertSelection node.
+   */
+  static InsertSelectionPtr Create(
+      const LogicalPtr &destination,
+      const LogicalPtr &selection) {
+    return InsertSelectionPtr(new InsertSelection(destination, selection));
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
+
+ private:
+  InsertSelection(const LogicalPtr &destination,
+                  const LogicalPtr &selection)
+      : destination_(destination),
+        selection_(selection) {
+    addChild(destination);
+    addChild(selection);
+  }
+
+  LogicalPtr destination_;
+  LogicalPtr selection_;
+
+  DISALLOW_COPY_AND_ASSIGN(InsertSelection);
+};
+
+/** @} */
+
+}  // namespace logical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_INSERT_SELECTION_HPP_ */
diff --git a/query_optimizer/logical/LogicalType.hpp b/query_optimizer/logical/LogicalType.hpp
index cf5fd58..1b9366e 100644
--- a/query_optimizer/logical/LogicalType.hpp
+++ b/query_optimizer/logical/LogicalType.hpp
@@ -38,6 +38,7 @@
   kDropTable,
   kFilter,
   kHashJoin,
+  kInsertSelection,
   kInsertTuple,
   kMultiwayCartesianJoin,
   kNestedLoopsJoin,
diff --git a/query_optimizer/physical/CMakeLists.txt b/query_optimizer/physical/CMakeLists.txt
index 0bfa717..d833423 100644
--- a/query_optimizer/physical/CMakeLists.txt
+++ b/query_optimizer/physical/CMakeLists.txt
@@ -22,6 +22,7 @@
 add_library(quickstep_queryoptimizer_physical_DeleteTuples DeleteTuples.cpp DeleteTuples.hpp)
 add_library(quickstep_queryoptimizer_physical_DropTable DropTable.cpp DropTable.hpp)
 add_library(quickstep_queryoptimizer_physical_HashJoin HashJoin.cpp HashJoin.hpp)
+add_library(quickstep_queryoptimizer_physical_InsertSelection InsertSelection.cpp InsertSelection.hpp)
 add_library(quickstep_queryoptimizer_physical_InsertTuple InsertTuple.cpp InsertTuple.hpp)
 add_library(quickstep_queryoptimizer_physical_Join ../../empty_src.cpp Join.hpp)
 add_library(quickstep_queryoptimizer_physical_NestedLoopsJoin NestedLoopsJoin.cpp NestedLoopsJoin.hpp)
@@ -121,6 +122,14 @@
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_utility_Cast
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_physical_InsertSelection
+                      glog
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_physical_InsertTuple
                       glog
                       quickstep_queryoptimizer_OptimizerTree
@@ -253,6 +262,7 @@
                       quickstep_queryoptimizer_physical_DeleteTuples
                       quickstep_queryoptimizer_physical_DropTable
                       quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_InsertSelection
                       quickstep_queryoptimizer_physical_InsertTuple
                       quickstep_queryoptimizer_physical_Join
                       quickstep_queryoptimizer_physical_NestedLoopsJoin
diff --git a/query_optimizer/physical/InsertSelection.cpp b/query_optimizer/physical/InsertSelection.cpp
new file mode 100644
index 0000000..0e45222
--- /dev/null
+++ b/query_optimizer/physical/InsertSelection.cpp
@@ -0,0 +1,45 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "query_optimizer/physical/InsertSelection.hpp"
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+void InsertSelection::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
+  non_container_child_field_names->push_back("destination");
+  non_container_child_fields->push_back(destination_);
+
+  non_container_child_field_names->push_back("selection");
+  non_container_child_fields->push_back(selection_);
+}
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep
diff --git a/query_optimizer/physical/InsertSelection.hpp b/query_optimizer/physical/InsertSelection.hpp
new file mode 100644
index 0000000..8bdcfcb
--- /dev/null
+++ b/query_optimizer/physical/InsertSelection.hpp
@@ -0,0 +1,129 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_INSERT_SELECTION_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_INSERT_SELECTION_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+/** \addtogroup OptimizerPhysical
+ *  @{
+ */
+
+class InsertSelection;
+typedef std::shared_ptr<const InsertSelection> InsertSelectionPtr;
+
+/**
+ * @brief Inserts the results from a SELECT query into a relation.
+ */
+class InsertSelection : public Physical {
+ public:
+  PhysicalType getPhysicalType() const override {
+    return PhysicalType::kInsertSelection;
+  }
+
+  std::string getName() const override { return "InsertSelection"; }
+
+  /**
+   * @return The destination relation to insert the selection to.
+   */
+  const PhysicalPtr& destination() const { return destination_; }
+
+  /**
+   * @return The selection to be inserted.
+   */
+  const PhysicalPtr& selection() const { return selection_; }
+
+  PhysicalPtr copyWithNewChildren(
+      const std::vector<PhysicalPtr> &new_children) const override {
+    DCHECK_EQ(getNumChildren(), new_children.size());
+    return Create(new_children[0], new_children[1]);
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override {
+    return {};
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override {
+    return selection_->getOutputAttributes();
+  }
+
+  bool maybeCopyWithPrunedExpressions(
+      const expressions::UnorderedNamedExpressionSet &referenced_expressions,
+      PhysicalPtr *output) const override {
+    return false;
+  }
+
+  /**
+   * @brief Creates an InsertSelection physical node.
+   *
+   * @param destination The destination relation to insert the selection to.
+   * @param selection The selection to be inserted.
+   * @return An immutable InsertSelection node.
+   */
+  static InsertSelectionPtr Create(
+      const PhysicalPtr &destination,
+      const PhysicalPtr &selection) {
+    return InsertSelectionPtr(new InsertSelection(destination, selection));
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
+
+ private:
+  InsertSelection(const PhysicalPtr &destination,
+                  const PhysicalPtr &selection)
+      : destination_(destination),
+        selection_(selection) {
+    addChild(destination_);
+    addChild(selection_);
+  }
+
+  PhysicalPtr destination_;
+  PhysicalPtr selection_;
+
+  DISALLOW_COPY_AND_ASSIGN(InsertSelection);
+};
+
+/** @} */
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_INSERT_SELECTION_HPP_ */
diff --git a/query_optimizer/physical/PhysicalType.hpp b/query_optimizer/physical/PhysicalType.hpp
index 1803f54..b036557 100644
--- a/query_optimizer/physical/PhysicalType.hpp
+++ b/query_optimizer/physical/PhysicalType.hpp
@@ -37,6 +37,7 @@
   kDeleteTuples,
   kDropTable,
   kHashJoin,
+  kInsertSelection,
   kInsertTuple,
   kNestedLoopsJoin,
   kSample,
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index 461fbd6..22cf397 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -82,6 +82,7 @@
                       quickstep_queryoptimizer_logical_DeleteTuples
                       quickstep_queryoptimizer_logical_DropTable
                       quickstep_queryoptimizer_logical_Filter
+                      quickstep_queryoptimizer_logical_InsertSelection
                       quickstep_queryoptimizer_logical_InsertTuple
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_logical_MultiwayCartesianJoin
@@ -104,9 +105,11 @@
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
+                      quickstep_types_operations_unaryoperations_DateExtractOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_utility_Macros
                       quickstep_utility_PtrList
+                      quickstep_utility_PtrVector
                       quickstep_utility_SqlError
                       quickstep_utility_StringUtil)
 
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 3161bbf..7fb5a90 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -80,6 +80,7 @@
 #include "query_optimizer/logical/DeleteTuples.hpp"
 #include "query_optimizer/logical/DropTable.hpp"
 #include "query_optimizer/logical/Filter.hpp"
+#include "query_optimizer/logical/InsertSelection.hpp"
 #include "query_optimizer/logical/InsertTuple.hpp"
 #include "query_optimizer/logical/MultiwayCartesianJoin.hpp"
 #include "query_optimizer/logical/Project.hpp"
@@ -101,8 +102,10 @@
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
+#include "types/operations/unary_operations/DateExtractOperation.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 #include "utility/PtrList.hpp"
+#include "utility/PtrVector.hpp"
 #include "utility/SqlError.hpp"
 #include "utility/StringUtil.hpp"
 
@@ -275,38 +278,48 @@
       logical_plan_ = resolveDropTable(
           static_cast<const ParseStatementDropTable&>(parse_query));
       break;
-    case ParseStatement::kInsert:
+    case ParseStatement::kInsert: {
       context_->set_is_catalog_changed();
-      logical_plan_ =
-          resolveInsert(static_cast<const ParseStatementInsert&>(parse_query));
+      const ParseStatementInsert &insert_statement =
+          static_cast<const ParseStatementInsert&>(parse_query);
+      if (insert_statement.getInsertType() == ParseStatementInsert::InsertType::kTuple) {
+        logical_plan_ =
+            resolveInsertTuple(static_cast<const ParseStatementInsertTuple&>(insert_statement));
+      } else {
+        DCHECK(insert_statement.getInsertType() == ParseStatementInsert::InsertType::kSelection);
+        const ParseStatementInsertSelection &insert_selection_statement =
+            static_cast<const ParseStatementInsertSelection&>(insert_statement);
+
+        if (insert_selection_statement.with_clause() != nullptr) {
+          resolveWithClause(*insert_selection_statement.with_clause());
+        }
+        logical_plan_ = resolveInsertSelection(insert_selection_statement);
+
+        if (insert_selection_statement.with_clause() != nullptr) {
+          // Report an error if there is a WITH query that is not actually used.
+          if (!with_queries_info_.unreferenced_query_indexes.empty()) {
+            int unreferenced_with_query_index = *with_queries_info_.unreferenced_query_indexes.begin();
+            const ParseSubqueryTableReference &unreferenced_with_query =
+                (*insert_selection_statement.with_clause())[unreferenced_with_query_index];
+            THROW_SQL_ERROR_AT(&unreferenced_with_query)
+                << "WITH query "
+                << unreferenced_with_query.table_reference_signature()->table_alias()->value()
+                << " is defined but not used";
+          }
+        }
+      }
       break;
+    }
     case ParseStatement::kSelect: {
       const ParseStatementSelect &select_statement =
           static_cast<const ParseStatementSelect&>(parse_query);
       if (select_statement.with_clause() != nullptr) {
-        int index = 0;
-        for (const ParseSubqueryTableReference &with_table_reference : *select_statement.with_clause()) {
-          NameResolver name_resolver;
-          with_queries_info_.with_query_plans.emplace_back(
-              resolveTableReference(with_table_reference, &name_resolver));
-
-          const ParseString *reference_alias = with_table_reference.table_reference_signature()->table_alias();
-          const std::string lower_alias_name = ToLower(reference_alias->value());
-          if (with_queries_info_.with_query_name_to_vector_position.find(lower_alias_name)
-                  != with_queries_info_.with_query_name_to_vector_position.end()) {
-            THROW_SQL_ERROR_AT(reference_alias)
-                << "WITH query name " << reference_alias->value()
-                << " is specified more than once";
-          }
-
-          with_queries_info_.with_query_name_to_vector_position.emplace(lower_alias_name, index);
-          with_queries_info_.unreferenced_query_indexes.insert(index);
-          ++index;
-        }
+        resolveWithClause(*select_statement.with_clause());
       }
       logical_plan_ =
           resolveSelect(*select_statement.select_query(),
-                        "" /* select_name */);
+                        "" /* select_name */,
+                        nullptr /* No Type hints */);
       if (select_statement.with_clause() != nullptr) {
         // Report an error if there is a WITH query that is not actually used.
         if (!with_queries_info_.unreferenced_query_indexes.empty()) {
@@ -687,8 +700,82 @@
       resolveRelationName(drop_table_statement.relation_name()));
 }
 
-L::LogicalPtr Resolver::resolveInsert(
-    const ParseStatementInsert &insert_statement) {
+L::LogicalPtr Resolver::resolveInsertSelection(
+    const ParseStatementInsertSelection &insert_statement) {
+  NameResolver name_resolver;
+
+  // Resolve the destination relation.
+  const L::LogicalPtr destination_logical = resolveSimpleTableReference(
+      *insert_statement.relation_name(), nullptr /* reference alias */);
+  name_resolver.addRelation(insert_statement.relation_name(),
+                            destination_logical);
+  const std::vector<E::AttributeReferencePtr> destination_attributes =
+      destination_logical->getOutputAttributes();
+
+  // Gather type information of the destination relation columns.
+  std::vector<const Type *> type_hints;
+  for (E::AttributeReferencePtr attr : destination_attributes) {
+    type_hints.emplace_back(&attr->getValueType());
+  }
+
+  // Resolve the selection query.
+  const L::LogicalPtr selection_logical =
+      resolveSelect(*insert_statement.select_query(), "", &type_hints);
+  const std::vector<E::AttributeReferencePtr> selection_attributes =
+      selection_logical->getOutputAttributes();
+
+  // Check that number of columns match between the selection relation and
+  // the destination relation.
+  if (selection_attributes.size() != destination_attributes.size()) {
+    THROW_SQL_ERROR_AT(insert_statement.relation_name())
+        << "The relation " << insert_statement.relation_name()->value()
+        << " has " << std::to_string(destination_attributes.size())
+        << " columns, but " << std::to_string(selection_attributes.size())
+        << " columns are generated by the SELECT query";
+  }
+
+  // Add cast operation if the selection column type does not match the destination
+  // column type
+  std::vector<E::NamedExpressionPtr> cast_expressions;
+  for (std::vector<E::AttributeReferencePtr>::size_type aid = 0;
+       aid < destination_attributes.size();
+       ++aid) {
+    const Type& destination_type = destination_attributes[aid]->getValueType();
+    const Type& selection_type = selection_attributes[aid]->getValueType();
+    if (destination_type.equals(selection_type)) {
+      cast_expressions.emplace_back(selection_attributes[aid]);
+    } else {
+      // TODO(jianqiao): implement Cast operation for non-numeric types.
+      if (destination_type.getSuperTypeID() == Type::kNumeric
+          && selection_type.getSuperTypeID() == Type::kNumeric
+          && destination_type.isSafelyCoercibleFrom(selection_type)) {
+        // Add cast operation
+        const E::AttributeReferencePtr attr = selection_attributes[aid];
+        const E::ExpressionPtr cast_expr =
+            E::Cast::Create(attr, destination_type);
+        cast_expressions.emplace_back(
+            E::Alias::Create(context_->nextExprId(),
+                             cast_expr,
+                             attr->attribute_name(),
+                             attr->attribute_alias()));
+      } else {
+        THROW_SQL_ERROR_AT(insert_statement.relation_name())
+            << "The assigned value for the column "
+            << insert_statement.relation_name()->value() << "."
+            << destination_attributes[aid]->attribute_name() << " has type "
+            << selection_attributes[aid]->getValueType().getName()
+            << ", which cannot be safely coerced to the column's type "
+            << destination_attributes[aid]->getValueType().getName();
+     }
+    }
+  }
+  return L::InsertSelection::Create(
+      destination_logical,
+      L::Project::Create(selection_logical, cast_expressions));
+}
+
+L::LogicalPtr Resolver::resolveInsertTuple(
+    const ParseStatementInsertTuple &insert_statement) {
   NameResolver name_resolver;
 
   const L::LogicalPtr input_logical = resolveSimpleTableReference(
@@ -838,7 +925,8 @@
 
 L::LogicalPtr Resolver::resolveSelect(
     const ParseSelect &select_query,
-    const std::string &select_name) {
+    const std::string &select_name,
+    const std::vector<const Type*> *type_hints) {
   // Create a new name scope. We currently do not support correlated query.
   std::unique_ptr<NameResolver> name_resolver(new NameResolver());
 
@@ -865,6 +953,7 @@
   std::vector<bool> has_aggregate_per_expression;
   resolveSelectClause(select_query.selection(),
                       select_name,
+                      type_hints,
                       *name_resolver,
                       &query_aggregation_info,
                       &select_list_expressions,
@@ -1223,6 +1312,29 @@
   }
 }
 
+void Resolver::resolveWithClause(
+    const PtrVector<ParseSubqueryTableReference> &with_list) {
+  int index = 0;
+  for (const ParseSubqueryTableReference &with_table_reference : with_list) {
+    NameResolver name_resolver;
+    with_queries_info_.with_query_plans.emplace_back(
+        resolveTableReference(with_table_reference, &name_resolver));
+
+    const ParseString *reference_alias = with_table_reference.table_reference_signature()->table_alias();
+    const std::string lower_alias_name = ToLower(reference_alias->value());
+    if (with_queries_info_.with_query_name_to_vector_position.find(lower_alias_name)
+            != with_queries_info_.with_query_name_to_vector_position.end()) {
+      THROW_SQL_ERROR_AT(reference_alias)
+          << "WITH query name " << reference_alias->value()
+          << " is specified more than once";
+    }
+
+    with_queries_info_.with_query_name_to_vector_position.emplace(lower_alias_name, index);
+    with_queries_info_.unreferenced_query_indexes.insert(index);
+    ++index;
+  }
+}
+
 L::LogicalPtr Resolver::resolveFromClause(
     const PtrList<ParseTableReference> &from_list,
     NameResolver *name_resolver) {
@@ -1303,7 +1415,8 @@
       reference_alias = reference_signature->table_alias();
       logical_plan = resolveSelect(
           *static_cast<const ParseSubqueryTableReference&>(table_reference).subquery_expr()->query(),
-          reference_alias->value());
+          reference_alias->value(),
+          nullptr /* No Type hints */);
 
       if (reference_signature->column_aliases() != nullptr) {
         logical_plan = RenameOutputColumns(logical_plan, *reference_signature);
@@ -1440,6 +1553,7 @@
 void Resolver::resolveSelectClause(
     const ParseSelectionClause &parse_selection,
     const std::string &select_name,
+    const std::vector<const Type*> *type_hints,
     const NameResolver &name_resolver,
     QueryAggregationInfo *query_aggregation_info,
     std::vector<expressions::NamedExpressionPtr> *project_expressions,
@@ -1457,6 +1571,13 @@
     case ParseSelectionClause::kNonStar: {
       const ParseSelectionList &selection_list =
           static_cast<const ParseSelectionList&>(parse_selection);
+
+      // Ignore type hints if its size does not match the number of columns.
+      if (type_hints != nullptr && type_hints->size() != selection_list.select_item_list().size()) {
+        type_hints = nullptr;
+      }
+
+      std::vector<const Type*>::size_type tid = 0;
       for (const ParseSelectionItem &selection_item :
            selection_list.select_item_list()) {
         const ParseString *parse_alias = selection_item.alias();
@@ -1468,7 +1589,7 @@
             nullptr /* select_list_info */);
         const E::ScalarPtr project_scalar =
             resolveExpression(*parse_project_expression,
-                              nullptr,  // No Type hint.
+                              (type_hints == nullptr ? nullptr : type_hints->at(tid)),
                               &expr_resolution_info);
 
         // If the resolved expression is a named expression,
@@ -1514,6 +1635,7 @@
         project_expressions->push_back(project_named_expression);
         has_aggregate_per_expression->push_back(
             expr_resolution_info.hasAggregate());
+        ++tid;
       }
       break;
     }
@@ -1756,6 +1878,44 @@
       THROW_SQL_ERROR_AT(&parse_expression)
           << "Subquery expression in a non-FROM clause is not supported yet";
     }
+    case ParseExpression::kExtract: {
+      const ParseExtractFunction &parse_extract =
+          static_cast<const ParseExtractFunction&>(parse_expression);
+
+      const ParseString &extract_field = *parse_extract.extract_field();
+      const std::string lowered_unit = ToLower(extract_field.value());
+      DateExtractUnit extract_unit;
+      if (lowered_unit == "year") {
+        extract_unit = DateExtractUnit::kYear;
+      } else if (lowered_unit == "month") {
+        extract_unit = DateExtractUnit::kMonth;
+      } else if (lowered_unit == "day") {
+        extract_unit = DateExtractUnit::kDay;
+      } else if (lowered_unit == "hour") {
+        extract_unit = DateExtractUnit::kHour;
+      } else if (lowered_unit == "minute") {
+        extract_unit = DateExtractUnit::kMinute;
+      } else if (lowered_unit == "second") {
+        extract_unit = DateExtractUnit::kSecond;
+      } else {
+        THROW_SQL_ERROR_AT(&extract_field)
+            << "Invalid extract unit: " << extract_field.value();
+      }
+
+      const DateExtractOperation &op = DateExtractOperation::Instance(extract_unit);
+      const E::ScalarPtr argument = resolveExpression(
+          *parse_extract.date_expression(),
+          op.pushDownTypeHint(type_hint),
+          expression_resolution_info);
+
+      if (!op.canApplyToType(argument->getValueType())) {
+        THROW_SQL_ERROR_AT(parse_extract.date_expression())
+            << "Can not extract from argument of type: "
+            << argument->getValueType().getName();
+      }
+
+      return E::UnaryExpression::Create(op, argument);
+    }
     default:
       LOG(FATAL) << "Unknown scalar type: "
                  << parse_expression.getExpressionType();
@@ -1853,7 +2013,8 @@
   const E::AggregateFunctionPtr aggregate_function
       = E::AggregateFunction::Create(*aggregate,
                                      resolved_arguments,
-                                     expression_resolution_info->query_aggregation_info->has_group_by);
+                                     expression_resolution_info->query_aggregation_info->has_group_by,
+                                     parse_function_call.is_distinct());
   const std::string internal_alias = GenerateAggregateAttributeAlias(
       expression_resolution_info->query_aggregation_info->aggregate_expressions.size());
   const E::AliasPtr aggregate_alias = E::Alias::Create(context_->nextExprId(),
diff --git a/query_optimizer/resolver/Resolver.hpp b/query_optimizer/resolver/Resolver.hpp
index bcbf94d..3b75114 100644
--- a/query_optimizer/resolver/Resolver.hpp
+++ b/query_optimizer/resolver/Resolver.hpp
@@ -30,6 +30,7 @@
 #include "query_optimizer/expressions/Scalar.hpp"
 #include "query_optimizer/logical/Logical.hpp"
 #include "utility/Macros.hpp"
+#include "utility/PtrVector.hpp"
 
 namespace quickstep {
 
@@ -44,13 +45,15 @@
 class ParseSelect;
 class ParseSelectionClause;
 class ParseSimpleTableReference;
+class ParseSubqueryTableReference;
 class ParseStatement;
 class ParseStatementCopyFrom;
 class ParseStatementCreateTable;
 class ParseStatementCreateIndex;
 class ParseStatementDelete;
 class ParseStatementDropTable;
-class ParseStatementInsert;
+class ParseStatementInsertSelection;
+class ParseStatementInsertTuple;
 class ParseStatementSelect;
 class ParseStatementUpdate;
 class ParseString;
@@ -147,11 +150,14 @@
    *
    * @param select_statement The SELECT parse tree.
    * @param select_name The name for the SELECT query.
+   * @param type_hints Type hints for the expressions in the SELECT clause. Can
+   *                   be NULL if there is no expectation.
    * @return A logical plan for the SELECT query.
    */
   logical::LogicalPtr resolveSelect(
       const ParseSelect &select_statement,
-      const std::string &select_name);
+      const std::string &select_name,
+      const std::vector<const Type*> *type_hints);
 
   /**
    * @brief Resolves a CREATE TABLE query and returns a logical plan.
@@ -201,13 +207,22 @@
       const ParseStatementDropTable &drop_table_statement);
 
   /**
-   * @brief Resolves a INSERT query and returns a logical plan.
+   * @brief Resolves an INSERT VALUES query and returns a logical plan.
    *
    * @param insert_statement The INSERT parse tree.
    * @return A logical plan for the INSERT query.
    */
-  logical::LogicalPtr resolveInsert(
-      const ParseStatementInsert &insert_statement);
+  logical::LogicalPtr resolveInsertTuple(
+      const ParseStatementInsertTuple &insert_statement);
+
+  /**
+   * @brief Resolves an INSERT SELECT query and returns a logical plan.
+   *
+   * @param insert_statement The INSERT parse tree.
+   * @return A logical plan for the INSERT query.
+   */
+  logical::LogicalPtr resolveInsertSelection(
+      const ParseStatementInsertSelection &insert_statement);
 
   /**
    * @brief Resolves a COPY FROM query and returns a logical plan.
@@ -228,6 +243,15 @@
       const ParseStatementUpdate &update_statement);
 
   /**
+   * @brief Resolves WITH clause and stores the resolved information into
+   * the member object with_query_info_ of this class.
+   *
+   * @param with_list The list of subqueries in WITH clause.
+   */
+  void resolveWithClause(
+      const PtrVector<ParseSubqueryTableReference> &with_list);
+
+  /**
    * @brief Resolves SELECT-list expressions and returns the resolved
    *        expressions in \p project_expressions.
    * @warning \p project_expressions will be cleared first.
@@ -235,6 +259,8 @@
    * @param parse_selection The SELECT parse tree.
    * @param select_name The name of the SELECT query. It is used to populate the
    *                    relation name in the expressions.
+   * @param type_hints Type hints for the expressions in the SELECT clause. Can
+   *                   be NULL if there is no expectation.
    * @param name_resolver NameResolver to resolve the relation/attribute names.
    * @param query_aggregation_info Passed down to each expression to collects
    *                               aggregate expressions.
@@ -246,6 +272,7 @@
   void resolveSelectClause(
       const ParseSelectionClause &parse_selection,
       const std::string &select_name,
+      const std::vector<const Type*> *type_hints,
       const NameResolver &name_resolver,
       QueryAggregationInfo *query_aggregation_info,
       std::vector<expressions::NamedExpressionPtr> *project_expressions,
diff --git a/query_optimizer/strategy/CMakeLists.txt b/query_optimizer/strategy/CMakeLists.txt
index 3def33d..e4052cf 100644
--- a/query_optimizer/strategy/CMakeLists.txt
+++ b/query_optimizer/strategy/CMakeLists.txt
@@ -78,6 +78,7 @@
                       quickstep_queryoptimizer_logical_CreateTable
                       quickstep_queryoptimizer_logical_DeleteTuples
                       quickstep_queryoptimizer_logical_DropTable
+                      quickstep_queryoptimizer_logical_InsertSelection
                       quickstep_queryoptimizer_logical_InsertTuple
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_logical_Logical
@@ -94,6 +95,7 @@
                       quickstep_queryoptimizer_physical_CreateTable
                       quickstep_queryoptimizer_physical_DeleteTuples
                       quickstep_queryoptimizer_physical_DropTable
+                      quickstep_queryoptimizer_physical_InsertSelection
                       quickstep_queryoptimizer_physical_InsertTuple
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_physical_Sample
diff --git a/query_optimizer/strategy/OneToOne.cpp b/query_optimizer/strategy/OneToOne.cpp
index aff0d90..d4d79f9 100644
--- a/query_optimizer/strategy/OneToOne.cpp
+++ b/query_optimizer/strategy/OneToOne.cpp
@@ -31,6 +31,7 @@
 #include "query_optimizer/logical/CreateTable.hpp"
 #include "query_optimizer/logical/DeleteTuples.hpp"
 #include "query_optimizer/logical/DropTable.hpp"
+#include "query_optimizer/logical/InsertSelection.hpp"
 #include "query_optimizer/logical/InsertTuple.hpp"
 #include "query_optimizer/logical/LogicalType.hpp"
 #include "query_optimizer/logical/Sample.hpp"
@@ -45,6 +46,7 @@
 #include "query_optimizer/physical/CreateTable.hpp"
 #include "query_optimizer/physical/DeleteTuples.hpp"
 #include "query_optimizer/physical/DropTable.hpp"
+#include "query_optimizer/physical/InsertSelection.hpp"
 #include "query_optimizer/physical/InsertTuple.hpp"
 #include "query_optimizer/physical/Sample.hpp"
 #include "query_optimizer/physical/SharedSubplanReference.hpp"
@@ -133,6 +135,14 @@
       *physical_output = P::DropTable::Create(drop_table->catalog_relation());
       return true;
     }
+    case L::LogicalType::kInsertSelection: {
+      const L::InsertSelectionPtr insert_selection =
+          std::static_pointer_cast<const L::InsertSelection>(logical_input);
+      *physical_output = P::InsertSelection::Create(
+          physical_mapper_->createOrGetPhysicalFromLogical(insert_selection->destination()),
+          physical_mapper_->createOrGetPhysicalFromLogical(insert_selection->selection()));
+      return true;
+    }
     case L::LogicalType::kInsertTuple: {
       const L::InsertTuplePtr insert_tuple =
           std::static_pointer_cast<const L::InsertTuple>(logical_input);
diff --git a/query_optimizer/strategy/tests/Aggregate_unittest.cpp b/query_optimizer/strategy/tests/Aggregate_unittest.cpp
index 193435b..1f4eb95 100644
--- a/query_optimizer/strategy/tests/Aggregate_unittest.cpp
+++ b/query_optimizer/strategy/tests/Aggregate_unittest.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -64,13 +66,15 @@
   }
 
   E::AggregateFunctionPtr createSum(const E::ScalarPtr &argument,
-                                    const bool is_vector_aggregate) {
+                                    const bool is_vector_aggregate,
+                                    const bool is_distinct) {
     std::vector<E::ScalarPtr> args(1, argument);
 
     return E::AggregateFunction::Create(
         AggregateFunctionFactory::Get(AggregationID::kSum),
         args,
-        is_vector_aggregate);
+        is_vector_aggregate,
+        is_distinct);
   }
 
  private:
@@ -78,54 +82,63 @@
 };
 
 TEST_F(AggregateTest, BasicTest) {
-  const E::AggregateFunctionPtr sum = createSum(add_literal_0_, true);
-  const E::AliasPtr sum_alias = createAlias(sum,
-                                            "sum" /* attribute_name */,
-                                            "test_relation" /* relation_name */);
-  input_logical_ = L::Aggregate::Create(
-      logical_table_reference_0_,
-      {alias_0_} /* grouping_expressions */,
-      {sum_alias} /* aggregate_expressions */);
-  expected_physical_ = P::Aggregate::Create(
-      physical_table_reference_0_,
-      {alias_0_} /* grouping_expressions */,
-      {sum_alias} /* aggregate_expressions */,
-      nullptr /* filter_predicate */);
-  EXPECT_CORRECT_PHYSICAL();
+  for (const bool is_distinct : {true, false}) {
+    const E::AggregateFunctionPtr sum = createSum(add_literal_0_,
+                                                  true /* is_vector_attribute */,
+                                                  is_distinct);
+    const E::AliasPtr sum_alias = createAlias(sum,
+                                              "sum" /* attribute_name */,
+                                              "test_relation" /* relation_name */);
+    input_logical_ = L::Aggregate::Create(
+        logical_table_reference_0_,
+        {alias_0_} /* grouping_expressions */,
+        {sum_alias} /* aggregate_expressions */);
+    expected_physical_ = P::Aggregate::Create(
+        physical_table_reference_0_,
+        {alias_0_} /* grouping_expressions */,
+        {sum_alias} /* aggregate_expressions */,
+        nullptr /* filter_predicate */);
+    EXPECT_CORRECT_PHYSICAL();
+  }
 }
 
 TEST_F(AggregateTest, PullupSelection) {
-  const E::AggregateFunctionPtr sum = createSum(E::ToRef(alias_add_literal_0_), true);
-  const E::AliasPtr sum_alias = createAlias(sum,
-                                            "sum" /* attribute_name */,
-                                            "test_relation" /* relation_name */);
-  const E::AliasPtr alias_on_alias_reference = E::Alias::Create(
-      optimizer_context()->nextExprId(),
-      E::ToRef(alias_add_literal_0_),
-      "alias_on_alias_reference" /* attribute_name */,
-      "alias_on_alias_reference" /* attribute_alias */,
-      "test_relation" /* relation_name */);
+  for (const bool is_distinct : {true, false}) {
+    const E::AggregateFunctionPtr sum = createSum(E::ToRef(alias_add_literal_0_),
+                                                  true /* is_vector_attribute */,
+                                                  is_distinct);
+    const E::AliasPtr sum_alias = createAlias(sum,
+                                              "sum" /* attribute_name */,
+                                              "test_relation" /* relation_name */);
+    const E::AliasPtr alias_on_alias_reference = E::Alias::Create(
+        optimizer_context()->nextExprId(),
+        E::ToRef(alias_add_literal_0_),
+        "alias_on_alias_reference" /* attribute_name */,
+        "alias_on_alias_reference" /* attribute_alias */,
+        "test_relation" /* relation_name */);
 
-  input_logical_ = L::Aggregate::Create(
-      logical_project_on_filter_0_,
-      {E::ToRef(alias_0_), alias_on_alias_reference} /* grouping_expressions */,
-      {sum_alias} /* aggregate_expressions */);
+    input_logical_ = L::Aggregate::Create(
+        logical_project_on_filter_0_,
+        {E::ToRef(alias_0_), alias_on_alias_reference} /* grouping_expressions */,
+        {sum_alias} /* aggregate_expressions */);
 
-  const E::AggregateFunctionPtr sum_after_pullup = std::static_pointer_cast<const E::AggregateFunction>(
-      sum->copyWithNewChildren({alias_add_literal_0_->expression()}));
-  const E::AliasPtr sum_alias_after_pullup =
-      std::static_pointer_cast<const E::Alias>(sum_alias->copyWithNewChildren({sum_after_pullup}));
-  const E::AliasPtr alias_on_alias_reference_after_pullup =
-      std::static_pointer_cast<const E::Alias>(alias_on_alias_reference->copyWithNewChildren(
-          {alias_add_literal_0_->expression()} /* new_children */));
-  expected_physical_ = P::Aggregate::Create(
-      physical_table_reference_0_,
-      {alias_0_,
-       alias_on_alias_reference_after_pullup} /* grouping_expressions */,
-      {sum_alias_after_pullup} /* aggregate_expressions */,
-      logical_filter_0_->filter_predicate());
+    const E::AggregateFunctionPtr sum_after_pullup =
+        std::static_pointer_cast<const E::AggregateFunction>(
+            sum->copyWithNewChildren({alias_add_literal_0_->expression()}));
+    const E::AliasPtr sum_alias_after_pullup =
+        std::static_pointer_cast<const E::Alias>(sum_alias->copyWithNewChildren({sum_after_pullup}));
+    const E::AliasPtr alias_on_alias_reference_after_pullup =
+        std::static_pointer_cast<const E::Alias>(alias_on_alias_reference->copyWithNewChildren(
+            {alias_add_literal_0_->expression()} /* new_children */));
+    expected_physical_ = P::Aggregate::Create(
+        physical_table_reference_0_,
+        {alias_0_,
+         alias_on_alias_reference_after_pullup} /* grouping_expressions */,
+        {sum_alias_after_pullup} /* aggregate_expressions */,
+        logical_filter_0_->filter_predicate());
 
-  EXPECT_CORRECT_PHYSICAL();
+    EXPECT_CORRECT_PHYSICAL();
+  }
 }
 
 }  // namespace strategy
diff --git a/query_optimizer/tests/execution_generator/CMakeLists.txt b/query_optimizer/tests/execution_generator/CMakeLists.txt
index 3191a28..d70ea4c 100644
--- a/query_optimizer/tests/execution_generator/CMakeLists.txt
+++ b/query_optimizer/tests/execution_generator/CMakeLists.txt
@@ -23,16 +23,21 @@
          "${CMAKE_CURRENT_SOURCE_DIR}/Delete.test"
          "${CMAKE_CURRENT_BINARY_DIR}/Delete.test"
          "${CMAKE_CURRENT_BINARY_DIR}/Delete/")
+add_test(quickstep_queryoptimizer_tests_executiongenerator_distinct
+         "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
+         "${CMAKE_CURRENT_SOURCE_DIR}/Distinct.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/Distinct.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/Distinct/")
 add_test(quickstep_queryoptimizer_tests_executiongenerator_drop
          "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
          "${CMAKE_CURRENT_SOURCE_DIR}/Drop.test"
          "${CMAKE_CURRENT_BINARY_DIR}/Drop.test"
          "${CMAKE_CURRENT_BINARY_DIR}/Drop/")
- add_test(quickstep_queryoptimizer_tests_executiongenerator_index
-          "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
-          "${CMAKE_CURRENT_SOURCE_DIR}/Index.test"
-          "${CMAKE_CURRENT_BINARY_DIR}/Index.test"
-          "${CMAKE_CURRENT_BINARY_DIR}/Index/")
+add_test(quickstep_queryoptimizer_tests_executiongenerator_index
+         "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
+         "${CMAKE_CURRENT_SOURCE_DIR}/Index.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/Index.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/Index/")
 add_test(quickstep_queryoptimizer_tests_executiongenerator_insert
          "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
          "${CMAKE_CURRENT_SOURCE_DIR}/Insert.test"
@@ -63,6 +68,7 @@
 # duration of their test.
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Create)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Delete)
+file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Distinct)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Drop)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Index)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Insert)
diff --git a/query_optimizer/tests/execution_generator/Distinct.test b/query_optimizer/tests/execution_generator/Distinct.test
new file mode 100644
index 0000000..feab898
--- /dev/null
+++ b/query_optimizer/tests/execution_generator/Distinct.test
@@ -0,0 +1,70 @@
+#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+#     University of Wisconsin—Madison.
+#
+#   Licensed under the Apache License, Version 2.0 (the "License");
+#   you may not use this file except in compliance with the License.
+#   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.
+
+CREATE TABLE foo(x INT, y DOUBLE, z INT);
+
+INSERT INTO foo
+SELECT i,
+       (i + 0.5) % 100,
+       i % 3
+FROM generate_series(0, 29999) AS gs(i);
+
+
+SELECT COUNT(*),
+       COUNT(DISTINCT x),
+       COUNT(DISTINCT y),
+       COUNT(DISTINCT z)
+FROM foo;
+--
++--------------------+--------------------+--------------------+--------------------+
+|COUNT(*)            |COUNT(DISTINCT x)   |COUNT(DISTINCT y)   |COUNT(DISTINCT z)   |
++--------------------+--------------------+--------------------+--------------------+
+|               30000|               30000|                 100|                   3|
++--------------------+--------------------+--------------------+--------------------+
+==
+
+SELECT SUM(y),
+       SUM(DISTINCT y),
+       COUNT(DISTINCT y),
+       AVG(DISTINCT y),
+       z
+FROM foo
+GROUP BY z
+ORDER BY z;
+--
++------------------------+------------------------+--------------------+------------------------+-----------+
+|SUM(y)                  |SUM(DISTINCT y)         |COUNT(DISTINCT y)   |AVG(DISTINCT y)         |z          |
++------------------------+------------------------+--------------------+------------------------+-----------+
+|                  500000|                    5000|                 100|                      50|          0|
+|                  500000|                    5000|                 100|                      50|          1|
+|                  500000|                    5000|                 100|                      50|          2|
++------------------------+------------------------+--------------------+------------------------+-----------+
+==
+
+SELECT MAX(x) * SUM(DISTINCT y),
+       COUNT(DISTINCT x % y) + z,
+       z
+FROM foo
+GROUP BY z
+ORDER BY z;
+--
++------------------------+-------------------------+-----------+
+|(MAX(x)*SUM(DISTINCT y))|(COUNT(DISTINCT (x%y))+z)|z          |
++------------------------+-------------------------+-----------+
+|               149985000|                      196|          0|
+|               149990000|                      197|          1|
+|               149995000|                      195|          2|
++------------------------+-------------------------+-----------+
+==
diff --git a/query_optimizer/tests/execution_generator/Insert.test b/query_optimizer/tests/execution_generator/Insert.test
index 09fe9bf..fd4cfde 100644
--- a/query_optimizer/tests/execution_generator/Insert.test
+++ b/query_optimizer/tests/execution_generator/Insert.test
@@ -13,9 +13,9 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License.
 
-INSERT INTO TEST VALUES (100, 2, 1.1, 1.3, 'foo');
-INSERT INTO TEST VALUES (100, 3, 1.2, 1.4, 'foofoo');
-INSERT INTO TEST VALUES (NULL, 3, 1.2, NULL, 'foofoo');
+INSERT INTO test VALUES (100, 2, 1.1, 1.3, 'foo');
+INSERT INTO test VALUES (100, 3, 1.2, 1.4, 'foofoo');
+INSERT INTO test VALUES (NULL, 3, 1.2, NULL, 'foofoo');
 SELECT * FROM test WHERE int_col = 100;
 --
 +-----------+--------------------+---------------+------------------------+--------------------+
@@ -24,3 +24,90 @@
 |        100|                   2|     1.10000002|                     1.3|                 foo|
 |        100|                   3|     1.20000005|      1.3999999999999999|              foofoo|
 +-----------+--------------------+---------------+------------------------+--------------------+
+==
+
+CREATE TABLE bar1 (x INT NULL, y DOUBLE);
+
+INSERT INTO bar1
+SELECT int_col, long_col
+FROM test;
+
+SELECT *
+FROM bar1
+WHERE x = 100;
+--
++-----------+------------------------+
+|x          |y                       |
++-----------+------------------------+
+|        100|                       2|
+|        100|                       3|
++-----------+------------------------+
+==
+
+INSERT INTO bar1
+SELECT int_col, long_col
+FROM test;
+
+SELECT *
+FROM bar1
+WHERE x = 100;
+--
++-----------+------------------------+
+|x          |y                       |
++-----------+------------------------+
+|        100|                       2|
+|        100|                       3|
+|        100|                       2|
+|        100|                       3|
++-----------+------------------------+
+==
+
+CREATE TABLE bar2 (x INT, y INT, z FLOAT);
+
+WITH s(i) AS
+  ( SELECT *
+   FROM generate_series(1, 5)),
+     t(j) AS
+  ( SELECT *
+   FROM generate_series(1, 5))
+INSERT INTO bar2
+SELECT i, j, i*j
+FROM s, t
+WHERE i < j;
+
+SELECT * FROM bar2;
+--
++-----------+-----------+---------------+
+|x          |y          |z              |
++-----------+-----------+---------------+
+|          1|          2|              2|
+|          1|          3|              3|
+|          1|          4|              4|
+|          1|          5|              5|
+|          2|          3|              6|
+|          2|          4|              8|
+|          2|          5|             10|
+|          3|          4|             12|
+|          3|          5|             15|
+|          4|          5|             20|
++-----------+-----------+---------------+
+==
+
+CREATE TABLE bar3 (x INT, y DATE, z DOUBLE NULL, w VARCHAR(32));
+
+INSERT INTO bar3
+SELECT i, DATE '2016-1-1', NULL, 'abc'
+FROM generate_series(1, 5) AS gs(i);
+
+SELECT * FROM bar3;
+--
++-----------+-----------------------------------------+------------------------+--------------------------------+
+|x          |y                                        |z                       |w                               |
++-----------+-----------------------------------------+------------------------+--------------------------------+
+|          1|                      2016-01-01T00:00:00|                    NULL|                             abc|
+|          2|                      2016-01-01T00:00:00|                    NULL|                             abc|
+|          3|                      2016-01-01T00:00:00|                    NULL|                             abc|
+|          4|                      2016-01-01T00:00:00|                    NULL|                             abc|
+|          5|                      2016-01-01T00:00:00|                    NULL|                             abc|
++-----------+-----------------------------------------+------------------------+--------------------------------+
+==
diff --git a/query_optimizer/tests/execution_generator/Select.test b/query_optimizer/tests/execution_generator/Select.test
index a9067da..82f7816 100644
--- a/query_optimizer/tests/execution_generator/Select.test
+++ b/query_optimizer/tests/execution_generator/Select.test
@@ -632,6 +632,48 @@
 +--------------------+--------------------+--------------------+
 ==
 
+
+CREATE TABLE dates (value DATETIME);
+INSERT INTO dates VALUES (DATE '2016-01-02 10:20:30');
+INSERT INTO dates VALUES (DATE '2016-02-03 11:21:31');
+INSERT INTO dates VALUES (DATE '2016-03-04 12:22:32');
+
+SELECT EXTRACT(YEAR FROM value) * 10000 +
+       EXTRACT(MONTH FROM value) * 100 +
+       EXTRACT(DAY FROM value) AS date_digits
+FROM dates;
+--
++--------------------+
+|date_digits         |
++--------------------+
+|            20160102|
+|            20160203|
+|            20160304|
++--------------------+
+==
+
+SELECT SUM(EXTRACT(SECOND FROM value))
+FROM dates;
+--
++-------------------------------+
+|SUM(EXTRACT(SECOND FROM value))|
++-------------------------------+
+|                             93|
++-------------------------------+
+==
+
+SELECT EXTRACT(hour FROM value + INTERVAL '1 hour')
+FROM dates;
+--
++-----------------------------------------------------+
+|EXTRACT(hour FROM (value+DatetimeInterval('1 hour')))|
++-----------------------------------------------------+
+|                                                   11|
+|                                                   12|
+|                                                   13|
++-----------------------------------------------------+
+==
+
 # TODO(team): Support uncorrelated queries.
 # SELECT COUNT(*)
 # FROM test
diff --git a/query_optimizer/tests/execution_generator/StringPatternMatching.test b/query_optimizer/tests/execution_generator/StringPatternMatching.test
index de69aa6..122deeb 100644
--- a/query_optimizer/tests/execution_generator/StringPatternMatching.test
+++ b/query_optimizer/tests/execution_generator/StringPatternMatching.test
@@ -144,7 +144,8 @@
 SELECT name, email, organization
 FROM foo, bar
 WHERE email LIKE pattern
-  AND name NOT LIKE 'd%';
+  AND name NOT LIKE 'd%'
+ORDER BY name ASC;
 --
 +------------------------+------------------------+------------------------+
 |name                    |email                   |organization            |
diff --git a/query_optimizer/tests/logical_generator/Select.test b/query_optimizer/tests/logical_generator/Select.test
index 2129de1..2709ce3 100644
--- a/query_optimizer/tests/logical_generator/Select.test
+++ b/query_optimizer/tests/logical_generator/Select.test
@@ -596,3 +596,60 @@
 +-output_attributes=
   +-AttributeReference[id=7,name=,alias=(int_col+2),relation=subquery,
     type=Int NULL]
+==
+
+SELECT COUNT(DISTINCT int_col), SUM(float_col)
+FROM test
+GROUP BY long_col
+HAVING AVG(DISTINCT int_col + double_col) > AVG(DISTINCT float_col);
+--
+TopLevelPlan
++-plan=Project
+| +-input=Filter
+| | +-input=Aggregate
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-grouping_expressions=
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long]
+| | |   | +-AggregateFunction[function=COUNT,is_distinct=true]
+| | |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,
+| | |   | type=Double NULL]
+| | |   | +-AggregateFunction[function=SUM]
+| | |   |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | |   +-Alias[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| | |   | type=Double NULL]
+| | |   | +-AggregateFunction[function=AVG,is_distinct=true]
+| | |   |   +-Add
+| | |   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   |     +-AttributeReference[id=3,name=double_col,relation=test,
+| | |   |       type=Double NULL]
+| | |   +-Alias[id=9,name=,alias=$aggregate3,relation=$aggregate,
+| | |     type=Double NULL]
+| | |     +-AggregateFunction[function=AVG,is_distinct=true]
+| | |       +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-filter_predicate=Greater
+| |   +-AttributeReference[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| |   | type=Double NULL]
+| |   +-AttributeReference[id=9,name=,alias=$aggregate3,relation=$aggregate,
+| |     type=Double NULL]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,type=Long]
+|   | +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   type=Long]
+|   +-Alias[id=7,name=,alias=SUM(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,
+  | type=Long]
+  +-AttributeReference[id=7,name=,alias=SUM(float_col),relation=,
+    type=Double NULL]
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index 69911ec..d653a9b 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -1586,3 +1586,108 @@
 |     +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
 +-output_attributes=
   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+==
+
+SELECT COUNT(DISTINCT int_col), SUM(float_col)
+FROM test
+GROUP BY long_col
+HAVING AVG(DISTINCT int_col + double_col) > AVG(DISTINCT float_col);
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=Filter
+| | +-input=Aggregate
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-grouping_expressions=
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long]
+| | |   | +-AggregateFunction[function=COUNT,is_distinct=true]
+| | |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,
+| | |   | type=Double NULL]
+| | |   | +-AggregateFunction[function=SUM]
+| | |   |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | |   +-Alias[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| | |   | type=Double NULL]
+| | |   | +-AggregateFunction[function=AVG,is_distinct=true]
+| | |   |   +-Add
+| | |   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   |     +-AttributeReference[id=3,name=double_col,relation=test,
+| | |   |       type=Double NULL]
+| | |   +-Alias[id=9,name=,alias=$aggregate3,relation=$aggregate,
+| | |     type=Double NULL]
+| | |     +-AggregateFunction[function=AVG,is_distinct=true]
+| | |       +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-filter_predicate=Greater
+| |   +-AttributeReference[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| |   | type=Double NULL]
+| |   +-AttributeReference[id=9,name=,alias=$aggregate3,relation=$aggregate,
+| |     type=Double NULL]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,type=Long]
+|   | +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   type=Long]
+|   +-Alias[id=7,name=,alias=SUM(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,
+  | type=Long]
+  +-AttributeReference[id=7,name=,alias=SUM(float_col),relation=,
+    type=Double NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=Aggregate
+| | +-input=TableReference[relation=Test,alias=test]
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   type=VarChar(20) NULL]
+| | +-grouping_expressions=
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-aggregate_expressions=
+| |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long]
+| |   | +-AggregateFunction[function=COUNT,is_distinct=true]
+| |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,type=Double NULL]
+| |   | +-AggregateFunction[function=SUM]
+| |   |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |   +-Alias[id=8,name=,alias=$aggregate2,relation=$aggregate,type=Double NULL]
+| |   | +-AggregateFunction[function=AVG,is_distinct=true]
+| |   |   +-Add
+| |   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |   |     +-AttributeReference[id=3,name=double_col,relation=test,
+| |   |       type=Double NULL]
+| |   +-Alias[id=9,name=,alias=$aggregate3,relation=$aggregate,type=Double NULL]
+| |     +-AggregateFunction[function=AVG,is_distinct=true]
+| |       +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| +-filter_predicate=Greater
+| | +-AttributeReference[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| | | type=Double NULL]
+| | +-AttributeReference[id=9,name=,alias=$aggregate3,relation=$aggregate,
+| |   type=Double NULL]
+| +-project_expressions=
+|   +-Alias[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,type=Long]
+|   | +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   type=Long]
+|   +-Alias[id=7,name=,alias=SUM(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,
+  | type=Long]
+  +-AttributeReference[id=7,name=,alias=SUM(float_col),relation=,
+    type=Double NULL]
diff --git a/query_optimizer/tests/resolver/Aggregate.test b/query_optimizer/tests/resolver/Aggregate.test
index 39ca8e0..318378b 100644
--- a/query_optimizer/tests/resolver/Aggregate.test
+++ b/query_optimizer/tests/resolver/Aggregate.test
@@ -877,3 +877,63 @@
 |   +-AttributeReference[id=6,name=alias_col,relation=,type=Int NULL]
 +-output_attributes=
   +-AttributeReference[id=6,name=alias_col,relation=,type=Int NULL]
+==
+
+# DISTINCT aggregation
+SELECT COUNT(DISTINCT int_col),
+       SUM(DISTINCT long_col * float_col)
+FROM test
+GROUP BY char_col
+HAVING AVG(double_col) > MIN(DISTINCT int_col);
+--
+TopLevelPlan
++-plan=Project
+| +-input=Filter
+| | +-input=Aggregate
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-grouping_expressions=
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long]
+| | |   | +-AggregateFunction[function=COUNT,is_distinct=true]
+| | |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,
+| | |   | type=Double NULL]
+| | |   | +-AggregateFunction[function=SUM,is_distinct=true]
+| | |   |   +-Multiply
+| | |   |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | |   |     +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | |   +-Alias[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| | |   | type=Double NULL]
+| | |   | +-AggregateFunction[function=AVG]
+| | |   |   +-AttributeReference[id=3,name=double_col,relation=test,
+| | |   |     type=Double NULL]
+| | |   +-Alias[id=9,name=,alias=$aggregate3,relation=$aggregate,type=Int NULL]
+| | |     +-AggregateFunction[function=MIN,is_distinct=true]
+| | |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-filter_predicate=Greater
+| |   +-AttributeReference[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| |   | type=Double NULL]
+| |   +-AttributeReference[id=9,name=,alias=$aggregate3,relation=$aggregate,
+| |     type=Int NULL]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,type=Long]
+|   | +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   type=Long]
+|   +-Alias[id=7,name=,alias=SUM(DISTINCT (long_col*float_col)),relation=,
+|     type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,
+  | type=Long]
+  +-AttributeReference[id=7,name=,alias=SUM(DISTINCT (long_col*float_col)),
+    relation=,type=Double NULL]
+==
\ No newline at end of file
diff --git a/query_optimizer/tests/resolver/Select.test b/query_optimizer/tests/resolver/Select.test
index cbfe7c9..1290d5e 100644
--- a/query_optimizer/tests/resolver/Select.test
+++ b/query_optimizer/tests/resolver/Select.test
@@ -2242,6 +2242,91 @@
     relation=,type=YearMonthInterval]
 ==
 
+SELECT EXTRACT(YEAR FROM DATE '2016-01-02 10:20:30') * 10000 +
+       EXTRACT(MONTH FROM DATE '2016-01-02 10:20:30') * 100 +
+       EXTRACT(DAY FROM DATE '2016-01-02 10:20:30') AS date_digits
+FROM generate_series(1, 1);
+--
+TopLevelPlan
++-plan=Project
+| +-input=TableGenerator[function_name=generate_series]
+| | +-AttributeReference[id=0,name=generate_series,relation=generate_series,
+| |   type=Int]
+| +-project_list=
+|   +-Alias[id=1,name=date_digits,relation=,type=Long]
+|     +-Add
+|       +-Add
+|       | +-Multiply
+|       | | +-DateExtract(YEAR)
+|       | | | +-Operand=Literal[value=2016-01-02T10:20:30,type=Datetime]
+|       | | +-Literal[value=10000,type=Int]
+|       | +-Multiply
+|       |   +-DateExtract(MONTH)
+|       |   | +-Operand=Literal[value=2016-01-02T10:20:30,type=Datetime]
+|       |   +-Literal[value=100,type=Int]
+|       +-DateExtract(DAY)
+|         +-Operand=Literal[value=2016-01-02T10:20:30,type=Datetime]
++-output_attributes=
+  +-AttributeReference[id=1,name=date_digits,relation=,type=Long]
+==
+
+SELECT COUNT(DISTINCT int_col), SUM(float_col)
+FROM test
+GROUP BY long_col
+HAVING AVG(DISTINCT int_col + double_col) > AVG(DISTINCT float_col);
+--
+TopLevelPlan
++-plan=Project
+| +-input=Filter
+| | +-input=Aggregate
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-grouping_expressions=
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long]
+| | |   | +-AggregateFunction[function=COUNT,is_distinct=true]
+| | |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,
+| | |   | type=Double NULL]
+| | |   | +-AggregateFunction[function=SUM]
+| | |   |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | |   +-Alias[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| | |   | type=Double NULL]
+| | |   | +-AggregateFunction[function=AVG,is_distinct=true]
+| | |   |   +-Add
+| | |   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   |     +-AttributeReference[id=3,name=double_col,relation=test,
+| | |   |       type=Double NULL]
+| | |   +-Alias[id=9,name=,alias=$aggregate3,relation=$aggregate,
+| | |     type=Double NULL]
+| | |     +-AggregateFunction[function=AVG,is_distinct=true]
+| | |       +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-filter_predicate=Greater
+| |   +-AttributeReference[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| |   | type=Double NULL]
+| |   +-AttributeReference[id=9,name=,alias=$aggregate3,relation=$aggregate,
+| |     type=Double NULL]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,type=Long]
+|   | +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   type=Long]
+|   +-Alias[id=7,name=,alias=SUM(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=COUNT(DISTINCT int_col),relation=,
+  | type=Long]
+  +-AttributeReference[id=7,name=,alias=SUM(float_col),relation=,
+    type=Double NULL]
+==
+
 select interval '4 day' + interval '5 year'
 from test
 --
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index 3bbccf0..eca429c 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -210,6 +210,7 @@
     // Add an aggregate.
     serialization::Aggregate *aggr_proto = aggr_state_proto->add_aggregates();
     aggr_proto->mutable_function()->CopyFrom(AggregateFunctionFactory::Get(agg_type).getProto());
+    aggr_proto->set_is_distinct(false);
     if (is_expression) {
       unique_ptr<ScalarBinaryExpression> exp(
           new ScalarBinaryExpression(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd),
@@ -224,6 +225,7 @@
     // Add another aggregate.
     aggr_proto = aggr_state_proto->add_aggregates();
     aggr_proto->mutable_function()->CopyFrom(AggregateFunctionFactory::Get(agg_type).getProto());
+    aggr_proto->set_is_distinct(false);
     if (is_expression) {
       unique_ptr<ScalarBinaryExpression> exp(
           new ScalarBinaryExpression(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kMultiply),
@@ -260,7 +262,7 @@
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,
-                                          db_.get(),
+                                          *db_,
                                           storage_manager_.get(),
                                           foreman_client_id_,
                                           &bus_));
@@ -296,6 +298,7 @@
     // Add an aggregate.
     serialization::Aggregate *aggr_proto = aggr_state_proto->add_aggregates();
     aggr_proto->mutable_function()->CopyFrom(AggregateFunctionFactory::Get(agg_type).getProto());
+    aggr_proto->set_is_distinct(false);
 
     unique_ptr<ScalarAttribute> attr(new ScalarAttribute(*table_->getAttributeByName(stem + "-0")));
     aggr_proto->add_argument()->CopyFrom(attr->getProto());
@@ -303,6 +306,7 @@
     // Add another aggregate.
     aggr_proto = aggr_state_proto->add_aggregates();
     aggr_proto->mutable_function()->CopyFrom(AggregateFunctionFactory::Get(agg_type).getProto());
+    aggr_proto->set_is_distinct(false);
     attr.reset(new ScalarAttribute(*table_->getAttributeByName(stem + "-1")));
     aggr_proto->add_argument()->CopyFrom(attr->getProto());
 
@@ -340,7 +344,7 @@
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,
-                                          db_.get(),
+                                          *db_,
                                           storage_manager_.get(),
                                           foreman_client_id_,
                                           &bus_));
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index e5afe69..11540d1 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -342,7 +342,7 @@
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
-                                        db_.get(),
+                                        *db_,
                                         storage_manager_.get(),
                                         foreman_client_id_,
                                         &bus_));
@@ -484,7 +484,7 @@
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
-                                        db_.get(),
+                                        *db_,
                                         storage_manager_.get(),
                                         foreman_client_id_,
                                         &bus_));
@@ -634,7 +634,7 @@
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
-                                        db_.get(),
+                                        *db_,
                                         storage_manager_.get(),
                                         foreman_client_id_,
                                         &bus_));
@@ -769,7 +769,7 @@
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
-                                        db_.get(),
+                                        *db_,
                                         storage_manager_.get(),
                                         foreman_client_id_,
                                         &bus_));
@@ -938,7 +938,7 @@
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
-                                        db_.get(),
+                                        *db_,
                                         storage_manager_.get(),
                                         foreman_client_id_,
                                         &bus_));
@@ -1118,7 +1118,7 @@
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
-                                        db_.get(),
+                                        *db_,
                                         storage_manager_.get(),
                                         foreman_client_id_,
                                         &bus_));
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index 680bd25..8406c0a 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -1272,7 +1272,7 @@
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto_,
-                                          db_.get(),
+                                          *db_,
                                           storage_manager_.get(),
                                           foreman_client_id_,
                                           &bus_));
@@ -1537,7 +1537,7 @@
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto_,
-                                          db_.get(),
+                                          *db_,
                                           storage_manager_.get(),
                                           foreman_client_id_,
                                           &bus_));
@@ -1580,7 +1580,7 @@
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto_,
-                                          db_.get(),
+                                          *db_,
                                           storage_manager_.get(),
                                           foreman_client_id_,
                                           &bus_));
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index 74efd26..df88e91 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -351,7 +351,7 @@
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,
-                                          db_.get(),
+                                          *db_,
                                           storage_manager_.get(),
                                           thread_client_id_,
                                           &bus_));
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index dbfb17f..292794f 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -179,7 +179,7 @@
 
   // Setup query_context_.
   query_context_.reset(new QueryContext(query_context_proto,
-                                        db_.get(),
+                                        *db_,
                                         storage_manager_.get(),
                                         foreman_client_id_,
                                         &bus_));
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index a129cc7..a3a669c 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -59,6 +59,7 @@
     const CatalogRelationSchema &input_relation,
     const std::vector<const AggregateFunction*> &aggregate_functions,
     std::vector<std::vector<std::unique_ptr<const Scalar>>> &&arguments,
+    std::vector<bool> &&is_distinct,
     std::vector<std::unique_ptr<const Scalar>> &&group_by,
     const Predicate *predicate,
     const std::size_t estimated_num_entries,
@@ -68,6 +69,7 @@
       predicate_(predicate),
       group_by_list_(std::move(group_by)),
       arguments_(std::move(arguments)),
+      is_distinct_(std::move(is_distinct)),
       storage_manager_(storage_manager) {
   // Sanity checks: each aggregate has a corresponding list of arguments.
   DCHECK(aggregate_functions.size() == arguments_.size());
@@ -85,6 +87,7 @@
 
     handles_.emplace_back(new AggregationHandleDistinct());
     arguments_.push_back({});
+    is_distinct_.emplace_back(false);
 
     group_by_hashtables_.emplace_back(handles_.back()->createGroupByHashTable(
         hash_table_impl_type,
@@ -97,7 +100,8 @@
         = aggregate_functions.begin();
     std::vector<std::vector<std::unique_ptr<const Scalar>>>::const_iterator args_it
         = arguments_.begin();
-    for (; agg_func_it != aggregate_functions.end(); ++agg_func_it, ++args_it) {
+    std::vector<bool>::const_iterator is_distinct_it = is_distinct_.begin();
+    for (; agg_func_it != aggregate_functions.end(); ++agg_func_it, ++args_it, ++is_distinct_it) {
       // Get the Types of this aggregate's arguments so that we can create an
       // AggregationHandle.
       std::vector<const Type*> argument_types;
@@ -145,6 +149,25 @@
         arguments_as_attributes_.emplace_back(std::move(local_arguments_as_attributes));
 #endif
       }
+
+      // Initialize the corresponding distinctify hash table if this is a DISTINCT
+      // aggregation.
+      if (*is_distinct_it) {
+        std::vector<const Type*> key_types(group_by_types);
+        key_types.insert(key_types.end(), argument_types.begin(), argument_types.end());
+        // TODO(jianqiao): estimated_num_entries is quite inaccurate for estimating
+        // the number of entries in the distinctify hash table. We may estimate
+        // for each distinct aggregation an estimated_num_distinct_keys value during
+        // query optimization, if it worths.
+        distinctify_hashtables_.emplace_back(
+            handles_.back()->createDistinctifyHashTable(
+                hash_table_impl_type,
+                key_types,
+                estimated_num_entries,
+                storage_manager));
+      } else {
+        distinctify_hashtables_.emplace_back(nullptr);
+      }
     }
   }
 }
@@ -158,6 +181,7 @@
   // Rebuild contructor arguments from their representation in 'proto'.
   std::vector<const AggregateFunction*> aggregate_functions;
   std::vector<std::vector<std::unique_ptr<const Scalar>>> arguments;
+  std::vector<bool> is_distinct;
   for (int agg_idx = 0; agg_idx < proto.aggregates_size(); ++agg_idx) {
     const serialization::Aggregate &agg_proto = proto.aggregates(agg_idx);
 
@@ -171,6 +195,8 @@
           agg_proto.argument(argument_idx),
           database));
     }
+
+    is_distinct.emplace_back(agg_proto.is_distinct());
   }
 
   std::vector<std::unique_ptr<const Scalar>> group_by_expressions;
@@ -192,6 +218,7 @@
   return new AggregationOperationState(database.getRelationSchemaById(proto.relation_id()),
                                        aggregate_functions,
                                        std::move(arguments),
+                                       std::move(is_distinct),
                                        std::move(group_by_expressions),
                                        predicate.release(),
                                        proto.estimated_num_entries(),
@@ -256,7 +283,7 @@
   }
 }
 
-void AggregationOperationState::finalizeAggregate(InsertDestination *output_destination) const {
+void AggregationOperationState::finalizeAggregate(InsertDestination *output_destination) {
   if (group_by_list_.empty()) {
     finalizeSingleState(output_destination);
   } else {
@@ -268,8 +295,10 @@
     const std::vector<std::unique_ptr<AggregationState>> &local_state) {
   DEBUG_ASSERT(local_state.size() == single_states_.size());
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    handles_[agg_idx]->mergeStates(*local_state[agg_idx],
-                                   single_states_[agg_idx].get());
+    if (!is_distinct_[agg_idx]) {
+      handles_[agg_idx]->mergeStates(*local_state[agg_idx],
+                                     single_states_[agg_idx].get());
+    }
   }
 }
 
@@ -293,13 +322,28 @@
       local_arguments_as_attributes = &(arguments_as_attributes_[agg_idx]);
     }
 #endif
-    // Call StorageBlock::aggregate() to actually do the aggregation.
-    local_state.emplace_back(
-        block->aggregate(*handles_[agg_idx],
-                         arguments_[agg_idx],
-                         local_arguments_as_attributes,
-                         predicate_.get(),
-                         &reuse_matches));
+    if (is_distinct_[agg_idx]) {
+      // Call StorageBlock::aggregateDistinct() to put the arguments as keys
+      // directly into the (threadsafe) shared global distinctify HashTable
+      // for this aggregate.
+      block->aggregateDistinct(*handles_[agg_idx],
+                               arguments_[agg_idx],
+                               local_arguments_as_attributes,
+                               {}, /* group_by */
+                               predicate_.get(),
+                               distinctify_hashtables_[agg_idx].get(),
+                               &reuse_matches,
+                               nullptr /* reuse_group_by_vectors */);
+      local_state.emplace_back(nullptr);
+    } else {
+      // Call StorageBlock::aggregate() to actually do the aggregation.
+      local_state.emplace_back(
+          block->aggregate(*handles_[agg_idx],
+                           arguments_[agg_idx],
+                           local_arguments_as_attributes,
+                           predicate_.get(),
+                           &reuse_matches));
+    }
   }
 
   // Merge per-block aggregation states back with global state.
@@ -322,24 +366,38 @@
   for (std::size_t agg_idx = 0;
        agg_idx < handles_.size();
        ++agg_idx) {
-    // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
-    // directly into the (threadsafe) shared global HashTable for this
-    // aggregate.
-    //
-    // TODO(shoban): Implement optional code path for using local hash table per
-    // block, which can be merged with global hash table for all blocks
-    // aggregated on.
-    block->aggregateGroupBy(*handles_[agg_idx],
-                            arguments_[agg_idx],
-                            group_by_list_,
-                            predicate_.get(),
-                            group_by_hashtables_[agg_idx].get(),
-                            &reuse_matches,
-                            &reuse_group_by_vectors);
+    if (is_distinct_[agg_idx]) {
+      // Call StorageBlock::aggregateDistinct() to insert the GROUP BY expression
+      // values and the aggregation arguments together as keys directly into the
+      // (threadsafe) shared global distinctify HashTable for this aggregate.
+      block->aggregateDistinct(*handles_[agg_idx],
+                               arguments_[agg_idx],
+                               nullptr, /* arguments_as_attributes */
+                               group_by_list_,
+                               predicate_.get(),
+                               distinctify_hashtables_[agg_idx].get(),
+                               &reuse_matches,
+                               &reuse_group_by_vectors);
+    } else {
+      // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
+      // directly into the (threadsafe) shared global HashTable for this
+      // aggregate.
+      //
+      // TODO(shoban): Implement optional code path for using local hash table per
+      // block, which can be merged with global hash table for all blocks
+      // aggregated on.
+      block->aggregateGroupBy(*handles_[agg_idx],
+                              arguments_[agg_idx],
+                              group_by_list_,
+                              predicate_.get(),
+                              group_by_hashtables_[agg_idx].get(),
+                              &reuse_matches,
+                              &reuse_group_by_vectors);
+    }
   }
 }
 
-void AggregationOperationState::finalizeSingleState(InsertDestination *output_destination) const {
+void AggregationOperationState::finalizeSingleState(InsertDestination *output_destination) {
   // Simply build up a Tuple from the finalized values for each aggregate and
   // insert it in '*output_destination'.
   std::vector<TypedValue> attribute_values;
@@ -347,13 +405,18 @@
   for (std::size_t agg_idx = 0;
        agg_idx < handles_.size();
        ++agg_idx) {
+    if (is_distinct_[agg_idx]) {
+      single_states_[agg_idx].reset(
+          handles_[agg_idx]->aggregateOnDistinctifyHashTableForSingle(*distinctify_hashtables_[agg_idx]));
+    }
+
     attribute_values.emplace_back(handles_[agg_idx]->finalize(*single_states_[agg_idx]));
   }
 
   output_destination->insertTuple(Tuple(std::move(attribute_values)));
 }
 
-void AggregationOperationState::finalizeHashTable(InsertDestination *output_destination) const {
+void AggregationOperationState::finalizeHashTable(InsertDestination *output_destination) {
   // Each element of 'group_by_keys' is a vector of values for a particular
   // group (which is also the prefix of the finalized Tuple for that group).
   std::vector<std::vector<TypedValue>> group_by_keys;
@@ -363,11 +426,17 @@
   for (std::size_t agg_idx = 0;
        agg_idx < handles_.size();
        ++agg_idx) {
-    ColumnVector* col =
+    if (is_distinct_[agg_idx]) {
+      handles_[agg_idx]->aggregateOnDistinctifyHashTableForGroupBy(
+          *distinctify_hashtables_[agg_idx],
+          group_by_hashtables_[agg_idx].get());
+    }
+
+    ColumnVector* agg_result_col =
         handles_[agg_idx]->finalizeHashTable(*group_by_hashtables_[agg_idx],
                                              &group_by_keys);
-    if (col != nullptr) {
-      final_values.emplace_back(col);
+    if (agg_result_col != nullptr) {
+      final_values.emplace_back(agg_result_col);
     }
   }
 
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 565d1f8..b883ed1 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_AGGREGATION_OPERATION_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_AGGREGATION_OPERATION_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <vector>
 
@@ -79,6 +80,8 @@
    * @param arguments For each entry in aggregate_functions, a corresponding
    *        list of argument expressions to that aggregate. This is moved-from,
    *        with AggregationOperationState taking ownership.
+   * @param is_distinct For each entry in aggregate_functions, whether DISTINCT
+   *        should be applied to the entry's arguments.
    * @param group_by A list of expressions to compute the GROUP BY values. If
    *        empty, no grouping is used. This is moved-from, with
    *        AggregationOperationState taking ownership.
@@ -97,6 +100,7 @@
   AggregationOperationState(const CatalogRelationSchema &input_relation,
                             const std::vector<const AggregateFunction*> &aggregate_functions,
                             std::vector<std::vector<std::unique_ptr<const Scalar>>> &&arguments,
+                            std::vector<bool> &&is_distinct,
                             std::vector<std::unique_ptr<const Scalar>> &&group_by,
                             const Predicate *predicate,
                             const std::size_t estimated_num_entries,
@@ -153,7 +157,7 @@
    * @param output_destination An InsertDestination where the finalized output
    *        tuple(s) from this aggregate are to be written.
    **/
-  void finalizeAggregate(InsertDestination *output_destination) const;
+  void finalizeAggregate(InsertDestination *output_destination);
 
  private:
   // Merge locally (per storage block) aggregated states with global aggregation
@@ -164,8 +168,8 @@
   void aggregateBlockSingleState(const block_id input_block);
   void aggregateBlockHashTable(const block_id input_block);
 
-  void finalizeSingleState(InsertDestination *output_destination) const;
-  void finalizeHashTable(InsertDestination *output_destination) const;
+  void finalizeSingleState(InsertDestination *output_destination);
+  void finalizeHashTable(InsertDestination *output_destination);
 
   // Common state for all aggregates in this operation: the input relation, the
   // filter predicate (if any), and the list of GROUP BY expressions (if any).
@@ -178,6 +182,13 @@
   std::vector<std::unique_ptr<AggregationHandle>> handles_;
   std::vector<std::vector<std::unique_ptr<const Scalar>>> arguments_;
 
+  // For each aggregate, whether DISTINCT should be applied to the aggregate's
+  // arguments.
+  std::vector<bool> is_distinct_;
+
+  // Hash table for obtaining distinct (i.e. unique) arguments.
+  std::vector<std::unique_ptr<AggregationStateHashTableBase>> distinctify_hashtables_;
+
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   // If all an aggregate's argument expressions are simply attributes in
   // 'input_relation_', then this caches the attribute IDs of those arguments.
diff --git a/storage/AggregationOperationState.proto b/storage/AggregationOperationState.proto
index 13836a3..031f782 100644
--- a/storage/AggregationOperationState.proto
+++ b/storage/AggregationOperationState.proto
@@ -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.
@@ -24,6 +24,7 @@
 message Aggregate {
   required AggregateFunction function = 1;
   repeated Scalar argument = 2;
+  required bool is_distinct = 3;
 }
 
 message AggregationOperationState {
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 918a348..0ba699b 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -1,5 +1,7 @@
 #   Copyright 2011-2015 Quickstep Technologies LLC.
 #   Copyright 2015-2016 Pivotal Software, Inc.
+#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+#     University of Wisconsin—Madison.
 #
 #   Licensed under the Apache License, Version 2.0 (the "License");
 #   you may not use this file except in compliance with the License.
@@ -609,6 +611,7 @@
 target_link_libraries(quickstep_storage_PackedRowStoreTupleStorageSubBlock
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelationSchema
+                      quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_predicate_PredicateCost
                       quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_storage_StorageBlockInfo
diff --git a/storage/PackedRowStoreTupleStorageSubBlock.cpp b/storage/PackedRowStoreTupleStorageSubBlock.cpp
index 1328826..4cb6d02 100644
--- a/storage/PackedRowStoreTupleStorageSubBlock.cpp
+++ b/storage/PackedRowStoreTupleStorageSubBlock.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—Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -23,6 +25,7 @@
 
 #include "catalog/CatalogAttribute.hpp"
 #include "catalog/CatalogRelationSchema.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "storage/PackedRowStoreValueAccessor.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageBlockLayout.pb.h"
@@ -143,46 +146,55 @@
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+    const std::size_t num_attrs = relation_.size();
+    const std::vector<std::size_t> &attrs_max_size =
+        relation_.getMaximumAttributeByteLengths();
+
     if (num_nullable_attrs != 0) {
       while (this->hasSpaceToInsert<true>(1) && accessor->next()) {
-        attribute_id accessor_attr_id = 0;
-        for (CatalogRelationSchema::const_iterator attr_it = relation_.begin();
-             attr_it != relation_.end();
-             ++attr_it) {
-          const std::size_t attr_size = attr_it->getType().maximumByteLength();
-          const int nullable_idx = relation_.getNullableAttributeIndex(
-              attr_it->getID());
-          if (nullable_idx != -1) {
+        for (std::size_t curr_attr = 0; curr_attr < num_attrs; ++curr_attr) {
+          const std::size_t attr_size = attrs_max_size[curr_attr];
+          const attribute_id nullable_idx = relation_.getNullableAttributeIndex(curr_attr);
+          // If this attribute is nullable, check for a returned null value.
+          if (nullable_idx != kInvalidCatalogId) {
             const void *attr_value
-                = accessor->template getUntypedValue<true>(accessor_attr_id);
+                = accessor->template getUntypedValue<true>(curr_attr);
             if (attr_value == nullptr) {
-              null_bitmap_->setBit(header_->num_tuples * num_nullable_attrs + nullable_idx,
-                                   true);
+              null_bitmap_->setBit(
+                  header_->num_tuples * num_nullable_attrs + nullable_idx,
+                  true);
             } else {
               memcpy(dest_addr, attr_value, attr_size);
             }
           } else {
             memcpy(dest_addr,
-                   accessor->template getUntypedValue<false>(accessor_attr_id),
+                   accessor->template getUntypedValue<false>(curr_attr),
                    attr_size);
           }
-          ++accessor_attr_id;
           dest_addr += attr_size;
         }
         ++(header_->num_tuples);
       }
     } else {
+      // If the accessor is from a packed row store, we can optimize the
+      // memcpy by avoiding iterating over each attribute.
+      const bool fast_copy =
+          (accessor->getImplementationType() ==
+              ValueAccessor::Implementation::kCompressedPackedRowStore);
+      const std::size_t attrs_total_size = relation_.getMaximumByteLength();
       while (this->hasSpaceToInsert<false>(1) && accessor->next()) {
-        attribute_id accessor_attr_id = 0;
-        for (CatalogRelationSchema::const_iterator attr_it = relation_.begin();
-             attr_it != relation_.end();
-             ++attr_it) {
-          const std::size_t attr_size = attr_it->getType().maximumByteLength();
+        if (fast_copy) {
           memcpy(dest_addr,
-                 accessor->template getUntypedValue<false>(accessor_attr_id),
-                 attr_size);
-          ++accessor_attr_id;
-          dest_addr += attr_size;
+                 accessor->template getUntypedValue<false>(0),
+                 attrs_total_size);
+        } else {
+          for (std::size_t curr_attr = 0; curr_attr < num_attrs; ++curr_attr) {
+            const std::size_t attr_size = attrs_max_size[curr_attr];
+            memcpy(dest_addr,
+                   accessor->template getUntypedValue<false>(curr_attr),
+                   attr_size);
+            dest_addr += attr_size;
+          }
         }
         ++(header_->num_tuples);
       }
@@ -205,45 +217,42 @@
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+    const std::size_t num_attrs = relation_.size();
+    const std::vector<std::size_t> &attrs_max_size =
+        relation_.getMaximumAttributeByteLengths();
+
     if (num_nullable_attrs != 0) {
       while (this->hasSpaceToInsert<true>(1) && accessor->next()) {
-        std::vector<attribute_id>::const_iterator attribute_map_it = attribute_map.begin();
-        for (CatalogRelationSchema::const_iterator attr_it = relation_.begin();
-             attr_it != relation_.end();
-             ++attr_it) {
-          const std::size_t attr_size = attr_it->getType().maximumByteLength();
-          const int nullable_idx = relation_.getNullableAttributeIndex(
-              attr_it->getID());
-          if (nullable_idx != -1) {
+        for (std::size_t curr_attr = 0; curr_attr < num_attrs; ++curr_attr) {
+          const std::size_t attr_size = attrs_max_size[curr_attr];
+          const attribute_id nullable_idx = relation_.getNullableAttributeIndex(curr_attr);
+          // If this attribute is nullable, check for a returned null value.
+          if (nullable_idx != kInvalidCatalogId) {
             const void *attr_value
-                = accessor->template getUntypedValue<true>(*attribute_map_it);
+                = accessor->template getUntypedValue<true>(attribute_map[curr_attr]);
             if (attr_value == nullptr) {
-              null_bitmap_->setBit(header_->num_tuples * num_nullable_attrs + nullable_idx,
-                                   true);
+              null_bitmap_->setBit(
+                  header_->num_tuples * num_nullable_attrs + nullable_idx,
+                  true);
             } else {
               memcpy(dest_addr, attr_value, attr_size);
             }
           } else {
             memcpy(dest_addr,
-                   accessor->template getUntypedValue<false>(*attribute_map_it),
+                   accessor->template getUntypedValue<false>(attribute_map[curr_attr]),
                    attr_size);
           }
-          ++attribute_map_it;
           dest_addr += attr_size;
         }
         ++(header_->num_tuples);
       }
     } else {
       while (this->hasSpaceToInsert<false>(1) && accessor->next()) {
-        std::vector<attribute_id>::const_iterator attribute_map_it = attribute_map.begin();
-        for (CatalogRelationSchema::const_iterator attr_it = relation_.begin();
-             attr_it != relation_.end();
-             ++attr_it) {
-          const std::size_t attr_size = attr_it->getType().maximumByteLength();
+        for (std::size_t curr_attr = 0; curr_attr < num_attrs; ++curr_attr) {
+          const std::size_t attr_size = attrs_max_size[curr_attr];
           memcpy(dest_addr,
-                 accessor->template getUntypedValue<false>(*attribute_map_it),
+                 accessor->template getUntypedValue<false>(attribute_map[curr_attr]),
                  attr_size);
-          ++attribute_map_it;
           dest_addr += attr_size;
         }
         ++(header_->num_tuples);
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 6fd6bb2..e9ddb70 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.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.
@@ -486,6 +486,95 @@
                                              hash_table);
 }
 
+void StorageBlock::aggregateDistinct(
+    const AggregationHandle &handle,
+    const std::vector<std::unique_ptr<const Scalar>> &arguments,
+    const std::vector<attribute_id> *arguments_as_attributes,
+    const std::vector<std::unique_ptr<const Scalar>> &group_by,
+    const Predicate *predicate,
+    AggregationStateHashTableBase *distinctify_hash_table,
+    std::unique_ptr<TupleIdSequence> *reuse_matches,
+    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
+  DCHECK_GT(arguments.size(), 0u)
+      << "Called aggregateDistinct() with zero argument expressions";
+  DCHECK((group_by.size() == 0 || reuse_group_by_vectors != nullptr));
+
+  std::vector<attribute_id> key_ids;
+
+  // An intermediate ValueAccessor that stores the materialized 'arguments' for
+  // this aggregate, as well as the GROUP BY expression values.
+  ColumnVectorsValueAccessor temp_result;
+  {
+    std::unique_ptr<ValueAccessor> accessor;
+    if (predicate) {
+      if (!*reuse_matches) {
+        // If there is a filter predicate that hasn't already been evaluated,
+        // evaluate it now and save the results for other aggregates on this
+        // same block.
+        reuse_matches->reset(getMatchesForPredicate(predicate));
+      }
+
+      // Create a filtered ValueAccessor that only iterates over predicate
+      // matches.
+      accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
+    } else {
+      // Create a ValueAccessor that iterates over all tuples in this block
+      accessor.reset(tuple_store_->createValueAccessor());
+    }
+
+#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+    // If all the arguments to this aggregate are plain relation attributes,
+    // aggregate directly on a ValueAccessor from this block to avoid a copy.
+    if ((arguments_as_attributes != nullptr) && (!arguments_as_attributes->empty())) {
+      DCHECK_EQ(arguments.size(), arguments_as_attributes->size())
+          << "Mismatch between number of arguments and number of attribute_ids";
+      DCHECK_EQ(group_by.size(), 0u);
+      handle.insertValueAccessorIntoDistinctifyHashTable(
+          accessor.get(), *arguments_as_attributes, distinctify_hash_table);
+      return;
+    }
+#endif
+
+    SubBlocksReference sub_blocks_ref(*tuple_store_,
+                                      indices_,
+                                      indices_consistent_);
+    attribute_id attr_id = 0;
+
+    if (!group_by.empty()) {
+      // Put GROUP BY keys into 'temp_result'.
+      if (reuse_group_by_vectors->empty()) {
+        // Compute GROUP BY values from group_by Scalars, and store them in
+        // reuse_group_by_vectors for reuse by other aggregates on this same
+        // block.
+        reuse_group_by_vectors->reserve(group_by.size());
+        for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
+          reuse_group_by_vectors->emplace_back(
+              group_by_element->getAllValues(accessor.get(), &sub_blocks_ref));
+          temp_result.addColumn(reuse_group_by_vectors->back().get(), false);
+          key_ids.push_back(attr_id++);
+        }
+      } else {
+        // Reuse precomputed GROUP BY values from reuse_group_by_vectors.
+        DCHECK_EQ(group_by.size(), reuse_group_by_vectors->size())
+            << "Wrong number of reuse_group_by_vectors";
+        for (const std::unique_ptr<ColumnVector> &reuse_cv : *reuse_group_by_vectors) {
+          temp_result.addColumn(reuse_cv.get(), false);
+          key_ids.push_back(attr_id++);
+        }
+      }
+    }
+    // Compute argument vectors and add them to 'temp_result'.
+    for (const std::unique_ptr<const Scalar> &argument : arguments) {
+      temp_result.addColumn(argument->getAllValues(accessor.get(), &sub_blocks_ref));
+      key_ids.push_back(attr_id++);
+    }
+  }
+
+  handle.insertValueAccessorIntoDistinctifyHashTable(
+      &temp_result, key_ids, distinctify_hash_table);
+}
+
+
 // TODO(chasseur): Vectorization for updates.
 StorageBlock::UpdateResult StorageBlock::update(
     const unordered_map<attribute_id, unique_ptr<const Scalar>> &assignments,
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 3b45998..97813e2 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.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.
@@ -467,6 +467,50 @@
                             *reuse_group_by_vectors) const;
 
   /**
+   * @brief Inserts the GROUP BY expressions and aggregation arguments together
+   *        as keys into the distinctify hash table.
+   *
+   * This is the first step for DISTINCT aggregation. It populates the distinctify
+   * hash table so that arguments are distinctified within each GROUP BY group.
+   * Later, a second-round aggregation on the distinctify hash table will be
+   * performed to actually compute the aggregated result for each GROUP BY group.
+   *
+   * @param handle Aggregation handle to compute aggregates with.
+   * @param arguments The arguments to the aggregation function as Scalars.
+   * @param arguments_as_attributes If non-NULL, indicates a valid attribute_id
+   *        for each of the elements in arguments, and is used to elide a copy.
+   *        Has no effect if NULL, or if VECTOR_COPY_ELISION_LEVEL is NONE.
+   * @param group_by The list of GROUP BY attributes/expressions.
+   * @param predicate A predicate for selection. \c nullptr indicates that all
+   *        tuples should be aggregated on.
+   * @param distinctify_hash_table Hash table to store the arguments and GROUP
+   *        BY expressions together as hash table key and a bool constant \c true
+   *        as hash table value. (So the hash table actually serves as a hash set.)
+   * @param reuse_matches This parameter is used to store and reuse tuple-id
+   *        sequence of matches pre-computed in an earlier invocations of
+   *        aggregateGroupBy(). \c reuse_matches is never \c nullptr for ease of
+   *        use.  Current invocation of aggregateGroupBy() will reuse
+   *        TupleIdSequence if passed, otherwise computes a TupleIdSequence based
+   *        on \c predicate and stores in \c reuse_matches. We use
+   *        std::unique_ptr for each of use, since the caller will not have to
+   *        selective free.
+   * @param reuse_group_by_vectors This parameter is used to store and reuse
+   *        GROUP BY attribute vectors pre-computed in an earlier invocation of
+   *        aggregateGroupBy(). \c reuse_group_by_vectors is never \c nullptr
+   *        for ease of use. Current invocation of aggregateGroupBy() will reuse
+   *        ColumnVectors if non-empty, otherwise computes ColumnVectors based
+   *        on \c group_by and stores them in \c reuse_group_by_vectors.
+   */
+  void aggregateDistinct(const AggregationHandle &handle,
+                         const std::vector<std::unique_ptr<const Scalar>> &arguments,
+                         const std::vector<attribute_id> *arguments_as_attributes,
+                         const std::vector<std::unique_ptr<const Scalar>> &group_by,
+                         const Predicate *predicate,
+                         AggregationStateHashTableBase *distinctify_hash_table,
+                         std::unique_ptr<TupleIdSequence> *reuse_matches,
+                         std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
+
+  /**
    * @brief Perform an UPDATE query over the tuples in this StorageBlock.
    * @warning In some edge cases, calling this method may cause IndexSubBlocks
    *          in this block to become inconsistent (the TupleStorageSubBlock
diff --git a/transaction/AccessMode.cpp b/transaction/AccessMode.cpp
new file mode 100644
index 0000000..3618bfd
--- /dev/null
+++ b/transaction/AccessMode.cpp
@@ -0,0 +1,34 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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 "AccessMode.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+const bool AccessMode::kLockCompatibilityMatrix[kNumberLocks][kNumberLocks] = {
+/*           NL     IS     IX      S     SIX     X    */
+/*  NL  */ {true , true , true , true , true , true },
+/*  IS  */ {true , true , true , true , true , false},
+/*  IX  */ {true , true , true , false, false, false},
+/*  S   */ {true , true , false, true , false, false},
+/*  SIX */ {true , true , false, false, false, false},
+/*  X   */ {true , false, false, false, false, false}
+};
+
+}  // namespace transaction
+}  // namespace quickstep
diff --git a/transaction/AccessMode.hpp b/transaction/AccessMode.hpp
new file mode 100644
index 0000000..34ace36
--- /dev/null
+++ b/transaction/AccessMode.hpp
@@ -0,0 +1,158 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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_ACCESS_MODE_HPP_
+#define QUICKSTEP_TRANSACTION_ACCESS_MODE_HPP_
+
+#include <cstdint>
+#include <type_traits>
+
+namespace quickstep {
+namespace transaction {
+
+/** \addtogroup 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,
+  kNumAccessModeTypes,
+};
+
+/**
+ * @brief Class for representing resource lock's access mode
+ **/
+class AccessMode {
+ public:
+  /**
+   * @brief Only constructor for access mode.
+   *
+   * @param access_mode Mode type of the object.
+   **/
+  explicit AccessMode(const AccessModeType access_mode)
+      : access_mode_(access_mode) {}
+
+  /**
+   * @brief Checks whether this access mode is compatible with the other.
+   *
+   * @param other Other access mode that will be checked against to this one.
+   * @return True if they are compatible, otherwise false.
+   **/
+  inline bool isCompatible(const AccessMode &other) const {
+    const access_mode_underlying_type this_mode =
+        static_cast<access_mode_underlying_type>(access_mode_);
+    const access_mode_underlying_type other_mode =
+        static_cast<access_mode_underlying_type>(other.access_mode_);
+    return AccessMode::kLockCompatibilityMatrix[this_mode][other_mode];
+  }
+
+  /**
+   * @brief Checks whether this access mode is IS mode.
+   *
+   * @return True if it is IS mode, false otherwise.
+   **/
+  inline bool isIntentionShareLock() const {
+    return access_mode_ == AccessModeType::kIsLock;
+  }
+
+  /**
+   * @brief Checks whether this access mode is IX mode.
+   *
+   * @return True if it is IX mode, false otherwise.
+   **/
+  inline bool isIntentionExclusiveLock() const {
+    return access_mode_ == AccessModeType::kIxLock;
+  }
+
+  /**
+   * @brief Checks whether this access mdoe is SIX mode.
+   *
+   * @return True if it is SIX mode, false otherwise.
+   **/
+  inline bool isShareAndIntentionExclusiveLock() const {
+    return access_mode_ == AccessModeType::kSixLock;
+  }
+
+  /**
+   * @brief Checks whether this access mode is S mode.
+   *
+   * @return True if it is S mode, false otherwise.
+   **/
+  inline bool isShareLock() const {
+    return access_mode_ == AccessModeType::kSLock;
+  }
+
+  /**
+   * @brief Checks whether this access mode is X mode.
+   *
+   * @return True if it is X mode, false otherwise.
+   **/
+  inline bool isExclusiveLock() const {
+    return access_mode_ == AccessModeType::kXLock;
+  }
+
+  /**
+   * @brief Checks whether this access mode is in
+   *        the same level with other mode.
+   *
+   * @return True if both modes have the same level.
+   **/
+  inline bool operator==(const AccessMode &other) const {
+    return access_mode_ == other.access_mode_;
+  }
+
+  /**
+   * @brief Checks whether this access mode is in
+   *        the different level with other mode.
+   *
+   * @return True if the modes have different levels.
+   **/
+  inline bool operator!=(const AccessMode &other) const {
+    return access_mode_ != other.access_mode_;
+  }
+
+ private:
+  typedef std::underlying_type<AccessModeType>::type
+      access_mode_underlying_type;
+
+  // The compatibility matrix should be N by N. kNumberLocks == N.
+  static constexpr std::uint64_t kNumberLocks =
+      static_cast<access_mode_underlying_type>(
+          AccessModeType::kNumAccessModeTypes);
+
+  // Compatibility matrix for checking access modes.
+  // True means they are compatible.
+  static const bool kLockCompatibilityMatrix[kNumberLocks][kNumberLocks];
+  // Type of access, the possible values are
+  // NoLock, IsLock, IxLock, SLock, SixLock, XLock
+  AccessModeType access_mode_;
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TRANSACTION_ACCESS_MODE_HPP_
diff --git a/transaction/CMakeLists.txt b/transaction/CMakeLists.txt
index 2babf51..05fc96a 100644
--- a/transaction/CMakeLists.txt
+++ b/transaction/CMakeLists.txt
@@ -13,31 +13,89 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License.
 
+add_library(quickstep_transaction_AccessMode
+            AccessMode.cpp
+            AccessMode.hpp)
 add_library(quickstep_transaction_DirectedGraph
             ../empty_src.cpp
             DirectedGraph.hpp)
+add_library(quickstep_transaction_Lock
+            ../empty_src.cpp
+            Lock.hpp)
+add_library(quickstep_transaction_LockRequest
+            ../empty_src.cpp
+            LockRequest.hpp)
+add_library(quickstep_transaction_LockTable
+            LockTable.cpp
+            LockTable.hpp)
+add_library(quickstep_transaction_ResourceId
+            ResourceId.cpp
+            ResourceId.hpp)
 add_library(quickstep_transaction_StronglyConnectedComponents
             StronglyConnectedComponents.cpp
             StronglyConnectedComponents.hpp)
 add_library(quickstep_transaction_Transaction
             ../empty_src.cpp
             Transaction.hpp)
-
+add_library(quickstep_transaction_TransactionTable
+            TransactionTable.cpp
+            TransactionTable.hpp)
+          
 target_link_libraries(quickstep_transaction_DirectedGraph
-	              glog
+                      glog
                       quickstep_transaction_Transaction
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_transaction_Lock
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_ResourceId)
+target_link_libraries(quickstep_transaction_LockRequest
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction)
+target_link_libraries(quickstep_transaction_LockTable
+                      quickstep_threading_SharedMutex
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_Lock
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_transaction_ResourceId
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_utility_HashPair)
+target_link_libraries(quickstep_transaction_ResourceId
+                      glog)
 target_link_libraries(quickstep_transaction_StronglyConnectedComponents
                       quickstep_transaction_DirectedGraph
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_transaction_TransactionTable
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_Lock
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction
+                      quickstep_utility_Macros)
 
 add_library(quickstep_transaction
             ../empty_src.cpp
             TransactionModule.hpp)
 target_link_libraries(quickstep_transaction
+                      quickstep_transaction_AccessMode
                       quickstep_transaction_DirectedGraph
+                      quickstep_transaction_Lock
+                      quickstep_transaction_LockRequest
+                      quickstep_transaction_LockTable
+                      quickstep_transaction_ResourceId
                       quickstep_transaction_StronglyConnectedComponents
-                      quickstep_transaction_Transaction)
+                      quickstep_transaction_Transaction
+                      quickstep_transaction_TransactionTable)
+
+add_executable(AccessMode_unittest
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AccessMode_unittest.cpp")
+target_link_libraries(AccessMode_unittest
+                      gtest
+                      gtest_main
+                      quickstep_transaction_AccessMode)
+add_test(AccessMode_unittest AccessMode_unittest)
 
 add_executable(DirectedGraph_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/DirectedGraph_unittest.cpp")
@@ -48,6 +106,45 @@
                       quickstep_transaction_Transaction)
 add_test(DirectedGraph_unittest DirectedGraph_unittest)
 
+add_executable(Lock_unittest
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/Lock_unittest.cpp")
+target_link_libraries(Lock_unittest
+                      gtest
+                      gtest_main
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_Lock
+                      quickstep_transaction_ResourceId)
+add_test(Lock_unittest Lock_unittest)
+
+add_executable(LockRequest_unittest
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/LockRequest_unittest.cpp")
+target_link_libraries(LockRequest_unittest
+                      gtest
+                      gtest_main
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction)
+add_test(LockRequest_unittest LockRequest_unittest)
+
+add_executable(LockTable_unittest
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/LockTable_unittest.cpp")
+target_link_libraries(LockTable_unittest
+                      gtest
+                      gtest_main
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_LockTable
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction)
+add_test(LockTable_unittest LockTable_unittest)
+
+add_executable(ResourceId_unittest
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/ResourceId_unittest.cpp")
+target_link_libraries(ResourceId_unittest
+                      gtest
+                      gtest_main
+                      quickstep_transaction_ResourceId)
+add_test(ResourceId_unittest ResourceId_unittest)
+
 add_executable(StronglyConnectedComponents_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/StronglyConnectedComponents_unittest.cpp")
 target_link_libraries(StronglyConnectedComponents_unittest
@@ -56,3 +153,14 @@
                       quickstep_transaction_DirectedGraph
                       quickstep_transaction_StronglyConnectedComponents)
 add_test(StronglyConnectedComponents_unittest StronglyConnectedComponents_unittest)
+
+add_executable(TransactionTable_unittest
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/TransactionTable_unittest.cpp")
+target_link_libraries(TransactionTable_unittest
+                      gtest
+                      gtest_main
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction
+                      quickstep_transaction_TransactionTable)
+add_test(TransactionTable_unittest TransactionTable_unittest)
diff --git a/transaction/DirectedGraph.hpp b/transaction/DirectedGraph.hpp
index 5c62425..89ce9c6 100644
--- a/transaction/DirectedGraph.hpp
+++ b/transaction/DirectedGraph.hpp
@@ -19,6 +19,8 @@
 #define QUICKSTEP_TRANSACTION_DIRECTED_GRAPH_HPP_
 
 #include <algorithm>
+#include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <stack>
 #include <unordered_set>
@@ -30,7 +32,6 @@
 #include "glog/logging.h"
 
 namespace quickstep {
-
 namespace transaction {
 
 /** \addtogroup Transaction
@@ -200,7 +201,6 @@
 /** @} */
 
 }  // namespace transaction
-
 }  // namespace quickstep
 
 #endif  // QUICKSTEP_TRANSACTION_DIRECTED_GRAPH_HPP_
diff --git a/transaction/Lock.hpp b/transaction/Lock.hpp
new file mode 100644
index 0000000..27fe93a
--- /dev/null
+++ b/transaction/Lock.hpp
@@ -0,0 +1,103 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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_HPP_
+#define QUICKSTEP_TRANSACTION_LOCK_HPP_
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/ResourceId.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+/** \addtogroup Transaction
+ *  @{
+ */
+
+/**
+ * @brief Class for representing resource locks.
+ **/
+class Lock {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param rid Resource id of the resource on which the lock is taken.
+   * @param access_mode Mode of the lock access.
+   **/
+  Lock(const ResourceId &rid, const AccessMode &access_mode)
+      : rid_(rid),
+        access_mode_(access_mode) {
+  }
+
+  /**
+   * @brief Equality operator for lock.
+   *
+   * @param other Reference to the other Lock object that will be
+   *        compared against to this.
+   * @return True if both locks have equal resource ids
+   *         and access modes, false otherwise.
+   **/
+  inline bool operator==(const Lock &other) const {
+    return rid_ == other.rid_ && access_mode_ == other.access_mode_;
+  }
+
+  /**
+   * @brief Inequality operator for lock.
+   *
+   * @param other Reference to the other Lock object that will be
+   *        compared against to this.
+   * @return False if one of them has different resource id
+   *         or access mode, true otherwise.
+   **/
+  inline bool operator!=(const Lock &other) const {
+    return !(*this == other);
+  }
+
+  /**
+   * @brief Getter for resource id.
+   *
+   * @return Resource id of the lock.
+   **/
+  inline const ResourceId& getResourceId() const {
+    return rid_;
+  }
+
+  /**
+   * @brief Getter for access mode.
+   *
+   * @return Access mode of the lock.
+   **/
+  inline const AccessMode& getAccessMode() const {
+    return access_mode_;
+  }
+
+ private:
+  // Id of the resource that is locked.
+  const ResourceId rid_;
+
+  // Lock's access type.
+  AccessMode access_mode_;
+};
+
+/** @} */
+
+}  // namespace transaction
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TRANSACTION_LOCK_HPP_
diff --git a/transaction/LockRequest.hpp b/transaction/LockRequest.hpp
new file mode 100644
index 0000000..80daa4d
--- /dev/null
+++ b/transaction/LockRequest.hpp
@@ -0,0 +1,112 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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_REQUEST_HPP_
+#define QUICKSTEP_TRANSACTION_LOCK_REQUEST_HPP_
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+/** \addtogroup Transaction
+ *  @{
+ */
+
+/**
+ * @brief Enum class for representing request types.
+ **/
+enum class RequestType {
+  kAcquireLock = 0,
+  kReleaseLocks,
+};
+
+/**
+ * @brief Class for encapsulate lock request put into
+ *        the buffers.
+ **/
+class LockRequest {
+ public:
+  /**
+   * @brief Constructor for LockRequest.
+   *
+   * @param tid Id of the transaction that requests the lock.
+   * @param rid Id of the resource that is requested.
+   * @param access_mode Access mode of the request.
+   * @param type Type of the request.
+   */
+  LockRequest(const transaction_id tid,
+              const ResourceId &rid,
+              const AccessMode access_mode,
+              const RequestType request_type)
+      : tid_(tid),
+        rid_(rid),
+        access_mode_(access_mode),
+        request_type_(request_type) {
+  }
+
+  /**
+   * @brief Getter for transaction id.
+   *
+   * @return Transaction id of the request.
+   **/
+  inline transaction_id getTransactionId() const {
+    return tid_;
+  }
+
+  /**
+   * @brief Getter for resource id.
+   *
+   * @return Resource id of the request.
+   **/
+  inline const ResourceId& getResourceId() const {
+    return rid_;
+  }
+
+  /**
+   * @brief Getter for access mode.
+   *
+   * @return Access mode of the request.
+   **/
+  inline AccessMode getAccessMode() const {
+    return access_mode_;
+  }
+
+  /**
+   * @brief Getter for request type.
+   *
+   * @return Type of the request.
+   **/
+  inline RequestType getRequestType() const {
+    return request_type_;
+  }
+
+ private:
+  transaction_id tid_;
+  ResourceId rid_;
+  AccessMode access_mode_;
+  RequestType request_type_;
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif
diff --git a/transaction/LockTable.cpp b/transaction/LockTable.cpp
new file mode 100644
index 0000000..77986f6
--- /dev/null
+++ b/transaction/LockTable.cpp
@@ -0,0 +1,206 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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/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 {
+
+LockTableResult
+LockTable::putLock(const transaction_id tid,
+                   const ResourceId &rid,
+                   const AccessMode access_mode) {
+  // TODO(hakan): Lock upgrade is not supported.
+  lock_list_pair &lock_list_pair = internal_map_[rid];
+
+  // Each resource id entry has own list and pending list.
+  lock_own_list &lock_own_list = lock_list_pair.first;
+  lock_pending_list &lock_pending_list = lock_list_pair.second;
+
+  // Check this resource id already has the same lock request from
+  // the same transaction in the own list.
+  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;
+    }
+  }
+
+  // Check this resource id already has the same lock request from
+  // the same transaction in the pending list.
+  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;
+    }
+  }
+
+  // If the execution can reach this point, it means the resource id
+  // does not have duplicate lock record (for both in owned and pending).
+  if (lock_pending_list.empty()) {
+    for (lock_own_list::const_iterator it = lock_own_list.cbegin();
+         it != lock_own_list.cend(); ++it) {
+      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;
+      }
+    }
+
+    lock_own_list.push_back(std::make_pair(tid, Lock(rid, access_mode)));
+    return LockTableResult::kPLACED_IN_OWNED;
+  } 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;
+  }
+}
+
+LockTableResult
+LockTable::deleteLock(const transaction_id tid,
+                      const ResourceId &rid) {
+  lock_list_pair &lock_list_pair = internal_map_[rid];
+
+  // Each resource id has its own and pending locks list.
+  lock_own_list &lock_own_list = lock_list_pair.first;
+  lock_pending_list &lock_pending_list = lock_list_pair.second;
+
+  // Iterate over owned locks list to see the lock entry of the transaction
+  // on the resource id exists.
+  for (lock_own_list::const_iterator it = lock_own_list.begin();
+       it != lock_own_list.cend(); ++it) {
+    if (it->first == tid) {
+      // If it exists, erase it from the owned list.
+      lock_own_list.erase(it);
+
+      // Since we erased a lock entry from owned list, the first entries
+      // in the pending list can be pushed to owned list if they are
+      // compatible with the remaining owned entries.
+      movePendingToOwned(rid);
+
+      return LockTableResult::kDEL_FROM_OWNED;
+    }
+  }
+
+  // Iterate over pending locks list to check the lock entry of the transaction
+  // on this resource id exists.
+  for (lock_pending_list::const_iterator it = lock_pending_list.begin();
+         it != lock_pending_list.cend(); ++it) {
+    if (it->first == tid) {
+      // If it exists, erase it from pending list.
+      lock_pending_list.erase(it);
+      return LockTableResult::kDEL_FROM_PENDING;
+    }
+  }
+
+  // Execution reaches here, if we cannot find the corresponding lock entry
+  // in the both list.
+  return LockTableResult::kDEL_ERROR;
+}
+
+void LockTable::movePendingToOwned(const ResourceId &rid) {
+  lock_list_pair &lock_list_pair = internal_map_[rid];
+  lock_own_list &lock_own_list = lock_list_pair.first;
+  lock_pending_list &lock_pending_list = lock_list_pair.second;
+
+  // Iterate over pending list to pending requests compatible with the
+  // all entries in the resource ids owned lock list.
+  for (lock_pending_list::const_iterator pending_it = lock_pending_list.cbegin();
+         pending_it != lock_pending_list.cend(); ++pending_it) {
+    transaction_id pending_tid = pending_it->first;
+    AccessMode pending_mode = pending_it->second.getAccessMode();
+    bool is_compatible_with_own_list = true;
+
+    // Now compare against the all entries in the owned lock list.
+    for (lock_own_list::const_iterator owned_it = lock_own_list.cbegin();
+           owned_it != lock_pending_list.cend(); ++owned_it) {
+      AccessMode owned_mode = owned_it->second.getAccessMode();
+      if (!pending_mode.isCompatible(owned_mode)) {
+        // If it is not compatible, we will not move this entry.
+        is_compatible_with_own_list = false;
+        break;
+      }
+    }
+
+    // If this pending lock entry is compatible with the all entries in the
+    // owned lock list, we should move it from pending list to owned list.
+    if (is_compatible_with_own_list) {
+      // Erase the entry from the list. Get the new iterator.
+      pending_it = lock_pending_list.erase(pending_it);
+
+      // Put the corresponding entry to the own list.
+      lock_own_list.emplace_back(pending_tid, Lock(rid, pending_mode));
+
+      // Move iterator one step backward because erasing an element moves the
+      // iterator to the next element.
+      --pending_it;
+    } else {
+      // There is no need to iterate pending list anymore since
+      // we found first incompatible one. Checking, and accepting other pending
+      // entries may cause starvation.
+      break;
+    }
+  }
+}
+
+LockTable::iterator LockTable::begin() {
+  return internal_map_.begin();
+}
+
+LockTable::iterator LockTable::end() {
+  return internal_map_.end();
+}
+
+LockTable::const_iterator LockTable::begin() const {
+  return internal_map_.begin();
+}
+
+LockTable::const_iterator LockTable::end() const {
+  return internal_map_.end();
+}
+
+void LockTable::latchShared() {
+  mutex_.lockShared();
+}
+
+void LockTable::unlatchShared() {
+  mutex_.unlockShared();
+}
+
+void LockTable::latchExclusive() {
+  mutex_.lock();
+}
+
+void LockTable::unlatchExclusive() {
+  mutex_.unlock();
+}
+
+}  // namespace transaction
+}  // namespace quickstep
diff --git a/transaction/LockTable.hpp b/transaction/LockTable.hpp
new file mode 100644
index 0000000..5a0612e
--- /dev/null
+++ b/transaction/LockTable.hpp
@@ -0,0 +1,179 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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_TABLE_HPP_
+#define QUICKSTEP_TRANSACTION_LOCK_TABLE_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 {
+
+/** \addtogroup 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,
+};
+
+/**
+ * @brief LockTable class represents the hash map for RID and
+ *        list of locks on RID.
+ **/
+class LockTable {
+ public:
+  typedef std::pair<transaction_id, Lock> lock_entry;
+  typedef std::list<lock_entry> lock_own_list;
+  typedef std::list<lock_entry> lock_pending_list;
+  typedef std::pair<lock_own_list, lock_pending_list> lock_list_pair;
+  typedef std::unordered_map<ResourceId,
+                             lock_list_pair,
+                             ResourceId::ResourceIdHasher> internal_map_type;
+  typedef internal_map_type::iterator iterator;
+  typedef internal_map_type::const_iterator const_iterator;
+
+  /**
+   * @brief Constructor for LockTable.
+   **/
+  LockTable() {
+  }
+
+  /**
+   * @brief Puts the lock entry into the lock table for corresponding resource.
+   *
+   * @param tid Id of the transaction that requests the lock.
+   * @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
+   *         already granted,
+   *         LockTableResult::kALREADY_IN_PENDING if lock has been
+   *         already pending.
+   **/
+  LockTableResult putLock(const transaction_id tid,
+                          const ResourceId &rid,
+                          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
+   *         owned list,
+   *         LockTableResult::kDEL_FROM_PENDING if the lock is deleted from
+   *         pending list,
+   *         LockTableResult::kDEL_ERROR if the lock cannot be found
+   **/
+  LockTableResult deleteLock(const transaction_id tid,
+                             const ResourceId &rid);
+
+
+  /**
+   * @brief Iterator for begin position.
+   *
+   * @return Non-const Iterator which points to begin point
+   *         of the lock table.
+   **/
+  iterator begin();
+
+  /**
+   * @brief Iterator for end position.
+   *
+   * @return Non-const iterator which points to end point
+   *         of the lock table.
+   **/
+  iterator end();
+
+  /**
+   * @brief Iterator for begin position.
+   *
+   * @return Const iterator which points to the begin
+   *         point of the lock table.
+   **/
+  const_iterator begin() const;
+
+  /**
+   * @brief Iterator for end position.
+   *
+   * @return Const iterator which points to the end
+   *         point of the lock table.
+   **/
+  const_iterator end() const;
+
+  /**
+   * @brief Latch mutex in shared mode. Multiple shared mode
+   *        latch acquisition is compatible.
+   **/
+  void latchShared();
+
+  /**
+   * @brief Unlatch mutex in shared mode.
+   **/
+  void unlatchShared();
+
+  /**
+   * @brief Latch mutex in exclusive mode.
+   */
+  void latchExclusive();
+
+  /**
+   * @brief Unlatch mutex in exclusive mode.
+   */
+  void unlatchExclusive();
+
+ private:
+  // This method will be called after deletion of locks.
+  // After delete, some pending locks might be acquired.
+  void movePendingToOwned(const ResourceId &rid);
+
+  internal_map_type internal_map_;
+
+  // Mutex protects whole lock table.
+  SharedMutex mutex_;
+
+  DISALLOW_COPY_AND_ASSIGN(LockTable);
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif
diff --git a/transaction/ResourceId.cpp b/transaction/ResourceId.cpp
new file mode 100644
index 0000000..bae92f8
--- /dev/null
+++ b/transaction/ResourceId.cpp
@@ -0,0 +1,74 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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/ResourceId.hpp"
+
+#include <cstddef>
+#include <functional>
+#include <string>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "utility/HashPair.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace transaction {
+
+bool ResourceId::operator==(const ResourceId &other) const {
+  return db_id_ == other.db_id_
+      && rel_id_ == other.rel_id_
+      && block_id_ == other.block_id_
+      && tuple_id_ == other.tuple_id_;
+}
+
+ResourceId ResourceId::getParentResourceId() const {
+  if (isRelationAccess()) {
+    return ResourceId(db_id_);
+  } else if (isBlockAccess()) {
+    return ResourceId(db_id_, rel_id_);
+  } else if (isTupleAccess()) {
+    return ResourceId(db_id_, rel_id_, block_id_);
+  } else {
+    LOG(FATAL) << "Database level does not have any parent level.";
+  }
+}
+
+std::string ResourceId::toString() const {
+  return "ResourceId(" +
+    std::to_string(db_id_) + ", " +
+    std::to_string(rel_id_) + ", " +
+    std::to_string(block_id_) + ", " +
+    std::to_string(tuple_id_) + ")";
+}
+
+std::size_t
+ResourceId::ResourceIdHasher::operator()(const ResourceId &rid) const {
+  const std::size_t hash1 = std::hash<database_id>()(rid.db_id_);
+  const std::size_t hash2 = std::hash<relation_id>()(rid.rel_id_);
+  const std::size_t hash3 = std::hash<block_id>()(rid.block_id_);
+  const std::size_t hash4 = std::hash<tuple_id>()(rid.tuple_id_);
+
+  const std::size_t comb1 = CombineHashes(hash1, hash2);
+  const std::size_t comb2 = CombineHashes(hash3, hash4);
+
+  return CombineHashes(comb1, comb2);
+}
+
+}  // namespace transaction
+}  // namespace quickstep
diff --git a/transaction/ResourceId.hpp b/transaction/ResourceId.hpp
new file mode 100644
index 0000000..b9d1cdf
--- /dev/null
+++ b/transaction/ResourceId.hpp
@@ -0,0 +1,216 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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_RESOURCE_ID_HPP_
+#define QUICKSTEP_TRANSACTION_RESOURCE_ID_HPP_
+
+#include <cstddef>
+#include <limits>
+#include <string>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/StorageBlockInfo.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+/** \addtogroup Transaction
+ *  @{
+ */
+
+/**
+ * @brief Class for representing unique id for each database resource
+ *        e.g Tuple, block, relation, database etc...
+ **/
+class ResourceId {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param db_id Unique id of the database resource.
+   * @param rel_id Unique id of the relation resource in the database.
+   * @param block_id Unique id of the block resource in the relation.
+   * @param tuple_id Unique id of the tuple resource in the block.
+   **/
+  explicit ResourceId(const database_id db_id = kDatabaseIdPlaceholder,
+                      const relation_id rel_id = kRelationIdPlaceholder,
+                      const block_id block_id = kBlockIdPlaceholder,
+                      const tuple_id tuple_id = kTupleIdPlaceholder)
+      : db_id_(db_id),
+        rel_id_(rel_id),
+        block_id_(block_id),
+        tuple_id_(tuple_id) {
+  }
+
+  /**
+   * @brief Copy constructor.
+   *
+   * @param other The ResourceId that will be copied.
+   **/
+  ResourceId(const ResourceId &other)
+      : db_id_(other.db_id_),
+        rel_id_(other.rel_id_),
+        block_id_(other.block_id_),
+        tuple_id_(other.tuple_id_) {
+  }
+
+  /**
+   * @brief Hasher class for ResourceId class to use it in the hash map.
+   **/
+  struct ResourceIdHasher {
+    /**
+     * @brief Functor of the class.
+     *
+     * @param rid Resource id to be hashed.
+     * @return Hash of the resource id.
+     **/
+    std::size_t operator()(const ResourceId &rid) const;
+  };
+
+  /**
+   * @brief Equality operator for ResourceId class.
+   *
+   * @param other Resource to be compared.
+   * @return True if this and other are the id of the same
+   *         resource, false otherwise.
+   **/
+  bool operator==(const ResourceId &other) const;
+
+   /**
+   * @brief Inequality operator for ResourceId class.
+   *
+   * @param other Resource to be compared.
+   * @return True if this and other are the ids of the different
+   *         resources, false otherwise.
+   **/
+  inline bool operator!=(const ResourceId &other) const {
+    return !(*this == other);
+  }
+
+  /**
+   * @brief Checks whether this resource id has a parent in resource hierarchy.
+   *
+   * @return False if this resource id is database level, true otherwise.
+   **/
+  inline bool hasParent() const {
+    return !isDatabaseAccess();
+  }
+
+  /**
+   * @brief Getter for this resource id's parent in the hierarchy.
+   *
+   * @return The resource id of this resource id's parent.
+   **/
+  ResourceId getParentResourceId() const;
+
+  /**
+   * @brief Checks whether this is a database level resource id.
+   *
+   * @return True if this is a database level access, false otherwise.
+   **/
+  inline bool isDatabaseAccess() const {
+    return !isDatabaseIdPlaceholder()
+        && isRelationIdPlaceholder()
+        && isBlockIdPlaceholder()
+        && isTupleIdPlaceholder();
+  }
+
+  /**
+   * @brief Checks whether this is a relation level resource id.
+   *
+   * @return True if this is a relation level access, false otherwise.
+   **/
+  bool isRelationAccess() const {
+    return !isDatabaseIdPlaceholder()
+        && !isRelationIdPlaceholder()
+        && isBlockIdPlaceholder()
+        && isTupleIdPlaceholder();
+  }
+
+  /**
+   * @brief Checks whether this is a block level resource id.
+   *
+   * @return True if this is a block level access, false otherwise.
+   **/
+  bool isBlockAccess() const {
+    return !isDatabaseIdPlaceholder()
+        && !isRelationIdPlaceholder()
+        && !isBlockIdPlaceholder()
+        && isTupleIdPlaceholder();
+  }
+
+  /**
+   * @brief Checks whether this is a tuple level resource id.
+   *
+   * @return True if this is a tuple level access, false otherwise.
+   **/
+  bool isTupleAccess() const {
+    return !isDatabaseIdPlaceholder()
+        && !isRelationIdPlaceholder()
+        && !isBlockIdPlaceholder()
+        && !isTupleIdPlaceholder();
+  }
+
+  /**
+   * @brief This is a helper method for string representation
+   *        of the resource id.
+   *
+   * @return String representation of the reosurce id.
+   **/
+  std::string toString() const;
+
+ private:
+  // Negative value is invalid id for database id. Use -1 as placeholder.
+  static constexpr database_id kDatabaseIdPlaceholder = kInvalidCatalogId;
+
+  // Negative value is invalid id for relation id. Use -1 as placeholder.
+  static constexpr relation_id kRelationIdPlaceholder = kInvalidCatalogId;
+
+  // Zero is invalid id for block id. Use zero as the placeholder.
+  static constexpr block_id kBlockIdPlaceholder = kInvalidBlockId;
+
+  // Negative tuple id os invalid, therefore use a negative value.
+  static constexpr tuple_id kTupleIdPlaceholder = kInvalidCatalogId;
+
+  inline bool isDatabaseIdPlaceholder() const {
+    return db_id_ == kDatabaseIdPlaceholder;
+  }
+
+  inline bool isRelationIdPlaceholder() const {
+    return rel_id_ == kRelationIdPlaceholder;
+  }
+
+  inline bool isBlockIdPlaceholder() const {
+    return block_id_ == kBlockIdPlaceholder;
+  }
+
+  inline bool isTupleIdPlaceholder() const {
+    return tuple_id_ == kTupleIdPlaceholder;
+  }
+
+  const database_id db_id_;
+  const relation_id rel_id_;
+  const block_id block_id_;
+  const tuple_id tuple_id_;
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RESOURCE_ID_HPP_
diff --git a/transaction/StronglyConnectedComponents.cpp b/transaction/StronglyConnectedComponents.cpp
index 76da4bc..89daf46 100644
--- a/transaction/StronglyConnectedComponents.cpp
+++ b/transaction/StronglyConnectedComponents.cpp
@@ -17,13 +17,15 @@
 
 #include "transaction/StronglyConnectedComponents.hpp"
 
+#include <cstddef>
 #include <cstdint>
 #include <stack>
 #include <unordered_map>
 #include <vector>
 
-namespace quickstep {
+#include "transaction/DirectedGraph.hpp"
 
+namespace quickstep {
 namespace transaction {
 
 StronglyConnectedComponents::StronglyConnectedComponents(
@@ -118,5 +120,4 @@
 }
 
 }  // namespace transaction
-
 }  // namespace quickstep
diff --git a/transaction/StronglyConnectedComponents.hpp b/transaction/StronglyConnectedComponents.hpp
index 57a8d59..ca58a68 100644
--- a/transaction/StronglyConnectedComponents.hpp
+++ b/transaction/StronglyConnectedComponents.hpp
@@ -28,7 +28,6 @@
 #include "utility/Macros.hpp"
 
 namespace quickstep {
-
 namespace transaction {
 
 /** \addtogroup Transaction
@@ -115,7 +114,6 @@
 /** @} */
 
 }  // namespace transaction
-
 }  // namespace quickstep
 
 #endif  // QUICKSTEP_TRANSACTION_STRONGLY_CONNECTED_COMPONENTS_HPP_
diff --git a/transaction/Transaction.cpp b/transaction/Transaction.cpp
new file mode 100644
index 0000000..3478d01
--- /dev/null
+++ b/transaction/Transaction.cpp
@@ -0,0 +1,48 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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
diff --git a/transaction/Transaction.hpp b/transaction/Transaction.hpp
index 82ca436..69d4311 100644
--- a/transaction/Transaction.hpp
+++ b/transaction/Transaction.hpp
@@ -23,7 +23,6 @@
 #include <functional>
 
 namespace quickstep {
-
 namespace transaction {
 
 /** \addtogroup Transaction
@@ -115,7 +114,6 @@
 /** @} */
 
 }  // namespace transaction
-
 }  // namespace quickstep
 
 #endif  // QUICKSTEP_TRANSACTION_TRANSACTION_HPP_
diff --git a/transaction/TransactionTable.cpp b/transaction/TransactionTable.cpp
new file mode 100644
index 0000000..993703a
--- /dev/null
+++ b/transaction/TransactionTable.cpp
@@ -0,0 +1,140 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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/TransactionTable.hpp"
+
+#include <cstddef>
+#include <list>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+TransactionTableResult
+TransactionTable::putOwnEntry(const transaction_id tid,
+                              const ResourceId &rid,
+                              const AccessMode access_mode) {
+  transaction_list_pair &transaction_list_pair = internal_map_[tid];
+  transaction_own_list &transaction_own_list = transaction_list_pair.first;
+
+  transaction_own_list.push_back(std::make_pair(rid, Lock(rid, access_mode)));
+
+  return TransactionTableResult::kPlacedInOwned;
+}
+
+TransactionTableResult
+TransactionTable::putPendingEntry(const transaction_id tid,
+                                  const ResourceId &rid,
+                                  const AccessMode access_mode) {
+  transaction_list_pair &transaction_list_pair = internal_map_[tid];
+  transaction_pending_list &transaction_pending_list
+      = transaction_list_pair.second;
+
+  transaction_pending_list.push_back(std::make_pair(rid,
+                                                    Lock(rid, access_mode)));
+
+  return TransactionTableResult::kPlacedInPending;
+}
+
+TransactionTableResult
+TransactionTable::deleteOwnEntry(const transaction_id tid,
+                                 const ResourceId &rid,
+                                 const AccessMode access_mode) {
+  transaction_list_pair &transaction_list_pair = internal_map_[tid];
+  transaction_own_list &transaction_own_list = transaction_list_pair.first;
+
+  std::size_t original_size = transaction_own_list.size();
+  transaction_own_list.remove_if(
+      [&rid, &access_mode](transaction_entry &entry) {
+        return entry.second.getResourceId() == rid
+        && entry.second.getAccessMode() == access_mode;
+      });
+  if (transaction_own_list.size() == original_size) {
+    return TransactionTableResult::kDelError;
+  } else {
+    return TransactionTableResult::kDelFromOwned;
+  }
+}
+
+TransactionTableResult
+TransactionTable::deletePendingEntry(const transaction_id tid,
+                                     const ResourceId &rid,
+                                     const AccessMode access_mode) {
+  transaction_list_pair &transaction_list_pair = internal_map_[tid];
+  transaction_pending_list &transaction_pending_list
+      = transaction_list_pair.second;
+
+  std::size_t original_size = transaction_pending_list.size();
+  transaction_pending_list.remove_if(
+     [&rid, &access_mode] (transaction_entry &entry) {
+       return entry.second.getResourceId() == rid
+         && entry.second.getAccessMode() == access_mode;
+     });
+
+  if (transaction_pending_list.size() == original_size) {
+    return TransactionTableResult::kDelError;
+  } else {
+    return TransactionTableResult::kDelFromPending;
+  }
+}
+
+std::vector<ResourceId>
+TransactionTable::getResourceIdList(const transaction_id tid) {
+  std::vector<ResourceId> result;
+  const transaction_list_pair &transaction_list_pair = internal_map_[tid];
+  const transaction_own_list &transaction_own_list =
+      transaction_list_pair.second;
+  const transaction_pending_list
+      &transaction_pending_list = transaction_list_pair.first;
+
+  for (transaction_own_list::const_iterator it = transaction_own_list.begin();
+       it != transaction_own_list.end();
+       ++it) {
+    result.push_back(it->first);
+  }
+
+  for (transaction_pending_list::const_iterator
+           it = transaction_pending_list.begin();
+       it != transaction_pending_list.end();
+       ++it) {
+    result.push_back(it->first);
+  }
+
+  return result;
+}
+
+TransactionTableResult
+TransactionTable::deleteTransaction(const transaction_id tid) {
+  std::size_t original_size = internal_map_.size();
+  internal_map_.erase(tid);
+  std::size_t size_after_delete = internal_map_.size();
+
+  if (original_size == size_after_delete) {
+    return  TransactionTableResult::kTransactionDeleteError;
+  }
+
+  return TransactionTableResult::kTransactionDeleteOk;
+}
+
+}  // namespace transaction
+}  // namespace quickstep
diff --git a/transaction/TransactionTable.hpp b/transaction/TransactionTable.hpp
new file mode 100644
index 0000000..a5e1da4
--- /dev/null
+++ b/transaction/TransactionTable.hpp
@@ -0,0 +1,161 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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_TRANSACTION_TABLE_HPP_
+#define QUICKSTEP_TRANSACTION_TRANSACTION_TABLE_HPP_
+
+#include <list>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/Lock.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+/** \addtogroup Transaction
+ *  @{
+ */
+
+/**
+ * @brief Represents different result for TransactionTable's methods.
+ **/
+enum class TransactionTableResult {
+  kPlacedInOwned = 0,
+  kPlacedInPending,
+  kAlreadyInOwned,
+  kAlreadyInPending,
+  kDelFromOwned,
+  kDelFromPending,
+  kDelError,
+  kTransactionDeleteOk,
+  kTransactionDeleteError,
+};
+
+/**
+ * @brief Class for keeping track of the owner and pending list of transactions.
+ **/
+class TransactionTable {
+ public:
+  typedef std::pair<ResourceId, Lock> transaction_entry;
+  typedef std::list<transaction_entry> transaction_own_list;
+  typedef std::list<transaction_entry> transaction_pending_list;
+  typedef std::pair<transaction_own_list, transaction_pending_list>
+      transaction_list_pair;
+
+  /**
+   * @brief Contructor for TransactionTable.
+   **/
+  TransactionTable() {
+  }
+
+  /**
+   * @brief Puts a owned entry of the given resource id in the given
+   *        transaction's owned list.
+   *
+   * @param tid Transaction id of the requestor.
+   * @param rid Resource id of the corresponding lock.
+   * @param access_mode Access mode of the lock.
+   *
+   * @return TransactionTableResult::kPLACED_IN_OWNED since it is
+   *         always a successful operation on owned list.
+   **/
+  TransactionTableResult putOwnEntry(const transaction_id tid,
+                                     const ResourceId &rid,
+                                     const AccessMode access_mode);
+
+  /**
+   * @brief Puts a pending entry of the given resource id in the given
+   *        transaction's pending list.
+   *
+   * @param tid Transaction id of the requestor.
+   * @param rid Resource id of the corresponding lock.
+   * @param access_mode Access mode of the lock.
+   *
+   * @return TransactionTableResult::kPLACED_IN_PENDING
+   **/
+  TransactionTableResult putPendingEntry(const transaction_id tid,
+                                         const ResourceId &rid,
+                                         const AccessMode access_mode);
+
+  /**
+   * @brief Deletes the owned entry corresponding to the resource id
+   *        in the transaction's owned list.
+   *
+   * @param tid Transaction id of the owner.
+   * @param rid Resource id of the corresponding lock.
+   * @param access_mode Access mode of the lock.
+   *
+   * @return TransactionTableResult::kDEL_FROM_OWNED if the entry is deleted,
+   *         otherwise TransactionTable::kDEL_ERROR.
+   **/
+  TransactionTableResult deleteOwnEntry(const transaction_id tid,
+                                        const ResourceId &rid,
+                                        const AccessMode access_mode);
+
+  /**
+   * @brief Deletes the pending entry corresponding to the resource id
+   *        in the transaction's pending list.
+   * @param tid Transaction id of the owner.
+   * @param rid Resource id of the corresponding lock.
+   * @param access_mode Access mode of the lock.
+   *
+   * @return TransactionTableResult::kDEL_FROM_PENDING if the entry is
+   *         successfuly deleted, otherwise TransactionTableResult::k_DEL_ERROR.
+   **/
+  TransactionTableResult deletePendingEntry(const transaction_id tid,
+                                            const ResourceId &rid,
+                                            const AccessMode access_mode);
+
+  /**
+   * @brief Returns a vector of resource ids which the corresponding transaction
+   *        owns or pends.
+   *
+   * @param tid Transaction id of the corresponding transaction
+   *
+   * @return Vector of resource id that the transaction owns or pends.
+   **/
+  std::vector<ResourceId> getResourceIdList(const transaction_id tid);
+
+  /**
+   * @brief Deletes the transaction entry from transaction table.
+   *
+   * @param tid Transaction id of the corresponding transaction.
+   *
+   * @return TransactionTableResult::kTRANSACTION_DELETE_ERROR if there is no
+   *         entry for the transaction, otherwise
+   *         TransactionTableResult::kTRANSACTION_DELETE_OK.
+   **/
+  TransactionTableResult deleteTransaction(const transaction_id tid);
+
+ private:
+  std::unordered_map<transaction_id, transaction_list_pair> internal_map_;
+
+  DISALLOW_COPY_AND_ASSIGN(TransactionTable);
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif
diff --git a/transaction/tests/AccessMode_unittest.cpp b/transaction/tests/AccessMode_unittest.cpp
new file mode 100644
index 0000000..fa51525
--- /dev/null
+++ b/transaction/tests/AccessMode_unittest.cpp
@@ -0,0 +1,171 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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/AccessMode.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+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) {
+  }
+
+  const AccessMode nl_mode_;
+  const AccessMode is_mode_;
+  const AccessMode ix_mode_;
+  const AccessMode s_mode_;
+  const AccessMode six_mode_;
+  const AccessMode x_mode_;
+};
+
+TEST_F(AccessModeTest, ModeCompatibilty) {
+  EXPECT_TRUE(nl_mode_.isCompatible(nl_mode_));
+  EXPECT_TRUE(nl_mode_.isCompatible(is_mode_));
+  EXPECT_TRUE(nl_mode_.isCompatible(ix_mode_));
+  EXPECT_TRUE(nl_mode_.isCompatible(s_mode_));
+  EXPECT_TRUE(nl_mode_.isCompatible(six_mode_));
+  EXPECT_TRUE(nl_mode_.isCompatible(x_mode_));
+
+  EXPECT_TRUE(is_mode_.isCompatible(nl_mode_));
+  EXPECT_TRUE(is_mode_.isCompatible(is_mode_));
+  EXPECT_TRUE(is_mode_.isCompatible(ix_mode_));
+  EXPECT_TRUE(is_mode_.isCompatible(s_mode_));
+  EXPECT_TRUE(is_mode_.isCompatible(six_mode_));
+  EXPECT_FALSE(is_mode_.isCompatible(x_mode_));
+
+  EXPECT_TRUE(ix_mode_.isCompatible(nl_mode_));
+  EXPECT_TRUE(ix_mode_.isCompatible(is_mode_));
+  EXPECT_TRUE(ix_mode_.isCompatible(ix_mode_));
+  EXPECT_FALSE(ix_mode_.isCompatible(s_mode_));
+  EXPECT_FALSE(ix_mode_.isCompatible(six_mode_));
+  EXPECT_FALSE(ix_mode_.isCompatible(x_mode_));
+
+  EXPECT_TRUE(s_mode_.isCompatible(nl_mode_));
+  EXPECT_TRUE(s_mode_.isCompatible(is_mode_));
+  EXPECT_FALSE(s_mode_.isCompatible(ix_mode_));
+  EXPECT_TRUE(s_mode_.isCompatible(s_mode_));
+  EXPECT_FALSE(s_mode_.isCompatible(six_mode_));
+  EXPECT_FALSE(s_mode_.isCompatible(x_mode_));
+
+  EXPECT_TRUE(six_mode_.isCompatible(nl_mode_));
+  EXPECT_TRUE(six_mode_.isCompatible(is_mode_));
+  EXPECT_FALSE(six_mode_.isCompatible(ix_mode_));
+  EXPECT_FALSE(six_mode_.isCompatible(s_mode_));
+  EXPECT_FALSE(six_mode_.isCompatible(six_mode_));
+  EXPECT_FALSE(six_mode_.isCompatible(x_mode_));
+
+  EXPECT_TRUE(x_mode_.isCompatible(nl_mode_));
+  EXPECT_FALSE(x_mode_.isCompatible(is_mode_));
+  EXPECT_FALSE(x_mode_.isCompatible(ix_mode_));
+  EXPECT_FALSE(x_mode_.isCompatible(s_mode_));
+  EXPECT_FALSE(x_mode_.isCompatible(six_mode_));
+  EXPECT_FALSE(x_mode_.isCompatible(x_mode_));
+}
+
+TEST_F(AccessModeTest, ModeQueryChecks) {
+  EXPECT_FALSE(nl_mode_.isIntentionShareLock());
+  EXPECT_FALSE(nl_mode_.isIntentionExclusiveLock());
+  EXPECT_FALSE(nl_mode_.isShareLock());
+  EXPECT_FALSE(nl_mode_.isShareAndIntentionExclusiveLock());
+  EXPECT_FALSE(nl_mode_.isExclusiveLock());
+
+  EXPECT_TRUE(is_mode_.isIntentionShareLock());
+  EXPECT_FALSE(is_mode_.isIntentionExclusiveLock());
+  EXPECT_FALSE(is_mode_.isShareLock());
+  EXPECT_FALSE(is_mode_.isShareAndIntentionExclusiveLock());
+  EXPECT_FALSE(is_mode_.isExclusiveLock());
+
+  EXPECT_FALSE(ix_mode_.isIntentionShareLock());
+  EXPECT_TRUE(ix_mode_.isIntentionExclusiveLock());
+  EXPECT_FALSE(ix_mode_.isShareLock());
+  EXPECT_FALSE(ix_mode_.isShareAndIntentionExclusiveLock());
+  EXPECT_FALSE(ix_mode_.isExclusiveLock());
+
+  EXPECT_FALSE(s_mode_.isIntentionShareLock());
+  EXPECT_FALSE(s_mode_.isIntentionExclusiveLock());
+  EXPECT_TRUE(s_mode_.isShareLock());
+  EXPECT_FALSE(s_mode_.isShareAndIntentionExclusiveLock());
+  EXPECT_FALSE(s_mode_.isExclusiveLock());
+
+  EXPECT_FALSE(six_mode_.isIntentionShareLock());
+  EXPECT_FALSE(six_mode_.isIntentionExclusiveLock());
+  EXPECT_FALSE(six_mode_.isShareLock());
+  EXPECT_TRUE(six_mode_.isShareAndIntentionExclusiveLock());
+  EXPECT_FALSE(six_mode_.isExclusiveLock());
+
+  EXPECT_FALSE(x_mode_.isIntentionShareLock());
+  EXPECT_FALSE(x_mode_.isIntentionExclusiveLock());
+  EXPECT_FALSE(x_mode_.isShareLock());
+  EXPECT_FALSE(x_mode_.isShareAndIntentionExclusiveLock());
+  EXPECT_TRUE(x_mode_.isExclusiveLock());
+}
+
+TEST_F(AccessModeTest, Equality) {
+  EXPECT_EQ(nl_mode_, nl_mode_);
+  EXPECT_NE(nl_mode_, is_mode_);
+  EXPECT_NE(nl_mode_, ix_mode_);
+  EXPECT_NE(nl_mode_, s_mode_);
+  EXPECT_NE(nl_mode_, six_mode_);
+  EXPECT_NE(nl_mode_, x_mode_);
+
+  EXPECT_NE(is_mode_, nl_mode_);
+  EXPECT_EQ(is_mode_, is_mode_);
+  EXPECT_NE(is_mode_, ix_mode_);
+  EXPECT_NE(is_mode_, s_mode_);
+  EXPECT_NE(is_mode_, six_mode_);
+  EXPECT_NE(is_mode_, x_mode_);
+
+  EXPECT_NE(ix_mode_, nl_mode_);
+  EXPECT_NE(ix_mode_, is_mode_);
+  EXPECT_EQ(ix_mode_, ix_mode_);
+  EXPECT_NE(ix_mode_, s_mode_);
+  EXPECT_NE(ix_mode_, six_mode_);
+  EXPECT_NE(ix_mode_, x_mode_);
+
+  EXPECT_NE(s_mode_, nl_mode_);
+  EXPECT_NE(s_mode_, is_mode_);
+  EXPECT_NE(s_mode_, ix_mode_);
+  EXPECT_EQ(s_mode_, s_mode_);
+  EXPECT_NE(s_mode_, six_mode_);
+  EXPECT_NE(s_mode_, x_mode_);
+
+  EXPECT_NE(six_mode_, nl_mode_);
+  EXPECT_NE(six_mode_, is_mode_);
+  EXPECT_NE(six_mode_, ix_mode_);
+  EXPECT_NE(six_mode_, s_mode_);
+  EXPECT_EQ(six_mode_, six_mode_);
+  EXPECT_NE(six_mode_, x_mode_);
+
+  EXPECT_NE(x_mode_, nl_mode_);
+  EXPECT_NE(x_mode_, is_mode_);
+  EXPECT_NE(x_mode_, ix_mode_);
+  EXPECT_NE(x_mode_, s_mode_);
+  EXPECT_NE(x_mode_, six_mode_);
+  EXPECT_EQ(x_mode_, x_mode_);
+}
+
+}  // namespace transaction
+}  // namespace quickstep
diff --git a/transaction/tests/DirectedGraph_unittest.cpp b/transaction/tests/DirectedGraph_unittest.cpp
index 2f43642..43ad972 100644
--- a/transaction/tests/DirectedGraph_unittest.cpp
+++ b/transaction/tests/DirectedGraph_unittest.cpp
@@ -24,7 +24,6 @@
 #include "gtest/gtest.h"
 
 namespace quickstep {
-
 namespace transaction {
 
 TEST(DirectedGraphTest, AddNode) {
@@ -127,5 +126,4 @@
 }
 
 }  // namespace transaction
-
 }  // namespace quickstep
diff --git a/transaction/tests/LockRequest_unittest.cpp b/transaction/tests/LockRequest_unittest.cpp
new file mode 100644
index 0000000..0e4138a
--- /dev/null
+++ b/transaction/tests/LockRequest_unittest.cpp
@@ -0,0 +1,49 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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/LockRequest.hpp"
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+namespace transaction {
+
+class LockRequestTest : public ::testing::Test {
+ protected:
+  LockRequestTest()
+      : lock_request_(transaction_id(3),
+                      ResourceId(5),
+                      AccessMode(AccessModeType::kSLock),
+                      RequestType::kAcquireLock) {
+  }
+
+  const LockRequest lock_request_;
+};
+
+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(RequestType::kAcquireLock, lock_request_.getRequestType());
+}
+
+}  // namespace transaction
+}  // namespace quickstep
diff --git a/transaction/tests/LockTable_unittest.cpp b/transaction/tests/LockTable_unittest.cpp
new file mode 100644
index 0000000..577cb79
--- /dev/null
+++ b/transaction/tests/LockTable_unittest.cpp
@@ -0,0 +1,105 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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/LockTable.hpp"
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+namespace transaction {
+
+class LockTableTest : public ::testing::Test {
+ protected:
+  LockTableTest()
+      : tid_1_(1),
+        tid_2_(2),
+        tid_3_(3) {
+  }
+
+  LockTable lock_table_;
+  const transaction_id tid_1_;
+  const transaction_id tid_2_;
+  const transaction_id tid_3_;
+};
+
+TEST_F(LockTableTest, CompatibleRequestsFromDifferentTransactions) {
+  EXPECT_EQ(lock_table_.putLock(tid_1_,
+                                ResourceId(2),
+                                AccessMode(AccessModeType::kIsLock)),
+            LockTableResult::kPLACED_IN_OWNED);
+
+  // Acquire the same lock mode on same resource.
+  EXPECT_EQ(lock_table_.putLock(tid_1_,
+                                ResourceId(2),
+                                AccessMode(AccessModeType::kIsLock)),
+            LockTableResult::kALREADY_IN_OWNED);
+
+  // Another transaction acquires compatible lock on the same resource.
+  EXPECT_EQ(lock_table_.putLock(tid_2_,
+                                ResourceId(2),
+                                AccessMode(AccessModeType::kSLock)),
+            LockTableResult::kPLACED_IN_OWNED);
+}
+
+TEST_F(LockTableTest, IncompatibleRequestsFromDifferentTransactions) {
+  EXPECT_EQ(lock_table_.putLock(tid_1_,
+                                ResourceId(2),
+                                AccessMode(AccessModeType::kIsLock)),
+            LockTableResult::kPLACED_IN_OWNED);
+
+  // Acquire the same lock mode on same resource.
+  EXPECT_EQ(lock_table_.putLock(tid_1_,
+                                ResourceId(2),
+                                AccessMode(AccessModeType::kIsLock)),
+            LockTableResult::kALREADY_IN_OWNED);
+
+  // Another transaction acquires incompatible lock on the same resource.
+  EXPECT_EQ(lock_table_.putLock(tid_2_,
+                                ResourceId(2),
+                                AccessMode(AccessModeType::kXLock)),
+            LockTableResult::kPLACED_IN_PENDING);
+}
+
+TEST_F(LockTableTest, StarvationProtection) {
+  EXPECT_EQ(lock_table_.putLock(tid_1_,
+                                ResourceId(2),
+                                AccessMode(AccessModeType::kIsLock)),
+            LockTableResult::kPLACED_IN_OWNED);
+
+  // Another transaction requests incompatible lock on the same resource.
+  // It should wait for the previous transaction.
+  EXPECT_EQ(lock_table_.putLock(tid_2_,
+                                ResourceId(2),
+                                AccessMode(AccessModeType::kXLock)),
+            LockTableResult::kPLACED_IN_PENDING);
+
+  // 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_,
+                                ResourceId(2),
+                                AccessMode(AccessModeType::kIsLock)),
+            LockTableResult::kPLACED_IN_PENDING);
+}
+
+}  // namespace transaction
+}  // namespace quickstep
diff --git a/transaction/tests/Lock_unittest.cpp b/transaction/tests/Lock_unittest.cpp
new file mode 100644
index 0000000..2ab8b3e
--- /dev/null
+++ b/transaction/tests/Lock_unittest.cpp
@@ -0,0 +1,92 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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/Lock.hpp"
+
+#include <cstddef>
+#include <vector>
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/ResourceId.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+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)}),
+        resource_a_(3, 10, 2, 5),
+        resource_b_(4, 5, 3, 2),
+        locks_on_resource_a_({Lock(resource_a_, modes_[0]),
+                              Lock(resource_a_, modes_[1]),
+                              Lock(resource_a_, modes_[2]),
+                              Lock(resource_a_, modes_[3]),
+                              Lock(resource_a_, modes_[4]),
+                              Lock(resource_a_, modes_[5])}),
+        locks_on_resource_b_({Lock(resource_b_, modes_[0]),
+                             Lock(resource_b_, modes_[1]),
+                             Lock(resource_b_, modes_[2]),
+                             Lock(resource_b_, modes_[3]),
+                             Lock(resource_b_, modes_[4]),
+                             Lock(resource_b_, modes_[5])}) {
+  }
+
+  const std::vector<AccessMode> modes_;
+  const ResourceId resource_a_;
+  const ResourceId resource_b_;
+  const std::vector<Lock> locks_on_resource_a_;
+  const std::vector<Lock> locks_on_resource_b_;
+};
+
+TEST_F(LockTest, LockEquality) {
+  // Locks are equal if they are on the same resource with same
+  // access mode.
+  for (std::size_t i = 0; i < locks_on_resource_a_.size(); ++i) {
+    for (std::size_t j = 0; j < locks_on_resource_a_.size(); ++j) {
+      if (i == j) {
+        EXPECT_EQ(locks_on_resource_a_[i], locks_on_resource_a_[j]);
+      } else {
+        EXPECT_NE(locks_on_resource_a_[i], locks_on_resource_a_[j]);
+      }
+    }
+    // Locks are nver equal if they have different resource ids.
+    EXPECT_NE(locks_on_resource_a_[i], locks_on_resource_b_[i]);
+  }
+}
+
+TEST_F(LockTest, GetResourceId) {
+  for (std::size_t i = 0; i < locks_on_resource_a_.size(); ++i) {
+    EXPECT_EQ(resource_a_, locks_on_resource_a_[i].getResourceId());
+  }
+}
+
+TEST_F(LockTest, GetAccessMode) {
+  for (std::size_t i = 0; i < locks_on_resource_a_.size(); ++i) {
+    EXPECT_EQ(modes_[i], locks_on_resource_a_[i].getAccessMode());
+  }
+}
+
+}  // namespace transaction
+}  // namespace quickstep
diff --git a/transaction/tests/ResourceId_unittest.cpp b/transaction/tests/ResourceId_unittest.cpp
new file mode 100644
index 0000000..f3029f1
--- /dev/null
+++ b/transaction/tests/ResourceId_unittest.cpp
@@ -0,0 +1,113 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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/ResourceId.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+namespace transaction {
+
+class ResourceIdTest : public ::testing::Test {
+ protected:
+  ResourceIdTest()
+      : db1_(1),
+        db2_(2),
+        db2_rel2_(2, 2),
+        db2_rel4_(2, 4),
+        db2_rel2_block2_(2, 2, 2),
+        db2_rel2_block3_(2, 2, 3),
+        db2_rel4_block5_(2, 4, 5),
+        db2_rel4_block7_(2, 4, 7),
+        db2_rel2_block2_tpl4_(2, 2, 2, 4),
+        db2_rel2_block2_tpl7_(2, 2, 2, 7),
+        db2_rel4_block5_tpl3_(2, 4, 5, 7),
+        db2_rel4_block5_tpl8_(2, 4, 5, 8) {
+  }
+
+  const ResourceId db1_;
+  const ResourceId db2_;
+  const ResourceId db2_rel2_;
+  const ResourceId db2_rel4_;
+  const ResourceId db2_rel2_block2_;
+  const ResourceId db2_rel2_block3_;
+  const ResourceId db2_rel4_block5_;
+  const ResourceId db2_rel4_block7_;
+  const ResourceId db2_rel2_block2_tpl4_;
+  const ResourceId db2_rel2_block2_tpl7_;
+  const ResourceId db2_rel4_block5_tpl3_;
+  const ResourceId db2_rel4_block5_tpl8_;
+};
+
+TEST_F(ResourceIdTest, CheckAccessLevels) {
+  EXPECT_TRUE(db2_.isDatabaseAccess());
+  EXPECT_FALSE(db2_rel4_.isDatabaseAccess());
+  EXPECT_FALSE(db2_rel4_block5_.isDatabaseAccess());
+  EXPECT_FALSE(db2_rel4_block5_tpl8_.isDatabaseAccess());
+
+  EXPECT_FALSE(db2_.isRelationAccess());
+  EXPECT_TRUE(db2_rel4_.isRelationAccess());
+  EXPECT_FALSE(db2_rel4_block5_.isRelationAccess());
+  EXPECT_FALSE(db2_rel4_block5_tpl8_.isRelationAccess());
+
+  EXPECT_FALSE(db2_.isBlockAccess());
+  EXPECT_FALSE(db2_rel4_.isBlockAccess());
+  EXPECT_TRUE(db2_rel4_block5_.isBlockAccess());
+  EXPECT_FALSE(db2_rel4_block5_tpl8_.isBlockAccess());
+
+  EXPECT_FALSE(db2_.isTupleAccess());
+  EXPECT_FALSE(db2_rel4_.isTupleAccess());
+  EXPECT_FALSE(db2_rel4_block5_.isTupleAccess());
+  EXPECT_TRUE(db2_rel4_block5_tpl8_.isTupleAccess());
+}
+
+TEST_F(ResourceIdTest, Equality) {
+  // Copy some resource ids.
+  const ResourceId db1_copy(db1_);
+  const ResourceId db2_rel2_copy(db2_rel2_);
+  const ResourceId db2_rel4_block5_copy(db2_rel4_block5_);
+  const ResourceId db2_rel4_block5_tpl8_copy(db2_rel4_block5_tpl8_);
+
+  // Self comparison must be equal.
+  EXPECT_EQ(db1_, db1_);
+  EXPECT_EQ(db2_rel2_, db2_rel2_);
+  EXPECT_EQ(db2_rel4_block5_, db2_rel4_block5_);
+  EXPECT_EQ(db2_rel4_block5_tpl3_, db2_rel4_block5_tpl3_);
+
+  // If resources are different, than it must have different ids.
+  EXPECT_NE(db1_, db2_);
+  EXPECT_NE(db2_rel2_, db2_rel4_);
+  EXPECT_NE(db2_rel4_block5_, db2_rel4_block7_);
+  EXPECT_NE(db2_rel4_block5_tpl3_, db2_rel4_block5_tpl8_);
+
+  // Comparison with a copy must be equal.
+  EXPECT_EQ(db1_copy, db1_);
+  EXPECT_EQ(db2_rel2_copy, db2_rel2_);
+  EXPECT_EQ(db2_rel4_block5_copy, db2_rel4_block5_);
+  EXPECT_EQ(db2_rel4_block5_tpl8_copy, db2_rel4_block5_tpl8_);
+}
+
+TEST_F(ResourceIdTest, ParentResourceIds) {
+  EXPECT_EQ(db2_, db2_rel2_.getParentResourceId());
+  EXPECT_EQ(db2_rel2_, db2_rel2_block2_.getParentResourceId());
+  EXPECT_EQ(db2_rel2_, db2_rel2_block3_.getParentResourceId());
+  EXPECT_EQ(db2_rel2_block2_, db2_rel2_block2_tpl4_.getParentResourceId());
+  EXPECT_EQ(db2_rel2_block2_, db2_rel2_block2_tpl7_.getParentResourceId());
+}
+
+}  // namespace transaction
+}  // namespace quickstep
diff --git a/transaction/tests/StronglyConnectedComponents_unittest.cpp b/transaction/tests/StronglyConnectedComponents_unittest.cpp
index 84851ab..79d6881 100644
--- a/transaction/tests/StronglyConnectedComponents_unittest.cpp
+++ b/transaction/tests/StronglyConnectedComponents_unittest.cpp
@@ -30,7 +30,6 @@
 #include "gtest/gtest.h"
 
 namespace quickstep {
-
 namespace transaction {
 
 class GraphConfiguration {
@@ -225,5 +224,4 @@
 }
 
 }  // namespace transaction
-
 }  // namespace quickstep
diff --git a/transaction/tests/TransactionTable_unittest.cpp b/transaction/tests/TransactionTable_unittest.cpp
new file mode 100644
index 0000000..f5b5bc9
--- /dev/null
+++ b/transaction/tests/TransactionTable_unittest.cpp
@@ -0,0 +1,132 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin—Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   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/TransactionTable.hpp"
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+namespace transaction {
+
+class TransactionTableTest : public ::testing::Test {
+ protected:
+  TransactionTableTest()
+      : tid_1_(1),
+        tid_2_(2),
+        tid_3_(3) {
+    }
+
+  TransactionTable transaction_table_;
+  const transaction_id tid_1_;
+  const transaction_id tid_2_;
+  const transaction_id tid_3_;
+};
+
+TEST_F(TransactionTableTest, NormalOperations) {
+  EXPECT_EQ(transaction_table_.putOwnEntry(tid_1_,
+                                           ResourceId(3),
+                                           AccessMode(AccessModeType::kIsLock)),
+            TransactionTableResult::kPlacedInOwned);
+
+  EXPECT_EQ(transaction_table_.putPendingEntry(tid_1_,
+                                               ResourceId(5),
+                                               AccessMode(AccessModeType::kXLock)),
+            TransactionTableResult::kPlacedInPending);
+}
+
+TEST_F(TransactionTableTest, DeleteEntryOperations) {
+  EXPECT_EQ(transaction_table_.deleteOwnEntry(tid_2_,
+                                              ResourceId(5),
+                                              AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kDelError);
+
+  EXPECT_EQ(transaction_table_.putOwnEntry(tid_2_,
+                                           ResourceId(5),
+                                           AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kPlacedInOwned);
+
+  // 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_,
+                                              ResourceId(5),
+                                              AccessMode(AccessModeType::kXLock)),
+            TransactionTableResult::kDelError);
+
+  // Transaction 3 does not have a lock on this resource id.
+  EXPECT_EQ(transaction_table_.deleteOwnEntry(tid_3_,
+                                              ResourceId(5),
+                                              AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kDelError);
+
+  // 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_,
+                                              ResourceId(5),
+                                              AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kDelFromOwned);
+
+  // Repeat the previous sequence, with pending list.
+  EXPECT_EQ(transaction_table_.deletePendingEntry(tid_2_,
+                                                  ResourceId(5),
+                                                  AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kDelError);
+
+  EXPECT_EQ(transaction_table_.putPendingEntry(tid_2_,
+                                               ResourceId(5),
+                                               AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kPlacedInPending);
+
+  EXPECT_EQ(transaction_table_.deletePendingEntry(tid_2_,
+                                                  ResourceId(5),
+                                                  AccessMode(AccessModeType::kXLock)),
+            TransactionTableResult::kDelError);
+
+  EXPECT_EQ(transaction_table_.deletePendingEntry(tid_3_,
+                                                  ResourceId(5),
+                                                  AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kDelError);
+
+  EXPECT_EQ(transaction_table_.deletePendingEntry(tid_2_,
+                                                  ResourceId(5),
+                                                  AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kDelFromPending);
+}
+
+TEST_F(TransactionTableTest, TransactionEntries) {
+  EXPECT_EQ(transaction_table_.deleteTransaction(tid_1_),
+            TransactionTableResult::kTransactionDeleteError);
+
+  EXPECT_EQ(transaction_table_.putOwnEntry(tid_1_,
+                                           ResourceId(4),
+                                           AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kPlacedInOwned);
+
+  EXPECT_EQ(transaction_table_.deleteTransaction(tid_1_),
+            TransactionTableResult::kTransactionDeleteOk);
+
+  EXPECT_EQ(transaction_table_.deleteOwnEntry(tid_1_,
+                                              ResourceId(4),
+                                              AccessMode(AccessModeType::kSLock)),
+            TransactionTableResult::kDelError);
+}
+
+}  // namespace transaction
+}  // namespace quickstep