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/04/06 23:23:23 UTC

[1/2] incubator-kudu git commit: Run RAT as part of release source tarball generation

Repository: incubator-kudu
Updated Branches:
  refs/heads/master a6dfbf6de -> 4c4c3ae59


Run RAT as part of release source tarball generation

Tested manually on a failed build and a "good" build.

Change-Id: I8bdadd1eeb124331af2a4c3c4b537d044b0aceae
Reviewed-on: http://gerrit.cloudera.org:8080/2724
Tested-by: Kudu Jenkins
Reviewed-by: Jean-Daniel Cryans


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

Branch: refs/heads/master
Commit: e0b3940839c58402799d75e278f257bd0b7e1180
Parents: a6dfbf6
Author: Mike Percy <mp...@apache.org>
Authored: Mon Apr 4 12:38:23 2016 -0700
Committer: Mike Percy <mp...@apache.org>
Committed: Wed Apr 6 19:20:56 2016 +0000

----------------------------------------------------------------------
 build-support/build_source_release.py | 44 ++++++++++++++++++++++++++++++
 build-support/release/README.adoc     | 13 +++++----
 2 files changed, 52 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/e0b39408/build-support/build_source_release.py
----------------------------------------------------------------------
diff --git a/build-support/build_source_release.py b/build-support/build_source_release.py
index d6973d9..10650b3 100755
--- a/build-support/build_source_release.py
+++ b/build-support/build_source_release.py
@@ -20,8 +20,11 @@
 import hashlib
 import logging
 import os
+import shutil
 import subprocess
 import sys
+import tempfile
+import urllib
 
 from kudu_util import check_output, confirm_prompt, Colors, get_my_email
 
@@ -131,6 +134,46 @@ def gen_checksum_files(tarball_path):
     print Colors.GREEN + ("Generated %s:\t" % extension) + Colors.RESET, path
 
 
+def run_rat(tarball_path):
+  """
+  Run Apache RAT on the source tarball.
+
+  Raises an exception on failure.
+  """
+  if not confirm_prompt("Would you like to run Apache RAT (Release Audit Tool) now?"):
+    return
+
+  # TODO: Cache and call the jar from the maven repo?
+  rat_url = "http://central.maven.org/maven2/org/apache/rat/apache-rat/0.11/apache-rat-0.11.jar"
+
+  tmpdir_path = tempfile.mkdtemp()
+  rat_report_result = ''
+  try:
+    rat_jar_dest = "%s/%s" % (tmpdir_path, os.path.basename(rat_url))
+
+    print "> Downloading RAT jar from " + rat_url
+    urllib.urlretrieve(rat_url, rat_jar_dest)
+
+    print "> Running RAT..."
+    xml = subprocess.check_output(["java", "-jar", rat_jar_dest, "-x", tarball_path])
+    rat_report_dest = "%s/%s" % (tmpdir_path, "rat_report.xml")
+    with open(rat_report_dest, "w") as f:
+        f.write(xml)
+
+    print "> Parsing RAT report..."
+    rat_report_result = subprocess.check_output(
+        ["./build-support/release/check-rat-report.py",
+         "./build-support/release/rat_exclude_files.txt",
+         rat_report_dest],
+        stderr=subprocess.STDOUT)
+    print Colors.GREEN + "RAT: LICENSES APPROVED" + Colors.RESET
+  except subprocess.CalledProcessError as e:
+    print Colors.RED + "RAT: LICENSES NOT APPROVED" + Colors.RESET
+    print e.output
+    raise e
+  finally:
+    shutil.rmtree(tmpdir_path)
+
 def main():
   # Change into the source repo so that we can run git commands without having to
   # specify cwd=BUILD_SUPPORT every time.
@@ -140,6 +183,7 @@ def main():
   tarball_path = create_tarball()
   gen_checksum_files(tarball_path)
   sign_tarball(tarball_path)
+  run_rat(tarball_path)
 
   print Colors.GREEN + "Release successfully generated!" + Colors.RESET
   print

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/e0b39408/build-support/release/README.adoc
----------------------------------------------------------------------
diff --git a/build-support/release/README.adoc b/build-support/release/README.adoc
index b54f0ef..cb24852 100644
--- a/build-support/release/README.adoc
+++ b/build-support/release/README.adoc
@@ -15,13 +15,16 @@
 // specific language governing permissions and limitations
 // under the License.
 
-= How to run RAT on a release tarball
+= How to run Apache RAT on a release tarball
 
 Kudu has a script and an excludes file to allow for running RAT on a release
 artifact. You can use the following steps to verify a source tarball.
 
-1. Download a binary artifact of RAT from
-   http://creadur.apache.org/rat/download_rat.cgi and unpack it.
-2. Run RAT: `java -jar apache-rat-0.11.jar -x apache-kudu-X.Y.Z.tar.gz > rat-report.xml`
-3. ./build-support/release/check-rat-report.py build-support/release/rat_exclude_files.txt rat-report.xml
+1. Download a binary artifact of RAT:
+   `wget http://central.maven.org/maven2/org/apache/rat/apache-rat/0.11/apache-rat-0.11.jar`
+2. Run RAT:
+   `java -jar apache-rat-0.11.jar -x apache-kudu-X.Y.Z.tar.gz > rat-report.xml`
+3. Run the report checker:
+   `./build-support/release/check-rat-report.py build-support/release/rat_exclude_files.txt rat-report.xml`
 
+The above steps are automatically performed as part of the `build-support/build_source_release.py` script.


[2/2] incubator-kudu git commit: KUDU-1396. Fix index building when an individual value overflows block

Posted by to...@apache.org.
KUDU-1396. Fix index building when an individual value overflows block

This fixes an infinite recursion when building the index for a cfile
in the case that a single value is larger than the configured index
block size. Previously, we would see that the block was full, propagate
the key upwards, see that the new block was full, and continue until
we crashed.

The fix is relatively simple: we need to make sure that an index block
has at least two elements before we propagate upwards.

This adds a new cfile test which writes strings that are larger than
64KB.

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


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

Branch: refs/heads/master
Commit: 4c4c3ae59e81d2b80209b94e0dfe49768f21ba8e
Parents: e0b3940
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Apr 5 23:56:06 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Apr 6 21:22:58 2016 +0000

----------------------------------------------------------------------
 src/kudu/cfile/cfile-test-base.h | 19 +++------
 src/kudu/cfile/cfile-test.cc     | 72 +++++++++++++++++++++++++----------
 src/kudu/cfile/index_block.cc    |  4 --
 src/kudu/cfile/index_block.h     |  6 ++-
 src/kudu/cfile/index_btree.cc    | 11 +++---
 5 files changed, 66 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/4c4c3ae5/src/kudu/cfile/cfile-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile-test-base.h b/src/kudu/cfile/cfile-test-base.h
index acff8fe..2aa408f 100644
--- a/src/kudu/cfile/cfile-test-base.h
+++ b/src/kudu/cfile/cfile-test-base.h
@@ -22,6 +22,7 @@
 #include <algorithm>
 #include <stdlib.h>
 #include <string>
+#include <vector>
 
 #include "kudu/cfile/cfile-test-base.h"
 #include "kudu/cfile/cfile_reader.h"
@@ -232,27 +233,17 @@ class StringDataGenerator : public DataGenerator<STRING, HAS_NULLS> {
   }
 
   Slice BuildTestValue(size_t block_index, size_t value) OVERRIDE {
-    char *buf = data_buffer_[block_index].data;
-    int len = snprintf(buf, kItemBufferSize - 1, format_, value);
-    DCHECK_LT(len, kItemBufferSize);
-    return Slice(buf, len);
+    data_buffers_[block_index] = StringPrintf(format_, value);
+    return Slice(data_buffers_[block_index]);
   }
 
   void Resize(size_t num_entries) OVERRIDE {
-    if (num_entries > this->block_entries()) {
-      data_buffer_.reset(new Buffer[num_entries]);
-    }
+    data_buffers_.resize(num_entries);
     DataGenerator<STRING, HAS_NULLS>::Resize(num_entries);
   }
 
  private:
-  static const int kItemBufferSize = 16;
-
-  struct Buffer {
-    char data[kItemBufferSize];
-  };
-
-  gscoped_array<Buffer> data_buffer_;
+  std::vector<std::string> data_buffers_;
   const char* format_;
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/4c4c3ae5/src/kudu/cfile/cfile-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile-test.cc b/src/kudu/cfile/cfile-test.cc
index dcdaf48..5f38306 100644
--- a/src/kudu/cfile/cfile-test.cc
+++ b/src/kudu/cfile/cfile-test.cc
@@ -255,7 +255,7 @@ class TestCFile : public CFileTestBase {
     ASSERT_EQ(num_entries, count);
   }
 
-  void TestReadWriteStrings(EncodingType encoding);
+  void TestReadWriteStrings(EncodingType encoding, const char* format);
 
 #ifdef NDEBUG
   void TestWrite100MFileStrings(EncodingType encoding) {
@@ -470,12 +470,13 @@ void EncodeStringKey(const Schema &schema, const Slice& key,
   encoded_key->reset(kb.BuildEncodedKey());
 }
 
-void TestCFile::TestReadWriteStrings(EncodingType encoding) {
+void TestCFile::TestReadWriteStrings(EncodingType encoding,
+                                     const char* str_format = "hello %04d") {
   Schema schema({ ColumnSchema("key", STRING) }, 1);
 
   const int nrows = 10000;
   BlockId block_id;
-  StringDataGenerator<false> generator("hello %04d");
+  StringDataGenerator<false> generator(str_format);
   WriteTestFile(&generator, encoding, NO_COMPRESSION, nrows,
                 SMALL_BLOCKSIZE | WRITE_VALIDX, &block_id);
 
@@ -500,7 +501,7 @@ void TestCFile::TestReadWriteStrings(EncodingType encoding) {
   Slice s;
 
   CopyOne<STRING>(iter.get(), &s, &arena);
-  ASSERT_EQ(string("hello 5000"), s.ToString());
+  ASSERT_EQ(StringPrintf(str_format, 5000), s.ToString());
 
   // Seek to last key exactly, should succeed
   ASSERT_OK(iter->SeekToOrdinal(9999));
@@ -517,25 +518,29 @@ void TestCFile::TestReadWriteStrings(EncodingType encoding) {
   gscoped_ptr<EncodedKey> encoded_key;
   bool exact;
 
-  // Seek in between each key
+  // Seek in between each key.
+  // (seek to "hello 0000.5" through "hello 9999.5")
+  string buf;
   for (int i = 1; i < 10000; i++) {
+    arena.Reset();
     SCOPED_TRACE(i);
-    char buf[100];
-    snprintf(buf, sizeof(buf), "hello %04d.5", i - 1);
+    SStringPrintf(&buf, str_format, i - 1);
+    buf.append(".5");
     s = Slice(buf);
     EncodeStringKey(schema, s, &encoded_key);
     ASSERT_OK(iter->SeekAtOrAfter(*encoded_key, &exact));
     ASSERT_FALSE(exact);
     ASSERT_EQ(i, iter->GetCurrentOrdinal());
     CopyOne<STRING>(iter.get(), &s, &arena);
-    ASSERT_EQ(StringPrintf("hello %04d", i), s.ToString());
+    ASSERT_EQ(StringPrintf(str_format, i), s.ToString());
   }
 
   // Seek exactly to each key
+  // (seek to "hello 0000" through "hello 9999")
   for (int i = 0; i < 9999; i++) {
+    arena.Reset();
     SCOPED_TRACE(i);
-    char buf[100];
-    snprintf(buf, sizeof(buf), "hello %04d", i);
+    SStringPrintf(&buf, str_format, i);
     s = Slice(buf);
     EncodeStringKey(schema, s, &encoded_key);
     ASSERT_OK(iter->SeekAtOrAfter(*encoded_key, &exact));
@@ -547,32 +552,40 @@ void TestCFile::TestReadWriteStrings(EncodingType encoding) {
   }
 
   // after last entry
-  s = "hello 9999x";
+  // (seek to "hello 9999.x")
+  buf = StringPrintf(str_format, 9999) + "x";
+  s = Slice(buf);
   EncodeStringKey(schema, s, &encoded_key);
   EXPECT_TRUE(iter->SeekAtOrAfter(*encoded_key, &exact).IsNotFound());
 
   // before first entry
-  s = "hello";
+  // (seek to "hello 000", which falls before "hello 0000")
+  buf = StringPrintf(str_format, 0);
+  buf.resize(buf.size() - 1);
+  s = Slice(buf);
   EncodeStringKey(schema, s, &encoded_key);
   ASSERT_OK(iter->SeekAtOrAfter(*encoded_key, &exact));
-  ASSERT_FALSE(exact);
-  ASSERT_EQ(0u, iter->GetCurrentOrdinal());
+  EXPECT_FALSE(exact);
+  EXPECT_EQ(0u, iter->GetCurrentOrdinal());
   CopyOne<STRING>(iter.get(), &s, &arena);
-  ASSERT_EQ(string("hello 0000"), s.ToString());
+  EXPECT_EQ(StringPrintf(str_format, 0), s.ToString());
 
   // Seek to start of file by ordinal
   ASSERT_OK(iter->SeekToFirst());
   ASSERT_EQ(0u, iter->GetCurrentOrdinal());
   CopyOne<STRING>(iter.get(), &s, &arena);
-  ASSERT_EQ(string("hello 0000"), s.ToString());
+  ASSERT_EQ(StringPrintf(str_format, 0), s.ToString());
 
   // Reseek to start and fetch all data.
+  // We fetch in 10 smaller chunks to avoid using too much RAM for the
+  // case where the values are large.
   ASSERT_OK(iter->SeekToFirst());
-
-  ScopedColumnBlock<STRING> cb(10000);
-  size_t n = 10000;
-  ASSERT_OK(iter->CopyNextValues(&n, &cb));
-  ASSERT_EQ(10000, n);
+  for (int i = 0; i < 10; i++) {
+    ScopedColumnBlock<STRING> cb(10000);
+    size_t n = 1000;
+    ASSERT_OK(iter->CopyNextValues(&n, &cb));
+    ASSERT_EQ(1000, n);
+  }
 }
 
 
@@ -585,6 +598,23 @@ TEST_P(TestCFileBothCacheTypes, TestReadWriteStringsDictEncoding) {
   TestReadWriteStrings(DICT_ENCODING);
 }
 
+// Regression test for properly handling cells that are larger
+// than the index block and/or data block size.
+//
+// This test is disabled in TSAN because it's single-threaded anyway
+// and runs extremely slowly with TSAN enabled.
+#ifndef THREAD_SANITIZER
+TEST_P(TestCFileBothCacheTypes, TestReadWriteLargeStrings) {
+  // Pad the values out to a length of ~65KB.
+  const char* kFormat = "%066000d";
+  TestReadWriteStrings(PLAIN_ENCODING, kFormat);
+  if (AllowSlowTests()) {
+    TestReadWriteStrings(DICT_ENCODING, kFormat);
+    TestReadWriteStrings(PREFIX_ENCODING, kFormat);
+  }
+}
+#endif
+
 // Test that metadata entries stored in the cfile are persisted.
 TEST_P(TestCFileBothCacheTypes, TestMetadata) {
   BlockId block_id;

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/4c4c3ae5/src/kudu/cfile/index_block.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/index_block.cc b/src/kudu/cfile/index_block.cc
index b5908d2..90b8b05 100644
--- a/src/kudu/cfile/index_block.cc
+++ b/src/kudu/cfile/index_block.cc
@@ -235,10 +235,6 @@ void IndexBlockReader::GetKeyPointer(int idx_in_block, const uint8_t **ptr,
   }
 }
 
-size_t IndexBlockBuilder::Count() const {
-  return entry_offsets_.size();
-}
-
 void IndexBlockBuilder::Reset() {
   buffer_.clear();
   entry_offsets_.clear();

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/4c4c3ae5/src/kudu/cfile/index_block.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/index_block.h b/src/kudu/cfile/index_block.h
index 7f7fe5a..2fbacf9 100644
--- a/src/kudu/cfile/index_block.h
+++ b/src/kudu/cfile/index_block.h
@@ -66,7 +66,11 @@ class IndexBlockBuilder {
   // The pointed-to data is only valid until the next call to this builder.
   Status GetFirstKey(Slice *key) const;
 
-  size_t Count() const;
+  // Return the number of entries already added to this index
+  // block.
+  size_t count() const {
+    return entry_offsets_.size();
+  }
 
   // Return an estimate of the post-encoding size of this
   // index block. This estimate should be conservative --

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/4c4c3ae5/src/kudu/cfile/index_btree.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/index_btree.cc b/src/kudu/cfile/index_btree.cc
index f9065eb..c31c915 100644
--- a/src/kudu/cfile/index_btree.cc
+++ b/src/kudu/cfile/index_btree.cc
@@ -60,12 +60,11 @@ Status IndexTreeBuilder::Append(
   IndexBlockBuilder* idx_block = idx_blocks_[level].get();
   idx_block->Add(key, block_ptr);
 
+  // This index block is full, and there are at least two entries,
+  // flush it.
   size_t est_size = idx_block->EstimateEncodedSize();
-  if (est_size > options_->index_block_size) {
-    DCHECK(idx_block->Count() > 1)
-      << "Index block full with only one entry - this would create "
-      << "an infinite loop";
-    // This index block is full, flush it.
+  if (est_size > options_->index_block_size &&
+      idx_block->count() > 1) {
     BlockPointer index_block_ptr;
     RETURN_NOT_OK(FinishBlockAndPropagate(level));
   }
@@ -109,7 +108,7 @@ Status IndexTreeBuilder::FinishBlockAndPropagate(size_t level) {
   // and then the file completes.
   //
   // TODO: add a test case which exercises this explicitly.
-  if (idx_block->Count() == 0) {
+  if (idx_block->count() == 0) {
     return Status::OK();
   }