You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2016/08/10 23:29:42 UTC

kudu git commit: [KuduPartialRow::Set{Binary, String}()] copy input data

Repository: kudu
Updated Branches:
  refs/heads/master 76ec7e66e -> 48766a4ce


[KuduPartialRow::Set{Binary,String}()] copy input data

KuduPartialRow::SetBinary()/SetString() behavior was optimized
to omit copying of the passed data.  However, a user of the API
might assume these methods are safe to use along with other setters
as SetInt32, SetDouble, etc. where the string or binary data
goes out of scope (or deallocated) before AppendToPB() is called.
To play safe, the behavior of these methods has been changed
to immediately copy the input data.

This is a safe modification, but it is not backward-compatible
in semver notation since it changes the semantics
of already existing methods.  However, we don't bump API version
since Kudu is not 1.0 yet.  As for the ABI, it remains compatible
with the prior versions: the existing client C++ code
will still compile and link.

This approach may add some performance regression issues for
already existing clients, but hopefully it is negligible for most
of the C++ clients around.  As for the Impala-Kudu integration,
it seems current Impala code uses SetString() only in one place
at be/src/sec/kudu-table-sink.cc, and that can be addressed
separately.

An alternative approach might be to deprecate
KuduPartialRow::Set{Binary,String}() in favor of the newly introduced
KuduPartialRow::Set{Binary,String}NoCopy() methods.  That approach
would spare us from performance regression worries and compatibility
issues.  However, after some discussion, introducing the copying
behavior of KuduPartialRow::SetBinary()/SetString() methods
appeared to be more beneficial in the long run
from the usability perspective.

Change-Id: I764339a0e3ffbf6abd5372e682c9fa1792bdd52b
Reviewed-on: http://gerrit.cloudera.org:8080/3868
Tested-by: Kudu Jenkins
Reviewed-by: Mike Percy <mp...@apache.org>


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

Branch: refs/heads/master
Commit: 48766a4ce17d422ced9a6ec78c9a9969ac44d8c9
Parents: 76ec7e6
Author: Alexey Serbin <as...@cloudera.com>
Authored: Thu Jul 21 18:28:07 2016 -0700
Committer: Mike Percy <mp...@apache.org>
Committed: Wed Aug 10 23:11:19 2016 +0000

----------------------------------------------------------------------
 src/kudu/client/predicate-test.cc               |   4 +-
 src/kudu/common/key_util-test.cc                |   4 +-
 src/kudu/common/partial_row-test.cc             | 124 ++++++++++++++++++-
 src/kudu/common/partial_row.cc                  |  44 ++++---
 src/kudu/common/partial_row.h                   |  90 +++++++++++---
 src/kudu/common/row_operations-test.cc          |  10 +-
 .../integration-tests/client_failover-itest.cc  |   4 +-
 .../integration-tests/table_locations-itest.cc  |  20 +--
 .../integration-tests/write_throttling-itest.cc |   4 +-
 src/kudu/master/sys_catalog.cc                  |  20 +--
 src/kudu/tablet/major_delta_compaction-test.cc  |  12 +-
 src/kudu/tserver/tablet_copy_session-test.cc    |   2 +-
 12 files changed, 266 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/client/predicate-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/predicate-test.cc b/src/kudu/client/predicate-test.cc
index 8170670..1a56da8 100644
--- a/src/kudu/client/predicate-test.cc
+++ b/src/kudu/client/predicate-test.cc
@@ -803,7 +803,7 @@ TEST_F(PredicateTest, TestStringPredicates) {
   for (const string& value : values) {
       unique_ptr<KuduInsert> insert(table->NewInsert());
       ASSERT_OK(insert->mutable_row()->SetInt64("key", i++));
-      ASSERT_OK(insert->mutable_row()->SetString("value", value));
+      ASSERT_OK(insert->mutable_row()->SetStringNoCopy("value", value));
       ASSERT_OK(session->Apply(insert.release()));
   }
   unique_ptr<KuduInsert> null_insert(table->NewInsert());
@@ -824,7 +824,7 @@ TEST_F(PredicateTest, TestBinaryPredicates) {
   for (const string& value : values) {
       unique_ptr<KuduInsert> insert(table->NewInsert());
       ASSERT_OK(insert->mutable_row()->SetInt64("key", i++));
-      ASSERT_OK(insert->mutable_row()->SetBinary("value", value));
+      ASSERT_OK(insert->mutable_row()->SetBinaryNoCopy("value", value));
       ASSERT_OK(session->Apply(insert.release()));
   }
   unique_ptr<KuduInsert> null_insert(table->NewInsert());

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/common/key_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/key_util-test.cc b/src/kudu/common/key_util-test.cc
index 004e78d..b159a19 100644
--- a/src/kudu/common/key_util-test.cc
+++ b/src/kudu/common/key_util-test.cc
@@ -98,7 +98,7 @@ TEST_F(KeyUtilTest, TestIncrementCompositeIntStringPrimaryKey) {
 
   // Normal increment.
   EXPECT_OK(p_row.SetInt32(0, 1000));
-  EXPECT_OK(p_row.SetString(1, "hello"));
+  EXPECT_OK(p_row.SetStringNoCopy(1, "hello"));
   EXPECT_TRUE(key_util::IncrementPrimaryKey(&row, &arena_));
   EXPECT_EQ("int32 k1=1000, string k2=hello\\000", p_row.ToString());
 
@@ -118,7 +118,7 @@ TEST_F(KeyUtilTest, TestIncrementCompositeStringIntPrimaryKey) {
   ContiguousRow row(&schema, row_data(&p_row));
 
   // Normal increment.
-  EXPECT_OK(p_row.SetString(0, "hello"));
+  EXPECT_OK(p_row.SetStringNoCopy(0, "hello"));
   EXPECT_OK(p_row.SetInt32(1, 1000));
   EXPECT_TRUE(key_util::IncrementPrimaryKey(&row, &arena_));
   EXPECT_EQ("string k1=hello, int32 k2=1001", p_row.ToString());

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/common/partial_row-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partial_row-test.cc b/src/kudu/common/partial_row-test.cc
index 9661140..9f7bc79 100644
--- a/src/kudu/common/partial_row-test.cc
+++ b/src/kudu/common/partial_row-test.cc
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <functional>
+
 #include <gtest/gtest.h>
 
 #include "kudu/common/partial_row.h"
@@ -34,8 +36,78 @@ class PartialRowTest : public KuduTest {
               1) {
     SeedRandom();
   }
+
  protected:
+  // A couple of typedefs to facilitate transformation of KuduPartialRow member
+  // function pointers into std::function<...> wrappers.
+  //
+  // The typedefs and explicit casting below via static_cast<>
+  // would not be necessary if the
+  // KuduPartialRow::Set{Binary,String}{,Copy,NoCopy}() and
+  // KuduPartialRow::Get{Binary,String}() methods had no overloaded
+  // counterparts for column name-based and index-based operations.
+  typedef Status (KuduPartialRow::*BinarySetter)(int, const Slice&);
+  typedef Status (KuduPartialRow::*BinaryGetter)(int, Slice*) const;
+
+  // Expected behavior of the
+  // KuduPartialRow::Set{Binary,String}{,Copy,NoCopy}() methods:
+  // whether the source data is copied or not.
+  enum CopyBehavior {
+    COPY,
+    NO_COPY,
+  };
+
   Schema schema_;
+
+  // Utility method to perform checks on copy/no-copy behavior of the
+  // PartialRow::Set{Binary,String}{,Copy,NoCopy}() methods.
+  void BinaryDataSetterTest(
+      std::function<Status(const KuduPartialRow&, int, Slice*)> getter,
+      std::function<Status(KuduPartialRow&, int, const Slice&)> setter,
+      int column_idx, CopyBehavior copy_behavior) {
+
+    KuduPartialRow row(&schema_);
+    string src_data = "src-data";
+    ASSERT_OK(setter(row, column_idx, src_data));
+
+    Slice column_slice;
+    ASSERT_OK(getter(row, column_idx, &column_slice));
+
+    // Check that the row's column contains the right data.
+    EXPECT_EQ("src-data", column_slice.ToString());
+
+    switch (copy_behavior) {
+      case COPY:
+        // Check that the row keeps an independent copy of the source data.
+        EXPECT_NE(reinterpret_cast<uintptr_t>(src_data.data()),
+                  reinterpret_cast<uintptr_t>(column_slice.data()));
+        break;
+      case NO_COPY:
+        // Check that the row keeps a reference to the source data.
+        EXPECT_EQ(reinterpret_cast<uintptr_t>(src_data.data()),
+                  reinterpret_cast<uintptr_t>(column_slice.data()));
+        break;
+      default:
+        ASSERT_TRUE(false) << "unexpected copy behavior specified";
+        break;  // unreachable
+    }
+
+    // Additional, more high-level check.
+    src_data.replace(0, src_data.find("-"), "new");
+    ASSERT_EQ("new-data", src_data);
+
+    switch (copy_behavior) {
+      case COPY:
+        EXPECT_EQ("src-data", column_slice.ToString());
+        break;
+      case NO_COPY:
+        EXPECT_EQ("new-data", column_slice.ToString());
+        break;
+      default:
+        ASSERT_TRUE(false) << "unexpected copy behavior specified";
+        break;  // unreachable
+    }
+  }
 };
 
 TEST_F(PartialRowTest, UnitTest) {
@@ -167,8 +239,8 @@ TEST_F(PartialRowTest, TestCopy) {
   // Check a copy with a borrowed value.
   string borrowed_string = "borrowed-string";
   string borrowed_binary = "borrowed-binary";
-  ASSERT_OK(row.SetString(2, borrowed_string));
-  ASSERT_OK(row.SetBinary(3, borrowed_binary));
+  ASSERT_OK(row.SetStringNoCopy(2, borrowed_string));
+  ASSERT_OK(row.SetBinaryNoCopy(3, borrowed_binary));
 
   copy = row;
   ASSERT_OK(copy.GetString(2, &string_val));
@@ -184,4 +256,52 @@ TEST_F(PartialRowTest, TestCopy) {
   EXPECT_EQ("mutated--binary", string_val.ToString());
 }
 
+// Check that PartialRow::SetBinaryCopy() copies the input data.
+TEST_F(PartialRowTest, TestSetBinaryCopy) {
+  BinaryDataSetterTest(
+      static_cast<BinaryGetter>(&KuduPartialRow::GetBinary),
+      static_cast<BinarySetter>(&KuduPartialRow::SetBinaryCopy),
+      3, COPY);
+}
+
+// Check that PartialRow::SetStringCopy() copies the input data.
+TEST_F(PartialRowTest, TestSetStringCopy) {
+  BinaryDataSetterTest(
+      static_cast<BinaryGetter>(&KuduPartialRow::GetString),
+      static_cast<BinarySetter>(&KuduPartialRow::SetStringCopy),
+      2, COPY);
+}
+
+// Check that PartialRow::SetBinaryNoCopy() does not copy the input data.
+TEST_F(PartialRowTest, TestSetBinaryNoCopy) {
+  BinaryDataSetterTest(
+      static_cast<BinaryGetter>(&KuduPartialRow::GetBinary),
+      static_cast<BinarySetter>(&KuduPartialRow::SetBinaryNoCopy),
+      3, NO_COPY);
+}
+
+// Check that PartialRow::SetStringNoCopy() does not copy the input data.
+TEST_F(PartialRowTest, TestSetStringNoCopy) {
+  BinaryDataSetterTest(
+      static_cast<BinaryGetter>(&KuduPartialRow::GetString),
+      static_cast<BinarySetter>(&KuduPartialRow::SetStringNoCopy),
+      2, NO_COPY);
+}
+
+// Check that PartialRow::SetBinary() copies the input data.
+TEST_F(PartialRowTest, TestSetBinary) {
+  BinaryDataSetterTest(
+      static_cast<BinaryGetter>(&KuduPartialRow::GetBinary),
+      static_cast<BinarySetter>(&KuduPartialRow::SetBinary),
+      3, COPY);
+}
+
+// Check that PartialRow::SetString() copies the input data.
+TEST_F(PartialRowTest, TestSetString) {
+  BinaryDataSetterTest(
+      static_cast<BinaryGetter>(&KuduPartialRow::GetString),
+      static_cast<BinarySetter>(&KuduPartialRow::SetString),
+      2, COPY);
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/common/partial_row.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/partial_row.cc b/src/kudu/common/partial_row.cc
index 1ff6a54..cfecdc7 100644
--- a/src/kudu/common/partial_row.cc
+++ b/src/kudu/common/partial_row.cc
@@ -248,12 +248,6 @@ Status KuduPartialRow::SetFloat(const Slice& col_name, float val) {
 Status KuduPartialRow::SetDouble(const Slice& col_name, double val) {
   return Set<TypeTraits<DOUBLE> >(col_name, val);
 }
-Status KuduPartialRow::SetString(const Slice& col_name, const Slice& val) {
-  return Set<TypeTraits<STRING> >(col_name, val, false);
-}
-Status KuduPartialRow::SetBinary(const Slice& col_name, const Slice& val) {
-  return Set<TypeTraits<BINARY> >(col_name, val, false);
-}
 Status KuduPartialRow::SetBool(int col_idx, bool val) {
   return Set<TypeTraits<BOOL> >(col_idx, val);
 }
@@ -272,12 +266,6 @@ Status KuduPartialRow::SetInt64(int col_idx, int64_t val) {
 Status KuduPartialRow::SetTimestamp(int col_idx, int64_t val) {
   return Set<TypeTraits<TIMESTAMP> >(col_idx, val);
 }
-Status KuduPartialRow::SetString(int col_idx, const Slice& val) {
-  return Set<TypeTraits<STRING> >(col_idx, val, false);
-}
-Status KuduPartialRow::SetBinary(int col_idx, const Slice& val) {
-  return Set<TypeTraits<BINARY> >(col_idx, val, false);
-}
 Status KuduPartialRow::SetFloat(int col_idx, float val) {
   return Set<TypeTraits<FLOAT> >(col_idx, val);
 }
@@ -285,19 +273,45 @@ Status KuduPartialRow::SetDouble(int col_idx, double val) {
   return Set<TypeTraits<DOUBLE> >(col_idx, val);
 }
 
+Status KuduPartialRow::SetBinary(const Slice& col_name, const Slice& val) {
+  return SetBinaryCopy(col_name, val);
+}
+Status KuduPartialRow::SetString(const Slice& col_name, const Slice& val) {
+  return SetStringCopy(col_name, val);
+}
+Status KuduPartialRow::SetBinary(int col_idx, const Slice& val) {
+  return SetBinaryCopy(col_idx, val);
+}
+Status KuduPartialRow::SetString(int col_idx, const Slice& val) {
+  return SetStringCopy(col_idx, val);
+}
+
 Status KuduPartialRow::SetBinaryCopy(const Slice& col_name, const Slice& val) {
   return SetSliceCopy<TypeTraits<BINARY> >(col_name, val);
 }
-Status KuduPartialRow::SetBinaryCopy(int col_idx, const Slice& val) {
-  return SetSliceCopy<TypeTraits<BINARY> >(col_idx, val);
-}
 Status KuduPartialRow::SetStringCopy(const Slice& col_name, const Slice& val) {
   return SetSliceCopy<TypeTraits<STRING> >(col_name, val);
 }
+Status KuduPartialRow::SetBinaryCopy(int col_idx, const Slice& val) {
+  return SetSliceCopy<TypeTraits<BINARY> >(col_idx, val);
+}
 Status KuduPartialRow::SetStringCopy(int col_idx, const Slice& val) {
   return SetSliceCopy<TypeTraits<STRING> >(col_idx, val);
 }
 
+Status KuduPartialRow::SetBinaryNoCopy(const Slice& col_name, const Slice& val) {
+  return Set<TypeTraits<BINARY> >(col_name, val, false);
+}
+Status KuduPartialRow::SetStringNoCopy(const Slice& col_name, const Slice& val) {
+  return Set<TypeTraits<STRING> >(col_name, val, false);
+}
+Status KuduPartialRow::SetBinaryNoCopy(int col_idx, const Slice& val) {
+  return Set<TypeTraits<BINARY> >(col_idx, val, false);
+}
+Status KuduPartialRow::SetStringNoCopy(int col_idx, const Slice& val) {
+  return Set<TypeTraits<STRING> >(col_idx, val, false);
+}
+
 template<typename T>
 Status KuduPartialRow::SetSliceCopy(const Slice& col_name, const Slice& val) {
   auto relocated = new uint8_t[val.size()];

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/common/partial_row.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/partial_row.h b/src/kudu/common/partial_row.h
index e9519ab..452aba8 100644
--- a/src/kudu/common/partial_row.h
+++ b/src/kudu/common/partial_row.h
@@ -123,12 +123,15 @@ class KUDU_EXPORT KuduPartialRow {
   Status SetDouble(int col_idx, double val) WARN_UNUSED_RESULT;
   ///@}
 
-  /// @name Setters for string/binary columns by name.
+  /// @name Setters for binary/string columns by name (copying).
   ///
-  /// Set the string/binary value for a column by name.
+  /// Set the binary/string value for a column by name, copying the specified
+  /// data immediately.
   ///
-  /// @note These methods do not copy the value, so the slice must remain valid
-  ///   until corresponding data is sent to the server.
+  /// @note The copying behavior is new for these methods starting Kudu 0.10.
+  ///   Prior to Kudu 0.10, these methods behaved like
+  ///   KuduPartialRow::SetStringNoCopy() and KuduPartialRow::SetBinaryNoCopy()
+  ///   correspondingly.
   ///
   /// @param [in] col_name
   ///   Name of the target column.
@@ -137,21 +140,24 @@ class KUDU_EXPORT KuduPartialRow {
   /// @return Operation result status.
   ///
   ///@{
-  Status SetString(const Slice& col_name, const Slice& val) WARN_UNUSED_RESULT;
   Status SetBinary(const Slice& col_name, const Slice& val) WARN_UNUSED_RESULT;
+  Status SetString(const Slice& col_name, const Slice& val) WARN_UNUSED_RESULT;
   ///@}
 
-  /// @name Setters for string/binary columns by index.
+  /// @name Setters for binary/string columns by index (copying).
   ///
-  /// Set the string/binary value for a column by index.
+  /// Set the binary/string value for a column by index, copying the specified
+  /// data immediately.
   ///
-  /// These setters are same as the corresponding column-name-based setters,
+  /// These setters are the same as the corresponding column-name-based setters,
   /// but with numeric column indexes. These are faster since they avoid
   /// hashmap lookups, so should be preferred in performance-sensitive code
   /// (e.g. bulk loaders).
   ///
-  /// @note These methods do not copy the value, so the slice must remain valid
-  ///   until corresponding data is sent to the server.
+  /// @note The copying behavior is new for these methods starting Kudu 0.10.
+  ///   Prior to Kudu 0.10, these methods behaved like
+  ///   KuduPartialRow::SetStringNoCopy() and KuduPartialRow::SetBinaryNoCopy()
+  ///   correspondingly.
   ///
   /// @param [in] col_idx
   ///   The index of the target column.
@@ -164,9 +170,10 @@ class KUDU_EXPORT KuduPartialRow {
   Status SetString(int col_idx, const Slice& val) WARN_UNUSED_RESULT;
   ///@}
 
-  /// @name Setters for string/binary columns by name (copying).
+  /// @name Setters for binary/string columns by name (copying).
   ///
-  /// Set the string/binary value for a column by name.
+  /// Set the binary/string value for a column by name, copying the specified
+  /// data immediately.
   ///
   /// @param [in] col_name
   ///   Name of the target column.
@@ -175,13 +182,19 @@ class KUDU_EXPORT KuduPartialRow {
   /// @return Operation result status.
   ///
   ///@{
-  Status SetStringCopy(const Slice& col_name, const Slice& val) WARN_UNUSED_RESULT;
   Status SetBinaryCopy(const Slice& col_name, const Slice& val) WARN_UNUSED_RESULT;
+  Status SetStringCopy(const Slice& col_name, const Slice& val) WARN_UNUSED_RESULT;
   ///@}
 
-  /// @name Setters for string/binary columns by index (copying).
+  /// @name Setters for binary/string columns by index (copying).
   ///
-  /// Set the string/binary value for a column by index.
+  /// Set the binary/string value for a column by index, copying the specified
+  /// data immediately.
+  ///
+  /// These setters are the same as the corresponding column-name-based setters,
+  /// but with numeric column indexes. These are faster since they avoid
+  /// hashmap lookups, so should be preferred in performance-sensitive code
+  /// (e.g. bulk loaders).
   ///
   /// @param [in] col_idx
   ///   The index of the target column.
@@ -194,6 +207,53 @@ class KUDU_EXPORT KuduPartialRow {
   Status SetBinaryCopy(int col_idx, const Slice& val) WARN_UNUSED_RESULT;
   ///@}
 
+  /// @name Setters for binary/string columns by name (non-copying).
+  ///
+  /// Set the binary/string value for a column by name, not copying the
+  /// specified data.
+  ///
+  /// @note The specified data must remain valid until the corresponding
+  ///   RPC calls are completed to be able to access error buffers,
+  ///   if any errors happened (the errors can be fetched using the
+  ///   KuduSession::GetPendingErrors() method).
+  ///
+  /// @param [in] col_name
+  ///   Name of the target column.
+  /// @param [in] val
+  ///   The value to set.
+  /// @return Operation result status.
+  ///
+  ///@{
+  Status SetBinaryNoCopy(const Slice& col_name, const Slice& val) WARN_UNUSED_RESULT;
+  Status SetStringNoCopy(const Slice& col_name, const Slice& val) WARN_UNUSED_RESULT;
+  ///@}
+
+  /// @name Setters for binary/string columns by index (non-copying).
+  ///
+  /// Set the binary/string value for a column by index, not copying the
+  /// specified data.
+  ///
+  /// These setters are the same as the corresponding column-name-based setters,
+  /// but with numeric column indexes. These are faster since they avoid
+  /// hashmap lookups, so should be preferred in performance-sensitive code
+  /// (e.g. bulk loaders).
+  ///
+  /// @note The specified data must remain valid until the corresponding
+  ///   RPC calls are completed to be able to access error buffers,
+  ///   if any errors happened (the errors can be fetched using the
+  ///   KuduSession::GetPendingErrors() method).
+  ///
+  /// @param [in] col_idx
+  ///   The index of the target column.
+  /// @param [in] val
+  ///   The value to set.
+  /// @return Operation result status.
+  ///
+  ///@{
+  Status SetBinaryNoCopy(int col_idx, const Slice& val) WARN_UNUSED_RESULT;
+  Status SetStringNoCopy(int col_idx, const Slice& val) WARN_UNUSED_RESULT;
+  ///@}
+
   /// Set column value to @c NULL; the column is identified by its name.
   ///
   /// This will only succeed on nullable columns. Use Unset() to restore

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/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 0d29e44..ed0fcc5 100644
--- a/src/kudu/common/row_operations-test.cc
+++ b/src/kudu/common/row_operations-test.cc
@@ -512,7 +512,7 @@ TEST_F(RowOperationsTest, TestProjectUpdates) {
             TestProjection(RowOperationsPB::UPDATE, client_row, server_schema));
 
   // Specify the key and update both columns
-  ASSERT_OK(client_row.SetString("string_val", "foo"));
+  ASSERT_OK(client_row.SetStringNoCopy("string_val", "foo"));
   EXPECT_EQ("MUTATE (int32 key=12345) SET int_val=12345, string_val=foo",
             TestProjection(RowOperationsPB::UPDATE, client_row, server_schema));
 
@@ -556,7 +556,7 @@ TEST_F(RowOperationsTest, DISABLED_TestProjectUpdatesSubsetOfColumns) {
 
   KuduPartialRow client_row(&client_schema);
   ASSERT_OK(client_row.SetInt32("key", 12345));
-  ASSERT_OK(client_row.SetString("string_val", "foo"));
+  ASSERT_OK(client_row.SetStringNoCopy("string_val", "foo"));
   EXPECT_EQ("MUTATE (int32 key=12345) SET string_val=foo",
             TestProjection(RowOperationsPB::UPDATE, client_row, server_schema));
 }
@@ -601,7 +601,7 @@ TEST_F(RowOperationsTest, TestProjectDeletes) {
             TestProjection(RowOperationsPB::DELETE, client_row, server_schema));
 
   // Extra column set (incorrect)
-  ASSERT_OK(client_row.SetString("string_val", "hello"));
+  ASSERT_OK(client_row.SetStringNoCopy("string_val", "hello"));
   EXPECT_EQ("error: Invalid argument: DELETE should not have a value for column: "
             "string_val[string NULLABLE]",
             TestProjection(RowOperationsPB::DELETE, client_row, server_schema));
@@ -629,8 +629,8 @@ TEST_F(RowOperationsTest, SplitKeyRoundTrip) {
   ASSERT_OK(row.SetInt16("int16", int16_expected));
   ASSERT_OK(row.SetInt32("int32", int32_expected));
   ASSERT_OK(row.SetInt64("int64", int64_expected));
-  ASSERT_OK(row.SetString("string", "string-value"));
-  ASSERT_OK(row.SetBinary("binary", "binary-value"));
+  ASSERT_OK(row.SetStringNoCopy("string", "string-value"));
+  ASSERT_OK(row.SetBinaryNoCopy("binary", "binary-value"));
   ASSERT_OK(row.SetTimestamp("timestamp", 9));
 
   RowOperationsPB pb;

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/integration-tests/client_failover-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/client_failover-itest.cc b/src/kudu/integration-tests/client_failover-itest.cc
index 1542679..71a0a00 100644
--- a/src/kudu/integration-tests/client_failover-itest.cc
+++ b/src/kudu/integration-tests/client_failover-itest.cc
@@ -115,7 +115,7 @@ TEST_P(ClientFailoverParamITest, TestDeleteLeaderWhileScanning) {
   KuduInsert* insert = table->NewInsert();
   ASSERT_OK(insert->mutable_row()->SetInt32(0, 0));
   ASSERT_OK(insert->mutable_row()->SetInt32(1, 1));
-  ASSERT_OK(insert->mutable_row()->SetString(2, "a"));
+  ASSERT_OK(insert->mutable_row()->SetStringNoCopy(2, "a"));
   ASSERT_OK(session->Apply(insert));
   ASSERT_OK(session->Flush());
   ASSERT_EQ(1, CountTableRows(table.get()));
@@ -190,7 +190,7 @@ TEST_P(ClientFailoverParamITest, TestDeleteLeaderWhileScanning) {
     KuduUpdate* update = table->NewUpdate();
     ASSERT_OK(update->mutable_row()->SetInt32(0, 0));
     ASSERT_OK(update->mutable_row()->SetInt32(1, 2));
-    ASSERT_OK(update->mutable_row()->SetString(2, "b"));
+    ASSERT_OK(update->mutable_row()->SetStringNoCopy(2, "b"));
     ASSERT_OK(session->Apply(update));
     ASSERT_OK(session->Flush());
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/integration-tests/table_locations-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/table_locations-itest.cc b/src/kudu/integration-tests/table_locations-itest.cc
index 3f90b18..db71877 100644
--- a/src/kudu/integration-tests/table_locations-itest.cc
+++ b/src/kudu/integration-tests/table_locations-itest.cc
@@ -121,18 +121,18 @@ TEST_F(TableLocationsTest, TestGetTableLocations) {
   KuduPartialRow row(&schema);
 
   vector<KuduPartialRow> splits(6, row);
-  ASSERT_OK(splits[0].SetString(0, "aa"));
-  ASSERT_OK(splits[1].SetString(0, "ab"));
-  ASSERT_OK(splits[2].SetString(0, "ac"));
-  ASSERT_OK(splits[3].SetString(0, "ca"));
-  ASSERT_OK(splits[4].SetString(0, "cb"));
-  ASSERT_OK(splits[5].SetString(0, "cc"));
+  ASSERT_OK(splits[0].SetStringNoCopy(0, "aa"));
+  ASSERT_OK(splits[1].SetStringNoCopy(0, "ab"));
+  ASSERT_OK(splits[2].SetStringNoCopy(0, "ac"));
+  ASSERT_OK(splits[3].SetStringNoCopy(0, "ca"));
+  ASSERT_OK(splits[4].SetStringNoCopy(0, "cb"));
+  ASSERT_OK(splits[5].SetStringNoCopy(0, "cc"));
 
   vector<pair<KuduPartialRow, KuduPartialRow>> bounds(2, { row, row });
-  ASSERT_OK(bounds[0].first.SetString(0, "a"));
-  ASSERT_OK(bounds[0].second.SetString(0, "b"));
-  ASSERT_OK(bounds[1].first.SetString(0, "c"));
-  ASSERT_OK(bounds[1].second.SetString(0, "d"));
+  ASSERT_OK(bounds[0].first.SetStringNoCopy(0, "a"));
+  ASSERT_OK(bounds[0].second.SetStringNoCopy(0, "b"));
+  ASSERT_OK(bounds[1].first.SetStringNoCopy(0, "c"));
+  ASSERT_OK(bounds[1].second.SetStringNoCopy(0, "d"));
 
   ASSERT_OK(CreateTable(table_name, schema, splits, bounds));
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/integration-tests/write_throttling-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/write_throttling-itest.cc b/src/kudu/integration-tests/write_throttling-itest.cc
index cd8ee78..48c57ba 100644
--- a/src/kudu/integration-tests/write_throttling-itest.cc
+++ b/src/kudu/integration-tests/write_throttling-itest.cc
@@ -89,7 +89,7 @@ TEST_F(WriteThrottlingTest, ThrottleWriteRpcPerSec) {
       gscoped_ptr<KuduInsert> insert(table_->NewInsert());
       KuduPartialRow* row = insert->mutable_row();
       CHECK_OK(row->SetInt64("key", t * TARGET_QPS + i));
-      CHECK_OK(row->SetString("string_val", string_val));
+      CHECK_OK(row->SetStringNoCopy("string_val", string_val));
       CHECK_OK(session->Apply(insert.release()));
     }
     MonoTime end = MonoTime::Now(MonoTime::FINE);
@@ -121,7 +121,7 @@ TEST_F(WriteThrottlingTest, ThrottleWriteBytesPerSec) {
       gscoped_ptr<KuduInsert> insert(table_->NewInsert());
       KuduPartialRow* row = insert->mutable_row();
       CHECK_OK(row->SetInt64("key", t * TARGET_QPS + i));
-      CHECK_OK(row->SetString("string_val", string_val));
+      CHECK_OK(row->SetStringNoCopy("string_val", string_val));
       CHECK_OK(session->Apply(insert.release()));
     }
     MonoTime end = MonoTime::Now(MonoTime::FINE);

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/master/sys_catalog.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index 824561f..3775009 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -420,8 +420,8 @@ Status SysCatalogTable::ReqAddTable(WriteRequestPB* req, const TableInfo* table)
 
   KuduPartialRow row(&schema_);
   CHECK_OK(row.SetInt8(kSysCatalogTableColType, TABLES_ENTRY));
-  CHECK_OK(row.SetString(kSysCatalogTableColId, table->id()));
-  CHECK_OK(row.SetString(kSysCatalogTableColMetadata, metadata_buf));
+  CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, table->id()));
+  CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColMetadata, metadata_buf));
   RowOperationsPBEncoder enc(req->mutable_row_operations());
   enc.Add(RowOperationsPB::INSERT, row);
   return Status::OK();
@@ -436,8 +436,8 @@ Status SysCatalogTable::ReqUpdateTable(WriteRequestPB* req, const TableInfo* tab
 
   KuduPartialRow row(&schema_);
   CHECK_OK(row.SetInt8(kSysCatalogTableColType, TABLES_ENTRY));
-  CHECK_OK(row.SetString(kSysCatalogTableColId, table->id()));
-  CHECK_OK(row.SetString(kSysCatalogTableColMetadata, metadata_buf));
+  CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, table->id()));
+  CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColMetadata, metadata_buf));
   RowOperationsPBEncoder enc(req->mutable_row_operations());
   enc.Add(RowOperationsPB::UPDATE, row);
   return Status::OK();
@@ -446,7 +446,7 @@ Status SysCatalogTable::ReqUpdateTable(WriteRequestPB* req, const TableInfo* tab
 Status SysCatalogTable::ReqDeleteTable(WriteRequestPB* req, const TableInfo* table) {
   KuduPartialRow row(&schema_);
   CHECK_OK(row.SetInt8(kSysCatalogTableColType, TABLES_ENTRY));
-  CHECK_OK(row.SetString(kSysCatalogTableColId, table->id()));
+  CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, table->id()));
   RowOperationsPBEncoder enc(req->mutable_row_operations());
   enc.Add(RowOperationsPB::DELETE, row);
   return Status::OK();
@@ -513,8 +513,8 @@ Status SysCatalogTable::ReqAddTablets(WriteRequestPB* req,
     }
 
     CHECK_OK(row.SetInt8(kSysCatalogTableColType, TABLETS_ENTRY));
-    CHECK_OK(row.SetString(kSysCatalogTableColId, tablet->tablet_id()));
-    CHECK_OK(row.SetString(kSysCatalogTableColMetadata, metadata_buf));
+    CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, tablet->tablet_id()));
+    CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColMetadata, metadata_buf));
     enc.Add(RowOperationsPB::INSERT, row);
   }
 
@@ -533,8 +533,8 @@ Status SysCatalogTable::ReqUpdateTablets(WriteRequestPB* req,
     }
 
     CHECK_OK(row.SetInt8(kSysCatalogTableColType, TABLETS_ENTRY));
-    CHECK_OK(row.SetString(kSysCatalogTableColId, tablet->tablet_id()));
-    CHECK_OK(row.SetString(kSysCatalogTableColMetadata, metadata_buf));
+    CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, tablet->tablet_id()));
+    CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColMetadata, metadata_buf));
     enc.Add(RowOperationsPB::UPDATE, row);
   }
 
@@ -547,7 +547,7 @@ Status SysCatalogTable::ReqDeleteTablets(WriteRequestPB* req,
   RowOperationsPBEncoder enc(req->mutable_row_operations());
   for (auto tablet : tablets) {
     CHECK_OK(row.SetInt8(kSysCatalogTableColType, TABLETS_ENTRY));
-    CHECK_OK(row.SetString(kSysCatalogTableColId, tablet->tablet_id()));
+    CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, tablet->tablet_id()));
     enc.Add(RowOperationsPB::DELETE, row);
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/tablet/major_delta_compaction-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/major_delta_compaction-test.cc b/src/kudu/tablet/major_delta_compaction-test.cc
index 7c81cad..0c609c2 100644
--- a/src/kudu/tablet/major_delta_compaction-test.cc
+++ b/src/kudu/tablet/major_delta_compaction-test.cc
@@ -86,11 +86,11 @@ class TestMajorDeltaCompaction : public KuduRowSetTest {
       row.val4 = StringPrintf("b %08d", i * 10);
 
       int col = 0;
-      CHECK_OK(ins_row.SetString(col++, row.key));
+      CHECK_OK(ins_row.SetStringNoCopy(col++, row.key));
       CHECK_OK(ins_row.SetInt32(col++, row.val1));
-      CHECK_OK(ins_row.SetString(col++, row.val2));
+      CHECK_OK(ins_row.SetStringNoCopy(col++, row.val2));
       CHECK_OK(ins_row.SetInt32(col++, row.val3));
-      CHECK_OK(ins_row.SetString(col++, row.val4));
+      CHECK_OK(ins_row.SetStringNoCopy(col++, row.val4));
       ASSERT_OK_FAST(writer.Insert(ins_row));
       expected_state_.push_back(row);
     }
@@ -102,7 +102,7 @@ class TestMajorDeltaCompaction : public KuduRowSetTest {
     KuduPartialRow del_row(&client_schema_);
 
     for (int i = nrows - 1; i >= 0; i--) {
-      CHECK_OK(del_row.SetString(0, expected_state_[i].key));
+      CHECK_OK(del_row.SetStringNoCopy(0, expected_state_[i].key));
       ASSERT_OK(writer.Delete(del_row));
       expected_state_.pop_back();
     }
@@ -119,7 +119,7 @@ class TestMajorDeltaCompaction : public KuduRowSetTest {
       ExpectedRow* row = &expected_state_[idx];
       if ((idx % 2 == 0) == even) {
         // Set key
-        CHECK_OK(prow.SetString(0, row->key));
+        CHECK_OK(prow.SetStringNoCopy(0, row->key));
 
         // Update the data
         row->val1 *= 2;
@@ -129,7 +129,7 @@ class TestMajorDeltaCompaction : public KuduRowSetTest {
         // Apply the updates.
         CHECK_OK(prow.SetInt32(1, row->val1));
         CHECK_OK(prow.SetInt32(3, row->val3));
-        CHECK_OK(prow.SetString(4, row->val4));
+        CHECK_OK(prow.SetStringNoCopy(4, row->val4));
         ASSERT_OK(writer.Update(prow));
       }
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/48766a4c/src/kudu/tserver/tablet_copy_session-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_session-test.cc b/src/kudu/tserver/tablet_copy_session-test.cc
index aa104b5..e75a3b9 100644
--- a/src/kudu/tserver/tablet_copy_session-test.cc
+++ b/src/kudu/tserver/tablet_copy_session-test.cc
@@ -156,7 +156,7 @@ class TabletCopyTest : public KuduTabletTest {
       KuduPartialRow row(&client_schema_);
 
       string key = Substitute("key$0", i);
-      ASSERT_OK(row.SetString(0, key));
+      ASSERT_OK(row.SetStringNoCopy(0, key));
       ASSERT_OK(row.SetInt32(1, i));
       enc.Add(RowOperationsPB::INSERT, row);