You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@kudu.apache.org by "Adar Dembo (Code Review)" <ge...@cloudera.org> on 2017/06/29 00:24:45 UTC

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Hello David Ribeiro Alves, Mike Percy,

I'd like you to do a code review.  Please visit

    http://gerrit.cloudera.org:8080/7328

to review the following change.

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................

time_manager: fix unsynchronized access to GetSerialTimestamp()

This produced the occasional race, which I've attached to the bottom of the
commit message.

I also revoked friendship for RaftConsensus. This was done solely to get
access to GetSerialTimestamp; making the function public seems like the
lesser of two evils to me.

WARNING: ThreadSanitizer: data race (pid=15727)
  Read of size 8 at 0x7b1c00013818 by thread T126 (mutexes: write M2582, write M2285):
    #0 kudu::operator<(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:105:22 (libtserver.so+0xe1768)
    #1 kudu::operator>(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:109:14 (libtserver.so+0xd9170)
    #2 kudu::operator<=(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:113:16 (libtablet.so+0x19e990)
    #3 kudu::consensus::TimeManager::GetSafeTimeUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:291:11 (libconsensus.so+0xbddad)
    #4 kudu::consensus::TimeManager::GetSafeTime() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:267:10 (libconsensus.so+0xbdeb9)
    #5 kudu::consensus::PeerMessageQueue::RequestForPeer(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::ConsensusRequestPB*, std::__1::vector<scoped_refptr<kudu::consensus::RefCountedReplicate>, std::__1::allocator<scoped_refptr<kudu::consensus::RefCountedReplicate> > >*, bool*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:469:50 (libconsensus.so+0x70a3a)
    #6 kudu::consensus::Peer::SendNextRequest(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:177:22 (libconsensus.so+0x649c2)
    #7 kudu::consensus::Peer::SignalRequest(bool)::$_0::operator()() const /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:134:3 (libconsensus.so+0x67a02)
    #8 boost::detail::function::void_function_obj_invoker0<kudu::consensus::Peer::SignalRequest(bool)::$_0, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libconsensus.so+0x67809)
    #9 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #10 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #11 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #12 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #13 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #14 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #15 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #16 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #17 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Previous write of size 8 at 0x7b1c00013818 by thread T76 (mutexes: write M2279):
    #0 kudu::consensus::TimeManager::GetSerialTimestamp() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:310:28 (libconsensus.so+0xbcbbb)
    #1 kudu::consensus::RaftConsensus::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const&, kudu::tserver::TabletServerErrorPB_Code*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1668:36 (libconsensus.so+0xacabd)
    #2 kudu::tserver::ConsensusServiceImpl::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const*, kudu::consensus::UnsafeChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:941:25 (libtserver.so+0xc98cf)
    #3 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:160:13 (libconsensus_proto.so+0x825b4)
    #4 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_7JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x82541)
    #5 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_7PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x82541)
    #6 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x82454)
    #7 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #8 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #9 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #10 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #11 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Location is heap block of size 104 at 0x7b1c000137f0 allocated by thread T123:
    #0 operator new(unsigned long) /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_new_delete.cc:41 (kudu-tserver+0x4c0d63)
    #1 kudu::tablet::TabletReplica::Init(std::__1::shared_ptr<kudu::tablet::Tablet> const&, scoped_refptr<kudu::server::Clock> const&, std::__1::shared_ptr<kudu::rpc::Messenger> const&, scoped_refptr<kudu::rpc::ResultTracker> const&, scoped_refptr<kudu::log::Log> const&, scoped_refptr<kudu::MetricEntity> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:162:45 (libtablet.so+0x139f6a)
    #2 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:770:19 (libtserver.so+0xe6991)
    #3 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #4 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #5 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #6 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #7 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #8 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #9 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #10 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #11 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2582 (0x7b54000b0130) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x6427c)
    #5 kudu::consensus::Peer::Init() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:119 (libconsensus.so+0x6427c)
    #6 kudu::consensus::Peer::NewRemotePeer(kudu::consensus::RaftPeerPB const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::PeerMessageQueue*, kudu::ThreadPool*, gscoped_ptr<kudu::consensus::PeerProxy, kudu::DefaultDeleter<kudu::consensus::PeerProxy> >, std::__1::shared_ptr<kudu::consensus::Peer>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:97:3 (libconsensus.so+0x64101)
    #7 kudu::consensus::PeerManager::UpdateRaftConfig(kudu::consensus::RaftConfigPB const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/peer_manager.cc:73:5 (libconsensus.so+0x908e1)
    #8 kudu::consensus::RaftConsensus::RefreshConsensusQueueAndPeersUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2044:3 (libconsensus.so+0xa254f)
    #9 kudu::consensus::RaftConsensus::AddPendingOperationUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:668:9 (libconsensus.so+0xa3787)
    #10 kudu::consensus::RaftConsensus::AppendNewRoundToQueueUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:626:3 (libconsensus.so+0xa2d76)
    #11 kudu::consensus::RaftConsensus::ReplicateConfigChangeUnlocked(kudu::consensus::RaftConfigPB const&, kudu::consensus::RaftConfigPB const&, kudu::Callback<void ()(kudu::Status const&)> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2025:3 (libconsensus.so+0xac5f3)
    #12 kudu::consensus::RaftConsensus::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const&, kudu::Callback<void ()(kudu::Status const&)> const&, boost::optional<kudu::tserver::TabletServerErrorPB_Code>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1621:5 (libconsensus.so+0xa5c0b)
    #13 kudu::tserver::ConsensusServiceImpl::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const*, kudu::consensus::ChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:917:25 (libtserver.so+0xc8d55)
    #14 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:140:13 (libconsensus_proto.so+0x81e94)
    #15 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_5JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x81e21)
    #16 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_5PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x81e21)
    #17 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x81d34)
    #18 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #19 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #20 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #21 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #22 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #23 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #24 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #25 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #26 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2285 (0x7b1c000137f8) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0xbc8ee)
    #5 kudu::consensus::TimeManager::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:78 (libconsensus.so+0xbc8ee)
    #6 kudu::consensus::PeerMessageQueue::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:186:18 (libconsensus.so+0x6dbe9)
    #7 kudu::consensus::RaftConsensus::BecomeReplicaUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:593:11 (libconsensus.so+0x9e9fa)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:340:5 (libconsensus.so+0x9d41f)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2279 (0x7b540006fe44) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x9cf38)
    #5 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:282 (libconsensus.so+0x9cf38)
    #6 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #7 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #8 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #9 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #10 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #11 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #12 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #13 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #14 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #15 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #16 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #17 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #18 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #19 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #20 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T126 'cc8abe-raft [wo' (tid=17191, running) created by thread T123 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitClosure(kudu::Callback<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:430:10 (libkudu_util.so+0x1c15c9)
    #7 kudu::consensus::RaftConsensus::MarkDirty(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2243:3 (libconsensus.so+0x9f83a)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:351:3 (libconsensus.so+0x9d5f2)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T76 'rpc worker-1602' (tid=16028, running) created by main thread at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::rpc::ServicePool::*)(), kudu::rpc::ServicePool*>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::rpc::ServicePool::* const&)(), kudu::rpc::ServicePool* const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:158:12 (libkrpc.so+0xe9615)
    #3 kudu::rpc::ServicePool::Init(int) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:81:5 (libkrpc.so+0xe7ea5)
    #4 kudu::RpcServer::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/rpc_server.cc:144:3 (libserver_process.so+0x561d1)
    #5 kudu::server::ServerBase::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/server_base.cc:364:23 (libserver_process.so+0x5c1cf)
    #6 kudu::tserver::TabletServer::Start() /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server.cc:117:3 (libtserver.so+0xc1441)
    #7 kudu::tserver::TabletServerMain(int, char**) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:77:3 (kudu-tserver+0x4c3778)
    #8 main /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:91:10 (kudu-tserver+0x4c33ee)

  Thread T123 'tablet-open [wo' (tid=17160, finished) created by thread T56 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitFunc(boost::function<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:434:10 (libkudu_util.so+0x1c1729)
    #7 kudu::tserver::TSTabletManager::CreateNewTablet(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Partition const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Schema const&, kudu::PartitionSchema const&, kudu::consensus::RaftConfigPB, scoped_refptr<kudu::tablet::TabletReplica>*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:281:3 (libtserver.so+0xe79a1)
    #8 kudu::tserver::TabletServiceAdminImpl::CreateTablet(kudu::tserver::CreateTabletRequestPB const*, kudu::tserver::CreateTabletResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:678:34 (libtserver.so+0xc51ab)
    #9 kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/tserver/tserver_admin.service.cc:58:13 (libtserver_admin_proto.so+0x27ec4)
    #10 _ZNSt3__18__invokeIRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_1JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libtserver_admin_proto.so+0x27e51)
    #11 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_1PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libtserver_admin_proto.so+0x27e51)
    #12 std::__1::__function::__func<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1, std::__1::allocator<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libtserver_admin_proto.so+0x27d64)
    #13 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #14 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #15 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #16 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #17 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #18 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #19 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #20 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #21 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
---
M src/kudu/consensus/time_manager.cc
M src/kudu/consensus/time_manager.h
2 files changed, 27 insertions(+), 7 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/28/7328/1
-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Mike Percy <mp...@apache.org>

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Posted by "Adar Dembo (Code Review)" <ge...@cloudera.org>.
Hello Mike Percy, Todd Lipcon, Kudu Jenkins,

I'd like you to reexamine a change.  Please visit

    http://gerrit.cloudera.org:8080/7328

to look at the new patch set (#3).

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................

time_manager: fix unsynchronized access to GetSerialTimestamp()

This access produced the occasional race, which I've attached to the bottom
of the commit message.

I also revoked friendship for RaftConsensus. This was done solely to get
access to GetSerialTimestamp; making the function public seems like the
lesser of two evils to me.

WARNING: ThreadSanitizer: data race (pid=15727)
  Read of size 8 at 0x7b1c00013818 by thread T126 (mutexes: write M2582, write M2285):
    #0 kudu::operator<(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:105:22 (libtserver.so+0xe1768)
    #1 kudu::operator>(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:109:14 (libtserver.so+0xd9170)
    #2 kudu::operator<=(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:113:16 (libtablet.so+0x19e990)
    #3 kudu::consensus::TimeManager::GetSafeTimeUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:291:11 (libconsensus.so+0xbddad)
    #4 kudu::consensus::TimeManager::GetSafeTime() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:267:10 (libconsensus.so+0xbdeb9)
    #5 kudu::consensus::PeerMessageQueue::RequestForPeer(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::ConsensusRequestPB*, std::__1::vector<scoped_refptr<kudu::consensus::RefCountedReplicate>, std::__1::allocator<scoped_refptr<kudu::consensus::RefCountedReplicate> > >*, bool*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:469:50 (libconsensus.so+0x70a3a)
    #6 kudu::consensus::Peer::SendNextRequest(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:177:22 (libconsensus.so+0x649c2)
    #7 kudu::consensus::Peer::SignalRequest(bool)::$_0::operator()() const /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:134:3 (libconsensus.so+0x67a02)
    #8 boost::detail::function::void_function_obj_invoker0<kudu::consensus::Peer::SignalRequest(bool)::$_0, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libconsensus.so+0x67809)
    #9 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #10 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #11 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #12 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #13 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #14 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #15 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #16 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #17 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Previous write of size 8 at 0x7b1c00013818 by thread T76 (mutexes: write M2279):
    #0 kudu::consensus::TimeManager::GetSerialTimestamp() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:310:28 (libconsensus.so+0xbcbbb)
    #1 kudu::consensus::RaftConsensus::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const&, kudu::tserver::TabletServerErrorPB_Code*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1668:36 (libconsensus.so+0xacabd)
    #2 kudu::tserver::ConsensusServiceImpl::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const*, kudu::consensus::UnsafeChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:941:25 (libtserver.so+0xc98cf)
    #3 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:160:13 (libconsensus_proto.so+0x825b4)
    #4 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_7JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x82541)
    #5 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_7PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x82541)
    #6 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x82454)
    #7 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #8 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #9 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #10 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #11 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Location is heap block of size 104 at 0x7b1c000137f0 allocated by thread T123:
    #0 operator new(unsigned long) /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_new_delete.cc:41 (kudu-tserver+0x4c0d63)
    #1 kudu::tablet::TabletReplica::Init(std::__1::shared_ptr<kudu::tablet::Tablet> const&, scoped_refptr<kudu::server::Clock> const&, std::__1::shared_ptr<kudu::rpc::Messenger> const&, scoped_refptr<kudu::rpc::ResultTracker> const&, scoped_refptr<kudu::log::Log> const&, scoped_refptr<kudu::MetricEntity> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:162:45 (libtablet.so+0x139f6a)
    #2 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:770:19 (libtserver.so+0xe6991)
    #3 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #4 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #5 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #6 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #7 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #8 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #9 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #10 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #11 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2582 (0x7b54000b0130) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x6427c)
    #5 kudu::consensus::Peer::Init() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:119 (libconsensus.so+0x6427c)
    #6 kudu::consensus::Peer::NewRemotePeer(kudu::consensus::RaftPeerPB const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::PeerMessageQueue*, kudu::ThreadPool*, gscoped_ptr<kudu::consensus::PeerProxy, kudu::DefaultDeleter<kudu::consensus::PeerProxy> >, std::__1::shared_ptr<kudu::consensus::Peer>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:97:3 (libconsensus.so+0x64101)
    #7 kudu::consensus::PeerManager::UpdateRaftConfig(kudu::consensus::RaftConfigPB const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/peer_manager.cc:73:5 (libconsensus.so+0x908e1)
    #8 kudu::consensus::RaftConsensus::RefreshConsensusQueueAndPeersUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2044:3 (libconsensus.so+0xa254f)
    #9 kudu::consensus::RaftConsensus::AddPendingOperationUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:668:9 (libconsensus.so+0xa3787)
    #10 kudu::consensus::RaftConsensus::AppendNewRoundToQueueUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:626:3 (libconsensus.so+0xa2d76)
    #11 kudu::consensus::RaftConsensus::ReplicateConfigChangeUnlocked(kudu::consensus::RaftConfigPB const&, kudu::consensus::RaftConfigPB const&, kudu::Callback<void ()(kudu::Status const&)> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2025:3 (libconsensus.so+0xac5f3)
    #12 kudu::consensus::RaftConsensus::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const&, kudu::Callback<void ()(kudu::Status const&)> const&, boost::optional<kudu::tserver::TabletServerErrorPB_Code>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1621:5 (libconsensus.so+0xa5c0b)
    #13 kudu::tserver::ConsensusServiceImpl::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const*, kudu::consensus::ChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:917:25 (libtserver.so+0xc8d55)
    #14 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:140:13 (libconsensus_proto.so+0x81e94)
    #15 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_5JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x81e21)
    #16 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_5PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x81e21)
    #17 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x81d34)
    #18 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #19 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #20 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #21 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #22 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #23 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #24 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #25 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #26 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2285 (0x7b1c000137f8) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0xbc8ee)
    #5 kudu::consensus::TimeManager::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:78 (libconsensus.so+0xbc8ee)
    #6 kudu::consensus::PeerMessageQueue::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:186:18 (libconsensus.so+0x6dbe9)
    #7 kudu::consensus::RaftConsensus::BecomeReplicaUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:593:11 (libconsensus.so+0x9e9fa)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:340:5 (libconsensus.so+0x9d41f)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2279 (0x7b540006fe44) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x9cf38)
    #5 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:282 (libconsensus.so+0x9cf38)
    #6 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #7 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #8 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #9 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #10 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #11 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #12 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #13 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #14 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #15 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #16 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #17 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #18 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #19 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #20 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T126 'cc8abe-raft [wo' (tid=17191, running) created by thread T123 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitClosure(kudu::Callback<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:430:10 (libkudu_util.so+0x1c15c9)
    #7 kudu::consensus::RaftConsensus::MarkDirty(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2243:3 (libconsensus.so+0x9f83a)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:351:3 (libconsensus.so+0x9d5f2)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T76 'rpc worker-1602' (tid=16028, running) created by main thread at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::rpc::ServicePool::*)(), kudu::rpc::ServicePool*>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::rpc::ServicePool::* const&)(), kudu::rpc::ServicePool* const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:158:12 (libkrpc.so+0xe9615)
    #3 kudu::rpc::ServicePool::Init(int) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:81:5 (libkrpc.so+0xe7ea5)
    #4 kudu::RpcServer::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/rpc_server.cc:144:3 (libserver_process.so+0x561d1)
    #5 kudu::server::ServerBase::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/server_base.cc:364:23 (libserver_process.so+0x5c1cf)
    #6 kudu::tserver::TabletServer::Start() /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server.cc:117:3 (libtserver.so+0xc1441)
    #7 kudu::tserver::TabletServerMain(int, char**) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:77:3 (kudu-tserver+0x4c3778)
    #8 main /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:91:10 (kudu-tserver+0x4c33ee)

  Thread T123 'tablet-open [wo' (tid=17160, finished) created by thread T56 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitFunc(boost::function<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:434:10 (libkudu_util.so+0x1c1729)
    #7 kudu::tserver::TSTabletManager::CreateNewTablet(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Partition const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Schema const&, kudu::PartitionSchema const&, kudu::consensus::RaftConfigPB, scoped_refptr<kudu::tablet::TabletReplica>*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:281:3 (libtserver.so+0xe79a1)
    #8 kudu::tserver::TabletServiceAdminImpl::CreateTablet(kudu::tserver::CreateTabletRequestPB const*, kudu::tserver::CreateTabletResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:678:34 (libtserver.so+0xc51ab)
    #9 kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/tserver/tserver_admin.service.cc:58:13 (libtserver_admin_proto.so+0x27ec4)
    #10 _ZNSt3__18__invokeIRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_1JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libtserver_admin_proto.so+0x27e51)
    #11 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_1PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libtserver_admin_proto.so+0x27e51)
    #12 std::__1::__function::__func<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1, std::__1::allocator<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libtserver_admin_proto.so+0x27d64)
    #13 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #14 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #15 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #16 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #17 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #18 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #19 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #20 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #21 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
---
M src/kudu/consensus/time_manager-test.cc
M src/kudu/consensus/time_manager.cc
M src/kudu/consensus/time_manager.h
3 files changed, 40 insertions(+), 12 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/28/7328/3
-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Posted by "Adar Dembo (Code Review)" <ge...@cloudera.org>.
Adar Dembo has posted comments on this change.

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................


Patch Set 1:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/7328/1/src/kudu/consensus/time_manager.h
File src/kudu/consensus/time_manager.h:

PS1, Line 130: to
> nit: to be
I just moved this from a different part of the file, but I fixed it anyway.


-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-HasComments: Yes

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Posted by "Todd Lipcon (Code Review)" <ge...@cloudera.org>.
Todd Lipcon has submitted this change and it was merged.

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................


time_manager: fix unsynchronized access to GetSerialTimestamp()

This access produced the occasional race, which I've attached to the bottom
of the commit message.

I also revoked friendship for RaftConsensus. This was done solely to get
access to GetSerialTimestamp; making the function public seems like the
lesser of two evils to me.

WARNING: ThreadSanitizer: data race (pid=15727)
  Read of size 8 at 0x7b1c00013818 by thread T126 (mutexes: write M2582, write M2285):
    #0 kudu::operator<(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:105:22 (libtserver.so+0xe1768)
    #1 kudu::operator>(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:109:14 (libtserver.so+0xd9170)
    #2 kudu::operator<=(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:113:16 (libtablet.so+0x19e990)
    #3 kudu::consensus::TimeManager::GetSafeTimeUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:291:11 (libconsensus.so+0xbddad)
    #4 kudu::consensus::TimeManager::GetSafeTime() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:267:10 (libconsensus.so+0xbdeb9)
    #5 kudu::consensus::PeerMessageQueue::RequestForPeer(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::ConsensusRequestPB*, std::__1::vector<scoped_refptr<kudu::consensus::RefCountedReplicate>, std::__1::allocator<scoped_refptr<kudu::consensus::RefCountedReplicate> > >*, bool*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:469:50 (libconsensus.so+0x70a3a)
    #6 kudu::consensus::Peer::SendNextRequest(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:177:22 (libconsensus.so+0x649c2)
    #7 kudu::consensus::Peer::SignalRequest(bool)::$_0::operator()() const /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:134:3 (libconsensus.so+0x67a02)
    #8 boost::detail::function::void_function_obj_invoker0<kudu::consensus::Peer::SignalRequest(bool)::$_0, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libconsensus.so+0x67809)
    #9 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #10 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #11 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #12 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #13 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #14 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #15 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #16 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #17 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Previous write of size 8 at 0x7b1c00013818 by thread T76 (mutexes: write M2279):
    #0 kudu::consensus::TimeManager::GetSerialTimestamp() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:310:28 (libconsensus.so+0xbcbbb)
    #1 kudu::consensus::RaftConsensus::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const&, kudu::tserver::TabletServerErrorPB_Code*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1668:36 (libconsensus.so+0xacabd)
    #2 kudu::tserver::ConsensusServiceImpl::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const*, kudu::consensus::UnsafeChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:941:25 (libtserver.so+0xc98cf)
    #3 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:160:13 (libconsensus_proto.so+0x825b4)
    #4 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_7JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x82541)
    #5 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_7PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x82541)
    #6 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x82454)
    #7 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #8 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #9 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #10 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #11 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Location is heap block of size 104 at 0x7b1c000137f0 allocated by thread T123:
    #0 operator new(unsigned long) /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_new_delete.cc:41 (kudu-tserver+0x4c0d63)
    #1 kudu::tablet::TabletReplica::Init(std::__1::shared_ptr<kudu::tablet::Tablet> const&, scoped_refptr<kudu::server::Clock> const&, std::__1::shared_ptr<kudu::rpc::Messenger> const&, scoped_refptr<kudu::rpc::ResultTracker> const&, scoped_refptr<kudu::log::Log> const&, scoped_refptr<kudu::MetricEntity> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:162:45 (libtablet.so+0x139f6a)
    #2 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:770:19 (libtserver.so+0xe6991)
    #3 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #4 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #5 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #6 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #7 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #8 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #9 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #10 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #11 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2582 (0x7b54000b0130) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x6427c)
    #5 kudu::consensus::Peer::Init() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:119 (libconsensus.so+0x6427c)
    #6 kudu::consensus::Peer::NewRemotePeer(kudu::consensus::RaftPeerPB const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::PeerMessageQueue*, kudu::ThreadPool*, gscoped_ptr<kudu::consensus::PeerProxy, kudu::DefaultDeleter<kudu::consensus::PeerProxy> >, std::__1::shared_ptr<kudu::consensus::Peer>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:97:3 (libconsensus.so+0x64101)
    #7 kudu::consensus::PeerManager::UpdateRaftConfig(kudu::consensus::RaftConfigPB const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/peer_manager.cc:73:5 (libconsensus.so+0x908e1)
    #8 kudu::consensus::RaftConsensus::RefreshConsensusQueueAndPeersUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2044:3 (libconsensus.so+0xa254f)
    #9 kudu::consensus::RaftConsensus::AddPendingOperationUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:668:9 (libconsensus.so+0xa3787)
    #10 kudu::consensus::RaftConsensus::AppendNewRoundToQueueUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:626:3 (libconsensus.so+0xa2d76)
    #11 kudu::consensus::RaftConsensus::ReplicateConfigChangeUnlocked(kudu::consensus::RaftConfigPB const&, kudu::consensus::RaftConfigPB const&, kudu::Callback<void ()(kudu::Status const&)> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2025:3 (libconsensus.so+0xac5f3)
    #12 kudu::consensus::RaftConsensus::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const&, kudu::Callback<void ()(kudu::Status const&)> const&, boost::optional<kudu::tserver::TabletServerErrorPB_Code>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1621:5 (libconsensus.so+0xa5c0b)
    #13 kudu::tserver::ConsensusServiceImpl::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const*, kudu::consensus::ChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:917:25 (libtserver.so+0xc8d55)
    #14 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:140:13 (libconsensus_proto.so+0x81e94)
    #15 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_5JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x81e21)
    #16 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_5PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x81e21)
    #17 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x81d34)
    #18 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #19 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #20 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #21 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #22 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #23 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #24 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #25 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #26 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2285 (0x7b1c000137f8) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0xbc8ee)
    #5 kudu::consensus::TimeManager::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:78 (libconsensus.so+0xbc8ee)
    #6 kudu::consensus::PeerMessageQueue::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:186:18 (libconsensus.so+0x6dbe9)
    #7 kudu::consensus::RaftConsensus::BecomeReplicaUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:593:11 (libconsensus.so+0x9e9fa)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:340:5 (libconsensus.so+0x9d41f)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2279 (0x7b540006fe44) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x9cf38)
    #5 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:282 (libconsensus.so+0x9cf38)
    #6 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #7 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #8 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #9 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #10 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #11 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #12 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #13 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #14 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #15 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #16 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #17 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #18 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #19 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #20 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T126 'cc8abe-raft [wo' (tid=17191, running) created by thread T123 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitClosure(kudu::Callback<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:430:10 (libkudu_util.so+0x1c15c9)
    #7 kudu::consensus::RaftConsensus::MarkDirty(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2243:3 (libconsensus.so+0x9f83a)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:351:3 (libconsensus.so+0x9d5f2)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T76 'rpc worker-1602' (tid=16028, running) created by main thread at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::rpc::ServicePool::*)(), kudu::rpc::ServicePool*>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::rpc::ServicePool::* const&)(), kudu::rpc::ServicePool* const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:158:12 (libkrpc.so+0xe9615)
    #3 kudu::rpc::ServicePool::Init(int) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:81:5 (libkrpc.so+0xe7ea5)
    #4 kudu::RpcServer::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/rpc_server.cc:144:3 (libserver_process.so+0x561d1)
    #5 kudu::server::ServerBase::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/server_base.cc:364:23 (libserver_process.so+0x5c1cf)
    #6 kudu::tserver::TabletServer::Start() /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server.cc:117:3 (libtserver.so+0xc1441)
    #7 kudu::tserver::TabletServerMain(int, char**) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:77:3 (kudu-tserver+0x4c3778)
    #8 main /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:91:10 (kudu-tserver+0x4c33ee)

  Thread T123 'tablet-open [wo' (tid=17160, finished) created by thread T56 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitFunc(boost::function<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:434:10 (libkudu_util.so+0x1c1729)
    #7 kudu::tserver::TSTabletManager::CreateNewTablet(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Partition const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Schema const&, kudu::PartitionSchema const&, kudu::consensus::RaftConfigPB, scoped_refptr<kudu::tablet::TabletReplica>*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:281:3 (libtserver.so+0xe79a1)
    #8 kudu::tserver::TabletServiceAdminImpl::CreateTablet(kudu::tserver::CreateTabletRequestPB const*, kudu::tserver::CreateTabletResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:678:34 (libtserver.so+0xc51ab)
    #9 kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/tserver/tserver_admin.service.cc:58:13 (libtserver_admin_proto.so+0x27ec4)
    #10 _ZNSt3__18__invokeIRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_1JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libtserver_admin_proto.so+0x27e51)
    #11 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_1PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libtserver_admin_proto.so+0x27e51)
    #12 std::__1::__function::__func<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1, std::__1::allocator<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libtserver_admin_proto.so+0x27d64)
    #13 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #14 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #15 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #16 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #17 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #18 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #19 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #20 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #21 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Reviewed-on: http://gerrit.cloudera.org:8080/7328
Reviewed-by: Mike Percy <mp...@apache.org>
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
---
M src/kudu/consensus/time_manager-test.cc
M src/kudu/consensus/time_manager.cc
M src/kudu/consensus/time_manager.h
3 files changed, 40 insertions(+), 12 deletions(-)

Approvals:
  Mike Percy: Looks good to me, approved
  Todd Lipcon: Looks good to me, approved
  Kudu Jenkins: Verified



-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Posted by "Todd Lipcon (Code Review)" <ge...@cloudera.org>.
Todd Lipcon has posted comments on this change.

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................


Patch Set 1: Code-Review+1

-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-HasComments: No

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Posted by "Mike Percy (Code Review)" <ge...@cloudera.org>.
Mike Percy has posted comments on this change.

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................


Patch Set 4: Code-Review+2

(1 comment)

http://gerrit.cloudera.org:8080/#/c/7328/1/src/kudu/consensus/time_manager.h
File src/kudu/consensus/time_manager.h:

PS1, Line 130: to
> I just moved this from a different part of the file, but I fixed it anyway.
Ah, right.


-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-HasComments: Yes

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Posted by "Adar Dembo (Code Review)" <ge...@cloudera.org>.
Hello Mike Percy, Todd Lipcon, Kudu Jenkins,

I'd like you to reexamine a change.  Please visit

    http://gerrit.cloudera.org:8080/7328

to look at the new patch set (#4).

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................

time_manager: fix unsynchronized access to GetSerialTimestamp()

This access produced the occasional race, which I've attached to the bottom
of the commit message.

I also revoked friendship for RaftConsensus. This was done solely to get
access to GetSerialTimestamp; making the function public seems like the
lesser of two evils to me.

WARNING: ThreadSanitizer: data race (pid=15727)
  Read of size 8 at 0x7b1c00013818 by thread T126 (mutexes: write M2582, write M2285):
    #0 kudu::operator<(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:105:22 (libtserver.so+0xe1768)
    #1 kudu::operator>(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:109:14 (libtserver.so+0xd9170)
    #2 kudu::operator<=(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:113:16 (libtablet.so+0x19e990)
    #3 kudu::consensus::TimeManager::GetSafeTimeUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:291:11 (libconsensus.so+0xbddad)
    #4 kudu::consensus::TimeManager::GetSafeTime() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:267:10 (libconsensus.so+0xbdeb9)
    #5 kudu::consensus::PeerMessageQueue::RequestForPeer(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::ConsensusRequestPB*, std::__1::vector<scoped_refptr<kudu::consensus::RefCountedReplicate>, std::__1::allocator<scoped_refptr<kudu::consensus::RefCountedReplicate> > >*, bool*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:469:50 (libconsensus.so+0x70a3a)
    #6 kudu::consensus::Peer::SendNextRequest(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:177:22 (libconsensus.so+0x649c2)
    #7 kudu::consensus::Peer::SignalRequest(bool)::$_0::operator()() const /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:134:3 (libconsensus.so+0x67a02)
    #8 boost::detail::function::void_function_obj_invoker0<kudu::consensus::Peer::SignalRequest(bool)::$_0, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libconsensus.so+0x67809)
    #9 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #10 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #11 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #12 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #13 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #14 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #15 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #16 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #17 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Previous write of size 8 at 0x7b1c00013818 by thread T76 (mutexes: write M2279):
    #0 kudu::consensus::TimeManager::GetSerialTimestamp() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:310:28 (libconsensus.so+0xbcbbb)
    #1 kudu::consensus::RaftConsensus::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const&, kudu::tserver::TabletServerErrorPB_Code*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1668:36 (libconsensus.so+0xacabd)
    #2 kudu::tserver::ConsensusServiceImpl::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const*, kudu::consensus::UnsafeChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:941:25 (libtserver.so+0xc98cf)
    #3 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:160:13 (libconsensus_proto.so+0x825b4)
    #4 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_7JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x82541)
    #5 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_7PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x82541)
    #6 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x82454)
    #7 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #8 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #9 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #10 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #11 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Location is heap block of size 104 at 0x7b1c000137f0 allocated by thread T123:
    #0 operator new(unsigned long) /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_new_delete.cc:41 (kudu-tserver+0x4c0d63)
    #1 kudu::tablet::TabletReplica::Init(std::__1::shared_ptr<kudu::tablet::Tablet> const&, scoped_refptr<kudu::server::Clock> const&, std::__1::shared_ptr<kudu::rpc::Messenger> const&, scoped_refptr<kudu::rpc::ResultTracker> const&, scoped_refptr<kudu::log::Log> const&, scoped_refptr<kudu::MetricEntity> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:162:45 (libtablet.so+0x139f6a)
    #2 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:770:19 (libtserver.so+0xe6991)
    #3 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #4 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #5 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #6 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #7 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #8 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #9 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #10 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #11 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2582 (0x7b54000b0130) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x6427c)
    #5 kudu::consensus::Peer::Init() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:119 (libconsensus.so+0x6427c)
    #6 kudu::consensus::Peer::NewRemotePeer(kudu::consensus::RaftPeerPB const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::PeerMessageQueue*, kudu::ThreadPool*, gscoped_ptr<kudu::consensus::PeerProxy, kudu::DefaultDeleter<kudu::consensus::PeerProxy> >, std::__1::shared_ptr<kudu::consensus::Peer>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:97:3 (libconsensus.so+0x64101)
    #7 kudu::consensus::PeerManager::UpdateRaftConfig(kudu::consensus::RaftConfigPB const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/peer_manager.cc:73:5 (libconsensus.so+0x908e1)
    #8 kudu::consensus::RaftConsensus::RefreshConsensusQueueAndPeersUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2044:3 (libconsensus.so+0xa254f)
    #9 kudu::consensus::RaftConsensus::AddPendingOperationUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:668:9 (libconsensus.so+0xa3787)
    #10 kudu::consensus::RaftConsensus::AppendNewRoundToQueueUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:626:3 (libconsensus.so+0xa2d76)
    #11 kudu::consensus::RaftConsensus::ReplicateConfigChangeUnlocked(kudu::consensus::RaftConfigPB const&, kudu::consensus::RaftConfigPB const&, kudu::Callback<void ()(kudu::Status const&)> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2025:3 (libconsensus.so+0xac5f3)
    #12 kudu::consensus::RaftConsensus::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const&, kudu::Callback<void ()(kudu::Status const&)> const&, boost::optional<kudu::tserver::TabletServerErrorPB_Code>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1621:5 (libconsensus.so+0xa5c0b)
    #13 kudu::tserver::ConsensusServiceImpl::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const*, kudu::consensus::ChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:917:25 (libtserver.so+0xc8d55)
    #14 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:140:13 (libconsensus_proto.so+0x81e94)
    #15 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_5JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x81e21)
    #16 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_5PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x81e21)
    #17 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x81d34)
    #18 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #19 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #20 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #21 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #22 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #23 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #24 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #25 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #26 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2285 (0x7b1c000137f8) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0xbc8ee)
    #5 kudu::consensus::TimeManager::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:78 (libconsensus.so+0xbc8ee)
    #6 kudu::consensus::PeerMessageQueue::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:186:18 (libconsensus.so+0x6dbe9)
    #7 kudu::consensus::RaftConsensus::BecomeReplicaUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:593:11 (libconsensus.so+0x9e9fa)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:340:5 (libconsensus.so+0x9d41f)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2279 (0x7b540006fe44) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x9cf38)
    #5 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:282 (libconsensus.so+0x9cf38)
    #6 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #7 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #8 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #9 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #10 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #11 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #12 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #13 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #14 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #15 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #16 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #17 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #18 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #19 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #20 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T126 'cc8abe-raft [wo' (tid=17191, running) created by thread T123 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitClosure(kudu::Callback<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:430:10 (libkudu_util.so+0x1c15c9)
    #7 kudu::consensus::RaftConsensus::MarkDirty(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2243:3 (libconsensus.so+0x9f83a)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:351:3 (libconsensus.so+0x9d5f2)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T76 'rpc worker-1602' (tid=16028, running) created by main thread at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::rpc::ServicePool::*)(), kudu::rpc::ServicePool*>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::rpc::ServicePool::* const&)(), kudu::rpc::ServicePool* const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:158:12 (libkrpc.so+0xe9615)
    #3 kudu::rpc::ServicePool::Init(int) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:81:5 (libkrpc.so+0xe7ea5)
    #4 kudu::RpcServer::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/rpc_server.cc:144:3 (libserver_process.so+0x561d1)
    #5 kudu::server::ServerBase::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/server_base.cc:364:23 (libserver_process.so+0x5c1cf)
    #6 kudu::tserver::TabletServer::Start() /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server.cc:117:3 (libtserver.so+0xc1441)
    #7 kudu::tserver::TabletServerMain(int, char**) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:77:3 (kudu-tserver+0x4c3778)
    #8 main /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:91:10 (kudu-tserver+0x4c33ee)

  Thread T123 'tablet-open [wo' (tid=17160, finished) created by thread T56 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitFunc(boost::function<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:434:10 (libkudu_util.so+0x1c1729)
    #7 kudu::tserver::TSTabletManager::CreateNewTablet(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Partition const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Schema const&, kudu::PartitionSchema const&, kudu::consensus::RaftConfigPB, scoped_refptr<kudu::tablet::TabletReplica>*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:281:3 (libtserver.so+0xe79a1)
    #8 kudu::tserver::TabletServiceAdminImpl::CreateTablet(kudu::tserver::CreateTabletRequestPB const*, kudu::tserver::CreateTabletResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:678:34 (libtserver.so+0xc51ab)
    #9 kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/tserver/tserver_admin.service.cc:58:13 (libtserver_admin_proto.so+0x27ec4)
    #10 _ZNSt3__18__invokeIRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_1JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libtserver_admin_proto.so+0x27e51)
    #11 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_1PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libtserver_admin_proto.so+0x27e51)
    #12 std::__1::__function::__func<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1, std::__1::allocator<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libtserver_admin_proto.so+0x27d64)
    #13 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #14 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #15 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #16 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #17 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #18 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #19 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #20 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #21 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
---
M src/kudu/consensus/time_manager-test.cc
M src/kudu/consensus/time_manager.cc
M src/kudu/consensus/time_manager.h
3 files changed, 40 insertions(+), 12 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/28/7328/4
-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Posted by "Mike Percy (Code Review)" <ge...@cloudera.org>.
Mike Percy has posted comments on this change.

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................


Patch Set 1: Code-Review+1

(1 comment)

http://gerrit.cloudera.org:8080/#/c/7328/1/src/kudu/consensus/time_manager.h
File src/kudu/consensus/time_manager.h:

PS1, Line 130: to
nit: to be


-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-HasComments: Yes

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Posted by "Todd Lipcon (Code Review)" <ge...@cloudera.org>.
Todd Lipcon has posted comments on this change.

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................


Patch Set 4: Code-Review+2

-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-HasComments: No

[kudu-CR] time manager: fix unsynchronized access to GetSerialTimestamp()

Posted by "Adar Dembo (Code Review)" <ge...@cloudera.org>.
Hello Mike Percy, Todd Lipcon, Kudu Jenkins,

I'd like you to reexamine a change.  Please visit

    http://gerrit.cloudera.org:8080/7328

to look at the new patch set (#2).

Change subject: time_manager: fix unsynchronized access to GetSerialTimestamp()
......................................................................

time_manager: fix unsynchronized access to GetSerialTimestamp()

This produced the occasional race, which I've attached to the bottom of the
commit message.

I also revoked friendship for RaftConsensus. This was done solely to get
access to GetSerialTimestamp; making the function public seems like the
lesser of two evils to me.

WARNING: ThreadSanitizer: data race (pid=15727)
  Read of size 8 at 0x7b1c00013818 by thread T126 (mutexes: write M2582, write M2285):
    #0 kudu::operator<(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:105:22 (libtserver.so+0xe1768)
    #1 kudu::operator>(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:109:14 (libtserver.so+0xd9170)
    #2 kudu::operator<=(kudu::Timestamp const&, kudu::Timestamp const&) /data/jenkins-workspace/kudu-workspace/src/kudu/common/timestamp.h:113:16 (libtablet.so+0x19e990)
    #3 kudu::consensus::TimeManager::GetSafeTimeUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:291:11 (libconsensus.so+0xbddad)
    #4 kudu::consensus::TimeManager::GetSafeTime() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:267:10 (libconsensus.so+0xbdeb9)
    #5 kudu::consensus::PeerMessageQueue::RequestForPeer(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::ConsensusRequestPB*, std::__1::vector<scoped_refptr<kudu::consensus::RefCountedReplicate>, std::__1::allocator<scoped_refptr<kudu::consensus::RefCountedReplicate> > >*, bool*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:469:50 (libconsensus.so+0x70a3a)
    #6 kudu::consensus::Peer::SendNextRequest(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:177:22 (libconsensus.so+0x649c2)
    #7 kudu::consensus::Peer::SignalRequest(bool)::$_0::operator()() const /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:134:3 (libconsensus.so+0x67a02)
    #8 boost::detail::function::void_function_obj_invoker0<kudu::consensus::Peer::SignalRequest(bool)::$_0, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libconsensus.so+0x67809)
    #9 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #10 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #11 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #12 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #13 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #14 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #15 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #16 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #17 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Previous write of size 8 at 0x7b1c00013818 by thread T76 (mutexes: write M2279):
    #0 kudu::consensus::TimeManager::GetSerialTimestamp() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:310:28 (libconsensus.so+0xbcbbb)
    #1 kudu::consensus::RaftConsensus::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const&, kudu::tserver::TabletServerErrorPB_Code*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1668:36 (libconsensus.so+0xacabd)
    #2 kudu::tserver::ConsensusServiceImpl::UnsafeChangeConfig(kudu::consensus::UnsafeChangeConfigRequestPB const*, kudu::consensus::UnsafeChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:941:25 (libtserver.so+0xc98cf)
    #3 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:160:13 (libconsensus_proto.so+0x825b4)
    #4 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_7JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x82541)
    #5 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_7PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x82541)
    #6 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_7>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x82454)
    #7 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #8 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #9 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #10 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #11 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Location is heap block of size 104 at 0x7b1c000137f0 allocated by thread T123:
    #0 operator new(unsigned long) /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_new_delete.cc:41 (kudu-tserver+0x4c0d63)
    #1 kudu::tablet::TabletReplica::Init(std::__1::shared_ptr<kudu::tablet::Tablet> const&, scoped_refptr<kudu::server::Clock> const&, std::__1::shared_ptr<kudu::rpc::Messenger> const&, scoped_refptr<kudu::rpc::ResultTracker> const&, scoped_refptr<kudu::log::Log> const&, scoped_refptr<kudu::MetricEntity> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:162:45 (libtablet.so+0x139f6a)
    #2 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:770:19 (libtserver.so+0xe6991)
    #3 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #4 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #5 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #6 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #7 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #8 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #9 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #10 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #11 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #12 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #13 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #14 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #15 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2582 (0x7b54000b0130) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x6427c)
    #5 kudu::consensus::Peer::Init() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:119 (libconsensus.so+0x6427c)
    #6 kudu::consensus::Peer::NewRemotePeer(kudu::consensus::RaftPeerPB const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::consensus::PeerMessageQueue*, kudu::ThreadPool*, gscoped_ptr<kudu::consensus::PeerProxy, kudu::DefaultDeleter<kudu::consensus::PeerProxy> >, std::__1::shared_ptr<kudu::consensus::Peer>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_peers.cc:97:3 (libconsensus.so+0x64101)
    #7 kudu::consensus::PeerManager::UpdateRaftConfig(kudu::consensus::RaftConfigPB const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/peer_manager.cc:73:5 (libconsensus.so+0x908e1)
    #8 kudu::consensus::RaftConsensus::RefreshConsensusQueueAndPeersUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2044:3 (libconsensus.so+0xa254f)
    #9 kudu::consensus::RaftConsensus::AddPendingOperationUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:668:9 (libconsensus.so+0xa3787)
    #10 kudu::consensus::RaftConsensus::AppendNewRoundToQueueUnlocked(scoped_refptr<kudu::consensus::ConsensusRound> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:626:3 (libconsensus.so+0xa2d76)
    #11 kudu::consensus::RaftConsensus::ReplicateConfigChangeUnlocked(kudu::consensus::RaftConfigPB const&, kudu::consensus::RaftConfigPB const&, kudu::Callback<void ()(kudu::Status const&)> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2025:3 (libconsensus.so+0xac5f3)
    #12 kudu::consensus::RaftConsensus::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const&, kudu::Callback<void ()(kudu::Status const&)> const&, boost::optional<kudu::tserver::TabletServerErrorPB_Code>*) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:1621:5 (libconsensus.so+0xa5c0b)
    #13 kudu::tserver::ConsensusServiceImpl::ChangeConfig(kudu::consensus::ChangeConfigRequestPB const*, kudu::consensus::ChangeConfigResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:917:25 (libtserver.so+0xc8d55)
    #14 kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/consensus/consensus.service.cc:140:13 (libconsensus_proto.so+0x81e94)
    #15 _ZNSt3__18__invokeIRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_5JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libconsensus_proto.so+0x81e21)
    #16 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu9consensus18ConsensusServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_5PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libconsensus_proto.so+0x81e21)
    #17 std::__1::__function::__func<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5, std::__1::allocator<kudu::consensus::ConsensusServiceIf::ConsensusServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_5>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libconsensus_proto.so+0x81d34)
    #18 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #19 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #20 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #21 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #22 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #23 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #24 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #25 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #26 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2285 (0x7b1c000137f8) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0xbc8ee)
    #5 kudu::consensus::TimeManager::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/time_manager.cc:78 (libconsensus.so+0xbc8ee)
    #6 kudu::consensus::PeerMessageQueue::SetNonLeaderMode() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/consensus_queue.cc:186:18 (libconsensus.so+0x6dbe9)
    #7 kudu::consensus::RaftConsensus::BecomeReplicaUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:593:11 (libconsensus.so+0x9e9fa)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:340:5 (libconsensus.so+0x9d41f)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Mutex M2279 (0x7b540006fe44) created at:
    #0 __tsan_atomic32_compare_exchange_strong /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interface_atomic.cc:756 (kudu-tserver+0x47d998)
    #1 base::subtle::Acquire_CompareAndSwap(int volatile*, int, int) /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/atomicops-internals-tsan.h:83:3 (libtserver.so+0x91fb7)
    #2 base::SpinLock::Lock() /data/jenkins-workspace/kudu-workspace/src/kudu/gutil/spinlock.h:73:9 (libtserver.so+0x91f20)
    #3 kudu::simple_spinlock::lock() /data/jenkins-workspace/kudu-workspace/src/kudu/util/locks.h:45:8 (libtserver.so+0x91ed9)
    #4 std::__1::lock_guard<kudu::simple_spinlock>::lock_guard(kudu::simple_spinlock&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__mutex_base:108:27 (libconsensus.so+0x9cf38)
    #5 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:282 (libconsensus.so+0x9cf38)
    #6 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #7 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #8 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #9 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #10 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #11 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #12 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #13 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #14 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #15 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #16 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #17 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #18 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #19 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #20 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T126 'cc8abe-raft [wo' (tid=17191, running) created by thread T123 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitClosure(kudu::Callback<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:430:10 (libkudu_util.so+0x1c15c9)
    #7 kudu::consensus::RaftConsensus::MarkDirty(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:2243:3 (libconsensus.so+0x9f83a)
    #8 kudu::consensus::RaftConsensus::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/consensus/raft_consensus.cc:351:3 (libconsensus.so+0x9d5f2)
    #9 kudu::tablet::TabletReplica::Start(kudu::consensus::ConsensusBootstrapInfo const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tablet/tablet_replica.cc:196:3 (libtablet.so+0x13a974)
    #10 kudu::tserver::TSTabletManager::OpenTablet(scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:785:18 (libtserver.so+0xe6a4b)
    #11 boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>::operator()(kudu::tserver::TSTabletManager*, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:280:29 (libtserver.so+0xf1bc7)
    #12 void boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > >::operator()<boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:398:9 (libtserver.so+0xf1b06)
    #13 boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libtserver.so+0xf1a63)
    #14 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, kudu::tserver::TSTabletManager, scoped_refptr<kudu::tablet::TabletMetadata> const&, scoped_refptr<kudu::tserver::TransitionInProgressDeleter> const&>, boost::_bi::list3<boost::_bi::value<kudu::tserver::TSTabletManager*>, boost::_bi::value<scoped_refptr<kudu::tablet::TabletMetadata> >, boost::_bi::value<scoped_refptr<kudu::tserver::TransitionInProgressDeleter> > > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libtserver.so+0xf17d1)
    #15 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #16 kudu::FunctionRunnable::Run() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:56:5 (libkudu_util.so+0x1c4b1d)
    #17 kudu::ThreadPool::DispatchThread(bool) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:621:22 (libkudu_util.so+0x1c1e04)
    #18 boost::_mfi::mf1<void, kudu::ThreadPool, bool>::operator()(kudu::ThreadPool*, bool) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:165:29 (libkudu_util.so+0x1ca38e)
    #19 void boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> >::operator()<boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf1<void, kudu::ThreadPool, bool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:319:9 (libkudu_util.so+0x1ca2cd)
    #20 boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkudu_util.so+0x1ca233)
    #21 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, kudu::ThreadPool, bool>, boost::_bi::list2<boost::_bi::value<kudu::ThreadPool*>, boost::_bi::value<bool> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkudu_util.so+0x1c9fd1)
    #22 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #23 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

  Thread T76 'rpc worker-1602' (tid=16028, running) created by main thread at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::rpc::ServicePool::*)(), kudu::rpc::ServicePool*>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::rpc::ServicePool::* const&)(), kudu::rpc::ServicePool* const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:158:12 (libkrpc.so+0xe9615)
    #3 kudu::rpc::ServicePool::Init(int) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:81:5 (libkrpc.so+0xe7ea5)
    #4 kudu::RpcServer::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/rpc_server.cc:144:3 (libserver_process.so+0x561d1)
    #5 kudu::server::ServerBase::RegisterService(gscoped_ptr<kudu::rpc::ServiceIf, kudu::DefaultDeleter<kudu::rpc::ServiceIf> >) /data/jenkins-workspace/kudu-workspace/src/kudu/server/server_base.cc:364:23 (libserver_process.so+0x5c1cf)
    #6 kudu::tserver::TabletServer::Start() /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server.cc:117:3 (libtserver.so+0xc1441)
    #7 kudu::tserver::TabletServerMain(int, char**) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:77:3 (kudu-tserver+0x4c3778)
    #8 main /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_server_main.cc:91:10 (kudu-tserver+0x4c33ee)

  Thread T123 'tablet-open [wo' (tid=17160, finished) created by thread T56 at:
    #0 pthread_create /data/jenkins-workspace/kudu-workspace/thirdparty/src/llvm-4.0.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:897 (kudu-tserver+0x4549db)
    #1 kudu::Thread::StartThread(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, boost::function<void ()()> const&, unsigned long, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:514:15 (libkudu_util.so+0x1b8f57)
    #2 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), kudu::ThreadPool*, bool>(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, void (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, scoped_refptr<kudu::Thread>*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.h:164:12 (libkudu_util.so+0x1c38b6)
    #3 kudu::ThreadPool::CreateThreadUnlocked() /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:683:14 (libkudu_util.so+0x1c1491)
    #4 kudu::ThreadPool::DoSubmit(std::__1::shared_ptr<kudu::Runnable>, kudu::ThreadPoolToken*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:477:21 (libkudu_util.so+0x1bf8e1)
    #5 kudu::ThreadPool::Submit(std::__1::shared_ptr<kudu::Runnable>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:438:10 (libkudu_util.so+0x1c168f)
    #6 kudu::ThreadPool::SubmitFunc(boost::function<void ()()>) /data/jenkins-workspace/kudu-workspace/src/kudu/util/threadpool.cc:434:10 (libkudu_util.so+0x1c1729)
    #7 kudu::tserver::TSTabletManager::CreateNewTablet(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Partition const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, kudu::Schema const&, kudu::PartitionSchema const&, kudu::consensus::RaftConfigPB, scoped_refptr<kudu::tablet::TabletReplica>*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/ts_tablet_manager.cc:281:3 (libtserver.so+0xe79a1)
    #8 kudu::tserver::TabletServiceAdminImpl::CreateTablet(kudu::tserver::CreateTabletRequestPB const*, kudu::tserver::CreateTabletResponsePB*, kudu::rpc::RpcContext*) /data/jenkins-workspace/kudu-workspace/src/kudu/tserver/tablet_service.cc:678:34 (libtserver.so+0xc51ab)
    #9 kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/build/tsan/src/kudu/tserver/tserver_admin.service.cc:58:13 (libtserver_admin_proto.so+0x27ec4)
    #10 _ZNSt3__18__invokeIRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS1_12MetricEntityEERKS4_INS1_3rpc13ResultTrackerEEE3$_1JPKN6google8protobuf7MessageEPSI_PNS9_10RpcContextEEEEDTclclsr3std3__1E7forwardIT_Efp_Espclsr3std3__1E7forwardIT0_Efp0_EEEOSO_DpOSP_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/type_traits:4301:1 (libtserver_admin_proto.so+0x27e51)
    #11 _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZN4kudu7tserver26TabletServerAdminServiceIfC1ERK13scoped_refptrINS3_12MetricEntityEERKS6_INS3_3rpc13ResultTrackerEEE3$_1PKN6google8protobuf7MessageEPSK_PNSB_10RpcContextEEEEvDpOT_ /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/__functional_base:359 (libtserver_admin_proto.so+0x27e51)
    #12 std::__1::__function::__func<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1, std::__1::allocator<kudu::tserver::TabletServerAdminServiceIf::TabletServerAdminServiceIf(scoped_refptr<kudu::MetricEntity> const&, scoped_refptr<kudu::rpc::ResultTracker> const&)::$_1>, void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*&&, google::protobuf::Message*&&, kudu::rpc::RpcContext*&&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1552:12 (libtserver_admin_proto.so+0x27d64)
    #13 std::__1::function<void ()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*)>::operator()(google::protobuf::Message const*, google::protobuf::Message*, kudu::rpc::RpcContext*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/c++/v1/functional:1914:12 (libkrpc.so+0xe72e9)
    #14 kudu::rpc::GeneratedServiceIf::Handle(kudu::rpc::InboundCall*) /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_if.cc:134:3 (libkrpc.so+0xe6bf4)
    #15 kudu::rpc::ServicePool::RunThread() /data/jenkins-workspace/kudu-workspace/src/kudu/rpc/service_pool.cc:210:15 (libkrpc.so+0xe82cd)
    #16 boost::_mfi::mf0<void, kudu::rpc::ServicePool>::operator()(kudu::rpc::ServicePool*) const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/mem_fn_template.hpp:49:29 (libkrpc.so+0xea2c6)
    #17 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, kudu::rpc::ServicePool>&, boost::_bi::list0&, int) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:259:9 (libkrpc.so+0xea21a)
    #18 boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >::operator()() /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/bind/bind.hpp:1222:16 (libkrpc.so+0xea1a3)
    #19 boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf0<void, kudu::rpc::ServicePool>, boost::_bi::list1<boost::_bi::value<kudu::rpc::ServicePool*> > >, void>::invoke(boost::detail::function::function_buffer&) /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:159:11 (libkrpc.so+0xe9fa9)
    #20 boost::function0<void>::operator()() const /data/jenkins-workspace/kudu-workspace/thirdparty/installed/tsan/include/boost/function/function_template.hpp:770:14 (libkrpc.so+0xb0111)
    #21 kudu::Thread::SuperviseThread(void*) /data/jenkins-workspace/kudu-workspace/src/kudu/util/thread.cc:591:3 (libkudu_util.so+0x1b975e)

Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
---
M src/kudu/consensus/time_manager-test.cc
M src/kudu/consensus/time_manager.cc
M src/kudu/consensus/time_manager.h
3 files changed, 40 insertions(+), 12 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/28/7328/2
-- 
To view, visit http://gerrit.cloudera.org:8080/7328
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I581eaa49ed3bf705121bb9c00b58499482ed2f39
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: David Ribeiro Alves <da...@gmail.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>