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/01/16 02:33:14 UTC

[1/4] incubator-kudu git commit: Add ccache-clang to build-support

Repository: incubator-kudu
Updated Branches:
  refs/heads/master 9daafa5e7 -> 704496559


Add ccache-clang to build-support

This is a wrapper which allows using ccache with clang. This also adjusts
the Jenkins build script to use it if ccache is installed.

This can dramatically improve performance of TSAN/ASAN builds which require
clang.

Change-Id: If7e160594f45d516fbb08121968a59a97289849e
Reviewed-on: http://gerrit.cloudera.org:8080/1790
Tested-by: Internal Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: c8049c19faf5432a04a1cb795b9032d2859c648b
Parents: 9daafa5
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Jan 14 17:58:22 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Sat Jan 16 00:42:29 2016 +0000

----------------------------------------------------------------------
 build-support/ccache-clang/clang        | 21 +++++++++++++++++++++
 build-support/ccache-clang/clang++      | 21 +++++++++++++++++++++
 build-support/jenkins/build-and-test.sh |  6 +++++-
 3 files changed, 47 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/c8049c19/build-support/ccache-clang/clang
----------------------------------------------------------------------
diff --git a/build-support/ccache-clang/clang b/build-support/ccache-clang/clang
new file mode 100755
index 0000000..a31b1ea
--- /dev/null
+++ b/build-support/ccache-clang/clang
@@ -0,0 +1,21 @@
+#!/bin/bash -e
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+ROOT=$(dirname $BASH_SOURCE)/../..
+CLANG=$ROOT/thirdparty/clang-toolchain/bin/clang
+CCACHE_CPP2=yes exec ccache $CLANG -Qunused-arguments `test -t 2 && echo -fcolor-diagnostics` "$@"

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/c8049c19/build-support/ccache-clang/clang++
----------------------------------------------------------------------
diff --git a/build-support/ccache-clang/clang++ b/build-support/ccache-clang/clang++
new file mode 100755
index 0000000..9cf7f90
--- /dev/null
+++ b/build-support/ccache-clang/clang++
@@ -0,0 +1,21 @@
+#!/bin/bash -e
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+ROOT=$(dirname $BASH_SOURCE)/../..
+CLANG=$ROOT/thirdparty/clang-toolchain/bin/clang++
+CCACHE_CPP2=yes exec ccache $CLANG -Qunused-arguments `test -t 2 && echo -fcolor-diagnostics` "$@"

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/c8049c19/build-support/jenkins/build-and-test.sh
----------------------------------------------------------------------
diff --git a/build-support/jenkins/build-and-test.sh b/build-support/jenkins/build-and-test.sh
index 4330b09..4f9c640 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -141,7 +141,11 @@ $ROOT/build-support/enable_devtoolset.sh thirdparty/build-if-necessary.sh
 THIRDPARTY_BIN=$(pwd)/thirdparty/installed/bin
 export PPROF_PATH=$THIRDPARTY_BIN/pprof
 
-CLANG=$(pwd)/thirdparty/clang-toolchain/bin/clang
+if which ccache >/dev/null ; then
+  CLANG=$(pwd)/build-support/ccache-clang/clang
+else
+  CLANG=$(pwd)/thirdparty/clang-toolchain/bin/clang
+fi
 
 # Configure the build
 #


[2/4] incubator-kudu git commit: TableInfo::RemoveTablet should use partition_key_start

Posted by to...@apache.org.
TableInfo::RemoveTablet should use partition_key_start

TableInfo::RemoveTablet try to remove tablet by tablet_id, but
tablet_map_ uses partition_key_start as key, so it does not work.
This commit fix this bug and add unit test for RemoveTablet.

Change-Id: I3b34ea8d4e04dd8a8b567dfc451edeabe34c7e2b
Reviewed-on: http://gerrit.cloudera.org:8080/1768
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Internal 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/a80b83fe
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kudu/tree/a80b83fe
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kudu/diff/a80b83fe

Branch: refs/heads/master
Commit: a80b83fe5e6a3cac1c310c2764fca0b6ca4a60e3
Parents: c8049c1
Author: Binglin Chang <de...@gmail.com>
Authored: Tue Jan 12 15:51:54 2016 +0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Sat Jan 16 00:42:45 2016 +0000

----------------------------------------------------------------------
 src/kudu/master/catalog_manager-test.cc |  8 ++++++++
 src/kudu/master/catalog_manager.cc      | 14 +++++---------
 src/kudu/master/catalog_manager.h       |  6 +++---
 3 files changed, 16 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/a80b83fe/src/kudu/master/catalog_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager-test.cc b/src/kudu/master/catalog_manager-test.cc
index d7d3ea6..2dee461 100644
--- a/src/kudu/master/catalog_manager-test.cc
+++ b/src/kudu/master/catalog_manager-test.cc
@@ -32,6 +32,7 @@ using strings::Substitute;
 TEST(TableInfoTest, TestAssignmentRanges) {
   const string table_id = CURRENT_TEST_NAME();
   scoped_refptr<TableInfo> table(new TableInfo(table_id));
+  vector<scoped_refptr<TabletInfo> > tablets;
 
   // Define & create the splits.
   const int kNumSplits = 3;
@@ -50,6 +51,8 @@ TEST(TableInfoTest, TestAssignmentRanges) {
     meta_lock.mutable_data()->pb.set_state(SysTabletsEntryPB::RUNNING);
 
     table->AddTablet(tablet);
+    meta_lock.Commit();
+    tablets.push_back(make_scoped_refptr(tablet));
   }
 
   // Ensure they give us what we are expecting.
@@ -73,6 +76,11 @@ TEST(TableInfoTest, TestAssignmentRanges) {
     ASSERT_EQ(tablet_id, (*tablets_in_range.begin())->tablet_id());
     LOG(INFO) << "Key " << start_key << " found in tablet " << tablet_id;
   }
+
+  BOOST_FOREACH(const scoped_refptr<TabletInfo>& tablet, tablets) {
+    ASSERT_TRUE(table->RemoveTablet(
+        tablet->metadata().state().pb.partition().partition_key_start()));
+  }
 }
 
 } // namespace master

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/a80b83fe/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 8a3b77b..8b1cfb8 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -2735,8 +2735,9 @@ Status CatalogManager::ProcessPendingAssignments(
     vector<string> tablet_ids_to_remove;
     for (scoped_refptr<TabletInfo>& new_tablet : new_tablets) {
       TableInfo* table = new_tablet->table().get();
-      TableMetadataLock l_table(table, TableMetadataLock::WRITE);
-      if (table->RemoveTablet(new_tablet->tablet_id())) {
+      TableMetadataLock l_table(table, TableMetadataLock::READ);
+      if (table->RemoveTablet(
+          new_tablet->metadata().dirty().pb.partition().partition_key_start())) {
         VLOG(1) << "Removed tablet " << new_tablet->tablet_id() << " from "
             "table " << l_table.data().name();
       }
@@ -3113,14 +3114,9 @@ std::string TableInfo::ToString() const {
   return Substitute("$0 [id=$1]", l.data().pb.name(), table_id_);
 }
 
-bool TableInfo::RemoveTablet(const string& tablet_id) {
+bool TableInfo::RemoveTablet(const std::string& partition_key_start) {
   boost::lock_guard<simple_spinlock> l(lock_);
-  if (tablet_map_.find(tablet_id) != tablet_map_.end()) {
-    CHECK_EQ(tablet_map_.erase(tablet_id), 1)
-        << "Unable to erase tablet " << tablet_id << " from TableInfo.";
-    return true;
-  }
-  return false;
+  return EraseKeyReturnValuePtr(&tablet_map_, partition_key_start) != NULL;
 }
 
 void TableInfo::AddTablet(TabletInfo *tablet) {

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/a80b83fe/src/kudu/master/catalog_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index 11c4b9f..f6c7db0 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -214,9 +214,9 @@ class TableInfo : public RefCountedThreadSafe<TableInfo> {
   // Add multiple tablets to this table.
   void AddTablets(const std::vector<TabletInfo*>& tablets);
 
-  // Return true if tablet with 'tablet_id' has been removed from
-  // 'tablet_map_' below.
-  bool RemoveTablet(const std::string& tablet_id);
+  // Return true if tablet with 'partition_key_start' has been
+  // removed from 'tablet_map_' below.
+  bool RemoveTablet(const std::string& partition_key_start);
 
   // This only returns tablets which are in RUNNING state.
   void GetTabletsInRange(const GetTableLocationsRequestPB* req,


[4/4] incubator-kudu git commit: Fix build failure with a BOOST_FOREACH that snuck in

Posted by to...@apache.org.
Fix build failure with a BOOST_FOREACH that snuck in

Committing patches out of order caused a new BOOST_FOREACH to sneak in after
the include was removed from a file. This converts it to C++11 foreach.

Change-Id: Ie6f4c652764833b155d958bf2ae2f75ef044a643
Reviewed-on: http://gerrit.cloudera.org:8080/1807
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 704496559510972960e39c9e65e1091073d46d50
Parents: 994fa40
Author: Todd Lipcon <to...@apache.org>
Authored: Fri Jan 15 17:30:14 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Sat Jan 16 01:32:44 2016 +0000

----------------------------------------------------------------------
 src/kudu/master/catalog_manager-test.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/70449655/src/kudu/master/catalog_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager-test.cc b/src/kudu/master/catalog_manager-test.cc
index 2dee461..b3dd302 100644
--- a/src/kudu/master/catalog_manager-test.cc
+++ b/src/kudu/master/catalog_manager-test.cc
@@ -77,7 +77,7 @@ TEST(TableInfoTest, TestAssignmentRanges) {
     LOG(INFO) << "Key " << start_key << " found in tablet " << tablet_id;
   }
 
-  BOOST_FOREACH(const scoped_refptr<TabletInfo>& tablet, tablets) {
+  for (const scoped_refptr<TabletInfo>& tablet : tablets) {
     ASSERT_TRUE(table->RemoveTablet(
         tablet->metadata().state().pb.partition().partition_key_start()));
   }


[3/4] incubator-kudu git commit: KUDU-815. Fix UBSAN error on Schemas with no columns

Posted by to...@apache.org.
KUDU-815. Fix UBSAN error on Schemas with no columns

This fixes a UBSAN error when calling .data() on an empty vector. In theory,
the C++11 standard allows this, but it appears that our libstdcxx
implementation ends up implementing it on top of vector::front(), which returns
a NULL reference in the case of an empty array. This is, of course, undefined
behavior.

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


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

Branch: refs/heads/master
Commit: 994fa40ec36f8874fb59585213b0363a9ecaa9be
Parents: a80b83f
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Jan 12 19:50:19 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Sat Jan 16 00:43:09 2016 +0000

----------------------------------------------------------------------
 src/kudu/common/id_mapping.cc  |  6 +++++-
 src/kudu/common/schema.cc      | 13 ++++++++++---
 src/kudu/util/memenv/memenv.cc |  4 +++-
 3 files changed, 18 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/994fa40e/src/kudu/common/id_mapping.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/id_mapping.cc b/src/kudu/common/id_mapping.cc
index 50a73c0..3a02520 100644
--- a/src/kudu/common/id_mapping.cc
+++ b/src/kudu/common/id_mapping.cc
@@ -24,7 +24,11 @@ namespace kudu {
 const int IdMapping::kNoEntry = -1;
 
 size_t IdMapping::memory_footprint_excluding_this() const {
-  return kudu_malloc_usable_size(entries_.data());
+  if (entries_.capacity() > 0) {
+    return kudu_malloc_usable_size(entries_.data());
+  } else {
+    return 0;
+  }
 }
 
 size_t IdMapping::memory_footprint_including_this() const {

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/994fa40e/src/kudu/common/schema.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/schema.cc b/src/kudu/common/schema.cc
index e722d20..0f16ffb 100644
--- a/src/kudu/common/schema.cc
+++ b/src/kudu/common/schema.cc
@@ -349,13 +349,20 @@ string Schema::DebugEncodedRowKey(Slice encoded_key, StartOrEnd start_or_end) co
 }
 
 size_t Schema::memory_footprint_excluding_this() const {
-  size_t size = kudu_malloc_usable_size(cols_.data());
+  size_t size = 0;
   for (const ColumnSchema& col : cols_) {
     size += col.memory_footprint_excluding_this();
   }
 
-  size += kudu_malloc_usable_size(col_ids_.data());
-  size += kudu_malloc_usable_size(col_offsets_.data());
+  if (cols_.capacity() > 0) {
+    size += kudu_malloc_usable_size(cols_.data());
+  }
+  if (col_ids_.capacity() > 0) {
+    size += kudu_malloc_usable_size(col_ids_.data());
+  }
+  if (col_offsets_.capacity() > 0) {
+    size += kudu_malloc_usable_size(col_offsets_.data());
+  }
   size += name_to_index_bytes_;
   size += id_to_index_.memory_footprint_excluding_this();
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/994fa40e/src/kudu/util/memenv/memenv.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/memenv/memenv.cc b/src/kudu/util/memenv/memenv.cc
index a80bac5..1a5063f 100644
--- a/src/kudu/util/memenv/memenv.cc
+++ b/src/kudu/util/memenv/memenv.cc
@@ -130,7 +130,9 @@ class FileState : public RefCountedThreadSafe<FileState> {
 
   size_t memory_footprint() const {
     size_t size = kudu_malloc_usable_size(this);
-    size += kudu_malloc_usable_size(blocks_.data());
+    if (blocks_.capacity() > 0) {
+      size += kudu_malloc_usable_size(blocks_.data());
+    }
     for (uint8_t* block : blocks_) {
       size += kudu_malloc_usable_size(block);
     }