You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by hb...@apache.org on 2016/06/20 17:14:00 UTC

[06/11] incubator-quickstep git commit: QUICKSTEP-18: Allow BasicColumnStoreTupleStorageSubBlock to be unsorted

QUICKSTEP-18: Allow BasicColumnStoreTupleStorageSubBlock to be unsorted

Review comments applied

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


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

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

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


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

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

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

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

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