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

[49/50] incubator-quickstep git commit: Removes packed row store.

Removes packed row store.


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

Branch: refs/heads/quickstep_partition_parser_support
Commit: 5fee82186da965eb204ec2435dab5b0f7690920e
Parents: c394405
Author: cramja <ma...@gmail.com>
Authored: Thu Nov 17 14:16:50 2016 -0600
Committer: cramja <ma...@gmail.com>
Committed: Sun Jan 8 14:55:21 2017 -0600

----------------------------------------------------------------------
 catalog/tests/Catalog_unittest.cpp              |  26 +-
 query_optimizer/OptimizerTree.hpp               |   4 -
 query_optimizer/resolver/Resolver.cpp           |   5 +-
 .../tests/logical_generator/Create.test         |   4 +-
 query_optimizer/tests/resolver/Create.test      |  56 +-
 relational_operators/CMakeLists.txt             |   2 -
 .../tests/SortMergeRunOperator_unittest.cpp     |   1 -
 .../SortRunGenerationOperator_unittest.cpp      |   1 -
 storage/PackedRowStoreTupleStorageSubBlock.cpp  | 491 ----------------
 storage/PackedRowStoreTupleStorageSubBlock.hpp  | 216 -------
 storage/PackedRowStoreValueAccessor.hpp         | 150 -----
 storage/SplitRowStoreTupleStorageSubBlock.cpp   |   4 +-
 storage/SplitRowStoreTupleStorageSubBlock.hpp   |   2 +-
 storage/StorageBlock.cpp                        |   7 -
 storage/StorageBlockInfo.cpp                    |   3 +-
 storage/StorageBlockInfo.hpp                    |   1 -
 storage/StorageBlockLayout.hpp                  |   4 +-
 storage/StorageBlockLayout.proto                |   9 +-
 storage/SubBlockTypeRegistry.hpp                |   4 +-
 storage/ValueAccessor.hpp                       |   1 -
 storage/ValueAccessorUtil.hpp                   |  11 -
 ...ColumnStoreTupleStorageSubBlock_unittest.cpp |   2 +-
 ...kedRowStoreTupleStorageSubBlock_unittest.cpp |   2 +-
 ...kedRowStoreTupleStorageSubBlock_unittest.cpp | 584 -------------------
 24 files changed, 44 insertions(+), 1546 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/catalog/tests/Catalog_unittest.cpp
----------------------------------------------------------------------
diff --git a/catalog/tests/Catalog_unittest.cpp b/catalog/tests/Catalog_unittest.cpp
index e430b89..276bfa9 100644
--- a/catalog/tests/Catalog_unittest.cpp
+++ b/catalog/tests/Catalog_unittest.cpp
@@ -52,7 +52,6 @@ QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(BasicColumnStoreTupleStorageSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(CSBTreeIndexSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(CompressedColumnStoreTupleStorageSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(CompressedPackedRowStoreTupleStorageSubBlock);
-QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(PackedRowStoreTupleStorageSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(SMAIndexSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(SplitRowStoreTupleStorageSubBlock);
 
@@ -79,8 +78,6 @@ class CatalogTest : public ::testing::Test {
     EXPECT_EQ(expected.sub_block_type(), checked.sub_block_type());
 
     switch (expected.sub_block_type()) {
-      case TupleStorageSubBlockDescription::PACKED_ROW_STORE:
-        break;
       case TupleStorageSubBlockDescription::BASIC_COLUMN_STORE:
         EXPECT_TRUE(expected.HasExtension(BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id));
         EXPECT_TRUE(checked.HasExtension(BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id));
@@ -436,27 +433,6 @@ TEST_F(CatalogTest, DroppedCatalogRelationSerializationTest) {
   checkCatalogSerialization();
 }
 
-TEST_F(CatalogTest, CatalogPackedRowStoreSerializationTest) {
-  CatalogRelation* const rel = createCatalogRelation("rel");
-  StorageBlockLayoutDescription layout_description(rel->getDefaultStorageBlockLayout().getDescription());
-
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_int", TypeFactory::GetType(kInt)));
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_long", TypeFactory::GetType(kLong)));
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_float", TypeFactory::GetType(kFloat)));
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_double", TypeFactory::GetType(kDouble)));
-
-  const std::size_t str_type_length = 20;
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_char", TypeFactory::GetType(kChar, str_type_length)));
-  // NOTE(zuyu): PackedRowStoreTupleStorageSubBlock does NOT support variable-length attributes.
-
-  layout_description.mutable_tuple_store_description()->set_sub_block_type(
-      TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-
-  rel->setDefaultStorageBlockLayout(new StorageBlockLayout(*rel, layout_description));
-
-  checkCatalogSerialization();
-}
-
 TEST_F(CatalogTest, CatalogBasicColumnStoreSerializationTest) {
   CatalogRelation* const rel = createCatalogRelation("rel");
   StorageBlockLayoutDescription layout_description(rel->getDefaultStorageBlockLayout().getDescription());
@@ -569,7 +545,7 @@ TEST_F(CatalogTest, CatalogIndexTest) {
   rel->addAttribute(new CatalogAttribute(nullptr, "attr_idx2", TypeFactory::GetType(kInt)));
 
   layout_description.mutable_tuple_store_description()->set_sub_block_type(
-      TupleStorageSubBlockDescription::PACKED_ROW_STORE);
+      TupleStorageSubBlockDescription::SPLIT_ROW_STORE);
 
   rel->setDefaultStorageBlockLayout(new StorageBlockLayout(*rel, layout_description));
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/query_optimizer/OptimizerTree.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/OptimizerTree.hpp b/query_optimizer/OptimizerTree.hpp
index 8ea2c6f..62df66d 100644
--- a/query_optimizer/OptimizerTree.hpp
+++ b/query_optimizer/OptimizerTree.hpp
@@ -233,10 +233,6 @@ OptimizerProtoRepresentation<TreeNodeType>* getOptimizerRepresentationForProto(
   const ::quickstep::TupleStorageSubBlockDescription &storage_block_description
       = description->tuple_store_description();
   switch (storage_block_description.sub_block_type()) {
-    case TupleStorageSubBlockDescription::PACKED_ROW_STORE: {
-      node->addProperty("blocktype", "rowstore");
-      break;
-    }
     case TupleStorageSubBlockDescription::SPLIT_ROW_STORE: {
       node->addProperty("blocktype", "split_rowstore");
       break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 38ec24a..2580342 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -520,10 +520,7 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
         << "TYPE property must be specified and be a string.";
   }
   const std::string type_string = ToLower(type_parse_string->value());
-  if (type_string.compare("rowstore") == 0) {
-    description->set_sub_block_type(
-        quickstep::TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-  } else if (type_string.compare("split_rowstore") == 0) {
+  if (type_string.compare("split_rowstore") == 0) {
     description->set_sub_block_type(
         quickstep::TupleStorageSubBlockDescription::SPLIT_ROW_STORE);
   } else if (type_string.compare("columnstore") == 0) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/query_optimizer/tests/logical_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Create.test b/query_optimizer/tests/logical_generator/Create.test
index a35a92a..aeff9ec 100644
--- a/query_optimizer/tests/logical_generator/Create.test
+++ b/query_optimizer/tests/logical_generator/Create.test
@@ -16,12 +16,12 @@
 # under the License.
 
 [default optimized_logical_plan]
-CREATE TABLE foo (attr int) WITH BLOCKPROPERTIES (TYPE rowstore, BLOCKSIZEMB 10)
+CREATE TABLE foo (attr int) WITH BLOCKPROPERTIES (TYPE split_rowstore, BLOCKSIZEMB 10)
 --
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
 | +-block_properties=ProtoDescription
-| | +-Property=ProtoProperty[Property=blocktype,Value=rowstore]
+| | +-Property=ProtoProperty[Property=blocktype,Value=split_rowstore]
 | | +-Property=ProtoProperty[Property=slots,Value=5]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/query_optimizer/tests/resolver/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Create.test b/query_optimizer/tests/resolver/Create.test
index 7cd980f..b04d785 100644
--- a/query_optimizer/tests/resolver/Create.test
+++ b/query_optimizer/tests/resolver/Create.test
@@ -90,20 +90,20 @@ create table "_qstemp_result_s" (col int)
 
 # Cannot repeat Block properties.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, TYPE columnstore);
+(TYPE columnstore, TYPE columnstore);
 --
-ERROR: Properties must be specified at most once. (2 : 17)
-(TYPE rowstore, TYPE columnstore);
-                ^
+ERROR: Properties must be specified at most once. (2 : 20)
+(TYPE columnstore, TYPE columnstore);
+                   ^
 ==
 
 # Unrecognized BLOCKPROPERTIES should throw.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, INVALID_PROP val);
+(TYPE split_rowstore, INVALID_PROP val);
 --
-ERROR: Unrecognized property name. (2 : 17)
-(TYPE rowstore, INVALID_PROP val);
-                ^
+ERROR: Unrecognized property name. (2 : 23)
+(TYPE split_rowstore, INVALID_PROP val);
+                      ^
 ==
 
 # TYPE property only accepts names of storage blocks.
@@ -126,11 +126,11 @@ BLOCKPROPERTIES (BLOCKSIZEMB 1...
 
 # Rowstores cannot have a sorted attribute.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, SORT attr);
+(TYPE split_rowstore, SORT attr);
 --
-ERROR: The SORT property does not apply to this block type. (2 : 22)
-(TYPE rowstore, SORT attr);
-                     ^
+ERROR: The SORT property does not apply to this block type. (2 : 28)
+(TYPE split_rowstore, SORT attr);
+                           ^
 ==
 
 # Columnstores require a sort attribute.
@@ -197,35 +197,35 @@ ERROR: The given attribute was not found. (2 : 44)
 
 # BLOCKSIZEMB property must be an integer.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, BLOCKSIZEMB ten);
+(TYPE split_rowstore, BLOCKSIZEMB ten);
 --
-ERROR: The BLOCKSIZEMB property must be an integer. (2 : 17)
-(TYPE rowstore, BLOCKSIZEMB ten);
-                ^
+ERROR: The BLOCKSIZEMB property must be an integer. (2 : 23)
+(TYPE split_rowstore, BLOCKSIZEMB ten);
+                      ^
 ==
 
 # BLOCKSIZEMB property must be multiple times of the slot size.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, BLOCKSIZEMB 25);
+(TYPE split_rowstore, BLOCKSIZEMB 25);
 --
-ERROR: The BLOCKSIZEMB property must be multiple times of 2MB. (2 : 17)
-(TYPE rowstore, BLOCKSIZEMB 25);
-                ^
+ERROR: The BLOCKSIZEMB property must be multiple times of 2MB. (2 : 23)
+(TYPE split_rowstore, BLOCKSIZEMB 25);
+                      ^
 ==
 
 # BLOCKSIZEMB must be greater than the minimum (defined in StorageConstants.hpp).
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, BLOCKSIZEMB 0);
+(TYPE split_rowstore, BLOCKSIZEMB 0);
 --
-ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 17)
-(TYPE rowstore, BLOCKSIZEMB 0);
-                ^
+ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 23)
+(TYPE split_rowstore, BLOCKSIZEMB 0);
+                      ^
 ==
 
 # BLOCKSIZEMB must be less than the maximum (defined in StorageConstants.hpp).
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, BLOCKSIZEMB 2000);
+(TYPE split_rowstore, BLOCKSIZEMB 2000);
 --
-ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 17)
-(TYPE rowstore, BLOCKSIZEMB 2000);
-                ^
+ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 23)
+(TYPE split_rowstore, BLOCKSIZEMB 2000);
+                      ^

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 9e4b1b6..526820e 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -674,7 +674,6 @@ target_link_libraries(SortMergeRunOperator_unittest
                       quickstep_storage_CountedReference
                       quickstep_storage_InsertDestination
                       quickstep_storage_InsertDestination_proto
-                      quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_storage_SplitRowStoreValueAccessor
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -723,7 +722,6 @@ target_link_libraries(SortRunGenerationOperator_unittest
                       quickstep_storage_CompressedPackedRowStoreValueAccessor
                       quickstep_storage_CountedReference
                       quickstep_storage_InsertDestination
-                      quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_storage_SplitRowStoreValueAccessor
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index b37d81f..a9128b1 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -51,7 +51,6 @@
 #include "storage/CountedReference.hpp"
 #include "storage/InsertDestination.hpp"
 #include "storage/InsertDestination.pb.h"
-#include "storage/PackedRowStoreValueAccessor.hpp"
 #include "storage/SplitRowStoreValueAccessor.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index d09ff07..99fafa8 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -47,7 +47,6 @@
 #include "storage/CountedReference.hpp"
 #include "storage/InsertDestination.hpp"
 #include "storage/InsertDestination.pb.h"
-#include "storage/PackedRowStoreValueAccessor.hpp"
 #include "storage/SplitRowStoreValueAccessor.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/PackedRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreTupleStorageSubBlock.cpp b/storage/PackedRowStoreTupleStorageSubBlock.cpp
deleted file mode 100644
index 0ad4a4c..0000000
--- a/storage/PackedRowStoreTupleStorageSubBlock.cpp
+++ /dev/null
@@ -1,491 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#include "storage/PackedRowStoreTupleStorageSubBlock.hpp"
-
-#include <cstddef>
-#include <cstring>
-#include <vector>
-
-#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"
-#include "storage/StorageErrors.hpp"
-#include "storage/SubBlockTypeRegistry.hpp"
-#include "storage/TupleIdSequence.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#include "types/Type.hpp"
-#include "types/TypedValue.hpp"
-#include "types/containers/Tuple.hpp"
-#include "utility/BitVector.hpp"
-#include "utility/Macros.hpp"
-
-using std::vector;
-using std::memcpy;
-using std::size_t;
-
-namespace quickstep {
-
-QUICKSTEP_REGISTER_TUPLE_STORE(PackedRowStoreTupleStorageSubBlock, PACKED_ROW_STORE);
-
-PackedRowStoreTupleStorageSubBlock::PackedRowStoreTupleStorageSubBlock(
-    const CatalogRelationSchema &relation,
-    const TupleStorageSubBlockDescription &description,
-    const bool new_block,
-    void *sub_block_memory,
-    const std::size_t sub_block_memory_size)
-    : TupleStorageSubBlock(relation,
-                           description,
-                           new_block,
-                           sub_block_memory,
-                           sub_block_memory_size),
-      header_(static_cast<PackedRowStoreHeader*>(sub_block_memory)),
-      null_bitmap_bytes_(0) {
-  if (!DescriptionIsValid(relation_, description_)) {
-    FATAL_ERROR("Attempted to construct a PackedRowStoreTupleStorageSubBlock from an invalid description.");
-  }
-
-  if (sub_block_memory_size < sizeof(PackedRowStoreHeader)) {
-    throw BlockMemoryTooSmall("PackedRowStoreTupleStorageSubBlock", sub_block_memory_size);
-  }
-
-  if (relation_.hasNullableAttributes()) {
-    // Compute on the order of bits to account for bits in null_bitmap_.
-    tuple_id row_capacity = ((sub_block_memory_size_ - sizeof(PackedRowStoreHeader)) << 3)
-                            / ((relation.getFixedByteLength() << 3) + relation.numNullableAttributes());
-    null_bitmap_bytes_ = BitVector<false>::BytesNeeded(row_capacity * relation.numNullableAttributes());
-
-    if (sub_block_memory_size < sizeof(PackedRowStoreHeader) + null_bitmap_bytes_) {
-      if (relation_.getFixedByteLength() == 0) {
-        // Special case: relation consists entirely of NullType attributes.
-        row_capacity = BitVector<false>::MaxCapacityForBytes(
-                           sub_block_memory_size - sizeof(PackedRowStoreHeader))
-                       / relation.numNullableAttributes();
-        null_bitmap_bytes_ = sub_block_memory_size - sizeof(PackedRowStoreHeader);
-      } else {
-        throw BlockMemoryTooSmall("PackedRowStoreTupleStorageSubBlock", sub_block_memory_size);
-      }
-    }
-
-    null_bitmap_.reset(new BitVector<false>(static_cast<char*>(sub_block_memory_)
-                                                + sizeof(PackedRowStoreHeader),
-                                            row_capacity * relation.numNullableAttributes()));
-    tuple_storage_ = static_cast<char*>(sub_block_memory_)
-                         + sizeof(PackedRowStoreHeader)
-                         + null_bitmap_bytes_;
-  } else {
-    tuple_storage_ = static_cast<char*>(sub_block_memory_)
-                         + sizeof(PackedRowStoreHeader);
-  }
-
-  if (new_block) {
-    header_->num_tuples = 0;
-    if (relation_.hasNullableAttributes()) {
-      null_bitmap_->clear();
-    }
-  }
-}
-
-bool PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(
-    const CatalogRelationSchema &relation,
-    const TupleStorageSubBlockDescription &description) {
-  // Make sure description is initialized and specifies PackedRowStore.
-  if (!description.IsInitialized()) {
-    return false;
-  }
-  if (description.sub_block_type() != TupleStorageSubBlockDescription::PACKED_ROW_STORE) {
-    return false;
-  }
-
-  // Make sure relation is not variable-length.
-  if (relation.isVariableLength()) {
-    return false;
-  }
-
-  return true;
-}
-
-std::size_t PackedRowStoreTupleStorageSubBlock::EstimateBytesPerTuple(
-    const CatalogRelationSchema &relation,
-    const TupleStorageSubBlockDescription &description) {
-  DEBUG_ASSERT(DescriptionIsValid(relation, description));
-
-  // NOTE(chasseur): We round-up the number of bytes needed in the NULL bitmap
-  // to avoid estimating 0 bytes needed for a relation with less than 8
-  // attributes which are all NullType.
-  return relation.getFixedByteLength()
-         + ((relation.numNullableAttributes() + 7) >> 3);
-}
-
-tuple_id PackedRowStoreTupleStorageSubBlock::bulkInsertTuples(ValueAccessor *accessor) {
-  const tuple_id original_num_tuples = header_->num_tuples;
-  char *dest_addr = static_cast<char*>(tuple_storage_)
-                      + header_->num_tuples * relation_.getFixedByteLength();
-  const unsigned num_nullable_attrs = relation_.numNullableAttributes();
-
-  InvokeOnAnyValueAccessor(
-      accessor,
-      [this, &dest_addr, &num_nullable_attrs](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()) {
-        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>(curr_attr);
-            if (attr_value == nullptr) {
-              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>(curr_attr),
-                   attr_size);
-          }
-          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()) {
-        if (fast_copy) {
-          memcpy(dest_addr,
-                 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);
-      }
-    }
-  });
-
-  return header_->num_tuples - original_num_tuples;
-}
-
-tuple_id PackedRowStoreTupleStorageSubBlock::bulkInsertTuplesWithRemappedAttributes(
-    const std::vector<attribute_id> &attribute_map,
-    ValueAccessor *accessor) {
-  DEBUG_ASSERT(attribute_map.size() == relation_.size());
-
-  const tuple_id original_num_tuples = header_->num_tuples;
-  char *dest_addr = static_cast<char*>(tuple_storage_)
-                      + header_->num_tuples * relation_.getFixedByteLength();
-  const unsigned num_nullable_attrs = relation_.numNullableAttributes();
-
-  InvokeOnAnyValueAccessor(
-      accessor,
-      [this, &num_nullable_attrs, &attribute_map, &dest_addr](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()) {
-        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[curr_attr]);
-            if (attr_value == nullptr) {
-              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[curr_attr]),
-                   attr_size);
-          }
-          dest_addr += attr_size;
-        }
-        ++(header_->num_tuples);
-      }
-    } else {
-      while (this->hasSpaceToInsert<false>(1) && accessor->next()) {
-        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[curr_attr]),
-                 attr_size);
-          dest_addr += attr_size;
-        }
-        ++(header_->num_tuples);
-      }
-    }
-  });
-
-  return header_->num_tuples - original_num_tuples;
-}
-
-const void* PackedRowStoreTupleStorageSubBlock::getAttributeValue(
-    const tuple_id tuple,
-    const attribute_id attr) const {
-  DEBUG_ASSERT(hasTupleWithID(tuple));
-  DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
-
-  const int nullable_idx = relation_.getNullableAttributeIndex(attr);
-  if ((nullable_idx != -1)
-      && null_bitmap_->getBit(tuple * relation_.numNullableAttributes() + nullable_idx)) {
-    return nullptr;
-  }
-
-  return static_cast<char*>(tuple_storage_)                // Start of actual tuple storage.
-         + (tuple * relation_.getFixedByteLength())        // Tuples prior to 'tuple'.
-         + relation_.getFixedLengthAttributeOffset(attr);  // Attribute offset within tuple.
-}
-
-TypedValue PackedRowStoreTupleStorageSubBlock::getAttributeValueTyped(
-    const tuple_id tuple,
-    const attribute_id attr) const {
-  const Type &attr_type = relation_.getAttributeById(attr)->getType();
-  const void *untyped_value = getAttributeValue(tuple, attr);
-  return (untyped_value == nullptr)
-      ? attr_type.makeNullValue()
-      : attr_type.makeValue(untyped_value, attr_type.maximumByteLength());
-}
-
-ValueAccessor* PackedRowStoreTupleStorageSubBlock::createValueAccessor(
-    const TupleIdSequence *sequence) const {
-  PackedRowStoreValueAccessor *base_accessor
-      = new PackedRowStoreValueAccessor(relation_,
-                                        relation_,
-                                        header_->num_tuples,
-                                        tuple_storage_,
-                                        null_bitmap_.get());
-  if (sequence == nullptr) {
-    return base_accessor;
-  } else {
-    return new TupleIdSequenceAdapterValueAccessor<PackedRowStoreValueAccessor>(
-        base_accessor,
-        *sequence);
-  }
-}
-
-void PackedRowStoreTupleStorageSubBlock::setAttributeValueInPlaceTyped(const tuple_id tuple,
-                                                                       const attribute_id attr,
-                                                                       const TypedValue &value) {
-  DEBUG_ASSERT(hasTupleWithID(tuple));
-  DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
-  DEBUG_ASSERT(value.isPlausibleInstanceOf(relation_.getAttributeById(attr)->getType().getSignature()));
-
-  const int nullable_idx = relation_.getNullableAttributeIndex(attr);
-  if (nullable_idx != -1) {
-    if (value.isNull()) {
-      null_bitmap_->setBit(tuple * relation_.numNullableAttributes() + nullable_idx, true);
-      return;
-    } else {
-      null_bitmap_->setBit(tuple * relation_.numNullableAttributes() + nullable_idx, false);
-    }
-  }
-
-  char *base_addr = static_cast<char*>(tuple_storage_)                // Start of actual tuple storage.
-                    + (tuple * relation_.getFixedByteLength())        // Tuples prior to 'tuple'.
-                    + relation_.getFixedLengthAttributeOffset(attr);  // Attribute offset within tuple.
-
-  value.copyInto(base_addr);
-}
-
-bool PackedRowStoreTupleStorageSubBlock::deleteTuple(const tuple_id tuple) {
-  DEBUG_ASSERT(hasTupleWithID(tuple));
-
-  if (tuple == header_->num_tuples - 1) {
-    // If deleting the last tuple, simply truncate.
-    --(header_->num_tuples);
-    if (null_bitmap_.get() != nullptr) {
-      null_bitmap_->setBitRange(tuple * relation_.numNullableAttributes(),
-                                relation_.numNullableAttributes(),
-                                false);
-    }
-    return false;
-  } else {
-    const size_t tuple_length = relation_.getFixedByteLength();
-
-    char *dest_addr = static_cast<char*>(tuple_storage_)  // Start of actual tuple storage.
-                      + (tuple * tuple_length);           // Prior tuples.
-    char *src_addr = dest_addr + tuple_length;  // Start of subsequent tuples.
-    const size_t copy_bytes = (header_->num_tuples - tuple - 1) * tuple_length;  // Bytes in subsequent tuples.
-    memmove(dest_addr, src_addr, copy_bytes);
-
-    if (null_bitmap_.get() != nullptr) {
-      null_bitmap_->shiftTailForward(tuple * relation_.numNullableAttributes(),
-                                     relation_.numNullableAttributes());
-    }
-
-    --(header_->num_tuples);
-
-    return true;
-  }
-}
-
-bool PackedRowStoreTupleStorageSubBlock::bulkDeleteTuples(TupleIdSequence *tuples) {
-  if (tuples->empty()) {
-    // Nothing to do.
-    return false;
-  }
-
-  const tuple_id front = tuples->front();
-  const tuple_id back = tuples->back();
-  const tuple_id num_tuples = tuples->numTuples();
-
-  if ((back == header_->num_tuples - 1)
-       && (back - front == num_tuples - 1)) {
-    // Just truncate the back.
-    header_->num_tuples = front;
-    if (null_bitmap_.get() != nullptr) {
-      null_bitmap_->setBitRange(header_->num_tuples * relation_.numNullableAttributes(),
-                                num_tuples * relation_.numNullableAttributes(),
-                                false);
-    }
-    return false;
-  }
-
-  // Pack the non-deleted tuples.
-  const size_t tuple_length = relation_.getFixedByteLength();
-  tuple_id dest_tid = front;
-  tuple_id src_tid = dest_tid;
-
-  TupleIdSequence::const_iterator it = tuples->begin();
-  for (tuple_id current_id = front;
-       current_id < header_->num_tuples;
-       ++current_id, ++src_tid) {
-    if (current_id == *it) {
-      // Don't copy a deleted tuple.
-
-      if (null_bitmap_.get() != nullptr) {
-        // Erase the deleted tuple's entries in the null bitmap.
-        null_bitmap_->shiftTailForward(dest_tid * relation_.numNullableAttributes(),
-                                       relation_.numNullableAttributes());
-      }
-
-      ++it;
-      if (it == tuples->end()) {
-        // No more to delete, so copy all the remaining tuples in one go.
-        memmove(static_cast<char*>(tuple_storage_) + dest_tid * tuple_length,
-                static_cast<char*>(tuple_storage_) + (src_tid + 1) * tuple_length,
-                (header_->num_tuples - back - 1) * tuple_length);
-        break;
-      }
-    } else {
-      // Copy the next tuple to the packed region.
-      memmove(static_cast<char*>(tuple_storage_) + dest_tid * tuple_length,
-              static_cast<char*>(tuple_storage_) + src_tid * tuple_length,
-              tuple_length);
-      ++dest_tid;
-    }
-  }
-
-  header_->num_tuples -= static_cast<tuple_id>(num_tuples);
-
-  return true;
-}
-
-template <bool nullable_attrs>
-bool PackedRowStoreTupleStorageSubBlock::hasSpaceToInsert(const tuple_id num_tuples) const {
-  if (sizeof(PackedRowStoreHeader)
-          + null_bitmap_bytes_
-          + (header_->num_tuples + num_tuples) * relation_.getFixedByteLength()
-      <= sub_block_memory_size_) {
-    if (nullable_attrs) {
-      return static_cast<std::size_t>(header_->num_tuples + num_tuples) < null_bitmap_->size();
-    } else {
-      return true;
-    }
-  } else {
-    return false;
-  }
-}
-
-// Make sure both versions get compiled in.
-template bool PackedRowStoreTupleStorageSubBlock::hasSpaceToInsert<false>(
-    const tuple_id num_tuples) const;
-template bool PackedRowStoreTupleStorageSubBlock::hasSpaceToInsert<true>(
-    const tuple_id num_tuples) const;
-
-template <bool nullable_attrs>
-TupleStorageSubBlock::InsertResult PackedRowStoreTupleStorageSubBlock::insertTupleImpl(
-    const Tuple &tuple) {
-#ifdef QUICKSTEP_DEBUG
-  paranoidInsertTypeCheck(tuple);
-#endif
-  if (!hasSpaceToInsert<nullable_attrs>(1)) {
-    return InsertResult(-1, false);
-  }
-
-  char *base_addr = static_cast<char*>(tuple_storage_)                       // Start of actual tuple-storage region.
-                    + header_->num_tuples * relation_.getFixedByteLength();  // Existing tuples.
-
-  Tuple::const_iterator value_it = tuple.begin();
-  CatalogRelationSchema::const_iterator attr_it = relation_.begin();
-
-  while (value_it != tuple.end()) {
-    if (nullable_attrs) {
-      const int nullable_idx = relation_.getNullableAttributeIndex(attr_it->getID());
-      if ((nullable_idx != -1) && value_it->isNull()) {
-        null_bitmap_->setBit(header_->num_tuples * relation_.numNullableAttributes()
-                                 + nullable_idx,
-                             true);
-      } else {
-        value_it->copyInto(base_addr);
-      }
-    } else {
-      value_it->copyInto(base_addr);
-    }
-
-    base_addr += attr_it->getType().maximumByteLength();
-
-    ++value_it;
-    ++attr_it;
-  }
-
-  ++(header_->num_tuples);
-
-  return InsertResult(header_->num_tuples - 1, false);
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/PackedRowStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreTupleStorageSubBlock.hpp b/storage/PackedRowStoreTupleStorageSubBlock.hpp
deleted file mode 100644
index 0cd41f4..0000000
--- a/storage/PackedRowStoreTupleStorageSubBlock.hpp
+++ /dev/null
@@ -1,216 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#ifndef QUICKSTEP_STORAGE_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_
-#define QUICKSTEP_STORAGE_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_
-
-#include <memory>
-#include <unordered_map>
-#include <vector>
-
-#include "expressions/predicate/PredicateCost.hpp"
-#include "storage/SubBlockTypeRegistryMacros.hpp"
-#include "storage/TupleStorageSubBlock.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/BitVector.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class CatalogRelationSchema;
-class ComparisonPredicate;
-class TupleStorageSubBlockDescription;
-class ValueAccessor;
-
-QUICKSTEP_DECLARE_SUB_BLOCK_TYPE_REGISTERED(PackedRowStoreTupleStorageSubBlock);
-
-/** \addtogroup Storage
- *  @{
- */
-
-/**
- * @brief An implementation of TupleStorageSubBlock as a packed row-store (i.e.
- *        an array of fixed-length values with no holes).
- * @warning This implementation does NOT support variable-length attributes. It
- *          is an error to attempt to construct a
- *          PackedRowStoreTupleStorageSubBlock for a relation with any
- *          variable-length attributes.
- **/
-class PackedRowStoreTupleStorageSubBlock: public TupleStorageSubBlock {
- public:
-  PackedRowStoreTupleStorageSubBlock(const CatalogRelationSchema &relation,
-                                     const TupleStorageSubBlockDescription &description,
-                                     const bool new_block,
-                                     void *sub_block_memory,
-                                     const std::size_t sub_block_memory_size);
-
-  ~PackedRowStoreTupleStorageSubBlock() override {
-  }
-
-  /**
-   * @brief Determine whether a TupleStorageSubBlockDescription is valid for
-   *        this type of TupleStorageSubBlock.
-   *
-   * @param relation The relation a tuple store described by description would
-   *        belong to.
-   * @param description A description of the parameters for this type of
-   *        TupleStorageSubBlock, which will be checked for validity.
-   * @return Whether description is well-formed and valid for this type of
-   *         TupleStorageSubBlock belonging to relation (i.e. whether a
-   *         TupleStorageSubBlock of this type, belonging to relation, can be
-   *         constructed according to description).
-   **/
-  static bool DescriptionIsValid(const CatalogRelationSchema &relation,
-                                 const TupleStorageSubBlockDescription &description);
-
-  /**
-   * @brief Estimate the average number of bytes (including any applicable
-   *        overhead) used to store a single tuple in this type of
-   *        TupleStorageSubBlock. Used by StorageBlockLayout::finalize() to
-   *        divide block memory amongst sub-blocks.
-   * @warning description must be valid. DescriptionIsValid() should be called
-   *          first if necessary.
-   *
-   * @param relation The relation tuples belong to.
-   * @param description A description of the parameters for this type of
-   *        TupleStorageSubBlock.
-   * @return The average/ammortized number of bytes used to store a single
-   *         tuple of relation in a TupleStorageSubBlock of this type described
-   *         by description.
-   **/
-  static std::size_t EstimateBytesPerTuple(const CatalogRelationSchema &relation,
-                                           const TupleStorageSubBlockDescription &description);
-
-  bool supportsUntypedGetAttributeValue(const attribute_id attr) const override {
-    return true;
-  }
-
-  bool supportsAdHocInsert() const override {
-    return true;
-  }
-
-  bool adHocInsertIsEfficient() const override {
-    return true;
-  }
-
-  TupleStorageSubBlockType getTupleStorageSubBlockType() const override {
-    return kPackedRowStore;
-  }
-
-  bool isEmpty() const override {
-    return (header_->num_tuples == 0);
-  }
-
-  bool isPacked() const override {
-    return true;
-  }
-
-  tuple_id getMaxTupleID() const override {
-    return header_->num_tuples - 1;
-  }
-
-  bool hasTupleWithID(const tuple_id tuple) const override {
-    return ((tuple >=0) && (tuple < header_->num_tuples));
-  }
-
-  InsertResult insertTuple(const Tuple &tuple) override {
-    if (null_bitmap_.get() == nullptr) {
-      return insertTupleImpl<false>(tuple);
-    } else {
-      return insertTupleImpl<true>(tuple);
-    }
-  }
-
-  inline bool insertTupleInBatch(const Tuple &tuple) override {
-    const InsertResult result = insertTuple(tuple);
-    return (result.inserted_id >= 0);
-  }
-
-  tuple_id bulkInsertTuples(ValueAccessor *accessor) override;
-
-  tuple_id bulkInsertTuplesWithRemappedAttributes(
-      const std::vector<attribute_id> &attribute_map,
-      ValueAccessor *accessor) override;
-
-  const void* getAttributeValue(const tuple_id tuple,
-                                const attribute_id attr) const override;
-
-  TypedValue getAttributeValueTyped(const tuple_id tuple,
-                                    const attribute_id attr) const override;
-
-  ValueAccessor* createValueAccessor(
-      const TupleIdSequence *sequence = nullptr) const override;
-
-  bool canSetAttributeValuesInPlaceTyped(
-      const tuple_id tuple,
-      const std::unordered_map<attribute_id, TypedValue> &new_values) const override {
-    return true;
-  }
-
-  void setAttributeValueInPlaceTyped(const tuple_id tuple,
-                                     const attribute_id attr,
-                                     const TypedValue &value) override;
-
-  bool deleteTuple(const tuple_id tuple) override;
-  bool bulkDeleteTuples(TupleIdSequence *tuples) override;
-
-  predicate_cost_t estimatePredicateEvaluationCost(
-      const ComparisonPredicate &predicate) const override {
-    return predicate_cost::kRowScan;
-  }
-
-  void rebuild() override {
-  }
-
-  bool isInsertOrderPreserving() const override {
-    return true;
-  }
-
- private:
-  struct PackedRowStoreHeader {
-    tuple_id num_tuples;
-  };
-
-  // If 'nullable_attrs' is true, extra branches are enabled to deal with NULLs
-  // and set bits in '*null_bitmap_' appropriately. If no attributes of the
-  // relation are nullable, then 'nullable_attrs' is false and this function
-  // has a simpler and faster implementation with no NULL-checking.
-  template <bool nullable_attrs>
-  InsertResult insertTupleImpl(const Tuple &tuple);
-
-  // Similar to insertTupleImpl(), the 'nullable_attrs' template parameter
-  // distinguishes between a version of this function that includes a check for
-  // space in '*null_bitmap_' and a simpler version that skips the check for
-  // relations that have no nullable attributes (and therefore no NULL-bitmap).
-  template <bool nullable_attrs>
-  bool hasSpaceToInsert(const tuple_id num_tuples) const;
-
-  PackedRowStoreHeader *header_;
-  std::unique_ptr<BitVector<false>> null_bitmap_;
-  std::size_t null_bitmap_bytes_;
-  void *tuple_storage_;
-
-  DISALLOW_COPY_AND_ASSIGN(PackedRowStoreTupleStorageSubBlock);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_STORAGE_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/PackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreValueAccessor.hpp b/storage/PackedRowStoreValueAccessor.hpp
deleted file mode 100644
index 9d43955..0000000
--- a/storage/PackedRowStoreValueAccessor.hpp
+++ /dev/null
@@ -1,150 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#ifndef QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
-#define QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
-
-#include "catalog/CatalogRelationSchema.hpp"
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "types/Type.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/BitVector.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class PackedRowStoreTupleStorageSubBlock;
-
-class PackedRowStoreValueAccessorHelper {
- public:
-  PackedRowStoreValueAccessorHelper(const CatalogRelationSchema &relation,
-                                    const tuple_id num_tuples,
-                                    const void *tuple_storage,
-                                    const BitVector<false> *null_bitmap)
-      : relation_(relation),
-        num_tuples_(num_tuples),
-        tuple_storage_(tuple_storage),
-        null_bitmap_(null_bitmap) {
-  }
-
-  inline tuple_id numPackedTuples() const {
-    return num_tuples_;
-  }
-
-  /**
-   * @brief Returns whether this accessor has a fast strided ColumnAccessor available
-   *        that can be used to optimize memory access in a tight loop iteration
-   *        over the underlying storage block.
-   *
-   * @return true if fast ColumnAccessor is supported, otherwise false.
-   */
-  inline bool isColumnAccessorSupported() const {
-    return true;
-  }
-
-  /**
-   * @brief Get a pointer to a ColumnAccessor object that provides a fast strided memory
-   *        access on the underlying storage block.
-   * @note The ownership of the returned object lies with the caller.
-   * @warning This method should only be called if isColumnAccessorSupported() method
-   *          returned true. If ColumnAccessor is not supported this method will return a nullptr.
-   *
-   * @param current_tuple_position A constant reference to the tuple position in the containing
-   *        ValueAccessor. This reference value is shared between the containing ValueAccessor &
-   *        a ColumnAccessor. However, a ColumnAccessor *CANNOT* modify this tuple position.
-   * @param attr_id The attribute id on which this ColumnAccessor will be created.
-   *
-   * @return A pointer to a ColumnAccessor object with specific properties set that can be used
-   *         in a tight loop iterations over the underlying storage block.
-   **/
-  template <bool check_null = true>
-  inline const ColumnAccessor<check_null>* getColumnAccessor(const tuple_id &current_tuple_position,
-                                                             const attribute_id attr_id) const {
-    DCHECK(relation_.hasAttributeWithId(attr_id));
-    const void* base_location = static_cast<const char*>(tuple_storage_)
-        + relation_.getFixedLengthAttributeOffset(attr_id);
-    const std::size_t stride = relation_.getFixedByteLength();
-
-    std::unique_ptr<ColumnAccessor<check_null>> column_accessor;
-    if (check_null) {
-      const int nullable_base = relation_.getNullableAttributeIndex(attr_id);
-      const unsigned nullable_stride = relation_.numNullableAttributes();
-      column_accessor.reset(new ColumnAccessor<check_null>(current_tuple_position,
-                                                           num_tuples_,
-                                                           base_location,
-                                                           stride,
-                                                           null_bitmap_,
-                                                           nullable_base,
-                                                           nullable_stride));
-    } else {
-      column_accessor.reset(new ColumnAccessor<check_null>(current_tuple_position,
-                                                           num_tuples_,
-                                                           base_location,
-                                                           stride));
-    }
-    return column_accessor.release();
-  }
-
-  template <bool check_null>
-  inline const void* getAttributeValue(const tuple_id tuple,
-                                       const attribute_id attr) const {
-    DEBUG_ASSERT(tuple < num_tuples_);
-    DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
-    if (check_null) {
-      const int nullable_idx = relation_.getNullableAttributeIndex(attr);
-      if ((nullable_idx != -1)
-          && null_bitmap_->getBit(tuple * relation_.numNullableAttributes() + nullable_idx)) {
-        return nullptr;
-      }
-    }
-
-    return static_cast<const char*>(tuple_storage_)          // Start of actual tuple storage.
-           + (tuple * relation_.getFixedByteLength())        // Tuples prior to 'tuple'.
-           + relation_.getFixedLengthAttributeOffset(attr);  // Attribute offset within tuple.
-  }
-
-  inline TypedValue getAttributeValueTyped(const tuple_id tuple,
-                                           const attribute_id attr) const {
-    const Type &attr_type = relation_.getAttributeById(attr)->getType();
-    const void *untyped_value = getAttributeValue<true>(tuple, attr);
-    return (untyped_value == nullptr)
-        ? attr_type.makeNullValue()
-        : attr_type.makeValue(untyped_value, attr_type.maximumByteLength());
-  }
-
- private:
-  const CatalogRelationSchema &relation_;
-  const tuple_id num_tuples_;
-  const void *tuple_storage_;
-  const BitVector<false> *null_bitmap_;
-
-  DISALLOW_COPY_AND_ASSIGN(PackedRowStoreValueAccessorHelper);
-};
-
-typedef PackedTupleStorageSubBlockValueAccessor<
-    PackedRowStoreTupleStorageSubBlock,
-    PackedRowStoreValueAccessorHelper,
-    ValueAccessor::Implementation::kPackedRowStore>
-        PackedRowStoreValueAccessor;
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/SplitRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreTupleStorageSubBlock.cpp b/storage/SplitRowStoreTupleStorageSubBlock.cpp
index 1e6f7ff..ad583eb 100644
--- a/storage/SplitRowStoreTupleStorageSubBlock.cpp
+++ b/storage/SplitRowStoreTupleStorageSubBlock.cpp
@@ -245,9 +245,7 @@ tuple_id SplitRowStoreTupleStorageSubBlock::bulkInsertDispatcher(
   CopyGroupList copy_groups;
   getCopyGroupsForAttributeMap(attribute_map, &copy_groups);
   auto impl = accessor->getImplementationType();
-  const bool is_rowstore_source =
-    (impl == ValueAccessor::Implementation::kPackedRowStore ||
-     impl == ValueAccessor::Implementation::kSplitRowStore);
+  const bool is_rowstore_source = impl == ValueAccessor::Implementation::kSplitRowStore;
   if (is_rowstore_source) {
     copy_groups.merge_contiguous();
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/SplitRowStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreTupleStorageSubBlock.hpp b/storage/SplitRowStoreTupleStorageSubBlock.hpp
index 89c756d..67d36fe 100644
--- a/storage/SplitRowStoreTupleStorageSubBlock.hpp
+++ b/storage/SplitRowStoreTupleStorageSubBlock.hpp
@@ -421,4 +421,4 @@ class SplitRowStoreTupleStorageSubBlock: public TupleStorageSubBlock {
 
 }  // namespace quickstep
 
-#endif  // QUICKSTEP_STORAGE_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_
+#endif  // QUICKSTEP_STORAGE_SPLIT_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 6267d6b..de2d25b 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -40,7 +40,6 @@
 #include "storage/HashTableBase.hpp"
 #include "storage/IndexSubBlock.hpp"
 #include "storage/InsertDestinationInterface.hpp"
-#include "storage/PackedRowStoreTupleStorageSubBlock.hpp"
 #include "storage/SMAIndexSubBlock.hpp"
 #include "storage/SplitRowStoreTupleStorageSubBlock.hpp"
 #include "storage/StorageBlockBase.hpp"
@@ -958,12 +957,6 @@ TupleStorageSubBlock* StorageBlock::CreateTupleStorageSubBlock(
     const std::size_t sub_block_memory_size) {
   DEBUG_ASSERT(description.IsInitialized());
   switch (description.sub_block_type()) {
-    case TupleStorageSubBlockDescription::PACKED_ROW_STORE:
-      return new PackedRowStoreTupleStorageSubBlock(relation,
-                                                    description,
-                                                    new_block,
-                                                    sub_block_memory,
-                                                    sub_block_memory_size);
     case TupleStorageSubBlockDescription::BASIC_COLUMN_STORE:
       return new BasicColumnStoreTupleStorageSubBlock(relation,
                                                       description,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlockInfo.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockInfo.cpp b/storage/StorageBlockInfo.cpp
index 8c40be6..2646e5f 100644
--- a/storage/StorageBlockInfo.cpp
+++ b/storage/StorageBlockInfo.cpp
@@ -37,7 +37,6 @@ string BlockIdUtil::ToString(const block_id block) {
 }
 
 const char *kTupleStorageSubBlockTypeNames[] = {
-  "PackedRowStore",
   "BasicColumnStore",
   "CompressedPackedRowStore",
   "CompressedColumnStore",
@@ -46,7 +45,7 @@ const char *kTupleStorageSubBlockTypeNames[] = {
 
 const char *kIndexSubBlockTypeNames[] = {
   "CSBTree",
+  "SMA",
 };
 
 }  // namespace quickstep
-

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlockInfo.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockInfo.hpp b/storage/StorageBlockInfo.hpp
index 8d443d5..361648f 100644
--- a/storage/StorageBlockInfo.hpp
+++ b/storage/StorageBlockInfo.hpp
@@ -118,7 +118,6 @@ const tuple_id kMaxTupleID = INT_MAX;
  * @brief Codes for the different implementations of TupleStorageSubBlock.
  **/
 enum TupleStorageSubBlockType {
-  kPackedRowStore = 0,
   kBasicColumnStore,
   kCompressedPackedRowStore,
   kCompressedColumnStore,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlockLayout.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockLayout.hpp b/storage/StorageBlockLayout.hpp
index 9827a4c..30a00f5 100644
--- a/storage/StorageBlockLayout.hpp
+++ b/storage/StorageBlockLayout.hpp
@@ -73,9 +73,7 @@ class StorageBlockLayout {
    * @brief Static method to generate a default layout for a particular
    *        relation.
    * @note The current policy is that a default layout takes up one slot, uses
-   *       PackedRowStoreTupleStorageSubBlock for fixed-length relations or
-   *       SplitRowStoreTupleStorageSubBlock for variable-length relations, and
-   *       has no indices.
+   *       SplitRowStoreTupleStorageSubBlock and uses no indices.
    *
    * @param relation The relation to generate a layout for.
    * @param relation_variable_length Whether relation is variable-length.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlockLayout.proto
----------------------------------------------------------------------
diff --git a/storage/StorageBlockLayout.proto b/storage/StorageBlockLayout.proto
index 6573470..8919505 100644
--- a/storage/StorageBlockLayout.proto
+++ b/storage/StorageBlockLayout.proto
@@ -22,11 +22,10 @@ package quickstep;
 // Options for TupleStorageSubBlocks.
 message TupleStorageSubBlockDescription {
   enum TupleStorageSubBlockType {
-    PACKED_ROW_STORE = 0;
-    BASIC_COLUMN_STORE = 1;
-    COMPRESSED_PACKED_ROW_STORE = 2;
-    COMPRESSED_COLUMN_STORE = 3;
-    SPLIT_ROW_STORE = 4;
+    BASIC_COLUMN_STORE = 0;
+    COMPRESSED_PACKED_ROW_STORE = 1;
+    COMPRESSED_COLUMN_STORE = 2;
+    SPLIT_ROW_STORE = 3;
   }
 
   required TupleStorageSubBlockType sub_block_type = 1;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/SubBlockTypeRegistry.hpp
----------------------------------------------------------------------
diff --git a/storage/SubBlockTypeRegistry.hpp b/storage/SubBlockTypeRegistry.hpp
index c362d9f..71e0629 100644
--- a/storage/SubBlockTypeRegistry.hpp
+++ b/storage/SubBlockTypeRegistry.hpp
@@ -47,7 +47,7 @@ class CatalogRelationSchema;
  *        (.cpp) file, and put an invocation of the
  *        QUICKSTEP_REGISTER_TUPLE_STORE() macro in the quickstep namespace.
  *        For example:
- *        QUICKSTEP_REGISTER_TUPLE_STORE(PackedRowStoreTupleStorageSubBlock, PACKED_ROW_STORE);
+ *        QUICKSTEP_REGISTER_TUPLE_STORE(SplitRowStoreTupleStorageSubBlock, SPLIT_ROW_STORE);
  *        The first argument to the macro is the name of the class, the second
  *        is the name of the class' corresponding case in the
  *        TupleStorageSubBlockDescription::TupleStorageSubBlockType enum.
@@ -55,7 +55,7 @@ class CatalogRelationSchema;
  *        file, and put an invocation of the
  *        QUICKSTEP_DECLARE_SUB_BLOCK_TYPE_REGISTERED() macro in the quickstep
  *        namespace like so:
- *        QUICKSTEP_DECLARE_SUB_BLOCK_TYPE_REGISTERED(PackedRowStoreTupleStorageSubBlock);
+ *        QUICKSTEP_DECLARE_SUB_BLOCK_TYPE_REGISTERED(SplitRowStoreTupleStorageSubBlock);
  *
  * Registration of IndexSubBlock implementations works the same way, except the
  * first macro used should be QUICKSTEP_REGISTER_INDEX() instead of

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index e4a2906..654bbf9 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -78,7 +78,6 @@ class ValueAccessor {
     kBasicColumnStore,
     kCompressedColumnStore,
     kCompressedPackedRowStore,
-    kPackedRowStore,
     kSplitRowStore,
     // Accessor for a group of ColumnVectors:
     kColumnVectors

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/ValueAccessorUtil.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessorUtil.hpp b/storage/ValueAccessorUtil.hpp
index 8be3785..4969156 100644
--- a/storage/ValueAccessorUtil.hpp
+++ b/storage/ValueAccessorUtil.hpp
@@ -25,7 +25,6 @@
 #include "storage/BasicColumnStoreValueAccessor.hpp"
 #include "storage/CompressedColumnStoreValueAccessor.hpp"
 #include "storage/CompressedPackedRowStoreValueAccessor.hpp"
-#include "storage/PackedRowStoreValueAccessor.hpp"
 #include "storage/SplitRowStoreValueAccessor.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
@@ -70,8 +69,6 @@ auto InvokeOnValueAccessorNotAdapter(
       return functor(static_cast<CompressedColumnStoreValueAccessor*>(accessor));
     case ValueAccessor::Implementation::kCompressedPackedRowStore:
       return functor(static_cast<CompressedPackedRowStoreValueAccessor*>(accessor));
-    case ValueAccessor::Implementation::kPackedRowStore:
-      return functor(static_cast<PackedRowStoreValueAccessor*>(accessor));
     case ValueAccessor::Implementation::kSplitRowStore:
       return functor(static_cast<SplitRowStoreValueAccessor*>(accessor));
     case ValueAccessor::Implementation::kColumnVectors:
@@ -121,10 +118,6 @@ auto InvokeOnTupleIdSequenceAdapterValueAccessor(
       return functor(
           static_cast<TupleIdSequenceAdapterValueAccessor<CompressedPackedRowStoreValueAccessor>*>(
               accessor));
-    case ValueAccessor::Implementation::kPackedRowStore:
-      return functor(
-          static_cast<TupleIdSequenceAdapterValueAccessor<PackedRowStoreValueAccessor>*>(
-              accessor));
     case ValueAccessor::Implementation::kSplitRowStore:
       return functor(
           static_cast<TupleIdSequenceAdapterValueAccessor<SplitRowStoreValueAccessor>*>(
@@ -177,10 +170,6 @@ auto InvokeOnOrderedTupleIdSequenceAdapterValueAccessor(
       return functor(
           static_cast<OrderedTupleIdSequenceAdapterValueAccessor<CompressedPackedRowStoreValueAccessor>*>(
               accessor));
-    case ValueAccessor::Implementation::kPackedRowStore:
-      return functor(
-          static_cast<OrderedTupleIdSequenceAdapterValueAccessor<PackedRowStoreValueAccessor>*>(
-              accessor));
     case ValueAccessor::Implementation::kSplitRowStore:
       return functor(
           static_cast<OrderedTupleIdSequenceAdapterValueAccessor<SplitRowStoreValueAccessor>*>(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
index a511fcb..d41a457 100644
--- a/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
@@ -859,7 +859,7 @@ TEST_P(BasicColumnStoreTupleStorageSubBlockTest, DescriptionIsValidTest) {
 
   // A description that specifies the wrong sub_block_type is not valid.
   tuple_store_description_->set_sub_block_type(
-      TupleStorageSubBlockDescription::PACKED_ROW_STORE);
+      TupleStorageSubBlockDescription::SPLIT_ROW_STORE);
   EXPECT_FALSE(BasicColumnStoreTupleStorageSubBlock::DescriptionIsValid(
                    *relation_,
                    *tuple_store_description_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
index 530507b..3feee5e 100644
--- a/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
@@ -679,7 +679,7 @@ TEST_P(CompressedPackedRowStoreTupleStorageSubBlockTest, DescriptionIsValidTest)
                                                                                 *tuple_store_description_));
 
   // A description that specifies the wrong sub_block_type is not valid.
-  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::PACKED_ROW_STORE);
+  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::SPLIT_ROW_STORE);
   EXPECT_FALSE(CompressedPackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*relation_,
                                                                                 *tuple_store_description_));
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
deleted file mode 100644
index bf3c605..0000000
--- a/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
+++ /dev/null
@@ -1,584 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#include <cstring>
-#include <memory>
-#include <sstream>
-#include <string>
-#include <unordered_map>
-#include <utility>
-#include <vector>
-
-#include "gtest/gtest.h"
-
-#include "catalog/CatalogAttribute.hpp"
-#include "catalog/CatalogRelation.hpp"
-#include "storage/PackedRowStoreTupleStorageSubBlock.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageBlockLayout.hpp"
-#include "storage/StorageBlockLayout.pb.h"
-#include "storage/StorageConstants.hpp"
-#include "storage/StorageErrors.hpp"
-#include "storage/TupleIdSequence.hpp"
-#include "storage/TupleStorageSubBlock.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#include "types/CharType.hpp"
-#include "types/DoubleType.hpp"
-#include "types/IntType.hpp"
-#include "types/Type.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypedValue.hpp"
-#include "types/TypeID.hpp"
-#include "types/containers/Tuple.hpp"
-#include "types/operations/comparisons/Comparison.hpp"
-#include "types/operations/comparisons/ComparisonFactory.hpp"
-#include "types/operations/comparisons/ComparisonID.hpp"
-#include "utility/BitVector.hpp"
-#include "utility/ScopedBuffer.hpp"
-
-using std::make_pair;
-using std::ostringstream;
-using std::pair;
-
-namespace quickstep {
-
-class PackedRowStoreTupleStorageSubBlockTest : public ::testing::TestWithParam<bool> {
- protected:
-  static const size_t kSubBlockSize = 0x100000;  // 1 MB
-  static const size_t kTupleLength = 24;
-
-  virtual void SetUp() {
-    // Create a sample relation with a variety of attribute types.
-    relation_.reset(new CatalogRelation(nullptr, "TestRelation"));
-
-    // An integer.
-    CatalogAttribute *current_attr = new CatalogAttribute(relation_.get(),
-                                                          "int_attr",
-                                                          TypeFactory::GetType(kInt, GetParam()));
-    ASSERT_EQ(0, relation_->addAttribute(current_attr));
-    eq_comp_int_.reset(ComparisonFactory::GetComparison(ComparisonID::kEqual).makeUncheckedComparatorForTypes(
-        current_attr->getType(),
-        current_attr->getType()));
-
-    // A double.
-    current_attr = new CatalogAttribute(relation_.get(),
-                                        "double_attr",
-                                        TypeFactory::GetType(kDouble, GetParam()));
-    ASSERT_EQ(1, relation_->addAttribute(current_attr));
-    eq_comp_double_.reset(ComparisonFactory::GetComparison(ComparisonID::kEqual).makeUncheckedComparatorForTypes(
-        current_attr->getType(),
-        current_attr->getType()));
-
-    // A string.
-    current_attr = new CatalogAttribute(relation_.get(),
-                                        "char_attr",
-                                        TypeFactory::GetType(kChar, 12, GetParam()));
-    ASSERT_EQ(2, relation_->addAttribute(current_attr));
-    eq_comp_char_.reset(ComparisonFactory::GetComparison(ComparisonID::kEqual).makeUncheckedComparatorForTypes(
-        current_attr->getType(),
-        current_attr->getType()));
-
-    tuple_store_description_.reset(new TupleStorageSubBlockDescription());
-    tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-
-    // Don't initialize the block yet. Different tests will use different
-    // params.
-    tuple_store_memory_.reset();
-    tuple_store_.reset();
-  }
-
-  void initializeNewBlock(const size_t block_size) {
-    tuple_store_memory_.reset(block_size);
-    tuple_store_.reset(new PackedRowStoreTupleStorageSubBlock(*relation_,
-                                                              *tuple_store_description_,
-                                                              true,
-                                                              tuple_store_memory_.get(),
-                                                              block_size));
-  }
-
-  int computeRowCapacity() {
-    if (GetParam()) {
-      // Estimate using the same heuristic as
-      // PackedRowStoreTupleStorageSubBlock's constructor.
-      int row_capacity = ((kSubBlockSize - sizeof(tuple_id)) << 3)
-                         / ((kTupleLength << 3) + relation_->numNullableAttributes());
-      const size_t null_bitmap_bytes_ = BitVector<false>::BytesNeeded(
-          row_capacity * relation_->numNullableAttributes());
-      return static_cast<int>(kSubBlockSize - sizeof(tuple_id) - null_bitmap_bytes_) / kTupleLength;
-    } else {
-      return static_cast<int>(kSubBlockSize - sizeof(tuple_id)) / kTupleLength;
-    }
-  }
-
-  // Caller takes ownership of new heap-created Tuple.
-  Tuple* createSampleTuple(const int base_value) const {
-    std::vector<TypedValue> attrs;
-
-    // int_attr
-    if (GetParam() && (base_value % 6 == 0)) {
-      // Throw in a NULL integer for every sixth value.
-      attrs.emplace_back(kInt);
-    } else {
-      attrs.emplace_back(base_value);
-    }
-
-    // double_attr
-    if (GetParam() && (base_value % 6 == 2)) {
-      // NULL very sixth value.
-      attrs.emplace_back(kDouble);
-    } else {
-      attrs.emplace_back(static_cast<double>(0.25 * base_value));
-    }
-
-    // char_attr
-    if (GetParam() && (base_value % 6 == 4)) {
-      // NULL very sixth value.
-      attrs.emplace_back(CharType::InstanceNullable(12).makeNullValue());
-    } else {
-      ostringstream char_buffer;
-      char_buffer << base_value;
-      std::string string_literal(char_buffer.str());
-      attrs.emplace_back(CharType::InstanceNonNullable(12).makeValue(string_literal.c_str(),
-                                                                     string_literal.size() + 1));
-      attrs.back().ensureNotReference();
-    }
-
-    return new Tuple(std::move(attrs));
-  }
-
-  void fillBlockWithSampleData() {
-    tuple_id current_tid = 0;
-    std::unique_ptr<Tuple> current_tuple(createSampleTuple(current_tid));
-    while (tuple_store_->insertTupleInBatch(*current_tuple)) {
-      ++current_tid;
-      current_tuple.reset(createSampleTuple(current_tid));
-    }
-
-    tuple_store_->rebuild();
-  }
-
-  void checkTupleValuesUntyped(const tuple_id tid,
-                               const int base_value) {
-    ASSERT_TRUE(tuple_store_->hasTupleWithID(tid));
-    ASSERT_TRUE(tuple_store_->supportsUntypedGetAttributeValue(0));
-    ASSERT_TRUE(tuple_store_->supportsUntypedGetAttributeValue(1));
-    ASSERT_TRUE(tuple_store_->supportsUntypedGetAttributeValue(2));
-
-    std::unique_ptr<Tuple> comp_tuple(createSampleTuple(base_value));
-
-    if (comp_tuple->getAttributeValue(0).isNull()) {
-      assert(nullptr == tuple_store_->getAttributeValue(tid, 0));
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_int_->compareDataPtrs(nullptr,
-                                                 tuple_store_->getAttributeValue(tid, 0)));
-    } else {
-      assert(eq_comp_int_->compareDataPtrs(comp_tuple->getAttributeValue(0).getDataPtr(),
-                                           tuple_store_->getAttributeValue(tid, 0)));
-    }
-
-    if (comp_tuple->getAttributeValue(1).isNull()) {
-      EXPECT_EQ(nullptr, tuple_store_->getAttributeValue(tid, 1));
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_double_->compareDataPtrs(nullptr,
-                                                    tuple_store_->getAttributeValue(tid, 1)));
-    } else {
-      EXPECT_TRUE(eq_comp_double_->compareDataPtrs(comp_tuple->getAttributeValue(1).getDataPtr(),
-                                                   tuple_store_->getAttributeValue(tid, 1)));
-    }
-
-    if (comp_tuple->getAttributeValue(2).isNull()) {
-      EXPECT_EQ(nullptr, tuple_store_->getAttributeValue(tid, 2));
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_char_->compareDataPtrs(nullptr,
-                                                  tuple_store_->getAttributeValue(tid, 2)));
-    } else {
-      EXPECT_TRUE(eq_comp_char_->compareDataPtrs(comp_tuple->getAttributeValue(2).getDataPtr(),
-                                                 tuple_store_->getAttributeValue(tid, 2)));
-    }
-  }
-
-  void checkTupleValuesTyped(const tuple_id tid,
-                             const int base_value) {
-    ASSERT_TRUE(tuple_store_->hasTupleWithID(tid));
-
-    std::unique_ptr<Tuple> comp_tuple(createSampleTuple(base_value));
-
-    if (comp_tuple->getAttributeValue(0).isNull()) {
-      EXPECT_TRUE(tuple_store_->getAttributeValueTyped(tid, 0).isNull());
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_int_->compareTypedValues(comp_tuple->getAttributeValue(0),
-                                                    tuple_store_->getAttributeValueTyped(tid, 0)));
-    } else {
-      EXPECT_TRUE(eq_comp_int_->compareTypedValues(comp_tuple->getAttributeValue(0),
-                                                   tuple_store_->getAttributeValueTyped(tid, 0)));
-    }
-
-    if (comp_tuple->getAttributeValue(1).isNull()) {
-      EXPECT_TRUE(tuple_store_->getAttributeValueTyped(tid, 1).isNull());
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_double_->compareTypedValues(comp_tuple->getAttributeValue(1),
-                                                       tuple_store_->getAttributeValueTyped(tid, 1)));
-    } else {
-      EXPECT_TRUE(eq_comp_double_->compareTypedValues(comp_tuple->getAttributeValue(1),
-                                                      tuple_store_->getAttributeValueTyped(tid, 1)));
-    }
-
-    if (comp_tuple->getAttributeValue(2).isNull()) {
-      EXPECT_TRUE(tuple_store_->getAttributeValueTyped(tid, 2).isNull());
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_char_->compareTypedValues(comp_tuple->getAttributeValue(2),
-                                                     tuple_store_->getAttributeValueTyped(tid, 2)));
-    } else {
-      EXPECT_TRUE(eq_comp_char_->compareTypedValues(comp_tuple->getAttributeValue(2),
-                                                    tuple_store_->getAttributeValueTyped(tid, 2)));
-    }
-  }
-
-  template<bool check_null>
-  void checkColumnAccessor() {
-    initializeNewBlock(kSubBlockSize);
-    fillBlockWithSampleData();
-    ASSERT_TRUE(tuple_store_->isPacked());
-    std::unique_ptr<PackedRowStoreValueAccessor> accessor(
-      static_cast<PackedRowStoreValueAccessor*>(tuple_store_->createValueAccessor()));
-    attribute_id  value_accessor_id = 0;
-    tuple_id tid = 0;
-    accessor->beginIteration();
-    ASSERT_TRUE(accessor->isColumnAccessorSupported());
-    std::unique_ptr<const ColumnAccessor<check_null>>
-    column_accessor(accessor->template getColumnAccessor<check_null>(value_accessor_id));
-    ASSERT_TRUE(column_accessor != nullptr);
-    while (accessor->next()) {
-      const void *va_value = column_accessor->getUntypedValue();
-      std::unique_ptr<Tuple> expected_tuple(createSampleTuple(tid));
-
-      if (expected_tuple->getAttributeValue(value_accessor_id).isNull()) {
-        ASSERT_TRUE(va_value == nullptr);
-      } else {
-        ASSERT_TRUE(eq_comp_int_->compareDataPtrs(expected_tuple->getAttributeValue(value_accessor_id).getDataPtr(),
-                                                  va_value));
-      }
-      ++tid;
-    }
-  }
-
-  std::unique_ptr<CatalogRelation> relation_;
-  ScopedBuffer tuple_store_memory_;
-  std::unique_ptr<TupleStorageSubBlockDescription> tuple_store_description_;
-  std::unique_ptr<PackedRowStoreTupleStorageSubBlock> tuple_store_;
-
-  std::unique_ptr<UncheckedComparator> eq_comp_int_;
-  std::unique_ptr<UncheckedComparator> eq_comp_double_;
-  std::unique_ptr<UncheckedComparator> eq_comp_char_;
-};
-
-typedef PackedRowStoreTupleStorageSubBlockTest PackedRowStoreTupleStorageSubBlockDeathTest;
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, DescriptionIsValidTest) {
-  // The descriptions we use for the other tests should be valid.
-  EXPECT_TRUE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*relation_,
-                                                                     *tuple_store_description_));
-
-  // An uninitialized description is not valid.
-  tuple_store_description_.reset(new TupleStorageSubBlockDescription());
-  EXPECT_FALSE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*relation_,
-                                                                      *tuple_store_description_));
-
-  // A description that specifies the wrong sub_block_type is not valid.
-  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::BASIC_COLUMN_STORE);
-  EXPECT_FALSE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*relation_,
-                                                                      *tuple_store_description_));
-
-  // A relation with a nullable attribute is OK.
-  std::unique_ptr<CatalogRelation> nullable_relation(new CatalogRelation(NULL, "nullable_relation"));
-  CatalogAttribute *nullable_attribute = new CatalogAttribute(nullable_relation.get(),
-                                                              "nullable_attr",
-                                                              TypeFactory::GetType(kInt, true));
-  ASSERT_EQ(0, nullable_relation->addAttribute(nullable_attribute));
-  tuple_store_description_.reset(new TupleStorageSubBlockDescription());
-  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-  EXPECT_TRUE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*nullable_relation,
-                                                                     *tuple_store_description_));
-
-  // A relation with a variable-length attribute can't be used with this block type.
-  std::unique_ptr<CatalogRelation> variable_length_relation(new CatalogRelation(NULL, "variable_length_relation"));
-  CatalogAttribute *variable_length_attribute = new CatalogAttribute(variable_length_relation.get(),
-                                                                    "variable_length_attr",
-                                                                     TypeFactory::GetType(kVarChar, 20, false));
-  ASSERT_EQ(0, variable_length_relation->addAttribute(variable_length_attribute));
-  EXPECT_FALSE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*variable_length_relation,
-                                                                      *tuple_store_description_));
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockDeathTest, ConstructWithInvalidDescriptionTest) {
-  tuple_store_description_.reset(new TupleStorageSubBlockDescription());
-  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::BASIC_COLUMN_STORE);
-  EXPECT_DEATH(initializeNewBlock(kSubBlockSize), "");
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, MemoryTooSmallTest) {
-  // 1 byte short.
-  EXPECT_THROW(initializeNewBlock(sizeof(tuple_id) - 1),
-               BlockMemoryTooSmall);
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, InsertTest) {
-  initializeNewBlock(kSubBlockSize);
-
-  int row_capacity = computeRowCapacity();
-
-  EXPECT_TRUE(tuple_store_->supportsAdHocInsert());
-  EXPECT_TRUE(tuple_store_->adHocInsertIsEfficient());
-  EXPECT_FALSE(tuple_store_->isCompressed());
-  EXPECT_TRUE(tuple_store_->isEmpty());
-
-  std::unique_ptr<Tuple> current_tuple;
-  for (int tuple_num = 0;
-       tuple_num < row_capacity;
-       ++tuple_num) {
-    current_tuple.reset(createSampleTuple(tuple_num));
-    TupleStorageSubBlock::InsertResult result = tuple_store_->insertTuple(*current_tuple);
-    ASSERT_EQ(tuple_num, result.inserted_id);
-    ASSERT_FALSE(result.ids_mutated);
-
-    EXPECT_FALSE(tuple_store_->isEmpty());
-    EXPECT_TRUE(tuple_store_->isPacked());
-    EXPECT_EQ(tuple_num, tuple_store_->getMaxTupleID());
-    EXPECT_EQ(tuple_num + 1, tuple_store_->numTuples());
-  }
-
-  current_tuple.reset(createSampleTuple(0));
-  TupleStorageSubBlock::InsertResult result = tuple_store_->insertTuple(*current_tuple);
-  EXPECT_EQ(-1, result.inserted_id);
-  EXPECT_FALSE(result.ids_mutated);
-
-  EXPECT_TRUE(tuple_store_->isPacked());
-  EXPECT_EQ(row_capacity - 1, tuple_store_->getMaxTupleID());
-  EXPECT_EQ(row_capacity, tuple_store_->numTuples());
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, InsertInBatchTest) {
-  initializeNewBlock(kSubBlockSize);
-
-  int row_capacity = computeRowCapacity();
-
-  EXPECT_TRUE(tuple_store_->supportsAdHocInsert());
-  EXPECT_TRUE(tuple_store_->adHocInsertIsEfficient());
-  EXPECT_FALSE(tuple_store_->isCompressed());
-  EXPECT_TRUE(tuple_store_->isEmpty());
-
-  std::unique_ptr<Tuple> current_tuple;
-  for (int tuple_num = 0;
-       tuple_num < row_capacity;
-       ++tuple_num) {
-    current_tuple.reset(createSampleTuple(tuple_num));
-    EXPECT_TRUE(tuple_store_->insertTupleInBatch(*current_tuple));
-
-    EXPECT_FALSE(tuple_store_->isEmpty());
-    EXPECT_TRUE(tuple_store_->isPacked());
-    EXPECT_EQ(tuple_num, tuple_store_->getMaxTupleID());
-    EXPECT_EQ(tuple_num + 1, tuple_store_->numTuples());
-  }
-
-  current_tuple.reset(createSampleTuple(0));
-  EXPECT_FALSE(tuple_store_->insertTupleInBatch(*current_tuple));
-
-  EXPECT_TRUE(tuple_store_->isPacked());
-  EXPECT_EQ(row_capacity - 1, tuple_store_->getMaxTupleID());
-  EXPECT_EQ(row_capacity, tuple_store_->numTuples());
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, ColumnAccessorTest) {
-  if (GetParam()) {   // when true, the attributes can be nullable.
-    checkColumnAccessor<true>();
-  } else {   // when false, the attributes are non-null.
-    checkColumnAccessor<false>();
-  }
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, GetAttributeValueTest) {
-  initializeNewBlock(kSubBlockSize);
-  fillBlockWithSampleData();
-  ASSERT_TRUE(tuple_store_->isPacked());
-
-  for (tuple_id tid = 0;
-       tid <= tuple_store_->getMaxTupleID();
-       ++tid) {
-    checkTupleValuesUntyped(tid, tid);
-  }
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, GetAttributeValueTypedTest) {
-  initializeNewBlock(kSubBlockSize);
-  fillBlockWithSampleData();
-  ASSERT_TRUE(tuple_store_->isPacked());
-
-  for (tuple_id tid = 0;
-       tid <= tuple_store_->getMaxTupleID();
-       ++tid) {
-    checkTupleValuesTyped(tid, tid);
-  }
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, SetAttributeValueTypedTest) {
-  initializeNewBlock(kSubBlockSize);
-  fillBlockWithSampleData();
-  ASSERT_TRUE(tuple_store_->isPacked());
-
-  // Alter every 16th tuple.
-  for (tuple_id tid = 0;
-       tid <= tuple_store_->getMaxTupleID();
-       tid += 16) {
-    std::unique_ptr<Tuple> mod_tuple(createSampleTuple(-tid));
-
-    std::unordered_map<attribute_id, TypedValue> new_values;
-    new_values.insert(make_pair(0, mod_tuple->getAttributeValue(0)));
-    new_values.insert(make_pair(1, mod_tuple->getAttributeValue(1)));
-    new_values.insert(make_pair(2, mod_tuple->getAttributeValue(2)));
-    ASSERT_TRUE(tuple_store_->canSetAttributeValuesInPlaceTyped(tid, new_values));
-
-    tuple_store_->setAttributeValueInPlaceTyped(tid, 0, mod_tuple->getAttributeValue(0));
-    tuple_store_->setAttributeValueInPlaceTyped(tid, 1, mod_tuple->getAttributeValue(1));
-    tuple_store_->setAttributeValueInPlaceTyped(tid, 2, mod_tuple->getAttributeValue(2));
-  }
-
-  // Check all values.
-  for (tuple_id tid = 0;
-       tid <= tuple_store_->getMaxTupleID();
-       ++tid) {
-    if (tid & 0xF) {
-      checkTupleValuesTyped(tid, tid);
-    } else {
-      checkTupleValuesTyped(tid, -tid);
-    }
-  }
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, DeleteTest) {
-  initializeNewBlock(kSubBlockSize);
-  fillBlockWithSampleData();
-  ASSERT_TRUE(tuple_store_->isPacked());
-
-  tuple_id original_num_tuples = tuple_store_->numTuples();
-
-  // Delete the last tuple.
-  EXPECT_FALSE(tuple_store_->deleteTuple(original_num_tuples - 1));
-
-  // Delete the first tuple.
-  EXPECT_TRUE(tuple_store_->deleteTuple(0));
-
-  // Delete a sequence of tuples.
-  TupleIdSequence delete_sequence(tuple_store_->getMaxTupleID() + 1);
-  for (tuple_id tid = 63;
-       tid <= tuple_store_->getMaxTupleID();
-       tid += 64) {
-    delete_sequence.set(tid, true);
-  }
-  EXPECT_TRUE(tuple_store_->bulkDeleteTuples(&delete_sequence));
-
-  EXPECT_EQ(static_cast<tuple_id>(original_num_tuples - 2 - delete_sequence.numTuples()),
-            tuple_store_->numTuples());
-
-  tuple_id remaining_tid = 0;
-  for (tuple_id original_tid = 0;
-       original_tid < (original_num_tuples - 1);
-       ++original_tid) {
-    if (original_tid & 0x3F) {
-      checkTupleValuesUntyped(remaining_tid, original_tid);
-      ++remaining_tid;
-    }
-  }
-}
-
-TEST(PackedRowStoreTupleStorageSubBlockNullTypeTest, NullTypeTest) {
-  // Set up a relation with a single NullType attribute.
-  CatalogRelation test_relation(nullptr, "TestRelation");
-  CatalogAttribute *nulltype_attr = new CatalogAttribute(&test_relation,
-                                                         "nulltype_attr",
-                                                         TypeFactory::GetType(kNullType, true));
-  ASSERT_EQ(0, test_relation.addAttribute(nulltype_attr));
-
-  // Set up a minimal StorageBlockLayoutDescription.
-  StorageBlockLayoutDescription layout_desc;
-  layout_desc.set_num_slots(1);
-  layout_desc.mutable_tuple_store_description()->set_sub_block_type(
-      TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-
-  // Check that the description is considered valid.
-  EXPECT_TRUE(StorageBlockLayout::DescriptionIsValid(test_relation, layout_desc));
-
-  StorageBlockLayout layout(test_relation, layout_desc);
-
-  // Construct an actual PackedRowStoreTupleStorageSubBlock.
-  ScopedBuffer tuple_store_memory(kSlotSizeBytes);
-  PackedRowStoreTupleStorageSubBlock tuple_store(test_relation,
-                                                 layout_desc.tuple_store_description(),
-                                                 true,
-                                                 tuple_store_memory.get(),
-                                                 kSlotSizeBytes);
-
-  // Insert some NullType values.
-  std::vector<TypedValue> attr_values;
-  attr_values.emplace_back(kNullType);
-  Tuple tuple(std::move(attr_values));
-
-  for (tuple_id tid = 0; tid < 100; ++tid) {
-    tuple_store.insertTuple(tuple);
-  }
-
-  EXPECT_EQ(100, tuple_store.numTuples());
-
-  // Delete some values.
-  TupleIdSequence delete_sequence(100);
-  delete_sequence.set(5, true);
-  delete_sequence.set(25, true);
-  delete_sequence.set(45, true);
-  delete_sequence.set(65, true);
-  delete_sequence.set(85, true);
-
-  EXPECT_TRUE(tuple_store.bulkDeleteTuples(&delete_sequence));
-  EXPECT_EQ(95, tuple_store.numTuples());
-  ASSERT_EQ(94, tuple_store.getMaxTupleID());
-
-  // Read out values.
-  for (tuple_id tid = 0; tid < 95; ++tid) {
-    ASSERT_TRUE(tuple_store.hasTupleWithID(tid));
-    EXPECT_EQ(nullptr, tuple_store.getAttributeValue(tid, 0));
-
-    TypedValue value = tuple_store.getAttributeValueTyped(tid, 0);
-    EXPECT_TRUE(value.isNull());
-    EXPECT_EQ(kNullType, value.getTypeID());
-  }
-}
-
-// Note: INSTANTIATE_TEST_CASE_P has variadic arguments part. If the variable argument part
-//       is empty, C++11 standard says it should produce a warning. A warning is converted
-//       to an error since we use -Werror as a compiler parameter. It causes Travis to build.
-//       This is the reason that we must give an empty string argument as a last parameter
-//       to supress warning that clang gives.
-INSTANTIATE_TEST_CASE_P(WithAndWithoutNullableAttributes,
-                        PackedRowStoreTupleStorageSubBlockTest,
-                        ::testing::Bool(),);  // NOLINT(whitespace/comma)
-
-INSTANTIATE_TEST_CASE_P(WithAndWithoutNullableAttributes,
-                        PackedRowStoreTupleStorageSubBlockDeathTest,
-                        ::testing::Bool(),);  // NOLINT(whitespace/comma)
-
-}  // namespace quickstep