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 2017/02/28 21:25:48 UTC

[1/5] kudu git commit: cmake: don't incorporate filenames in regexp patterns

Repository: kudu
Updated Branches:
  refs/heads/master 50274c8f8 -> 368f4f734


cmake: don't incorporate filenames in regexp patterns

It leads to incorrect match failures when the filenames contain regexp
metacharacters, the worst offender being '+'. I couldn't find a cmake
equivalent to java.util.regex.Pattern's \Q and \E control characters, so I
rewrote the matching logic with FIND and REPLACE instead.

With this change, building from within a directory like
kudu-1.2.0+cdh6.x+0/build/fastdebug correctly creates the 'latest' link.

Change-Id: Iffbb9cb377625b537212e8244976ffc1b898840b
Reviewed-on: http://gerrit.cloudera.org:8080/6161
Tested-by: Adar Dembo <ad...@cloudera.com>
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 8f54f6049434196e80f3f4adeb450348a013bad4
Parents: 50274c8
Author: Adar Dembo <ad...@cloudera.com>
Authored: Sun Feb 26 19:29:21 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 28 06:43:32 2017 +0000

----------------------------------------------------------------------
 CMakeLists.txt | 14 +++++++++-----
 1 file changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/8f54f604/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 6ec3917..84329a5 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -45,11 +45,15 @@ endif()
 # build/<first build directory>
 # build/<second build directory>
 # ...
-if ("${CMAKE_CURRENT_BINARY_DIR}" MATCHES "^${CMAKE_CURRENT_SOURCE_DIR}/build/[^/]+$")
-  if ("${CMAKE_CURRENT_BINARY_DIR}" MATCHES "^${CMAKE_CURRENT_SOURCE_DIR}/build/latest")
-    message(FATAL_ERROR "Should not run cmake inside the build/latest symlink. "
-            "First change directories into the destination of the symlink.")
-  endif()
+if ("${CMAKE_CURRENT_BINARY_DIR}" STREQUAL
+    "${CMAKE_CURRENT_SOURCE_DIR}/build/latest")
+  message(FATAL_ERROR "Should not run cmake inside the build/latest symlink. "
+    "First change directories into the destination of the symlink.")
+endif()
+string(REPLACE "${CMAKE_CURRENT_SOURCE_DIR}/build/" ""
+  BLESSED_BUILD_SUBDIR "${CMAKE_CURRENT_BINARY_DIR}")
+string(FIND "${BLESSED_BUILD_SUBDIR}" "/" SLASH_POS)
+if (SLASH_POS EQUAL -1)
   if (NOT APPLE)
     set(MORE_ARGS "-T")
   endif()


[4/5] kudu git commit: Misc optimizations to BinaryPlainBlockDecoder

Posted by to...@apache.org.
Misc optimizations to BinaryPlainBlockDecoder

Looking at a profile while running a YCSB read workload against a binary
built with clang, I noticed that BinaryPlainDecoder had done a really
poor job of inlining vector::push_back when building the offsets array.

This switches away from using a vector there and instead uses a simple
buffer/pointer view into the buffer. I also rearranged a bit of other
code and added PREDICT_FALSEs to try to get the code into a tighter
loop.

Change-Id: I5b5061818de36dc268cd5d4fc8553bceeca5dadd
Reviewed-on: http://gerrit.cloudera.org:8080/6159
Tested-by: Kudu Jenkins
Reviewed-by: David Ribeiro Alves <dr...@apache.org>


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

Branch: refs/heads/master
Commit: cbb07d23f18c055ff1ae659cfec19f734eeb0403
Parents: 92fe87f
Author: Todd Lipcon <to...@apache.org>
Authored: Sun Feb 26 18:51:09 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 28 21:24:41 2017 +0000

----------------------------------------------------------------------
 src/kudu/cfile/binary_plain_block.cc | 42 ++++++++++++++++---------------
 src/kudu/cfile/binary_plain_block.h  | 25 ++++++++++++++----
 2 files changed, 42 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/cbb07d23/src/kudu/cfile/binary_plain_block.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/binary_plain_block.cc b/src/kudu/cfile/binary_plain_block.cc
index a9eea75..430edcb 100644
--- a/src/kudu/cfile/binary_plain_block.cc
+++ b/src/kudu/cfile/binary_plain_block.cc
@@ -183,48 +183,50 @@ Status BinaryPlainBlockDecoder::ParseHeader() {
   const uint8_t *p = data_.data() + offsets_pos;
   const uint8_t *limit = data_.data() + data_.size();
 
-  offsets_.clear();
   // Reserve one extra element, which we'll fill in at the end
   // with an offset past the last element.
-  offsets_.reserve(num_elems_ + 1);
-
+  offsets_buf_.resize(sizeof(uint32_t) * (num_elems_ + 1));
+  uint32_t* dst_ptr = reinterpret_cast<uint32_t*>(offsets_buf_.data());
   size_t rem = num_elems_;
   while (rem >= 4) {
-    uint32_t ints[4];
-    if (p + 16 < limit) {
-      p = coding::DecodeGroupVarInt32_SSE(p, &ints[0], &ints[1], &ints[2], &ints[3]);
+    if (PREDICT_TRUE(p + 16 < limit)) {
+      p = coding::DecodeGroupVarInt32_SSE(
+          p, &dst_ptr[0], &dst_ptr[1], &dst_ptr[2], &dst_ptr[3]);
+
+      // The above function should add at most 17 (4 32-bit ints plus a selector byte) to
+      // 'p'. Thus, since we checked that (p + 16 < limit) above, we are guaranteed that
+      // (p <= limit) now.
+      DCHECK_LE(p, limit);
     } else {
-      p = coding::DecodeGroupVarInt32_SlowButSafe(p, &ints[0], &ints[1], &ints[2], &ints[3]);
-    }
-    if (p > limit) {
-      LOG(WARNING) << "bad block: " << HexDump(data_);
-      return Status::Corruption(
-        StringPrintf("unable to decode offsets in block"));
+      p = coding::DecodeGroupVarInt32_SlowButSafe(
+          p, &dst_ptr[0], &dst_ptr[1], &dst_ptr[2], &dst_ptr[3]);
+      if (PREDICT_FALSE(p > limit)) {
+        // Only need to check 'p' overrun in the slow path, because 'p' may have
+        // been within 16 bytes of 'limit'.
+        LOG(WARNING) << "bad block: " << HexDump(data_);
+        return Status::Corruption(StringPrintf("unable to decode offsets in block"));
+      }
     }
-
-    offsets_.push_back(ints[0]);
-    offsets_.push_back(ints[1]);
-    offsets_.push_back(ints[2]);
-    offsets_.push_back(ints[3]);
+    dst_ptr += 4;
     rem -= 4;
   }
 
   if (rem > 0) {
     uint32_t ints[4];
     p = coding::DecodeGroupVarInt32_SlowButSafe(p, &ints[0], &ints[1], &ints[2], &ints[3]);
-    if (p > limit) {
+    if (PREDICT_FALSE(p > limit)) {
       LOG(WARNING) << "bad block: " << HexDump(data_);
       return Status::Corruption(
         StringPrintf("unable to decode offsets in block"));
     }
 
     for (int i = 0; i < rem; i++) {
-      offsets_.push_back(ints[i]);
+      *dst_ptr++ = ints[i];
     }
   }
 
   // Add one extra entry pointing after the last item to make the indexing easier.
-  offsets_.push_back(offsets_pos);
+  *dst_ptr++ = offsets_pos;
 
   parsed_ = true;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/cbb07d23/src/kudu/cfile/binary_plain_block.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/binary_plain_block.h b/src/kudu/cfile/binary_plain_block.h
index 8c9cd49..2edba9d 100644
--- a/src/kudu/cfile/binary_plain_block.h
+++ b/src/kudu/cfile/binary_plain_block.h
@@ -122,9 +122,9 @@ class BinaryPlainBlockDecoder final : public BlockDecoder {
   }
 
   Slice string_at_index(size_t idx) const {
-    const uint32_t offset = offsets_[idx];
-    uint32_t len = offsets_[idx + 1] - offset;
-    return Slice(&data_[offset], len);
+    const uint32_t str_offset = offset(idx);
+    uint32_t len = offset(idx + 1) - str_offset;
+    return Slice(&data_[str_offset], len);
   }
 
   // Minimum length of a header.
@@ -139,13 +139,28 @@ class BinaryPlainBlockDecoder final : public BlockDecoder {
   template <typename CellHandler>
   Status HandleBatch(size_t* n, ColumnDataView* dst, CellHandler c);
 
+  // Return the offset within 'data_' where the string value with index 'idx'
+  // can be found.
+  uint32_t offset(int idx) const {
+    const uint8_t* p = &offsets_buf_[idx * sizeof(uint32_t)];
+    uint32_t ret;
+    memcpy(&ret, p, sizeof(uint32_t));
+    return ret;
+  }
+
   Slice data_;
   bool parsed_;
 
-  // The parsed offsets.
+  // A buffer for an array of 32-bit integers for the offsets of the underlying
+  // strings in 'data_'.
+  //
   // This array also contains one extra offset at the end, pointing
   // _after_ the last entry. This makes the code much simpler.
-  std::vector<uint32_t> offsets_;
+  //
+  // The array is stored inside a 'faststring' instead of a vector<uint32_t> to
+  // avoid the overhead of calling vector::push_back -- one would think it would
+  // be fully inlined away, but it's actually a perf win to do this.
+  faststring offsets_buf_;
 
   uint32_t num_elems_;
   rowid_t ordinal_pos_base_;


[3/5] kudu git commit: KUDU-1896 (part 1). Add redaction to JSON protobuf output

Posted by to...@apache.org.
KUDU-1896 (part 1). Add redaction to JSON protobuf output

This enables the redaction attribute when outputting a protobuf as JSON.
This is important in the context of the tracing endpoint as well as
/rpcz, to avoid such traces containing user data.

Change-Id: I506b3710d2b50ec7a8947c49ec208eb4b53b63ea
Reviewed-on: http://gerrit.cloudera.org:8080/6158
Reviewed-by: Hao Hao <ha...@cloudera.com>
Tested-by: Todd Lipcon <to...@apache.org>
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 92fe87f5e2db57f520ba96fc4b63f1d13a48b185
Parents: c11a315
Author: Todd Lipcon <to...@apache.org>
Authored: Sun Feb 26 16:12:24 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 28 21:13:04 2017 +0000

----------------------------------------------------------------------
 src/kudu/util/CMakeLists.txt        |  1 +
 src/kudu/util/jsonwriter-test.cc    | 35 +++++++++++++++++++++++++++++++-
 src/kudu/util/jsonwriter.cc         | 10 +++++++--
 src/kudu/util/jsonwriter.h          |  2 ++
 src/kudu/util/jsonwriter_test.proto |  8 +++++++-
 src/kudu/util/logging.h             |  4 ++++
 src/kudu/util/pb_util.cc            |  2 +-
 7 files changed, 57 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/92fe87f5/src/kudu/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index 27f5d70..685f01f 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -405,6 +405,7 @@ PROTOBUF_GENERATE_CPP(
   PROTO_FILES jsonwriter_test.proto)
 add_library(jsonwriter_test_proto ${JSONWRITER_TEST_PROTO_SRCS} ${JSONWRITER_TEST_PROTO_HDRS})
 target_link_libraries(jsonwriter_test_proto
+  pb_util_proto
   protobuf)
 
 #######################################

http://git-wip-us.apache.org/repos/asf/kudu/blob/92fe87f5/src/kudu/util/jsonwriter-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/jsonwriter-test.cc b/src/kudu/util/jsonwriter-test.cc
index 81e7d85..7f458fa 100644
--- a/src/kudu/util/jsonwriter-test.cc
+++ b/src/kudu/util/jsonwriter-test.cc
@@ -17,8 +17,10 @@
 
 #include <gtest/gtest.h>
 
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/jsonwriter.h"
 #include "kudu/util/jsonwriter_test.pb.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/test_util.h"
 
 using jsonwriter_test::TestAllTypes;
@@ -33,6 +35,7 @@ TEST_F(TestJsonWriter, TestPBEmpty) {
 }
 
 TEST_F(TestJsonWriter, TestPBAllFieldTypes) {
+  FLAGS_log_redact_user_data = true;
   TestAllTypes pb;
   pb.set_optional_int32(1);
   pb.set_optional_int64(2);
@@ -48,6 +51,7 @@ TEST_F(TestJsonWriter, TestPBAllFieldTypes) {
   pb.set_optional_double(12);
   pb.set_optional_bool(true);
   pb.set_optional_string("hello world");
+  pb.set_optional_redacted_string("secret!");
   pb.set_optional_nested_enum(TestAllTypes::FOO);
   ASSERT_EQ("{\n"
             "    \"optional_int32\": 1,\n"
@@ -64,6 +68,7 @@ TEST_F(TestJsonWriter, TestPBAllFieldTypes) {
             "    \"optional_double\": 12,\n"
             "    \"optional_bool\": true,\n"
             "    \"optional_string\": \"hello world\",\n"
+            "    \"optional_redacted_string\": \"<redacted>\",\n"
             "    \"optional_nested_enum\": \"FOO\"\n"
             "}", JsonWriter::ToJson(pb, JsonWriter::PRETTY));
   ASSERT_EQ("{"
@@ -81,15 +86,20 @@ TEST_F(TestJsonWriter, TestPBAllFieldTypes) {
             "\"optional_double\":12,"
             "\"optional_bool\":true,"
             "\"optional_string\":\"hello world\","
+            "\"optional_redacted_string\":\"<redacted>\","
             "\"optional_nested_enum\":\"FOO\""
             "}", JsonWriter::ToJson(pb, JsonWriter::COMPACT));
 
 }
 
 TEST_F(TestJsonWriter, TestPBRepeatedPrimitives) {
+  FLAGS_log_redact_user_data = true;
   TestAllTypes pb;
   for (int i = 0; i <= 3; i++) {
     pb.add_repeated_int32(i);
+    pb.add_repeated_string(strings::Substitute("hi $0", i));
+    pb.add_repeated_redacted_string("secret!");
+    pb.add_repeated_redacted_bytes("secret!");
   }
   ASSERT_EQ("{\n"
             "    \"repeated_int32\": [\n"
@@ -97,9 +107,32 @@ TEST_F(TestJsonWriter, TestPBRepeatedPrimitives) {
             "        1,\n"
             "        2,\n"
             "        3\n"
+            "    ],\n"
+            "    \"repeated_string\": [\n"
+            "        \"hi 0\",\n"
+            "        \"hi 1\",\n"
+            "        \"hi 2\",\n"
+            "        \"hi 3\"\n"
+            "    ],\n"
+            "    \"repeated_redacted_string\": [\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\"\n"
+            "    ],\n"
+            "    \"repeated_redacted_bytes\": [\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\",\n"
+            "        \"<redacted>\"\n"
             "    ]\n"
             "}", JsonWriter::ToJson(pb, JsonWriter::PRETTY));
-  ASSERT_EQ("{\"repeated_int32\":[0,1,2,3]}",
+  ASSERT_EQ("{\"repeated_int32\":[0,1,2,3],"
+            "\"repeated_string\":[\"hi 0\",\"hi 1\",\"hi 2\",\"hi 3\"],"
+            "\"repeated_redacted_string\":[\"<redacted>\",\"<redacted>\","
+            "\"<redacted>\",\"<redacted>\"],"
+            "\"repeated_redacted_bytes\":[\"<redacted>\",\"<redacted>\","
+            "\"<redacted>\",\"<redacted>\"]}",
             JsonWriter::ToJson(pb, JsonWriter::COMPACT));
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/92fe87f5/src/kudu/util/jsonwriter.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/jsonwriter.cc b/src/kudu/util/jsonwriter.cc
index 64f8706..ef9d5ba 100644
--- a/src/kudu/util/jsonwriter.cc
+++ b/src/kudu/util/jsonwriter.cc
@@ -22,10 +22,14 @@
 
 #include <glog/logging.h>
 #include <google/protobuf/descriptor.h>
+#include <google/protobuf/descriptor.pb.h>
 #include <google/protobuf/message.h>
 #include <rapidjson/prettywriter.h>
 #include <rapidjson/rapidjson.h>
 
+#include "kudu/util/logging.h"
+#include "kudu/util/pb_util.pb.h"
+
 using google::protobuf::FieldDescriptor;
 using google::protobuf::Message;
 using google::protobuf::Reflection;
@@ -213,7 +217,8 @@ void JsonWriter::ProtobufField(const Message& pb, const FieldDescriptor* field)
       String(reflection->GetEnum(pb, field)->name());
       break;
     case FieldDescriptor::CPPTYPE_STRING:
-      String(reflection->GetString(pb, field));
+      String(KUDU_MAYBE_REDACT_IF(field->options().GetExtension(REDACT),
+                                  reflection->GetString(pb, field)));
       break;
     case FieldDescriptor::CPPTYPE_MESSAGE:
       Protobuf(reflection->GetMessage(pb, field));
@@ -251,7 +256,8 @@ void JsonWriter::ProtobufRepeatedField(const Message& pb, const FieldDescriptor*
       String(reflection->GetRepeatedEnum(pb, field, index)->name());
       break;
     case FieldDescriptor::CPPTYPE_STRING:
-      String(reflection->GetRepeatedString(pb, field, index));
+      String(KUDU_MAYBE_REDACT_IF(field->options().GetExtension(REDACT),
+                                  reflection->GetRepeatedString(pb, field, index)));
       break;
     case FieldDescriptor::CPPTYPE_MESSAGE:
       Protobuf(reflection->GetRepeatedMessage(pb, field, index));

http://git-wip-us.apache.org/repos/asf/kudu/blob/92fe87f5/src/kudu/util/jsonwriter.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/jsonwriter.h b/src/kudu/util/jsonwriter.h
index 1909bc0..d3fb604 100644
--- a/src/kudu/util/jsonwriter.h
+++ b/src/kudu/util/jsonwriter.h
@@ -66,6 +66,8 @@ class JsonWriter {
   void String(const char* str);
   void String(const std::string& str);
 
+  // Convert the given protobuf message to JSON.
+  // The output respects redaction for 'string' and 'bytes' fields.
   void Protobuf(const google::protobuf::Message& message);
 
   template<typename T>

http://git-wip-us.apache.org/repos/asf/kudu/blob/92fe87f5/src/kudu/util/jsonwriter_test.proto
----------------------------------------------------------------------
diff --git a/src/kudu/util/jsonwriter_test.proto b/src/kudu/util/jsonwriter_test.proto
index d16cc11..cab7201 100644
--- a/src/kudu/util/jsonwriter_test.proto
+++ b/src/kudu/util/jsonwriter_test.proto
@@ -16,6 +16,8 @@
 // under the License.
 package jsonwriter_test;
 
+import "kudu/util/pb_util.proto";
+
 // This proto includes every type of field in both singular and repeated
 // forms. This is mostly copied from 'unittest.proto' in the protobuf source
 // (hence the odd field numbers which skip some).
@@ -45,7 +47,9 @@ message TestAllTypes {
   optional   double optional_double   = 12;
   optional     bool optional_bool     = 13;
   optional   string optional_string   = 14;
-  optional    bytes optional_bytes    = 15;
+  optional   string optional_redacted_string = 15 [ (kudu.REDACT) = true ];
+  optional    bytes optional_bytes    = 16;
+  optional    bytes optional_redacted_bytes = 17 [ (kudu.REDACT) = true ];
 
   optional NestedMessage optional_nested_message  = 18;
   optional NestedEnum optional_nested_enum     = 21;
@@ -66,6 +70,8 @@ message TestAllTypes {
   repeated     bool repeated_bool     = 43;
   repeated   string repeated_string   = 44;
   repeated    bytes repeated_bytes    = 45;
+  repeated   string repeated_redacted_string = 46 [ (kudu.REDACT) = true ];
+  repeated   string repeated_redacted_bytes = 47 [ (kudu.REDACT) = true ];
 
   repeated NestedMessage repeated_nested_message = 48;
   repeated NestedEnum repeated_nested_enum = 51;

http://git-wip-us.apache.org/repos/asf/kudu/blob/92fe87f5/src/kudu/util/logging.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/logging.h b/src/kudu/util/logging.h
index 5cbf820..e0b5dff 100644
--- a/src/kudu/util/logging.h
+++ b/src/kudu/util/logging.h
@@ -65,6 +65,10 @@
 #define KUDU_REDACT(expr) \
   (KUDU_SHOULD_REDACT() ? kudu::kRedactionMessage : (expr))
 
+// Like the above, but with the additional condition that redaction will only
+// be performed if 'cond' must be true.
+#define KUDU_MAYBE_REDACT_IF(cond, expr) \
+  ((KUDU_SHOULD_REDACT() && (cond)) ? kudu::kRedactionMessage : (expr))
 
 ////////////////////////////////////////
 // Redaction implementation details follow.

http://git-wip-us.apache.org/repos/asf/kudu/blob/92fe87f5/src/kudu/util/pb_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/pb_util.cc b/src/kudu/util/pb_util.cc
index 4c0ff5c..6deb726 100644
--- a/src/kudu/util/pb_util.cc
+++ b/src/kudu/util/pb_util.cc
@@ -567,7 +567,7 @@ class SecureFieldPrinter : public TextFormat::FieldValuePrinter {
   string PrintBytes(const string& val) const override {
     if (hide_next_string_) {
       hide_next_string_ = false;
-      return KUDU_REDACT(super::PrintString(val));
+      return KUDU_REDACT(super::PrintBytes(val));
     }
     return super::PrintBytes(val);
   }


[2/5] kudu git commit: KUDU-1899. Fix support for empty string keys

Posted by to...@apache.org.
KUDU-1899. Fix support for empty string keys

We've never prevented users from inserting a row with "" as a key. However, a
faulty assertion in diskrowset.cc caused the tablet server to crash on flush in
this case.

This adds a new end-to-end test via the client which inserts, updates, and deletes
a row with a "" key. It caused a crash prior to this patch.

Change-Id: I779f25afe6d39d91067b1e7c1238797ec2ac0295
Reviewed-on: http://gerrit.cloudera.org:8080/6163
Tested-by: Kudu Jenkins
Reviewed-by: David Ribeiro Alves <dr...@apache.org>


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

Branch: refs/heads/master
Commit: c11a315e73a20211caf71f2b3797e71af8449af4
Parents: 8f54f60
Author: Todd Lipcon <to...@apache.org>
Authored: Sun Feb 26 23:39:20 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 28 20:52:51 2017 +0000

----------------------------------------------------------------------
 src/kudu/client/client-test.cc | 80 +++++++++++++++++++++++++++++++++++++
 src/kudu/tablet/diskrowset.cc  |  3 +-
 2 files changed, 81 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c11a315e/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index bc57d6d..f7c8e0a 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -3950,6 +3950,86 @@ TEST_F(ClientTest, TestInsertTooLongEncodedPrimaryKey) {
             errors[0]->status().ToString());
 }
 
+// Test trying to insert a row with an empty string PK.
+// Regression test for KUDU-1899.
+TEST_F(ClientTest, TestInsertEmptyPK) {
+  const string kLongValue(10000, 'x');
+  const char* kTableName = "empty-pk";
+
+  // Create and open a table with a three-column composite PK.
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  KuduSchema schema;
+  KuduSchemaBuilder schema_builder;
+  schema_builder.AddColumn("k1")->Type(KuduColumnSchema::STRING)->NotNull();
+  schema_builder.AddColumn("v1")->Type(KuduColumnSchema::STRING)->NotNull();
+  schema_builder.SetPrimaryKey({"k1"});
+  ASSERT_OK(schema_builder.Build(&schema));
+  ASSERT_OK(table_creator->table_name(kTableName)
+            .schema(&schema)
+            .num_replicas(1)
+            .set_range_partition_columns({ "k1" })
+            .Create());
+  shared_ptr<KuduTable> table;
+  ASSERT_OK(client_->OpenTable(kTableName, &table));
+
+  // Find the tablet.
+  scoped_refptr<TabletPeer> tablet_peer;
+  ASSERT_TRUE(cluster_->mini_tablet_server(0)->server()->tablet_manager()->LookupTablet(
+          GetFirstTabletId(table.get()), &tablet_peer));
+
+  // Utility function to get the current value of the row.
+  const auto ReadRowAsString = [&]() {
+    vector<string> rows;
+    ScanTableToStrings(table.get(), &rows);
+    if (rows.empty()) return string("<none>");
+    CHECK_EQ(1, rows.size());
+    return rows[0];
+  };
+
+  shared_ptr<KuduSession> session(client_->NewSession());
+  ASSERT_OK(session->SetFlushMode(KuduSession::MANUAL_FLUSH));
+
+  // Insert a row with empty primary key.
+  {
+    unique_ptr<KuduInsert> insert(table->NewInsert());
+    ASSERT_OK(insert->mutable_row()->SetString(0, ""));
+    ASSERT_OK(insert->mutable_row()->SetString(1, "initial"));
+    ASSERT_OK(session->Apply(insert.release()));
+    ASSERT_OK(session->Flush());
+  }
+  ASSERT_EQ("(string k1=\"\", string v1=\"initial\")", ReadRowAsString());
+
+  // Make sure that Flush works properly, and the data is still readable.
+  ASSERT_OK(tablet_peer->tablet()->Flush());
+  ASSERT_EQ("(string k1=\"\", string v1=\"initial\")", ReadRowAsString());
+
+  // Perform an update.
+  {
+    unique_ptr<KuduUpdate> update(table->NewUpdate());
+    ASSERT_OK(update->mutable_row()->SetString(0, ""));
+    ASSERT_OK(update->mutable_row()->SetString(1, "updated"));
+    ASSERT_OK(session->Apply(update.release()));
+    ASSERT_OK(session->Flush());
+  }
+  ASSERT_EQ("(string k1=\"\", string v1=\"updated\")", ReadRowAsString());
+  ASSERT_OK(tablet_peer->tablet()->FlushAllDMSForTests());
+  ASSERT_EQ("(string k1=\"\", string v1=\"updated\")", ReadRowAsString());
+  ASSERT_OK(tablet_peer->tablet()->Compact(tablet::Tablet::FORCE_COMPACT_ALL));
+  ASSERT_EQ("(string k1=\"\", string v1=\"updated\")", ReadRowAsString());
+
+  // Perform a delete.
+  {
+    unique_ptr<KuduDelete> del(table->NewDelete());
+    ASSERT_OK(del->mutable_row()->SetString(0, ""));
+    ASSERT_OK(session->Apply(del.release()));
+    ASSERT_OK(session->Flush());
+  }
+  ASSERT_EQ("<none>", ReadRowAsString());
+  ASSERT_OK(tablet_peer->tablet()->FlushAllDMSForTests());
+  ASSERT_EQ("<none>", ReadRowAsString());
+  ASSERT_OK(tablet_peer->tablet()->Compact(tablet::Tablet::FORCE_COMPACT_ALL));
+  ASSERT_EQ("<none>", ReadRowAsString());
+}
 
 // Check the behavior of the latest observed timestamp when performing
 // write and read operations.

http://git-wip-us.apache.org/repos/asf/kudu/blob/c11a315e/src/kudu/tablet/diskrowset.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/diskrowset.cc b/src/kudu/tablet/diskrowset.cc
index 06b3897..014ecee 100644
--- a/src/kudu/tablet/diskrowset.cc
+++ b/src/kudu/tablet/diskrowset.cc
@@ -186,7 +186,7 @@ Status DiskRowSetWriter::AppendBlock(const RowBlock &block) {
     }
 
 #ifndef NDEBUG
-    CHECK_LT(Slice(prev_key).compare(enc_key), 0)
+    CHECK(prev_key.size() == 0 || Slice(prev_key).compare(enc_key) < 0)
       << KUDU_REDACT(enc_key.ToDebugString()) << " appended to file not > previous key "
       << KUDU_REDACT(Slice(prev_key).ToDebugString());
 #endif
@@ -214,7 +214,6 @@ Status DiskRowSetWriter::FinishAndReleaseBlocks(ScopedWritableBlockCloser* close
   }
 
   // Save the last encoded (max) key
-  CHECK_GT(last_encoded_key_.size(), 0);
   Slice last_enc_slice(last_encoded_key_);
   std::string first_encoded_key =
       key_index_writer()->GetMetaValueOrDie(DiskRowSet::kMinKeyMetaEntryName);


[5/5] kudu git commit: Micro-optimizations to try to inline DeltaKey::DecodeFrom

Posted by to...@apache.org.
Micro-optimizations to try to inline DeltaKey::DecodeFrom

This is a hot function in update-heavy workloads (eg YCSB workload A).
This is an attempt to make it inlinable, by marking it "hot" and
out-of-lining the error cases into a function marked "noinline".

Change-Id: I0611a3dd1309ab815880c2d151c1270a05f8e406
Reviewed-on: http://gerrit.cloudera.org:8080/6160
Reviewed-by: David Ribeiro Alves <dr...@apache.org>
Tested-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 368f4f734e4acbab2bdb19b824e2ba26cb2cfaa6
Parents: cbb07d2
Author: Todd Lipcon <to...@apache.org>
Authored: Sun Feb 26 19:01:49 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 28 21:24:47 2017 +0000

----------------------------------------------------------------------
 src/kudu/tablet/delta_key.cc |  5 +++++
 src/kudu/tablet/delta_key.h  | 16 +++++++++++-----
 2 files changed, 16 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/368f4f73/src/kudu/tablet/delta_key.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/delta_key.cc b/src/kudu/tablet/delta_key.cc
index 2bfce8f..879e4bd 100644
--- a/src/kudu/tablet/delta_key.cc
+++ b/src/kudu/tablet/delta_key.cc
@@ -34,5 +34,10 @@ const char* DeltaType_Name(DeltaType t) {
   return "UNKNOWN";
 }
 
+Status DeltaKey::DeltaKeyError(const Slice& orig, const char* err) {
+  return Status::Corruption(strings::Substitute("Bad delta key: $0", err),
+                            KUDU_REDACT(orig.ToDebugString(20)));
+}
+
 } // namespace tablet
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/368f4f73/src/kudu/tablet/delta_key.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/delta_key.h b/src/kudu/tablet/delta_key.h
index 402bb74..be5e946 100644
--- a/src/kudu/tablet/delta_key.h
+++ b/src/kudu/tablet/delta_key.h
@@ -72,17 +72,20 @@ class DeltaKey {
   // contain further data after that.
   // The 'key' slice is mutated so that, upon return, the decoded key has been removed from
   // its beginning.
-  Status DecodeFrom(Slice *key) {
+  //
+  // This function is called frequently, so is marked HOT to encourage inlining.
+  Status DecodeFrom(Slice *key) ATTRIBUTE_HOT {
     Slice orig(*key);
     if (!PREDICT_TRUE(DecodeRowId(key, &row_idx_))) {
-      return Status::Corruption("Bad delta key: bad rowid",
-                                KUDU_REDACT(orig.ToDebugString(20)));
+      // Out-of-line the error case to keep this function small and inlinable.
+      return DeltaKeyError(orig, "bad rowid");
     }
 
     if (!PREDICT_TRUE(timestamp_.DecodeFrom(key))) {
-      return Status::Corruption("Bad delta key: bad timestamp",
-                                KUDU_REDACT(orig.ToDebugString(20)));
+      // Out-of-line the error case to keep this function small and inlinable.
+      return DeltaKeyError(orig, "bad timestamp");
     }
+
     return Status::OK();
   }
 
@@ -102,6 +105,9 @@ class DeltaKey {
   const Timestamp &timestamp() const { return timestamp_; }
 
  private:
+  // Out-of-line error construction used by DecodeFrom.
+  static Status DeltaKeyError(const Slice& orig, const char* err);
+
   // The row which has been updated.
   rowid_t row_idx_;