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

[1/2] incubator-kudu git commit: Make ParseAddressList not OK empty address lists

Repository: incubator-kudu
Updated Branches:
  refs/heads/master ea9dedbd1 -> 11de5f2d1


Make ParseAddressList not OK empty address lists

ParseAddressList would return Status::OK, even when called to parse
an empty list of addresses. In some cases (KUDU-1490), this could
lead to a segfault if the caller doesn't check to see if the returned
vector is empty and tries to access the returned elements.

Current usages of ParseAddressList don't seem to have a standard behavior
that makes sense when called with an empty address string. Instead,
ParseAddressList should do the check and return Status::InvalidArgument.

Change-Id: Idedc2ad9f2ab321a5bb7aa65ea59b8f1a4186377
Reviewed-on: http://gerrit.cloudera.org:8080/3486
Tested-by: Kudu Jenkins
Reviewed-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/f38cfc8e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kudu/tree/f38cfc8e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kudu/diff/f38cfc8e

Branch: refs/heads/master
Commit: f38cfc8e9262781af7f9d42046ac46b01a69b1d9
Parents: ea9dedb
Author: Andrew Wong <an...@cloudera.com>
Authored: Fri Jun 24 13:25:35 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Sat Jun 25 00:27:43 2016 +0000

----------------------------------------------------------------------
 src/kudu/tools/kudu-ksck.cc   | 2 ++
 src/kudu/util/net/net_util.cc | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/f38cfc8e/src/kudu/tools/kudu-ksck.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-ksck.cc b/src/kudu/tools/kudu-ksck.cc
index a663750..dd98049 100644
--- a/src/kudu/tools/kudu-ksck.cc
+++ b/src/kudu/tools/kudu-ksck.cc
@@ -80,6 +80,8 @@ static void RunKsck(vector<string>* error_messages) {
                                        master::Master::kDefaultPort,
                                        &master_addrs),
                       error_messages, "Unable to parse master address");
+  // return early if the addresses could not be resolved
+  if (!error_messages->empty()) return;
 
   shared_ptr<KsckMaster> master;
   PUSH_PREPEND_NOT_OK(RemoteKsckMaster::Build(master_addrs[0], &master),

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/f38cfc8e/src/kudu/util/net/net_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/net_util.cc b/src/kudu/util/net/net_util.cc
index 0cb5101..6b1d4f6 100644
--- a/src/kudu/util/net/net_util.cc
+++ b/src/kudu/util/net/net_util.cc
@@ -168,8 +168,8 @@ Status ParseAddressList(const std::string& addr_list,
                         std::vector<Sockaddr>* addresses) {
   vector<HostPort> host_ports;
   RETURN_NOT_OK(HostPort::ParseStrings(addr_list, default_port, &host_ports));
+  if (host_ports.empty()) return Status::InvalidArgument("No address specified");
   unordered_set<Sockaddr> uniqued;
-
   for (const HostPort& host_port : host_ports) {
     vector<Sockaddr> this_addresses;
     RETURN_NOT_OK(host_port.ResolveAddresses(&this_addresses));


[2/2] incubator-kudu git commit: Add a Barrier implementation and use it in cbtree-test

Posted by ad...@apache.org.
Add a Barrier implementation and use it in cbtree-test

This removes another dependency on boost (boost::barrier).

Change-Id: I26d945dd83431e062f2a1b22512ddf4d37c89e01
Reviewed-on: http://gerrit.cloudera.org:8080/3494
Tested-by: Kudu 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/11de5f2d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kudu/tree/11de5f2d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kudu/diff/11de5f2d

Branch: refs/heads/master
Commit: 11de5f2d120afc4edd1ccee03c5aca2899f12832
Parents: f38cfc8
Author: Todd Lipcon <to...@cloudera.com>
Authored: Fri Jun 24 17:12:08 2016 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Sun Jun 26 23:33:36 2016 +0000

----------------------------------------------------------------------
 src/kudu/tablet/cbtree-test.cc | 38 ++++++++++-----------
 src/kudu/util/barrier.h        | 67 +++++++++++++++++++++++++++++++++++++
 2 files changed, 86 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/11de5f2d/src/kudu/tablet/cbtree-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/cbtree-test.cc b/src/kudu/tablet/cbtree-test.cc
index c29a57f..5750cfa 100644
--- a/src/kudu/tablet/cbtree-test.cc
+++ b/src/kudu/tablet/cbtree-test.cc
@@ -15,7 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <boost/thread/barrier.hpp>
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 #include <thread>
@@ -23,6 +22,7 @@
 
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/tablet/concurrent_btree.h"
+#include "kudu/util/barrier.h"
 #include "kudu/util/hexdump.h"
 #include "kudu/util/memory/memory.h"
 #include "kudu/util/memory/overwrite.h"
@@ -245,20 +245,20 @@ void VerifyRange(const CBTree<T> &tree,
 // into the given tree, then verifies that they are all
 // inserted properly
 template<class T>
-void InsertAndVerify(boost::barrier *go_barrier,
-                     boost::barrier *done_barrier,
+void InsertAndVerify(Barrier *go_barrier,
+                     Barrier *done_barrier,
                      gscoped_ptr<CBTree<T> > *tree,
                      int start_idx,
                      int end_idx) {
   while (true) {
-    go_barrier->wait();
+    go_barrier->Wait();
 
     if (tree->get() == nullptr) return;
 
     InsertRange(tree->get(), start_idx, end_idx);
     VerifyRange(*tree->get(), start_idx, end_idx);
 
-    done_barrier->wait();
+    done_barrier->Wait();
   }
 }
 
@@ -432,8 +432,8 @@ void TestCBTree::DoTestConcurrentInsert() {
 #endif
 
   vector<thread> threads;
-  boost::barrier go_barrier(num_threads + 1);
-  boost::barrier done_barrier(num_threads + 1);
+  Barrier go_barrier(num_threads + 1);
+  Barrier done_barrier(num_threads + 1);
 
 
   for (int i = 0; i < num_threads; i++) {
@@ -453,9 +453,9 @@ void TestCBTree::DoTestConcurrentInsert() {
 
   for (int trial = 0; trial < n_trials; trial++) {
     tree.reset(new CBTree<TraitsClass>());
-    go_barrier.wait();
+    go_barrier.Wait();
 
-    done_barrier.wait();
+    done_barrier.Wait();
 
     if (::testing::Test::HasFatalFailure()) {
       tree->DebugPrint();
@@ -464,7 +464,7 @@ void TestCBTree::DoTestConcurrentInsert() {
   }
 
   tree.reset(nullptr);
-  go_barrier.wait();
+  go_barrier.Wait();
 
   for (thread &thr : threads) {
     thr.join();
@@ -638,11 +638,11 @@ TEST_F(TestCBTree, TestIteratorSeekConditions) {
 //   go_barrier: waits on this barrier to start running
 //   done_barrier: waits on this barrier once finished.
 template<class T>
-static void ScanThread(boost::barrier *go_barrier,
-                       boost::barrier *done_barrier,
+static void ScanThread(Barrier *go_barrier,
+                       Barrier *done_barrier,
                        gscoped_ptr<CBTree<T> > *tree) {
   while (true) {
-    go_barrier->wait();
+    go_barrier->Wait();
     if (tree->get() == nullptr) return;
 
     int prev_count = 0;
@@ -672,7 +672,7 @@ static void ScanThread(boost::barrier *go_barrier,
       ASSERT_GE(count, prev_count);
     } while (count != prev_count || count == 0);
 
-    done_barrier->wait();
+    done_barrier->Wait();
   }
 }
 
@@ -693,8 +693,8 @@ TEST_F(TestCBTree, TestConcurrentIterateAndInsert) {
   }
 
   vector<thread> threads;
-  boost::barrier go_barrier(num_threads + 1);
-  boost::barrier done_barrier(num_threads + 1);
+  Barrier go_barrier(num_threads + 1);
+  Barrier done_barrier(num_threads + 1);
 
   for (int i = 0; i < num_ins_threads; i++) {
     threads.emplace_back(InsertAndVerify<SmallFanoutTraits>,
@@ -718,9 +718,9 @@ TEST_F(TestCBTree, TestConcurrentIterateAndInsert) {
   // on areas of the key space diminishes.
   for (int trial = 0; trial < trials; trial++) {
     tree.reset(new CBTree<SmallFanoutTraits>());
-    go_barrier.wait();
+    go_barrier.Wait();
 
-    done_barrier.wait();
+    done_barrier.Wait();
 
     if (::testing::Test::HasFatalFailure()) {
       tree->DebugPrint();
@@ -729,7 +729,7 @@ TEST_F(TestCBTree, TestConcurrentIterateAndInsert) {
   }
 
   tree.reset(nullptr);
-  go_barrier.wait();
+  go_barrier.Wait();
 
   for (thread& thr : threads) {
     thr.join();

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/11de5f2d/src/kudu/util/barrier.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/barrier.h b/src/kudu/util/barrier.h
new file mode 100644
index 0000000..901d46d
--- /dev/null
+++ b/src/kudu/util/barrier.h
@@ -0,0 +1,67 @@
+// 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.
+#pragma once
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/condition_variable.h"
+#include "kudu/util/mutex.h"
+
+namespace kudu {
+
+// Implementation of pthread-style Barriers.
+class Barrier {
+ public:
+  // Initialize the barrier with the given initial count.
+  explicit Barrier(int count) :
+      cond_(&mutex_),
+      count_(count),
+      initial_count_(count) {
+    DCHECK_GT(count, 0);
+  }
+
+  ~Barrier() {
+  }
+
+  // Wait until all threads have reached the barrier.
+  // Once all threads have reached the barrier, the barrier is reset
+  // to the initial count.
+  void Wait() {
+    ThreadRestrictions::AssertWaitAllowed();
+    MutexLock l(mutex_);
+    if (--count_ == 0) {
+      count_ = initial_count_;
+      cycle_count_++;
+      cond_.Broadcast();
+      return;
+    }
+
+    int initial_cycle = cycle_count_;
+    while (cycle_count_ == initial_cycle) {
+      cond_.Wait();
+    }
+  }
+
+ private:
+  Mutex mutex_;
+  ConditionVariable cond_;
+  int count_;
+  uint32_t cycle_count_ = 0;
+  const int initial_count_;
+  DISALLOW_COPY_AND_ASSIGN(Barrier);
+};
+
+} // namespace kudu