You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2016/08/14 22:34:28 UTC

[5/6] kudu git commit: Reorganize range partition client API

Reorganize range partition client API

This commit redesigns the client APIs dealing with adding and dropping range
partitions. The goal is to make them more consistent and easier to understand.
The biggest public facing change is that all methods which deal with range
partitions are now named that, and the 'bounds' nomenclature has mostly been
dropped. All of these methods are new since 0.9.1, so this shouldn't be a
breaking change (wire compatibility is not broken).

Additionally, range partitions can now be created, added, and dropped with
exclusive lower bounds and inclusive upper bounds. Originally I thought this
would be a pretty minor feature, but I think it's going to end up being useful
when creating higher level SQL bindings.

Finally, the error messages when altering tables have been significantly
improved with new pretty printing of the offending range partition. Before this
it would print the serialized range partition keys, which are pretty much
impossible to figure out.

An equivalent cleanup for the Java client will be in a follow up commit.

Change-Id: Ic731f0abcbb81cd4238cc960b150bf8cd1c1f0ea
Reviewed-on: http://gerrit.cloudera.org:8080/3882
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: ff589b52b867bc127bb671da87112c2881d6e99e
Parents: ea81e05
Author: Dan Burkert <da...@cloudera.com>
Authored: Wed Aug 10 01:44:40 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Sun Aug 14 22:33:42 2016 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/TestAlterTable.java  |  14 +-
 src/kudu/client/client-test.cc                  |  87 ++++-
 src/kudu/client/client.cc                       |  66 ++--
 src/kudu/client/client.h                        | 113 ++++---
 src/kudu/client/table_alterer-internal.cc       |  28 +-
 src/kudu/client/table_alterer-internal.h        |   2 +
 src/kudu/client/table_creator-internal.h        |  12 +-
 src/kudu/common/partition-test.cc               | 179 +++++++++-
 src/kudu/common/partition.cc                    | 323 ++++++++++++++++---
 src/kudu/common/partition.h                     |  26 ++
 src/kudu/common/row_operations-test.cc          |  14 +-
 src/kudu/common/row_operations.cc               |  11 +-
 src/kudu/common/wire_protocol.proto             |   8 +
 .../alter_table-randomized-test.cc              |  24 +-
 src/kudu/integration-tests/alter_table-test.cc  |  40 +--
 .../flex_partitioning-itest.cc                  |   2 +-
 src/kudu/master/catalog_manager.cc              |  60 +++-
 src/kudu/master/master-test.cc                  |   9 +-
 .../tablet/transactions/write_transaction.cc    |   2 +
 src/kudu/tserver/tablet_server-test.cc          |   4 +-
 20 files changed, 836 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
index 83b9767..6594e9b 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
@@ -180,7 +180,7 @@ public class TestAlterTable extends BaseKuduTest {
     } catch (KuduException e) {
       assertTrue(e.getStatus().isInvalidArgument());
       assertTrue(e.getStatus().getMessage().contains(
-          "New partition conflicts with existing partition"));
+          "New range partition conflicts with existing range partition"));
     }
     assertEquals(100, countRowsInTable(table));
 
@@ -192,7 +192,7 @@ public class TestAlterTable extends BaseKuduTest {
     } catch (KuduException e) {
       assertTrue(e.getStatus().isInvalidArgument());
       assertTrue(e.getStatus().getMessage().contains(
-          "New partition conflicts with existing partition"));
+          "New range partition conflicts with existing range partition"));
     }
     assertEquals(100, countRowsInTable(table));
 
@@ -204,7 +204,7 @@ public class TestAlterTable extends BaseKuduTest {
     } catch (KuduException e) {
       assertTrue(e.getStatus().isInvalidArgument());
       assertTrue(e.getStatus().getMessage().contains(
-          "New partition conflicts with existing partition"));
+          "New range partition conflicts with existing range partition"));
     }
     assertEquals(100, countRowsInTable(table));
 
@@ -222,7 +222,7 @@ public class TestAlterTable extends BaseKuduTest {
     } catch (KuduException e) {
       assertTrue(e.getStatus().isInvalidArgument());
       assertTrue(e.getStatus().getMessage().contains(
-          "New partition conflicts with existing partition"));
+          "New range partition conflicts with existing range partition"));
     }
     assertEquals(100, countRowsInTable(table));
 
@@ -234,7 +234,7 @@ public class TestAlterTable extends BaseKuduTest {
     } catch (KuduException e) {
       assertTrue(e.getStatus().isInvalidArgument());
       assertTrue(e.getStatus().getMessage(), e.getStatus().getMessage().contains(
-          "No tablet found for drop partition step"));
+          "No range partition found for drop range partition step"));
     }
     assertEquals(100, countRowsInTable(table));
 
@@ -248,7 +248,7 @@ public class TestAlterTable extends BaseKuduTest {
     } catch (KuduException e) {
       assertTrue(e.getStatus().isInvalidArgument());
       assertTrue(e.getStatus().getMessage().contains(
-          "No tablet found for drop partition step"));
+          "No range partition found for drop range partition step"));
     }
     assertEquals(100, countRowsInTable(table));
     assertFalse(syncClient.tableExists("foo"));
@@ -281,7 +281,7 @@ public class TestAlterTable extends BaseKuduTest {
     } catch (KuduException e) {
       assertTrue(e.getStatus().isInvalidArgument());
       assertTrue(e.getStatus().getMessage().contains(
-          "No tablet found for drop partition step"));
+          "No range partition found for drop range partition step"));
     }
     assertEquals(100, countRowsInTable(table));
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index f9d1d31..7420250 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -452,10 +452,10 @@ class ClientTest : public KuduTest {
 
     gscoped_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
     for (auto& split_row : split_rows) {
-      table_creator->add_range_split(split_row.release());
+      table_creator->add_range_partition_split(split_row.release());
     }
     for (auto& bound : range_bounds) {
-      table_creator->add_range_bound(bound.first.release(), bound.second.release());
+      table_creator->add_range_partition(bound.first.release(), bound.second.release());
     }
     ASSERT_OK(table_creator->table_name(table_name)
                             .schema(&schema_)
@@ -1139,15 +1139,14 @@ TEST_F(ClientTest, TestNonCoveringRangePartitions) {
     Status result = session->Apply(insert.release());
     EXPECT_TRUE(result.IsIOError());
     vector<KuduError*> errors;
+    ElementDeleter drop(&errors);
     bool overflowed;
     session->GetPendingErrors(&errors, &overflowed);
     EXPECT_FALSE(overflowed);
     EXPECT_EQ(1, errors.size());
     EXPECT_TRUE(errors[0]->status().IsNotFound());
-    STLDeleteElements(&errors);
   }
 
-
   // Scans
 
   { // full table scan
@@ -1229,6 +1228,86 @@ TEST_F(ClientTest, TestNonCoveringRangePartitions) {
   }
 }
 
+TEST_F(ClientTest, TestExclusiveInclusiveRangeBounds) {
+  // Create test table and insert test rows.
+  const string table_name = "TestExclusiveInclusiveRangeBounds";
+  shared_ptr<KuduTable> table;
+
+  unique_ptr<KuduPartialRow> lower_bound(schema_.NewRow());
+  ASSERT_OK(lower_bound->SetInt32("key", -1));
+  unique_ptr<KuduPartialRow> upper_bound(schema_.NewRow());
+  ASSERT_OK(upper_bound->SetInt32("key", 99));
+
+  gscoped_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  table_creator->add_range_partition(lower_bound.release(), upper_bound.release(),
+                                      KuduTableCreator::EXCLUSIVE_BOUND,
+                                      KuduTableCreator::INCLUSIVE_BOUND);
+  ASSERT_OK(table_creator->table_name(table_name)
+                          .schema(&schema_)
+                          .num_replicas(1)
+                          .set_range_partition_columns({ "key" })
+                          .Create());
+
+  lower_bound.reset(schema_.NewRow());
+  ASSERT_OK(lower_bound->SetInt32("key", 199));
+  upper_bound.reset(schema_.NewRow());
+  ASSERT_OK(upper_bound->SetInt32("key", 299));
+  unique_ptr<KuduTableAlterer> alterer(client_->NewTableAlterer(table_name));
+  alterer->AddRangePartition(lower_bound.release(), upper_bound.release(),
+                             KuduTableCreator::EXCLUSIVE_BOUND,
+                             KuduTableCreator::INCLUSIVE_BOUND);
+  ASSERT_OK(alterer->Alter());
+  ASSERT_OK(client_->OpenTable(table_name, &table));
+
+  ASSERT_NO_FATAL_FAILURE(InsertTestRows(table.get(), 100, 0));
+  ASSERT_NO_FATAL_FAILURE(InsertTestRows(table.get(), 100, 200));
+
+  // Insert out-of-range rows.
+  shared_ptr<KuduSession> session = client_->NewSession();
+  ASSERT_OK(session->SetFlushMode(KuduSession::AUTO_FLUSH_SYNC));
+  session->SetTimeoutMillis(60000);
+  vector<gscoped_ptr<KuduInsert>> out_of_range_inserts;
+  out_of_range_inserts.emplace_back(BuildTestRow(table.get(), -50));
+  out_of_range_inserts.emplace_back(BuildTestRow(table.get(), -1));
+  out_of_range_inserts.emplace_back(BuildTestRow(table.get(), 100));
+  out_of_range_inserts.emplace_back(BuildTestRow(table.get(), 150));
+  out_of_range_inserts.emplace_back(BuildTestRow(table.get(), 199));
+  out_of_range_inserts.emplace_back(BuildTestRow(table.get(), 300));
+  out_of_range_inserts.emplace_back(BuildTestRow(table.get(), 350));
+
+  for (auto& insert : out_of_range_inserts) {
+    Status result = session->Apply(insert.release());
+    EXPECT_TRUE(result.IsIOError());
+    vector<KuduError*> errors;
+    ElementDeleter drop(&errors);
+    bool overflowed;
+    session->GetPendingErrors(&errors, &overflowed);
+    EXPECT_FALSE(overflowed);
+    EXPECT_EQ(1, errors.size());
+    EXPECT_TRUE(errors[0]->status().IsNotFound());
+  }
+
+  ASSERT_EQ(200, CountTableRows(table.get()));
+
+  // Drop the range partitions by normal inclusive/exclusive bounds, and by
+  // exclusive/inclusive bounds.
+  alterer.reset(client_->NewTableAlterer(table_name));
+  lower_bound.reset(schema_.NewRow());
+  ASSERT_OK(lower_bound->SetInt32("key", 0));
+  upper_bound.reset(schema_.NewRow());
+  ASSERT_OK(upper_bound->SetInt32("key", 100));
+  alterer->DropRangePartition(lower_bound.release(), upper_bound.release());
+  lower_bound.reset(schema_.NewRow());
+  ASSERT_OK(lower_bound->SetInt32("key", 199));
+  upper_bound.reset(schema_.NewRow());
+  ASSERT_OK(upper_bound->SetInt32("key", 299));
+  alterer->DropRangePartition(lower_bound.release(), upper_bound.release(),
+                             KuduTableCreator::EXCLUSIVE_BOUND,
+                             KuduTableCreator::INCLUSIVE_BOUND);
+  ASSERT_OK(alterer->Alter());
+  ASSERT_EQ(0, CountTableRows(table.get()));
+}
+
 TEST_F(ClientTest, TestMetaCacheExpiry) {
   google::FlagSaver saver;
   FLAGS_table_locations_ttl_ms = 25;

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/client/client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 2d92e99..b1f19de 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -470,22 +470,28 @@ KuduTableCreator& KuduTableCreator::set_range_partition_columns(
   return *this;
 }
 
-KuduTableCreator& KuduTableCreator::add_range_split(KuduPartialRow* split_row) {
-  data_->range_splits_.emplace_back(split_row);
+KuduTableCreator& KuduTableCreator::add_range_partition_split(KuduPartialRow* split_row) {
+  data_->range_partition_splits_.emplace_back(split_row);
   return *this;
 }
 
 KuduTableCreator& KuduTableCreator::split_rows(const vector<const KuduPartialRow*>& rows) {
   for (const KuduPartialRow* row : rows) {
-    data_->range_splits_.emplace_back(const_cast<KuduPartialRow*>(row));
+    data_->range_partition_splits_.emplace_back(const_cast<KuduPartialRow*>(row));
   }
   return *this;
 }
 
-KuduTableCreator& KuduTableCreator::add_range_bound(KuduPartialRow* lower_bound,
-                                                    KuduPartialRow* upper_bound) {
-  data_->range_bounds_.emplace_back(unique_ptr<KuduPartialRow>(lower_bound),
-                                    unique_ptr<KuduPartialRow>(upper_bound));
+KuduTableCreator& KuduTableCreator::add_range_partition(KuduPartialRow* lower_bound,
+                                                        KuduPartialRow* upper_bound,
+                                                        RangePartitionBound lower_bound_type,
+                                                        RangePartitionBound upper_bound_type) {
+  data_->range_partition_bounds_.push_back({
+      unique_ptr<KuduPartialRow>(lower_bound),
+      unique_ptr<KuduPartialRow>(upper_bound),
+      lower_bound_type,
+      upper_bound_type,
+  });
   return *this;
 }
 
@@ -529,19 +535,30 @@ Status KuduTableCreator::Create() {
 
   RowOperationsPBEncoder encoder(req.mutable_split_rows_range_bounds());
 
-  for (const auto& row : data_->range_splits_) {
+  for (const auto& row : data_->range_partition_splits_) {
     if (!row) {
       return Status::InvalidArgument("range split row must not be null");
     }
     encoder.Add(RowOperationsPB::SPLIT_ROW, *row);
   }
 
-  for (const auto& bound : data_->range_bounds_) {
-    if (!bound.first || !bound.second) {
+  for (const auto& bound : data_->range_partition_bounds_) {
+    if (!bound.lower_bound || !bound.upper_bound) {
       return Status::InvalidArgument("range bounds must not be null");
     }
-    encoder.Add(RowOperationsPB::RANGE_LOWER_BOUND, *bound.first);
-    encoder.Add(RowOperationsPB::RANGE_UPPER_BOUND, *bound.second);
+
+    RowOperationsPB_Type lower_bound_type =
+      bound.lower_bound_type == KuduTableCreator::INCLUSIVE_BOUND ?
+      RowOperationsPB::RANGE_LOWER_BOUND :
+      RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND;
+
+    RowOperationsPB_Type upper_bound_type =
+      bound.upper_bound_type == KuduTableCreator::EXCLUSIVE_BOUND ?
+      RowOperationsPB::RANGE_UPPER_BOUND :
+      RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND;
+
+    encoder.Add(lower_bound_type, *bound.lower_bound);
+    encoder.Add(upper_bound_type, *bound.upper_bound);
   }
 
   req.mutable_partition_schema()->CopyFrom(data_->partition_schema_);
@@ -557,7 +574,7 @@ Status KuduTableCreator::Create() {
                                                            req,
                                                            *data_->schema_,
                                                            deadline,
-                                                           !data_->range_bounds_.empty()),
+                                                           !data_->range_partition_bounds_.empty()),
                         strings::Substitute("Error creating table $0 on the master",
                                             data_->table_name_));
 
@@ -847,8 +864,12 @@ KuduTableAlterer* KuduTableAlterer::DropColumn(const string& name) {
   return this;
 }
 
-KuduTableAlterer* KuduTableAlterer::AddRangePartition(KuduPartialRow* lower_bound,
-                                                      KuduPartialRow* upper_bound) {
+KuduTableAlterer* KuduTableAlterer::AddRangePartition(
+    KuduPartialRow* lower_bound,
+    KuduPartialRow* upper_bound,
+    KuduTableCreator::RangePartitionBound lower_bound_type,
+    KuduTableCreator::RangePartitionBound upper_bound_type) {
+
   if (lower_bound == nullptr || upper_bound == nullptr) {
     data_->status_ = Status::InvalidArgument("range partition bounds may not be null");
   }
@@ -864,14 +885,19 @@ KuduTableAlterer* KuduTableAlterer::AddRangePartition(KuduPartialRow* lower_boun
   Data::Step s { AlterTableRequestPB::ADD_RANGE_PARTITION,
                  nullptr,
                  unique_ptr<KuduPartialRow>(lower_bound),
-                 unique_ptr<KuduPartialRow>(upper_bound) };
+                 unique_ptr<KuduPartialRow>(upper_bound),
+                 lower_bound_type,
+                 upper_bound_type };
   data_->steps_.emplace_back(std::move(s));
   data_->has_alter_partitioning_steps = true;
   return this;
 }
 
-KuduTableAlterer* KuduTableAlterer::DropRangePartition(KuduPartialRow* lower_bound,
-                                                       KuduPartialRow* upper_bound) {
+KuduTableAlterer* KuduTableAlterer::DropRangePartition(
+    KuduPartialRow* lower_bound,
+    KuduPartialRow* upper_bound,
+    KuduTableCreator::RangePartitionBound lower_bound_type,
+    KuduTableCreator::RangePartitionBound upper_bound_type) {
   if (lower_bound == nullptr || upper_bound == nullptr) {
     data_->status_ = Status::InvalidArgument("range partition bounds may not be null");
   }
@@ -887,7 +913,9 @@ KuduTableAlterer* KuduTableAlterer::DropRangePartition(KuduPartialRow* lower_bou
   Data::Step s { AlterTableRequestPB::DROP_RANGE_PARTITION,
                  nullptr,
                  unique_ptr<KuduPartialRow>(lower_bound),
-                 unique_ptr<KuduPartialRow>(upper_bound) };
+                 unique_ptr<KuduPartialRow>(upper_bound),
+                 lower_bound_type,
+                 upper_bound_type };
   data_->steps_.emplace_back(std::move(s));
   data_->has_alter_partitioning_steps = true;
   return this;

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 44843a3..dbd0f6b 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -510,29 +510,18 @@ class KUDU_EXPORT KuduTableCreator {
   /// @return Reference to the modified table creator.
   KuduTableCreator& set_range_partition_columns(const std::vector<std::string>& columns);
 
-  /// Add a range partition split at the provided row.
-  ///
-  /// @param [in] split_row
-  ///   The row to use for partitioning. If the row is missing a value
-  ///   for any of the range partition columns, the logical minimum value
-  ///   for that column type will be used by default.
-  ///   The KuduObjectCreator object takes ownership of the parameter.
-  /// @return Reference to the modified table creator.
-  KuduTableCreator& add_range_split(KuduPartialRow* split_row);
-
-  /// @deprecated Use add_range_split() instead.
-  ///
-  /// @param [in] split_rows
-  ///   The row to use for partitioning.
-  /// @return Reference to the modified table creator.
-  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows);
+  /// Range partition bound type.
+  enum RangePartitionBound {
+    EXCLUSIVE_BOUND, ///< An exclusive bound.
+    INCLUSIVE_BOUND, ///< An inclusive bound.
+  };
 
-  /// Add a partition range bound to the table with an inclusive lower bound
-  /// and exclusive upper bound.
+  /// Add a range partition to the table.
   ///
-  /// Multiple range bounds may be added, but they must not overlap. All split
-  /// rows must fall in one of the range bounds. The lower bound must be less
-  /// than the upper bound.
+  /// Multiple range partitions may be added, but they must not overlap.  All
+  /// range splits specified by @c add_range_partition_split must fall in a
+  /// range partition. The lower bound must be less than or equal to the upper
+  /// bound.
   ///
   /// If this method is not called, the table's range will be unbounded.
   ///
@@ -545,12 +534,37 @@ class KUDU_EXPORT KuduTableCreator {
   /// @param [in] upper_bound
   ///   Row to use as an upper bound. The KuduTableCreator instance takes
   ///   ownership of this parameter. If row is empty, no upper bound is imposed
-  ///   on the table range. If a column of the @c lower_bound row is missing
+  ///   on the table range. If a column of the @c upper_bound row is missing
   ///   a value, the logical maximum value for that column type is used as the
   ///   default.
+  /// @param [in] lower_bound_type
+  ///   The type of the lower bound, either inclusive or exclusive. Defaults to
+  ///   inclusive.
+  /// @param [in] upper_bound_type
+  ///   The type of the lower bound, either inclusive or exclusive. Defaults to
+  ///   exclusive.
   /// @return Reference to the modified table creator.
-  KuduTableCreator& add_range_bound(KuduPartialRow* lower_bound,
-                                    KuduPartialRow* upper_bound);
+  KuduTableCreator& add_range_partition(KuduPartialRow* lower_bound,
+                                        KuduPartialRow* upper_bound,
+                                        RangePartitionBound lower_bound_type = INCLUSIVE_BOUND,
+                                        RangePartitionBound upper_bound_type = EXCLUSIVE_BOUND);
+
+  /// Add a range partition split at the provided row.
+  ///
+  /// @param [in] split_row
+  ///   The row to use for partitioning. If the row is missing a value
+  ///   for any of the range partition columns, the logical minimum value
+  ///   for that column type will be used by default.
+  ///   The KuduTableCreator object takes ownership of the parameter.
+  /// @return Reference to the modified table creator.
+  KuduTableCreator& add_range_partition_split(KuduPartialRow* split_row);
+
+  /// @deprecated Use @c add_range_partition_split() instead.
+  ///
+  /// @param [in] split_rows
+  ///   The row to use for partitioning.
+  /// @return Reference to the modified table creator.
+  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows);
 
   /// Set the table replication factor.
   ///
@@ -773,8 +787,8 @@ class KUDU_EXPORT KuduTableAlterer {
   /// @return Raw pointer to this alterer object.
   KuduTableAlterer* DropColumn(const std::string& name);
 
-  /// Add a range partition to the table with an inclusive lower bound and
-  /// exclusive upper bound.
+  /// Add a range partition to the table with the specified lower bound and
+  /// upper bound.
   ///
   /// @note The table alterer takes ownership of the rows.
   ///
@@ -789,22 +803,30 @@ class KUDU_EXPORT KuduTableAlterer {
   ///   defaults to one hour.
   ///
   /// @param [in] lower_bound
-  ///   The inclusive lower bound of the range partition to add. If the row is
-  ///   empty, then the lower bound is unbounded. If any of the columns are
-  ///   unset, the logical minimum value for the column's type will be used by
-  ///   default.
+  ///   The lower bound of the range partition to add. If the row is empty, then
+  ///   the lower bound is unbounded. If any of the columns are unset, the
+  ///   logical minimum value for the column's type will be used by default.
   /// @param [in] upper_bound
-  ///   The exclusive upper bound of the range partition to add. If the row is
-  ///   empty, then the upper bound is unbounded. If any of the individual
-  ///   columns are unset, the logical minimum value for the column' type will
-  ///   be used by default.
+  ///   The upper bound of the range partition to add. If the row is empty, then
+  ///   the upper bound is unbounded. If any of the individual columns are
+  ///   unset, the logical minimum value for the column' type will be used by
+  ///   default.
+  /// @param [in] lower_bound_type
+  ///   The type of the lower bound, either inclusive or exclusive. Defaults to
+  ///   inclusive.
+  /// @param [in] upper_bound_type
+  ///   The type of the lower bound, either inclusive or exclusive. Defaults to
+  ///   exclusive.
   /// @return Raw pointer to this alterer object.
-  KuduTableAlterer* AddRangePartition(KuduPartialRow* lower_bound,
-                                      KuduPartialRow* upper_bound);
+  KuduTableAlterer* AddRangePartition(
+      KuduPartialRow* lower_bound,
+      KuduPartialRow* upper_bound,
+      KuduTableCreator::RangePartitionBound lower_bound_type = KuduTableCreator::INCLUSIVE_BOUND,
+      KuduTableCreator::RangePartitionBound upper_bound_type = KuduTableCreator::EXCLUSIVE_BOUND);
 
-  /// Drop the range partition from the table with the specified inclusive lower
-  /// bound and exclusive upper bound. The bounds must match an existing range
-  /// partition exactly, and may not span multiple range partitions.
+  /// Drop the range partition from the table with the specified lower bound and
+  /// upper bound. The bounds must match an existing range partition exactly,
+  /// and may not span multiple range partitions.
   ///
   /// @note The table alterer takes ownership of the rows.
   ///
@@ -822,9 +844,18 @@ class KUDU_EXPORT KuduTableAlterer {
   ///   empty, then the upper bound is unbounded. If any of the individual
   ///   columns are unset, the logical minimum value for the column' type will
   ///   be used by default.
+  /// @param [in] lower_bound_type
+  ///   The type of the lower bound, either inclusive or exclusive. Defaults to
+  ///   inclusive.
+  /// @param [in] upper_bound_type
+  ///   The type of the lower bound, either inclusive or exclusive. Defaults to
+  ///   exclusive.
   /// @return Raw pointer to this alterer object.
-  KuduTableAlterer* DropRangePartition(KuduPartialRow* lower_bound,
-                                       KuduPartialRow* upper_bound);
+  KuduTableAlterer* DropRangePartition(
+      KuduPartialRow* lower_bound,
+      KuduPartialRow* upper_bound,
+      KuduTableCreator::RangePartitionBound lower_bound_type = KuduTableCreator::INCLUSIVE_BOUND,
+      KuduTableCreator::RangePartitionBound upper_bound_type = KuduTableCreator::EXCLUSIVE_BOUND);
 
   /// Set a timeout for the alteration operation.
   ///

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/client/table_alterer-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/table_alterer-internal.cc b/src/kudu/client/table_alterer-internal.cc
index 3eab03d..e229cae 100644
--- a/src/kudu/client/table_alterer-internal.cc
+++ b/src/kudu/client/table_alterer-internal.cc
@@ -111,16 +111,36 @@ Status KuduTableAlterer::Data::ToRequest(AlterTableRequestPB* req) {
       {
         RowOperationsPBEncoder encoder(pb_step->mutable_add_range_partition()
                                               ->mutable_range_bounds());
-        encoder.Add(RowOperationsPB::RANGE_LOWER_BOUND, *s.lower_bound);
-        encoder.Add(RowOperationsPB::RANGE_UPPER_BOUND, *s.upper_bound);
+        RowOperationsPB_Type lower_bound_type =
+          s.lower_bound_type == KuduTableCreator::INCLUSIVE_BOUND ?
+          RowOperationsPB::RANGE_LOWER_BOUND :
+          RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND;
+
+        RowOperationsPB_Type upper_bound_type =
+          s.upper_bound_type == KuduTableCreator::EXCLUSIVE_BOUND ?
+          RowOperationsPB::RANGE_UPPER_BOUND :
+          RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND;
+
+        encoder.Add(lower_bound_type, *s.lower_bound);
+        encoder.Add(upper_bound_type, *s.upper_bound);
         break;
       }
       case AlterTableRequestPB::DROP_RANGE_PARTITION:
       {
         RowOperationsPBEncoder encoder(pb_step->mutable_drop_range_partition()
                                               ->mutable_range_bounds());
-        encoder.Add(RowOperationsPB::RANGE_LOWER_BOUND, *s.lower_bound);
-        encoder.Add(RowOperationsPB::RANGE_UPPER_BOUND, *s.upper_bound);
+        RowOperationsPB_Type lower_bound_type =
+          s.lower_bound_type == KuduTableCreator::INCLUSIVE_BOUND ?
+          RowOperationsPB::RANGE_LOWER_BOUND :
+          RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND;
+
+        RowOperationsPB_Type upper_bound_type =
+          s.upper_bound_type == KuduTableCreator::EXCLUSIVE_BOUND ?
+          RowOperationsPB::RANGE_UPPER_BOUND :
+          RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND;
+
+        encoder.Add(lower_bound_type, *s.lower_bound);
+        encoder.Add(upper_bound_type, *s.upper_bound);
         break;
       }
       default:

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/client/table_alterer-internal.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/table_alterer-internal.h b/src/kudu/client/table_alterer-internal.h
index 4188baa..bf7cdc6 100644
--- a/src/kudu/client/table_alterer-internal.h
+++ b/src/kudu/client/table_alterer-internal.h
@@ -58,6 +58,8 @@ class KuduTableAlterer::Data {
     // [ADD|DROP]_RANGE_PARTITION.
     std::unique_ptr<KuduPartialRow> lower_bound;
     std::unique_ptr<KuduPartialRow> upper_bound;
+    KuduTableCreator::RangePartitionBound lower_bound_type;
+    KuduTableCreator::RangePartitionBound upper_bound_type;
   };
   std::vector<Step> steps_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/client/table_creator-internal.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/table_creator-internal.h b/src/kudu/client/table_creator-internal.h
index fc519ec..7a6753d 100644
--- a/src/kudu/client/table_creator-internal.h
+++ b/src/kudu/client/table_creator-internal.h
@@ -40,10 +40,16 @@ class KuduTableCreator::Data {
 
   const KuduSchema* schema_;
 
-  std::vector<std::unique_ptr<KuduPartialRow>> range_splits_;
+  std::vector<std::unique_ptr<KuduPartialRow>> range_partition_splits_;
 
-  std::vector<std::pair<std::unique_ptr<KuduPartialRow>,
-                         std::unique_ptr<KuduPartialRow>>> range_bounds_;
+  struct RangePartition {
+    std::unique_ptr<KuduPartialRow> lower_bound;
+    std::unique_ptr<KuduPartialRow> upper_bound;
+    RangePartitionBound lower_bound_type;
+    RangePartitionBound upper_bound_type;
+  };
+
+  std::vector<RangePartition> range_partition_bounds_;
 
   PartitionSchemaPB partition_schema_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/common/partition-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition-test.cc b/src/kudu/common/partition-test.cc
index 1600e3f..26e3e9a 100644
--- a/src/kudu/common/partition-test.cc
+++ b/src/kudu/common/partition-test.cc
@@ -28,6 +28,7 @@
 #include "kudu/common/row.h"
 #include "kudu/common/schema.h"
 #include "kudu/gutil/strings/escaping.h"
+#include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/hash_util.h"
 #include "kudu/util/test_util.h"
@@ -111,11 +112,12 @@ void CheckCreateRangePartitions(const vector<pair<optional<string>, optional<str
     EXPECT_EQ(lower, partitions[i].partition_key_start());
     EXPECT_EQ(upper, partitions[i].partition_key_end());
 
-    EXPECT_EQ(strings::Substitute("range: [($0$1), ($2$3))",
-                                  lower.empty() ? "" : "string col=",
-                                  lower.empty() ? "<start>" : strings::Utf8SafeCEscape(lower),
-                                  upper.empty() ? "" : "string col=",
-                                  upper.empty() ? "<end>" : strings::Utf8SafeCEscape(upper)),
+    string lower_debug = lower.empty() ? "<start>" :
+      strings::Substitute("(string col=$0)", strings::Utf8SafeCEscape(lower));
+    string upper_debug = upper.empty() ? "<end>" :
+      strings::Substitute("(string col=$0)", strings::Utf8SafeCEscape(upper));
+
+    EXPECT_EQ(strings::Substitute("range: [$0, $1)", lower_debug, upper_debug),
               partition_schema.PartitionDebugString(partitions[i], schema));
   }
 }
@@ -180,7 +182,7 @@ TEST(PartitionTest, TestCompoundRangeKeyEncoding) {
 
   EXPECT_TRUE(partitions[0].hash_buckets().empty());
 
-  EXPECT_EQ("range: [(<start>), (string c1=, string c2=, string c3=a))",
+  EXPECT_EQ("range: [<start>, (string c1=, string c2=, string c3=a))",
             partition_schema.PartitionDebugString(partitions[0], schema));
   EXPECT_EQ("range: [(string c1=, string c2=, string c3=a), "
                     "(string c1=, string c2=, string c3=b))",
@@ -188,7 +190,7 @@ TEST(PartitionTest, TestCompoundRangeKeyEncoding) {
   EXPECT_EQ("range: [(string c1=, string c2=b, string c3=c), "
             "(string c1=d, string c2=, string c3=f))",
             partition_schema.PartitionDebugString(partitions[2], schema));
-  EXPECT_EQ("range: [(string c1=e, string c2=, string c3=), (<end>))",
+  EXPECT_EQ("range: [(string c1=e, string c2=, string c3=), <end>)",
             partition_schema.PartitionDebugString(partitions[3], schema));
 }
 
@@ -510,7 +512,7 @@ TEST(PartitionTest, TestCreatePartitions) {
   EXPECT_EQ(string("a1\0\0b1\0\0c1", 10), partitions[0].range_key_end());
   EXPECT_EQ(string("", 0), partitions[0].partition_key_start());
   EXPECT_EQ(string("\0\0\0\0" "\0\0\0\0" "a1\0\0b1\0\0c1", 18), partitions[0].partition_key_end());
-  EXPECT_EQ("hash buckets: (0, 0), range: [(<start>), (string a=a1, string b=b1, string c=c1))",
+  EXPECT_EQ("hash buckets: (0, 0), range: [<start>, (string a=a1, string b=b1, string c=c1))",
             partition_schema.PartitionDebugString(partitions[0], schema));
 
   EXPECT_EQ(0, partitions[1].hash_buckets()[0]);
@@ -531,7 +533,7 @@ TEST(PartitionTest, TestCreatePartitions) {
   EXPECT_EQ(string("\0\0\0\0" "\0\0\0\0" "a2\0\0b2\0\0", 16), partitions[2].partition_key_start());
   EXPECT_EQ(string("\0\0\0\0" "\0\0\0\1", 8), partitions[2].partition_key_end());
   EXPECT_EQ("hash buckets: (0, 0), "
-            "range: [(string a=a2, string b=b2, string c=), (<end>))",
+            "range: [(string a=a2, string b=b2, string c=), <end>)",
             partition_schema.PartitionDebugString(partitions[2], schema));
 
   EXPECT_EQ(0, partitions[3].hash_buckets()[0]);
@@ -541,7 +543,7 @@ TEST(PartitionTest, TestCreatePartitions) {
   EXPECT_EQ(string("\0\0\0\0" "\0\0\0\1", 8), partitions[3].partition_key_start());
   EXPECT_EQ(string("\0\0\0\0" "\0\0\0\1" "a1\0\0b1\0\0c1", 18), partitions[3].partition_key_end());
   EXPECT_EQ("hash buckets: (0, 1), "
-            "range: [(<start>), (string a=a1, string b=b1, string c=c1))",
+            "range: [<start>, (string a=a1, string b=b1, string c=c1))",
             partition_schema.PartitionDebugString(partitions[3], schema));
 
   EXPECT_EQ(0, partitions[4].hash_buckets()[0]);
@@ -561,7 +563,7 @@ TEST(PartitionTest, TestCreatePartitions) {
   EXPECT_EQ(string("", 0), partitions[5].range_key_end());
   EXPECT_EQ(string("\0\0\0\0" "\0\0\0\1" "a2\0\0b2\0\0", 16), partitions[5].partition_key_start());
   EXPECT_EQ(string("\0\0\0\1", 4), partitions[5].partition_key_end());
-  EXPECT_EQ("hash buckets: (0, 1), range: [(string a=a2, string b=b2, string c=), (<end>))",
+  EXPECT_EQ("hash buckets: (0, 1), range: [(string a=a2, string b=b2, string c=), <end>)",
             partition_schema.PartitionDebugString(partitions[5], schema));
 
   EXPECT_EQ(1, partitions[6].hash_buckets()[0]);
@@ -570,7 +572,7 @@ TEST(PartitionTest, TestCreatePartitions) {
   EXPECT_EQ(string("a1\0\0b1\0\0c1", 10), partitions[6].range_key_end());
   EXPECT_EQ(string("\0\0\0\1", 4), partitions[6].partition_key_start());
   EXPECT_EQ(string("\0\0\0\1" "\0\0\0\0" "a1\0\0b1\0\0c1", 18), partitions[6].partition_key_end());
-  EXPECT_EQ("hash buckets: (1, 0), range: [(<start>), (string a=a1, string b=b1, string c=c1))",
+  EXPECT_EQ("hash buckets: (1, 0), range: [<start>, (string a=a1, string b=b1, string c=c1))",
             partition_schema.PartitionDebugString(partitions[6], schema));
 
   EXPECT_EQ(1, partitions[7].hash_buckets()[0]);
@@ -590,7 +592,7 @@ TEST(PartitionTest, TestCreatePartitions) {
   EXPECT_EQ(string("", 0), partitions[8].range_key_end());
   EXPECT_EQ(string("\0\0\0\1" "\0\0\0\0" "a2\0\0b2\0\0", 16), partitions[8].partition_key_start());
   EXPECT_EQ(string("\0\0\0\1" "\0\0\0\1", 8), partitions[8].partition_key_end());
-  EXPECT_EQ("hash buckets: (1, 0), range: [(string a=a2, string b=b2, string c=), (<end>))",
+  EXPECT_EQ("hash buckets: (1, 0), range: [(string a=a2, string b=b2, string c=), <end>)",
             partition_schema.PartitionDebugString(partitions[8], schema));
 
   EXPECT_EQ(1, partitions[9].hash_buckets()[0]);
@@ -599,7 +601,7 @@ TEST(PartitionTest, TestCreatePartitions) {
   EXPECT_EQ(string("a1\0\0b1\0\0c1", 10), partitions[9].range_key_end());
   EXPECT_EQ(string("\0\0\0\1" "\0\0\0\1", 8), partitions[9].partition_key_start());
   EXPECT_EQ(string("\0\0\0\1" "\0\0\0\1" "a1\0\0b1\0\0c1", 18), partitions[9].partition_key_end());
-  EXPECT_EQ("hash buckets: (1, 1), range: [(<start>), (string a=a1, string b=b1, string c=c1))",
+  EXPECT_EQ("hash buckets: (1, 1), range: [<start>, (string a=a1, string b=b1, string c=c1))",
             partition_schema.PartitionDebugString(partitions[9], schema));
 
   EXPECT_EQ(1, partitions[10].hash_buckets()[0]);
@@ -619,8 +621,155 @@ TEST(PartitionTest, TestCreatePartitions) {
   EXPECT_EQ(string("", 0), partitions[11].range_key_end());
   EXPECT_EQ(string("\0\0\0\1" "\0\0\0\1" "a2\0\0b2\0\0", 16), partitions[11].partition_key_start());
   EXPECT_EQ(string("", 0), partitions[11].partition_key_end());
-  EXPECT_EQ("hash buckets: (1, 1), range: [(string a=a2, string b=b2, string c=), (<end>))",
+  EXPECT_EQ("hash buckets: (1, 1), range: [(string a=a2, string b=b2, string c=), <end>)",
             partition_schema.PartitionDebugString(partitions[11], schema));
 }
 
+TEST(PartitionTest, TestIncrementRangePartitionBounds) {
+  // CREATE TABLE t (a INT8, b INT8, c INT8, PRIMARY KEY (a, b, c))
+  // PARITITION BY RANGE (a, b, c);
+  Schema schema({ ColumnSchema("c1", INT8),
+                  ColumnSchema("c2", INT8),
+                  ColumnSchema("c3", INT8) },
+                { ColumnId(0), ColumnId(1), ColumnId(2) }, 3);
+
+  PartitionSchemaPB schema_builder;
+  PartitionSchema partition_schema;
+  ASSERT_OK(PartitionSchema::FromPB(schema_builder, schema, &partition_schema));
+
+  vector<vector<boost::optional<int8_t>>> tests {
+    // Big list of test cases. First three columns are the input columns, final
+    // three columns are the expected output columns. For example,
+    { 1, 2, 3, 1, 2, 4 },
+    // corresponds to the test case:
+    // (1, 2, 3) -> (1, 2, 4)
+
+    { 1, 2, boost::none, 1, 2, -127 },
+    { 1, boost::none, 3, 1, boost::none, 4 },
+    { boost::none, 2, 3, boost::none, 2, 4 },
+    { 1, boost::none, boost::none, 1, boost::none, -127 },
+    { boost::none, boost::none, 3, boost::none, boost::none, 4 },
+    { boost::none, 2, boost::none, boost::none, 2, -127 },
+    { 1, 2, 127, 1, 3, boost::none },
+    { 1, 127, 3, 1, 127, 4},
+    { 1, 127, 127, 2, boost::none, boost::none },
+  };
+
+  auto check = [&] (const vector<boost::optional<int8_t>>& test, bool lower_bound) {
+    CHECK_EQ(6, test.size());
+    KuduPartialRow bound(&schema);
+    if (test[0]) ASSERT_OK(bound.SetInt8("c1", *test[0]));
+    if (test[1]) ASSERT_OK(bound.SetInt8("c2", *test[1]));
+    if (test[2]) ASSERT_OK(bound.SetInt8("c3", *test[2]));
+
+    vector<string> components;
+    partition_schema.AppendRangeDebugStringComponentsOrMin(bound, &components);
+    SCOPED_TRACE(JoinStrings(components, ", "));
+    SCOPED_TRACE(lower_bound ? "lower bound" : "upper bound");
+
+    if (lower_bound) {
+      ASSERT_OK(partition_schema.MakeLowerBoundRangePartitionKeyInclusive(&bound));
+    } else {
+      ASSERT_OK(partition_schema.MakeUpperBoundRangePartitionKeyExclusive(&bound));
+    }
+
+    int8_t val;
+    if (test[3]) {
+      ASSERT_OK(bound.GetInt8("c1", &val));
+      ASSERT_EQ(*test[3], val);
+    } else {
+      ASSERT_FALSE(bound.IsColumnSet("c1"));
+    }
+    if (test[4]) {
+      ASSERT_OK(bound.GetInt8("c2", &val));
+      ASSERT_EQ(*test[4], val);
+    } else {
+      ASSERT_FALSE(bound.IsColumnSet("c2"));
+    }
+    if (test[5]) {
+      ASSERT_OK(bound.GetInt8("c3", &val));
+      ASSERT_EQ(*test[5], val);
+    } else {
+      ASSERT_FALSE(bound.IsColumnSet("c3"));
+    }
+  };
+
+  for (const auto& test : tests) {
+    check(test, true);
+    check(test, false);
+  }
+
+  // Special cases:
+  // lower bound: (_, _, _) -> (_, _, -127)
+  check({ boost::none, boost::none, boost::none, boost::none, boost::none, -127 }, true);
+  // upper bound: (_, _, _) -> (_, _, _)
+  check({ boost::none, boost::none, boost::none, boost::none, boost::none, boost::none }, false);
+  // upper bound: (127, 127, 127) -> (_, _, _)
+  check({ 127, 127, 127, boost::none, boost::none, boost::none }, false);
+
+  // lower bound: (127, 127, 127) -> fail!
+    KuduPartialRow lower_bound(&schema);
+    ASSERT_OK(lower_bound.SetInt8("c1", 127));
+    ASSERT_OK(lower_bound.SetInt8("c2", 127));
+    ASSERT_OK(lower_bound.SetInt8("c3", 127));
+    Status s = partition_schema.MakeLowerBoundRangePartitionKeyInclusive(&lower_bound);
+    ASSERT_EQ("Invalid argument: Exclusive lower bound range partition key must not have "
+              "maximum values for all components: int8 c1=127, int8 c2=127, int8 c3=127",
+              s.ToString());
+}
+
+TEST(PartitionTest, TestIncrementRangePartitionStringBounds) {
+  // CREATE TABLE t (a STRING, b STRING, PRIMARY KEY (a, b))
+  // PARITITION BY RANGE (a, b, c);
+  Schema schema({ ColumnSchema("c1", STRING),
+                  ColumnSchema("c2", STRING) },
+                { ColumnId(0), ColumnId(1) }, 2);
+
+  PartitionSchemaPB schema_builder;
+  PartitionSchema partition_schema;
+  ASSERT_OK(PartitionSchema::FromPB(schema_builder, schema, &partition_schema));
+
+  vector<vector<boost::optional<string>>> tests {
+    { string("a"), string("b"), string("a"), string("b\0", 2) },
+    { string("a"), boost::none, string("a"), string("\0", 1) },
+  };
+
+  auto check = [&] (const vector<boost::optional<string>>& test, bool lower_bound) {
+    CHECK_EQ(4, test.size());
+    KuduPartialRow bound(&schema);
+    if (test[0]) ASSERT_OK(bound.SetString("c1", *test[0]));
+    if (test[1]) ASSERT_OK(bound.SetString("c2", *test[1]));
+
+    vector<string> components;
+    partition_schema.AppendRangeDebugStringComponentsOrMin(bound, &components);
+    SCOPED_TRACE(JoinStrings(components, ", "));
+    SCOPED_TRACE(lower_bound ? "lower bound" : "upper bound");
+
+    if (lower_bound) {
+      ASSERT_OK(partition_schema.MakeLowerBoundRangePartitionKeyInclusive(&bound));
+    } else {
+      ASSERT_OK(partition_schema.MakeUpperBoundRangePartitionKeyExclusive(&bound));
+    }
+
+    Slice val;
+    if (test[2]) {
+      ASSERT_OK(bound.GetString("c1", &val));
+      ASSERT_EQ(*test[2], val);
+    } else {
+      ASSERT_FALSE(bound.IsColumnSet("c1"));
+    }
+    if (test[3]) {
+      ASSERT_OK(bound.GetString("c2", &val));
+      ASSERT_EQ(*test[3], val);
+    } else {
+      ASSERT_FALSE(bound.IsColumnSet("c2"));
+    }
+  };
+
+  for (const auto& test : tests) {
+    check(test, true);
+    check(test, false);
+  }
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/common/partition.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition.cc b/src/kudu/common/partition.cc
index e89000f..b3b31c7 100644
--- a/src/kudu/common/partition.cc
+++ b/src/kudu/common/partition.cc
@@ -262,10 +262,8 @@ Status PartitionSchema::EncodeRangeBounds(const vector<pair<KuduPartialRow,
 
     if (!lower.empty() && !upper.empty() && lower >= upper) {
       return Status::InvalidArgument(
-          "range bound has lower bound equal to or above the upper bound",
-          strings::Substitute("lower bound: ($0), upper bound: ($1)",
-                              bound.first.ToString(),
-                              bound.second.ToString()));
+          "range partition lower bound must be less than or equal to the upper bound",
+          RangePartitionDebugString(bound.first, bound.second));
     }
     range_partitions->emplace_back(std::move(lower), std::move(upper));
   }
@@ -278,12 +276,14 @@ Status PartitionSchema::EncodeRangeBounds(const vector<pair<KuduPartialRow,
 
     if (first_upper.empty() || second_lower.empty() || first_upper > second_lower) {
       return Status::InvalidArgument(
-          "overlapping range bounds",
-          strings::Substitute("first bound: [($0), ($1)), second bound: [($2), ($3))",
-                              RangeKeyDebugString(range_partitions->at(i).first, schema),
-                              RangeKeyDebugString(range_partitions->at(i).second, schema),
-                              RangeKeyDebugString(range_partitions->at(i + 1).first, schema),
-                              RangeKeyDebugString(range_partitions->at(i + 1).second, schema)));
+          "overlapping range partitions",
+          strings::Substitute("first range partition: $0, second range partition: $1",
+                              RangePartitionDebugString(range_partitions->at(i).first,
+                                                        range_partitions->at(i).second,
+                                                        schema),
+                              RangePartitionDebugString(range_partitions->at(i + 1).first,
+                                                        range_partitions->at(i + 1).second,
+                                                        schema)));
     }
   }
 
@@ -545,44 +545,11 @@ string PartitionSchema::PartitionDebugString(const Partition& partition,
   }
 
   if (!range_schema_.column_ids.empty()) {
-    Arena arena(1024, 128 * 1024);
-    KuduPartialRow start_row(&schema);
-    KuduPartialRow end_row(&schema);
-
-    s.append("range: [(");
-
-    Slice encoded_range_key_start = partition.range_key_start();
-    Status status;
-    status = DecodeRangeKey(&encoded_range_key_start, &start_row, &arena);
-    if (status.ok()) {
-      if (IsRangePartitionKeyEmpty(start_row)) {
-        s.append("<start>");
-      } else {
-        vector<string> start_components;
-        AppendRangeDebugStringComponentsOrMin(start_row, &start_components);
-        s.append(JoinStrings(start_components, ", "));
-      }
-    } else {
-      s.append(Substitute("<decode-error: $0>", status.ToString()));
-    }
-    s.append("), (");
-
-    Slice encoded_range_key_end = partition.range_key_end();
-    status = DecodeRangeKey(&encoded_range_key_end, &end_row, &arena);
-    if (status.ok()) {
-      if (IsRangePartitionKeyEmpty(end_row)) {
-        s.append("<end>");
-      } else {
-        vector<string> end_components;
-        AppendRangeDebugStringComponentsOrMin(end_row, &end_components);
-        s.append(JoinStrings(end_components, ", "));
-      }
-    } else {
-      s.append(Substitute("<decode-error: $0>", status.ToString()));
-    }
-    s.append("))");
+    s.append("range: ");
+    s.append(RangePartitionDebugString(partition.range_key_start().ToString(),
+                                       partition.range_key_end().ToString(),
+                                       schema));
   }
-
   return s;
 }
 
@@ -696,6 +663,55 @@ string PartitionSchema::PartitionKeyDebugString(const string& key, const Schema&
   return JoinStrings(components, ", ");
 }
 
+string PartitionSchema::RangePartitionDebugString(const KuduPartialRow& lower_bound,
+                                                  const KuduPartialRow& upper_bound) const {
+  string out("[");
+  if (IsRangePartitionKeyEmpty(lower_bound)) {
+    out.append("<start>");
+  } else {
+    vector<string> components;
+    AppendRangeDebugStringComponentsOrMin(lower_bound, &components);
+    out.push_back('(');
+    out.append(JoinStrings(components, ", "));
+    out.push_back(')');
+  }
+  out.append(", ");
+  if (IsRangePartitionKeyEmpty(upper_bound)) {
+    out.append("<end>");
+  } else {
+    vector<string> components;
+    AppendRangeDebugStringComponentsOrMin(upper_bound, &components);
+    out.push_back('(');
+    out.append(JoinStrings(components, ", "));
+    out.push_back(')');
+  }
+  out.push_back(')');
+  return out;
+}
+
+string PartitionSchema::RangePartitionDebugString(const string& lower_bound,
+                                                  const string& upper_bound,
+                                                  const Schema& schema) const {
+  string out("[");
+  if (lower_bound.empty()) {
+    out.append("<start>");
+  } else {
+    out.push_back('(');
+    out.append(RangeKeyDebugString(lower_bound, schema));
+    out.push_back(')');
+  }
+  out.append(", ");
+  if (upper_bound.empty()) {
+    out.append("<end>");
+  } else {
+    out.push_back('(');
+    out.append(RangeKeyDebugString(upper_bound, schema));
+    out.push_back(')');
+  }
+  out.push_back(')');
+  return out;
+}
+
 string PartitionSchema::RangeKeyDebugString(const string& range_key, const Schema& schema) const {
   Arena arena(1024, 128 * 1024);
   KuduPartialRow row(&schema);
@@ -913,4 +929,217 @@ Status PartitionSchema::Validate(const Schema& schema) const {
   return Status::OK();
 }
 
+namespace {
+
+  // Increments an unset column in the row.
+  Status IncrementUnsetColumn(KuduPartialRow* row, int32_t idx) {
+    DCHECK(!row->IsColumnSet(idx));
+    switch (row->schema()->column(idx).type_info()->type()) {
+      case INT8:
+        RETURN_NOT_OK(row->SetInt8(idx, INT8_MIN + 1));
+        break;
+      case INT16:
+        RETURN_NOT_OK(row->SetInt16(idx, INT16_MIN + 1));
+        break;
+      case INT32:
+        RETURN_NOT_OK(row->SetInt32(idx, INT32_MIN + 1));
+        break;
+      case INT64:
+      case TIMESTAMP:
+        RETURN_NOT_OK(row->SetInt64(idx, INT64_MIN + 1));
+        break;
+      case STRING:
+        RETURN_NOT_OK(row->SetStringCopy(idx, Slice("\0", 1)));
+        break;
+      case BINARY:
+        RETURN_NOT_OK(row->SetBinaryCopy(idx, Slice("\0", 1)));
+        break;
+      default:
+        return Status::InvalidArgument("Invalid column type in range partition",
+                                       row->schema()->column(idx).ToString());
+    }
+    return Status::OK();
+  }
+
+  // Increments a column in the row, setting 'success' to true if the increment
+  // succeeds, or false if the column is already the maximum value.
+  Status IncrementColumn(KuduPartialRow* row, int32_t idx, bool* success) {
+    DCHECK(row->IsColumnSet(idx));
+    *success = true;
+    switch (row->schema()->column(idx).type_info()->type()) {
+      case INT8: {
+        int8_t value;
+        RETURN_NOT_OK(row->GetInt8(idx, &value));
+        if (value < INT8_MAX) {
+          RETURN_NOT_OK(row->SetInt8(idx, value + 1));
+        } else {
+          *success = false;
+        }
+        break;
+      }
+      case INT16: {
+        int16_t value;
+        RETURN_NOT_OK(row->GetInt16(idx, &value));
+        if (value < INT16_MAX) {
+          RETURN_NOT_OK(row->SetInt16(idx, value + 1));
+        } else {
+          *success = false;
+        }
+        break;
+      }
+      case INT32: {
+        int32_t value;
+        RETURN_NOT_OK(row->GetInt32(idx, &value));
+        if (value < INT32_MAX) {
+          RETURN_NOT_OK(row->SetInt32(idx, value + 1));
+        } else {
+          *success = false;
+        }
+        break;
+      }
+      case INT64:
+      case TIMESTAMP: {
+        int64_t value;
+        RETURN_NOT_OK(row->GetInt64(idx, &value));
+        if (value < INT64_MAX) {
+          RETURN_NOT_OK(row->SetInt64(idx, value + 1));
+        } else {
+          *success = false;
+        }
+        break;
+      }
+      case BINARY: {
+        Slice value;
+        RETURN_NOT_OK(row->GetBinary(idx, &value));
+        string incremented = value.ToString();
+        incremented.push_back('\0');
+        RETURN_NOT_OK(row->SetBinaryCopy(idx, incremented));
+        break;
+      }
+      case STRING: {
+        Slice value;
+        RETURN_NOT_OK(row->GetString(idx, &value));
+        string incremented = value.ToString();
+        incremented.push_back('\0');
+        RETURN_NOT_OK(row->SetStringCopy(idx, incremented));
+        break;
+      }
+      default:
+        return Status::InvalidArgument("Invalid column type in range partition",
+                                       row->schema()->column(idx).ToString());
+    }
+    return Status::OK();
+  }
+} // anonymous namespace
+
+Status PartitionSchema::IncrementRangePartitionKey(KuduPartialRow* row, bool* increment) const {
+  vector<int32_t> unset_idxs;
+  *increment = false;
+  for (auto itr = range_schema_.column_ids.rbegin();
+       itr != range_schema_.column_ids.rend(); ++itr) {
+    int32_t idx = row->schema()->find_column_by_id(*itr);
+    if (idx == Schema::kColumnNotFound) {
+      return Status::InvalidArgument(Substitute("range partition column ID $0 "
+                                                "not found in range partition key schema.",
+                                                *itr));
+    }
+
+    if (row->IsColumnSet(idx)) {
+      RETURN_NOT_OK(IncrementColumn(row, idx, increment));
+      if (*increment) break;
+    } else {
+      RETURN_NOT_OK(IncrementUnsetColumn(row, idx));
+      *increment = true;
+      break;
+    }
+    unset_idxs.push_back(idx);
+  }
+
+  if (*increment) {
+    for (int32_t idx : unset_idxs) {
+      RETURN_NOT_OK(row->Unset(idx));
+    }
+  }
+
+  return Status::OK();
+}
+
+Status PartitionSchema::MakeLowerBoundRangePartitionKeyInclusive(KuduPartialRow* row) const {
+  // To transform a lower bound range partition key from exclusive to inclusive,
+  // the key mut be incremented. To increment the key, start with the least
+  // significant column in the key (furthest right), and increment it.  If the
+  // increment fails because the value is already the maximum, move on to the
+  // next least significant column and attempt to increment it (and so on). When
+  // incrementing, an unset cell is equivalent to a cell set to the minimum
+  // value for its column (e.g. an unset Int8 column is incremented to -127
+  // (-2^7 + 1)). Finally, all columns less significant than the incremented
+  // column are unset (which means they are treated as the minimum value for
+  // that column). If all columns in the key are the maximum and can not be
+  // incremented, then the operation fails.
+  //
+  // A few examples, given a range partition of three Int8 columns. Underscore
+  // signifies unset:
+  //
+  // (1, 2, 3)       -> (1, 2, 4)
+  // (1, 2, 127)     -> (1, 3, _)
+  // (1, 127, 3)     -> (1, 127, 4)
+  // (1, _, 3)       -> (1, _, 4)
+  // (_, _, _)       -> (_, _, 1)
+  // (1, 127, 127)   -> (2, _, _)
+  // (127, 127, 127) -> fail!
+
+  bool increment;
+  RETURN_NOT_OK(IncrementRangePartitionKey(row, &increment));
+
+  if (!increment) {
+    vector<string> components;
+    AppendRangeDebugStringComponentsOrMin(*row, &components);
+    return Status::InvalidArgument("Exclusive lower bound range partition key must not "
+                                   "have maximum values for all components",
+                                   JoinStrings(components, ", "));
+  }
+
+  return Status::OK();
+}
+
+Status PartitionSchema::MakeUpperBoundRangePartitionKeyExclusive(KuduPartialRow* row) const {
+  // To transform an upper bound range partition key from inclusive to exclusive,
+  // the key must be incremented. Incrementing the key follows the same steps as
+  // turning an exclusive lower bound key into exclusive. Upper bound keys have
+  // two additional special cases:
+  //
+  // * For upper bound range partition keys with all columns unset, no
+  //   transformation is needed (all unset columns signifies unbounded,
+  //   so there is no difference between inclusive and exclusive).
+  //
+  // * For an upper bound key that can't be incremented because all components
+  //   are the maximum value, all columns are unset in order to transform it to
+  //   an unbounded upper bound (this is a special case increment).
+
+  bool all_unset = true;
+  for (ColumnId column_id : range_schema_.column_ids) {
+    int32_t idx = row->schema()->find_column_by_id(column_id);
+    if (idx == Schema::kColumnNotFound) {
+      return Status::InvalidArgument(Substitute("range partition column ID $0 "
+                                                "not found in range partition key schema.",
+                                                column_id));
+    }
+    all_unset = !row->IsColumnSet(idx);
+    if (!all_unset) break;
+  }
+
+  if (all_unset) return Status::OK();
+
+  bool increment;
+  RETURN_NOT_OK(IncrementRangePartitionKey(row, &increment));
+  if (!increment) {
+    for (ColumnId column_id : range_schema_.column_ids) {
+      int32_t idx = row->schema()->find_column_by_id(column_id);
+      RETURN_NOT_OK(row->Unset(idx));
+    }
+  }
+
+  return Status::OK();
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/common/partition.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/partition.h b/src/kudu/common/partition.h
index 347d809..41a6c24 100644
--- a/src/kudu/common/partition.h
+++ b/src/kudu/common/partition.h
@@ -178,6 +178,17 @@ class PartitionSchema {
   // Returns a text description of the encoded partition key suitable for debug printing.
   std::string PartitionKeyDebugString(const std::string& key, const Schema& schema) const;
 
+  // Returns a text description of the range partition with the provided
+  // inclusive lower bound and exclusive upper bound.
+  std::string RangePartitionDebugString(const KuduPartialRow& lower_bound,
+                                        const KuduPartialRow& upper_bound) const;
+
+  // Returns a text description of the range partition with the provided
+  // inclusive lower bound and exclusive upper bound.
+  std::string RangePartitionDebugString(const std::string& lower_bound,
+                                        const std::string& upper_bound,
+                                        const Schema& schema) const;
+
   // Returns a text description of the encoded range key suitable for debug printing.
   std::string RangeKeyDebugString(const std::string& range_key, const Schema& schema) const;
 
@@ -191,8 +202,18 @@ class PartitionSchema {
   // Returns true if the other partition schema is equivalent to this one.
   bool Equals(const PartitionSchema& other) const;
 
+  // Transforms an exclusive lower bound range partition key into an inclusive
+  // lower bound range partition key.
+  Status MakeLowerBoundRangePartitionKeyInclusive(KuduPartialRow* row) const;
+
+  // Transforms an inclusive upper bound range partition key into an exclusive
+  // upper bound range partition key.
+  Status MakeUpperBoundRangePartitionKeyExclusive(KuduPartialRow* row) const;
+
  private:
   friend class PartitionPruner;
+  FRIEND_TEST(PartitionTest, TestIncrementRangePartitionBounds);
+  FRIEND_TEST(PartitionTest, TestIncrementRangePartitionStringBounds);
 
   struct RangeSchema {
     std::vector<ColumnId> column_ids;
@@ -298,6 +319,11 @@ class PartitionSchema {
                           std::vector<std::string> splits,
                           std::vector<std::pair<std::string, std::string>>* bounds) const;
 
+  // Increments a range partition key, setting 'increment' to true if the
+  // increment succeeds, or false if all range partition columns are already the
+  // maximum value. Unset columns will be incremented to increment(min_value).
+  Status IncrementRangePartitionKey(KuduPartialRow* row, bool* increment) const;
+
   std::vector<HashBucketSchema> hash_bucket_schemas_;
   RangeSchema range_schema_;
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/common/row_operations-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_operations-test.cc b/src/kudu/common/row_operations-test.cc
index ed0fcc5..2c61261 100644
--- a/src/kudu/common/row_operations-test.cc
+++ b/src/kudu/common/row_operations-test.cc
@@ -96,7 +96,7 @@ void RowOperationsTest::CheckDecodeDoesntCrash(const Schema& client_schema,
 void RowOperationsTest::DoFuzzTest(const Schema& server_schema,
                                    const KuduPartialRow& row,
                                    int n_random_changes) {
-  for (int operation = 0; operation <= 4; operation++) {
+  for (int operation = 0; operation <= 8; operation++) {
     RowOperationsPB pb;
     RowOperationsPBEncoder enc(&pb);
 
@@ -116,6 +116,18 @@ void RowOperationsTest::DoFuzzTest(const Schema& server_schema,
       case 4:
         enc.Add(RowOperationsPB::SPLIT_ROW, row);
         break;
+      case 5:
+        enc.Add(RowOperationsPB::RANGE_LOWER_BOUND, row);
+        break;
+      case 6:
+        enc.Add(RowOperationsPB::RANGE_UPPER_BOUND, row);
+        break;
+      case 7:
+        enc.Add(RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND, row);
+        break;
+      case 8:
+        enc.Add(RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND, row);
+        break;
     }
 
     const Schema* client_schema = row.schema();

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/common/row_operations.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_operations.cc b/src/kudu/common/row_operations.cc
index af3a40b..17effcd 100644
--- a/src/kudu/common/row_operations.cc
+++ b/src/kudu/common/row_operations.cc
@@ -33,6 +33,8 @@ namespace kudu {
 
 string DecodedRowOperation::ToString(const Schema& schema) const {
   switch (type) {
+    case RowOperationsPB::UNKNOWN:
+      return "UNKNOWN";
     case RowOperationsPB::INSERT:
       return "INSERT " + schema.DebugRow(ConstContiguousRow(&schema, row_data));
     case RowOperationsPB::UPSERT:
@@ -48,9 +50,10 @@ string DecodedRowOperation::ToString(const Schema& schema) const {
       return Substitute("RANGE_LOWER_BOUND $0", split_row->ToString());
     case RowOperationsPB::RANGE_UPPER_BOUND:
       return Substitute("RANGE_UPPER_BOUND $0", split_row->ToString());
-    default:
-      LOG(DFATAL) << "Bad type: " << type;
-      return "<bad row operation>";
+    case RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND:
+      return Substitute("EXCLUSIVE_RANGE_LOWER_BOUND $0", split_row->ToString());
+    case RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND:
+      return Substitute("INCLUSIVE_RANGE_UPPER_BOUND $0", split_row->ToString());
   }
 }
 
@@ -578,6 +581,8 @@ Status RowOperationsPBDecoder::DecodeOperations(vector<DecodedRowOperation>* ops
       case RowOperationsPB::SPLIT_ROW:
       case RowOperationsPB::RANGE_LOWER_BOUND:
       case RowOperationsPB::RANGE_UPPER_BOUND:
+      case RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND:
+      case RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND:
         RETURN_NOT_OK(DecodeSplitRow(mapping, &op));
         break;
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/common/wire_protocol.proto
----------------------------------------------------------------------
diff --git a/src/kudu/common/wire_protocol.proto b/src/kudu/common/wire_protocol.proto
index 387439d..e1d2784 100644
--- a/src/kudu/common/wire_protocol.proto
+++ b/src/kudu/common/wire_protocol.proto
@@ -160,6 +160,14 @@ message RowOperationsPB {
     // Should be preceded by the associated lower bound. If all values are
     // missing, then signifies unbounded.
     RANGE_UPPER_BOUND = 7;
+    // Used when specifying an exclusive lower bound range on table creation.
+    // Should be followed by the associated upper bound. If all values are
+    // missing, then signifies unbounded.
+    EXCLUSIVE_RANGE_LOWER_BOUND = 8;
+    // Used when specifying an inclusive upper bound range on table creation.
+    // Should be preceded by the associated lower bound. If all values are
+    // missing, then signifies unbounded.
+    INCLUSIVE_RANGE_UPPER_BOUND = 9;
   }
 
   // The row data for each operation is stored in the following format:

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/integration-tests/alter_table-randomized-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/alter_table-randomized-test.cc b/src/kudu/integration-tests/alter_table-randomized-test.cc
index 9c0e0f1..717f0e3 100644
--- a/src/kudu/integration-tests/alter_table-randomized-test.cc
+++ b/src/kudu/integration-tests/alter_table-randomized-test.cc
@@ -323,7 +323,7 @@ struct MirrorTable {
       unique_ptr<KuduPartialRow> upper(schema.NewRow());
       RETURN_NOT_OK(lower->SetInt32("key", partition.first));
       RETURN_NOT_OK(upper->SetInt32("key", partition.second));
-      table_creator->add_range_bound(lower.release(), upper.release());
+      table_creator->add_range_partition(lower.release(), upper.release());
     }
 
     return table_creator->Create();
@@ -453,12 +453,28 @@ struct MirrorTable {
                                            ts_.range_partitions_.end(),
                                            ", ", "], (") << ")";
 
+    KuduTableCreator::RangePartitionBound lower_bound_type = KuduTableCreator::INCLUSIVE_BOUND;
+    KuduTableCreator::RangePartitionBound upper_bound_type = KuduTableCreator::EXCLUSIVE_BOUND;
+    int32_t lower_bound_value = bounds.first;
+    int32_t upper_bound_value = bounds.second;
+
+    if (ts_.rand_.OneIn(2) && lower_bound_value > INT32_MIN) {
+      lower_bound_type = KuduTableCreator::EXCLUSIVE_BOUND;
+      lower_bound_value -= 1;
+    }
+
+    if (ts_.rand_.OneIn(2)) {
+      upper_bound_type = KuduTableCreator::INCLUSIVE_BOUND;
+      upper_bound_value -= 1;
+    }
+
     unique_ptr<KuduPartialRow> lower_bound(schema.NewRow());
-    CHECK_OK(lower_bound->SetInt32("key", bounds.first));
+    CHECK_OK(lower_bound->SetInt32("key", lower_bound_value));
     unique_ptr<KuduPartialRow> upper_bound(schema.NewRow());
-    CHECK_OK(upper_bound->SetInt32("key", bounds.second));
+    CHECK_OK(upper_bound->SetInt32("key", upper_bound_value));
 
-    table_alterer->AddRangePartition(lower_bound.release(), upper_bound.release());
+    table_alterer->AddRangePartition(lower_bound.release(), upper_bound.release(),
+                                     lower_bound_type, upper_bound_type);
   }
 
   void DropARangePartition(KuduSchema& schema, KuduTableAlterer* table_alterer) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/integration-tests/alter_table-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc
index c91f4c9..0acacd9 100644
--- a/src/kudu/integration-tests/alter_table-test.cc
+++ b/src/kudu/integration-tests/alter_table-test.cc
@@ -567,11 +567,11 @@ Status AlterTableTest::CreateTable(const string& table_name,
                 .num_replicas(1);
 
   for (auto& split_row : split_rows) {
-    table_creator->add_range_split(split_row.release());
+    table_creator->add_range_partition_split(split_row.release());
   }
 
   for (auto& bound : bounds) {
-    table_creator->add_range_bound(bound.first.release(), bound.second.release());
+    table_creator->add_range_partition(bound.first.release(), bound.second.release());
   }
 
   return table_creator->Create();
@@ -1094,12 +1094,14 @@ TEST_F(AlterTableTest, TestAlterRangePartitioning) {
   table_alterer->DropRangePartition(lower.release(), upper.release());
   lower.reset(schema_.NewRow());
   upper.reset(schema_.NewRow());
-  ASSERT_OK(lower->SetInt32("c0", 50));
-  ASSERT_OK(upper->SetInt32("c0", 150));
-  table_alterer->AddRangePartition(lower.release(), upper.release());
+  ASSERT_OK(lower->SetInt32("c0", 49));
+  ASSERT_OK(upper->SetInt32("c0", 149));
+  table_alterer->AddRangePartition(lower.release(), upper.release(),
+                                   KuduTableCreator::EXCLUSIVE_BOUND,
+                                   KuduTableCreator::INCLUSIVE_BOUND);
   ASSERT_OK(table_alterer->wait(false)->Alter());
-  ASSERT_OK(InsertRowsSequential(table_name, 50, 75));
-  ASSERT_EQ(75, CountTableRows(table.get()));
+  ASSERT_OK(InsertRowsSequential(table_name, 50, 100));
+  ASSERT_EQ(100, CountTableRows(table.get()));
 
   // Replace the range partition with the same one.
   table_alterer.reset(client_->NewTableAlterer(table_name));
@@ -1167,7 +1169,7 @@ TEST_F(AlterTableTest, TestAlterRangePartitioningInvalid) {
   ASSERT_OK(table_creator->table_name(table_name)
                           .schema(&schema_)
                           .set_range_partition_columns({ "c0" })
-                          .add_range_bound(lower.release(), upper.release())
+                          .add_range_partition(lower.release(), upper.release())
                           .num_replicas(1)
                           .Create());
   shared_ptr<KuduTable> table;
@@ -1184,7 +1186,7 @@ TEST_F(AlterTableTest, TestAlterRangePartitioningInvalid) {
   table_alterer->AddRangePartition(lower.release(), upper.release());
   s = table_alterer->wait(false)->Alter();
   ASSERT_FALSE(s.ok());
-  ASSERT_STR_CONTAINS(s.ToString(), "New partition conflicts with existing partition");
+  ASSERT_STR_CONTAINS(s.ToString(), "New range partition conflicts with existing range partition");
   ASSERT_EQ(100, CountTableRows(table.get()));
 
   // ADD [50, 150) <- illegal (overlap)
@@ -1196,19 +1198,21 @@ TEST_F(AlterTableTest, TestAlterRangePartitioningInvalid) {
   table_alterer->AddRangePartition(lower.release(), upper.release());
   s = table_alterer->wait(false)->Alter();
   ASSERT_FALSE(s.ok());
-  ASSERT_STR_CONTAINS(s.ToString(), "New partition conflicts with existing partition");
+  ASSERT_STR_CONTAINS(s.ToString(), "New range partition conflicts with existing range partition");
   ASSERT_EQ(100, CountTableRows(table.get()));
 
-  // ADD [-50, 50) <- illegal (overlap)
+  // ADD (-50, 50] <- illegal (overlap)
   table_alterer.reset(client_->NewTableAlterer(table_name));
   lower.reset(schema_.NewRow());
   upper.reset(schema_.NewRow());
   ASSERT_OK(lower->SetInt32("c0", -50));
   ASSERT_OK(upper->SetInt32("c0", 50));
-  table_alterer->AddRangePartition(lower.release(), upper.release());
+  table_alterer->AddRangePartition(lower.release(), upper.release(),
+                                   KuduTableCreator::EXCLUSIVE_BOUND,
+                                   KuduTableCreator::INCLUSIVE_BOUND);
   s = table_alterer->wait(false)->Alter();
   ASSERT_FALSE(s.ok());
-  ASSERT_STR_CONTAINS(s.ToString(), "New partition conflicts with existing partition");
+  ASSERT_STR_CONTAINS(s.ToString(), "New range partition conflicts with existing range partition");
   ASSERT_EQ(100, CountTableRows(table.get()));
 
   // ADD [200, 300)
@@ -1226,7 +1230,7 @@ TEST_F(AlterTableTest, TestAlterRangePartitioningInvalid) {
   table_alterer->AddRangePartition(lower.release(), upper.release());
   s = table_alterer->wait(false)->Alter();
   ASSERT_FALSE(s.ok());
-  ASSERT_STR_CONTAINS(s.ToString(), "New partition conflicts with existing partition");
+  ASSERT_STR_CONTAINS(s.ToString(), "New range partition conflicts with existing range partition");
   ASSERT_FALSE(InsertRowsSequential(table_name, 200, 100).ok());
   ASSERT_EQ(100, CountTableRows(table.get()));
 
@@ -1235,7 +1239,7 @@ TEST_F(AlterTableTest, TestAlterRangePartitioningInvalid) {
   table_alterer->DropRangePartition(schema_.NewRow(), schema_.NewRow());
   s = table_alterer->Alter();
   ASSERT_FALSE(s.ok());
-  ASSERT_STR_CONTAINS(s.ToString(), "No tablet found for drop partition step");
+  ASSERT_STR_CONTAINS(s.ToString(), "No range partition found for drop range partition step");
   ASSERT_EQ(100, CountTableRows(table.get()));
 
   // DROP [50, 150)
@@ -1249,7 +1253,7 @@ TEST_F(AlterTableTest, TestAlterRangePartitioningInvalid) {
   table_alterer->RenameTo("foo");
   s = table_alterer->Alter();
   ASSERT_FALSE(s.ok());
-  ASSERT_STR_CONTAINS(s.ToString(), "No tablet found for drop partition step");
+  ASSERT_STR_CONTAINS(s.ToString(), "No range partition found for drop range partition step");
   ASSERT_EQ(100, CountTableRows(table.get()));
 
   // DROP [-50, 50)
@@ -1261,7 +1265,7 @@ TEST_F(AlterTableTest, TestAlterRangePartitioningInvalid) {
   table_alterer->DropRangePartition(lower.release(), upper.release());
   s = table_alterer->Alter();
   ASSERT_FALSE(s.ok());
-  ASSERT_STR_CONTAINS(s.ToString(), "No tablet found for drop partition step");
+  ASSERT_STR_CONTAINS(s.ToString(), "No range partition found for drop range partition step");
   ASSERT_EQ(100, CountTableRows(table.get()));
 
   // DROP [0, 100)
@@ -1297,7 +1301,7 @@ TEST_F(AlterTableTest, TestAlterRangePartitioningInvalid) {
   table_alterer->DropRangePartition(lower.release(), upper.release());
   s = table_alterer->wait(false)->Alter();
   ASSERT_FALSE(s.ok());
-  ASSERT_STR_CONTAINS(s.ToString(), "No tablet found for drop partition step");
+  ASSERT_STR_CONTAINS(s.ToString(), "No range partition found for drop range partition step");
   ASSERT_EQ(100, CountTableRows(table.get()));
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/integration-tests/flex_partitioning-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/flex_partitioning-itest.cc b/src/kudu/integration-tests/flex_partitioning-itest.cc
index 32282ee..35c38ec 100644
--- a/src/kudu/integration-tests/flex_partitioning-itest.cc
+++ b/src/kudu/integration-tests/flex_partitioning-itest.cc
@@ -217,7 +217,7 @@ class FlexPartitioningITest : public KuduTest {
       for (int i = 0; i < bound.second.size(); i++) {
         ASSERT_OK(upper->SetInt32(range_partition.columns[i], bound.second[i]));
       }
-      table_creator->add_range_bound(lower, upper);
+      table_creator->add_range_partition(lower, upper);
     }
 
     ASSERT_OK(table_creator->Create());

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index fbea196..85b6bab 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -53,6 +53,7 @@
 #include <vector>
 
 #include "kudu/cfile/type_encodings.h"
+#include "kudu/common/key_util.h"
 #include "kudu/common/partial_row.h"
 #include "kudu/common/partition.h"
 #include "kudu/common/row_operations.h"
@@ -831,13 +832,26 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
         split_rows.push_back(*op.split_row);
         break;
       }
-      case RowOperationsPB::RANGE_LOWER_BOUND: {
+      case RowOperationsPB::RANGE_LOWER_BOUND:
+      case RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND: {
         i += 1;
-        if (i >= ops.size() || ops[i].type != RowOperationsPB::RANGE_UPPER_BOUND) {
+        if (i >= ops.size() ||
+            (ops[i].type != RowOperationsPB::RANGE_UPPER_BOUND &&
+             ops[i].type != RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND)) {
           Status s = Status::InvalidArgument("Missing upper range bound in create table request");
           SetupError(resp->mutable_error(), MasterErrorPB::UNKNOWN_ERROR, s);
           return s;
         }
+
+        if (op.type == RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND) {
+          RETURN_NOT_OK(partition_schema.MakeLowerBoundRangePartitionKeyInclusive(
+                op.split_row.get()));
+        }
+        if (ops[i].type == RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND) {
+          RETURN_NOT_OK(partition_schema.MakeUpperBoundRangePartitionKeyExclusive(
+                ops[i].split_row.get()));
+        }
+
         range_bounds.emplace_back(*op.split_row, *ops[i].split_row);
         break;
       }
@@ -1253,13 +1267,24 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
                                      step.ShortDebugString());
     }
 
-    if (ops[0].type != RowOperationsPB::RANGE_LOWER_BOUND ||
-        ops[1].type != RowOperationsPB::RANGE_UPPER_BOUND) {
+    if ((ops[0].type != RowOperationsPB::RANGE_LOWER_BOUND &&
+         ops[0].type != RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND) ||
+        (ops[1].type != RowOperationsPB::RANGE_UPPER_BOUND &&
+         ops[1].type != RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND)) {
       return Status::InvalidArgument(
           "expected a lower bound and upper bound row op for alter range partition step",
           strings::Substitute("$0, $1", ops[0].ToString(schema), ops[1].ToString(schema)));
     }
 
+    if (ops[0].type == RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND) {
+      RETURN_NOT_OK(partition_schema.MakeLowerBoundRangePartitionKeyInclusive(
+            ops[0].split_row.get()));
+    }
+    if (ops[1].type == RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND) {
+      RETURN_NOT_OK(partition_schema.MakeUpperBoundRangePartitionKeyExclusive(
+            ops[1].split_row.get()));
+    }
+
     vector<Partition> partitions;
     RETURN_NOT_OK(partition_schema.CreatePartitions({}, {{ *ops[0].split_row, *ops[1].split_row }},
                                                     schema, &partitions));
@@ -1277,15 +1302,17 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
           if (existing_iter != existing_tablets.end()) {
             TabletMetadataLock metadata(existing_iter->second, TabletMetadataLock::READ);
             if (metadata.data().pb.partition().partition_key_start() < upper_bound) {
-              return Status::InvalidArgument("New partition conflicts with existing partition",
-                                             step.ShortDebugString());
+              return Status::InvalidArgument(
+                  "New range partition conflicts with existing range partition",
+                  partition_schema.RangePartitionDebugString(*ops[0].split_row, *ops[1].split_row));
             }
           }
           if (existing_iter != existing_tablets.begin()) {
             TabletMetadataLock metadata(std::prev(existing_iter)->second, TabletMetadataLock::READ);
             if (metadata.data().pb.partition().partition_key_end() > lower_bound) {
-              return Status::InvalidArgument("New partition conflicts with existing partition",
-                                             step.ShortDebugString());
+              return Status::InvalidArgument(
+                  "New range partition conflicts with existing range partition",
+                  partition_schema.RangePartitionDebugString(*ops[0].split_row, *ops[1].split_row));
             }
           }
 
@@ -1294,15 +1321,17 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
           if (new_iter != new_tablets.end()) {
             const auto& metadata = new_iter->second->mutable_metadata()->dirty();
             if (metadata.pb.partition().partition_key_start() < upper_bound) {
-              return Status::InvalidArgument("New partition conflicts with another new partition",
-                                             step.ShortDebugString());
+              return Status::InvalidArgument(
+                  "New range partition conflicts with another new range partition",
+                  partition_schema.RangePartitionDebugString(*ops[0].split_row, *ops[1].split_row));
             }
           }
           if (new_iter != new_tablets.begin()) {
             const auto& metadata = std::prev(new_iter)->second->mutable_metadata()->dirty();
             if (metadata.pb.partition().partition_key_end() > lower_bound) {
-              return Status::InvalidArgument("New partition conflicts with another new partition",
-                                             step.ShortDebugString());
+              return Status::InvalidArgument(
+                  "New range partition conflicts with another new range partition",
+                  partition_schema.RangePartitionDebugString(*ops[0].split_row, *ops[1].split_row));
             }
           }
 
@@ -1344,14 +1373,15 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
           } else if (found_new) {
             new_tablets.erase(new_iter);
           } else {
-            return Status::InvalidArgument("No tablet found for drop partition step",
-                                           step.ShortDebugString());
+            return Status::InvalidArgument(
+                "No range partition found for drop range partition step",
+                partition_schema.RangePartitionDebugString(*ops[0].split_row, *ops[1].split_row));
           }
         }
         break;
       }
       default: {
-        return Status::InvalidArgument("Unknown alter table partitioning step",
+        return Status::InvalidArgument("Unknown alter table range partitioning step",
                                        step.ShortDebugString());
       }
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/master/master-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 1579214..b4a00ac 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -481,7 +481,7 @@ TEST_F(MasterTest, TestCreateTableCheckRangeInvariants) {
     Status s = CreateTable(kTableName, kTableSchema, { }, { { a_lower, a_upper },
                                                             { b_lower, b_upper } });
     ASSERT_TRUE(s.IsInvalidArgument());
-    ASSERT_STR_CONTAINS(s.ToString(), "Invalid argument: overlapping range bounds");
+    ASSERT_STR_CONTAINS(s.ToString(), "Invalid argument: overlapping range partition");
   }
   { // Split row out of bounds (above).
     KuduPartialRow bound_lower(&kTableSchema);
@@ -520,7 +520,8 @@ TEST_F(MasterTest, TestCreateTableCheckRangeInvariants) {
     Status s = CreateTable(kTableName, kTableSchema, { }, { { bound_lower, bound_upper } });
     ASSERT_TRUE(s.IsInvalidArgument());
     ASSERT_STR_CONTAINS(s.ToString(),
-        "Invalid argument: range bound has lower bound equal to or above the upper bound");
+                        "Invalid argument: range partition lower bound must be "
+                        "less than or equal to the upper bound");
   }
   { // Lower bound equals upper bound.
     KuduPartialRow bound_lower(&kTableSchema);
@@ -531,8 +532,8 @@ TEST_F(MasterTest, TestCreateTableCheckRangeInvariants) {
     Status s = CreateTable(kTableName, kTableSchema, { }, { { bound_lower, bound_upper } });
     ASSERT_TRUE(s.IsInvalidArgument());
     ASSERT_STR_CONTAINS(s.ToString(),
-                        "Invalid argument: range bound has lower bound equal to or above the "
-                        "upper bound");
+                        "Invalid argument: range partition lower bound must be "
+                        "less than or equal to the upper bound");
   }
   { // Split equals lower bound
     KuduPartialRow bound_lower(&kTableSchema);

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/tablet/transactions/write_transaction.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/transactions/write_transaction.cc b/src/kudu/tablet/transactions/write_transaction.cc
index a36bc6c..44d3fa3 100644
--- a/src/kudu/tablet/transactions/write_transaction.cc
+++ b/src/kudu/tablet/transactions/write_transaction.cc
@@ -307,6 +307,8 @@ void WriteTransactionState::UpdateMetricsForOp(const RowOp& op) {
     case RowOperationsPB::SPLIT_ROW:
     case RowOperationsPB::RANGE_LOWER_BOUND:
     case RowOperationsPB::RANGE_UPPER_BOUND:
+    case RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND:
+    case RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND:
       break;
   }
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/ff589b52/src/kudu/tserver/tablet_server-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index 99b801d..3ba0b11 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -165,7 +165,7 @@ TEST_F(TabletServerTest, TestWebPages) {
   ASSERT_OK(c.FetchURL(Substitute("http://$0/tablets", addr),
                               &buf));
   ASSERT_STR_CONTAINS(buf.ToString(), kTabletId);
-  ASSERT_STR_CONTAINS(buf.ToString(), "<td>range: [(&lt;start&gt;), (&lt;end&gt;))</td>");
+  ASSERT_STR_CONTAINS(buf.ToString(), "<td>range: [&lt;start&gt;, &lt;end&gt;)</td>");
 
   // Tablet page should include the schema.
   ASSERT_OK(c.FetchURL(Substitute("http://$0/tablet?id=$1", addr, kTabletId),
@@ -188,7 +188,7 @@ TEST_F(TabletServerTest, TestWebPages) {
     // Check that the tablet entry shows up.
     ASSERT_STR_CONTAINS(buf.ToString(), "\"type\": \"tablet\"");
     ASSERT_STR_CONTAINS(buf.ToString(), "\"id\": \"TestTablet\"");
-    ASSERT_STR_CONTAINS(buf.ToString(), "\"partition\": \"range: [(<start>), (<end>))\"");
+    ASSERT_STR_CONTAINS(buf.ToString(), "\"partition\": \"range: [<start>, <end>)\"");
 
 
     // Check entity attributes.