You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2017/06/30 01:39:57 UTC

[1/3] kudu git commit: time_manager: fix unsynchronized access to GetSerialTimestamp()

Repository: kudu
Updated Branches:
  refs/heads/master 371a00b7a -> 332ebcd06


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>


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

Branch: refs/heads/master
Commit: b33689e70db2ccb17bc16bc8c5e63c85b0399f49
Parents: 371a00b
Author: Adar Dembo <ad...@cloudera.com>
Authored: Wed Jun 28 17:05:20 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Jun 30 01:37:54 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/time_manager-test.cc |  8 ++++----
 src/kudu/consensus/time_manager.cc      | 24 +++++++++++++++++++++++-
 src/kudu/consensus/time_manager.h       | 20 +++++++++++++-------
 3 files changed, 40 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/b33689e7/src/kudu/consensus/time_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/time_manager-test.cc b/src/kudu/consensus/time_manager-test.cc
index a97c547..3254dd3 100644
--- a/src/kudu/consensus/time_manager-test.cc
+++ b/src/kudu/consensus/time_manager-test.cc
@@ -81,9 +81,9 @@ TEST_F(TimeManagerTest, TestTimeManagerNonLeaderMode) {
   ASSERT_EQ(time_manager_->GetSafeTime(), init);
 
   // Check that 'before' is safe, as is 'init'. 'after' shouldn't be safe.
-  ASSERT_TRUE(time_manager_->IsTimestampSafeUnlocked(before));
-  ASSERT_TRUE(time_manager_->IsTimestampSafeUnlocked(init));
-  ASSERT_FALSE(time_manager_->IsTimestampSafeUnlocked(after));
+  ASSERT_TRUE(time_manager_->IsTimestampSafe(before));
+  ASSERT_TRUE(time_manager_->IsTimestampSafe(init));
+  ASSERT_FALSE(time_manager_->IsTimestampSafe(after));
 
   // Shouldn't be able to assign timestamps.
   ReplicateMsg message;
@@ -169,7 +169,7 @@ TEST_F(TimeManagerTest, TestTimeManagerLeaderMode) {
 
   // 'Now' should be safe.
   Timestamp now = clock_->Now();
-  ASSERT_TRUE(time_manager_->IsTimestampSafeUnlocked(now));
+  ASSERT_TRUE(time_manager_->IsTimestampSafe(now));
   ASSERT_GT(time_manager_->GetSafeTime(), now);
 
   // When changing to non-leader mode a timestamp after the last safe time shouldn't be

http://git-wip-us.apache.org/repos/asf/kudu/blob/b33689e7/src/kudu/consensus/time_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/time_manager.cc b/src/kudu/consensus/time_manager.cc
index 27c6791..23312f8 100644
--- a/src/kudu/consensus/time_manager.cc
+++ b/src/kudu/consensus/time_manager.cc
@@ -89,7 +89,7 @@ Status TimeManager::AssignTimestamp(ReplicateMsg* message) {
   Timestamp t;
   switch (GetMessageConsistencyMode(*message)) {
     case COMMIT_WAIT: t = GetSerialTimestampPlusMaxError(); break;
-    case CLIENT_PROPAGATED:  t = GetSerialTimestamp(); break;
+    case CLIENT_PROPAGATED:  t = GetSerialTimestampUnlocked(); break;
     default: return Status::NotSupported("Unsupported external consistency mode.");
   }
   message->set_timestamp(t.value());
@@ -140,6 +140,8 @@ void TimeManager::AdvanceSafeTime(Timestamp safe_time) {
 }
 
 bool TimeManager::HasAdvancedSafeTimeRecentlyUnlocked(string* error_message) {
+  DCHECK(lock_.is_locked());
+
   MonoDelta time_since_last_advance = MonoTime::Now() - last_advanced_safe_time_;
   int64_t max_last_advanced = FLAGS_missed_heartbeats_before_rejecting_snapshot_scans *
       FLAGS_raft_heartbeat_interval_ms;
@@ -157,6 +159,8 @@ bool TimeManager::HasAdvancedSafeTimeRecentlyUnlocked(string* error_message) {
 }
 
 bool TimeManager::IsSafeTimeLaggingUnlocked(Timestamp timestamp, string* error_message) {
+  DCHECK(lock_.is_locked());
+
   // Can't calculate safe time lag for the logical clock.
   if (PREDICT_FALSE(!clock_->HasPhysicalComponent())) return false;
   MonoDelta safe_time_diff = clock_->GetPhysicalComponentDifference(timestamp,
@@ -172,6 +176,8 @@ bool TimeManager::IsSafeTimeLaggingUnlocked(Timestamp timestamp, string* error_m
 }
 
 void TimeManager::MakeWaiterTimeoutMessageUnlocked(Timestamp timestamp, string* error_message) {
+  DCHECK(lock_.is_locked());
+
   string mode = mode_ == LEADER ? "LEADER" : "NON-LEADER";
   string clock_diff = clock_->HasPhysicalComponent() ? clock_->GetPhysicalComponentDifference(
       timestamp, last_safe_ts_).ToString() : "None (Logical clock)";
@@ -238,6 +244,8 @@ Status TimeManager::WaitUntilSafe(Timestamp timestamp, const MonoTime& deadline)
 }
 
 void TimeManager::AdvanceSafeTimeAndWakeUpWaitersUnlocked(Timestamp safe_time) {
+  DCHECK(lock_.is_locked());
+
   if (safe_time <= last_safe_ts_) {
     return;
   }
@@ -258,6 +266,11 @@ void TimeManager::AdvanceSafeTimeAndWakeUpWaitersUnlocked(Timestamp safe_time) {
   }
 }
 
+bool TimeManager::IsTimestampSafe(Timestamp timestamp) {
+  Lock l(lock_);
+  return IsTimestampSafeUnlocked(timestamp);
+}
+
 bool TimeManager::IsTimestampSafeUnlocked(Timestamp timestamp) {
   return timestamp <= GetSafeTimeUnlocked();
 }
@@ -268,6 +281,8 @@ Timestamp TimeManager::GetSafeTime()  {
 }
 
 Timestamp TimeManager::GetSafeTimeUnlocked() {
+  DCHECK(lock_.is_locked());
+
   switch (mode_) {
     case LEADER: {
       // In ASCII form, where 'S' represents a safe timestamp, 'A' represents the last assigned
@@ -307,6 +322,13 @@ Timestamp TimeManager::GetSafeTimeUnlocked() {
 }
 
 Timestamp TimeManager::GetSerialTimestamp() {
+  Lock l(lock_);
+  return GetSerialTimestampUnlocked();
+}
+
+Timestamp TimeManager::GetSerialTimestampUnlocked() {
+  DCHECK(lock_.is_locked());
+
   last_serial_ts_assigned_ = clock_->Now();
   return last_serial_ts_assigned_;
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/b33689e7/src/kudu/consensus/time_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/time_manager.h b/src/kudu/consensus/time_manager.h
index 83b507a..58720fe 100644
--- a/src/kudu/consensus/time_manager.h
+++ b/src/kudu/consensus/time_manager.h
@@ -126,10 +126,15 @@ class TimeManager : public RefCountedThreadSafe<TimeManager> {
   //
   // In non-leader mode returns the last safe time received from a leader.
   Timestamp GetSafeTime();
+
+  // Returns a timestamp that is guaranteed to be higher than all other timestamps
+  // that have been assigned by calls to GetSerialTimestamp() (in this or another
+  // replica).
+  Timestamp GetSerialTimestamp();
+
  private:
   FRIEND_TEST(TimeManagerTest, TestTimeManagerNonLeaderMode);
   FRIEND_TEST(TimeManagerTest, TestTimeManagerLeaderMode);
-  friend class RaftConsensus;
 
   // Returns whether we've advanced safe time recently.
   // If this returns false we might be partitioned or there might be election churn.
@@ -164,21 +169,22 @@ class TimeManager : public RefCountedThreadSafe<TimeManager> {
 
   // Returns whether 'timestamp' is safe.
   // Requires that we've waited for the local clock to move past 'timestamp'.
+  bool IsTimestampSafe(Timestamp timestamp);
+
+  // Internal, unlocked implementation of IsTimestampSafe().
   bool IsTimestampSafeUnlocked(Timestamp timestamp);
 
   // Advances safe time and wakes up any waiters.
   void AdvanceSafeTimeAndWakeUpWaitersUnlocked(Timestamp safe_time);
 
-  // Returns a timestamp that is guaranteed to higher than all other timestamps
-  // that have been assigned by calls to GetSerialTimestamp() (in this or another
-  // replica).
-  Timestamp GetSerialTimestamp();
+  // Internal, unlocked implementation of GetSerialTimestamp().
+  Timestamp GetSerialTimestampUnlocked();
 
-  // Like the above, but returns a serial timestamp plus the maximum error.
+  // Like GetSerialTimestamp(), but returns a serial timestamp plus the maximum error.
   // NOTE: GetSerialTimestamp() might still return timestamps that are smaller.
   Timestamp GetSerialTimestampPlusMaxError();
 
-  // Internal, unlocked implementation of GetSafeTime();
+  // Internal, unlocked implementation of GetSafeTime().
   Timestamp GetSafeTimeUnlocked();
 
   // Lock to protect the non-const fields below.


[2/3] kudu git commit: stopwatch: ensure LOG_TIMING always prints

Posted by to...@apache.org.
stopwatch: ensure LOG_TIMING always prints

On rare occasion in benchmark runs, the LOG_TIMING macro doesn't produce
any output. It's not entirely clear why this happens, but it seems the
only possibility is that the so-called monotonic clock is going
backwards. This change ensures that the macro always prints even if the
clock moves backwards.

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


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

Branch: refs/heads/master
Commit: 41a3aeabb902f5b46e97ea0edaaa2efd9f7cc006
Parents: b33689e
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Jun 29 11:14:33 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Jun 30 01:38:35 2017 +0000

----------------------------------------------------------------------
 src/kudu/util/stopwatch.h | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/41a3aeab/src/kudu/util/stopwatch.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/stopwatch.h b/src/kudu/util/stopwatch.h
index e86d90c..ea9a1be 100644
--- a/src/kudu/util/stopwatch.h
+++ b/src/kudu/util/stopwatch.h
@@ -327,7 +327,10 @@ class LogTiming {
   void Print(int64_t max_expected_millis) {
     stopwatch_.stop();
     CpuTimes times = stopwatch_.elapsed();
-    if (times.wall_millis() > max_expected_millis) {
+    // TODO(todd): for some reason, times.wall_millis() sometimes ends up negative
+    // on rare occasion, for unclear reasons, so we have to check max_expected_millis
+    // < 0 to be sure we always print when requested.
+    if (max_expected_millis < 0 || times.wall_millis() > max_expected_millis) {
       google::LogMessage(file_, line_, severity_).stream()
         << prefix_ << "Time spent " << description_ << ": "
         << times.ToString();


[3/3] kudu git commit: java: fix incorrect hashmap usage in Statistics

Posted by to...@apache.org.
java: fix incorrect hashmap usage in Statistics

65cb2edf5661599f689e28f4eec161fe062f7cb5 changed the hash map in the
Statistics class to use Strings instead of Slices as keys, but didn't
properly update all of the call sites.

Change-Id: I715e96a02ec83199b7e0678281c0857bf9258cf4
Reviewed-on: http://gerrit.cloudera.org:8080/7335
Reviewed-by: Jean-Daniel Cryans <jd...@apache.org>
Tested-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 332ebcd063b55923806b1704113d0623f7aae745
Parents: 41a3aea
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Jun 29 14:21:15 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Jun 30 01:39:09 2017 +0000

----------------------------------------------------------------------
 .../main/java/org/apache/kudu/client/Statistics.java   | 11 ++---------
 .../java/org/apache/kudu/client/TestStatistics.java    | 13 +++++++++++--
 2 files changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/332ebcd0/java/kudu-client/src/main/java/org/apache/kudu/client/Statistics.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/Statistics.java b/java/kudu-client/src/main/java/org/apache/kudu/client/Statistics.java
index b905b33..e47d3a6 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Statistics.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Statistics.java
@@ -17,7 +17,6 @@
 
 package org.apache.kudu.client;
 
-import java.nio.charset.Charset;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLongArray;
@@ -26,10 +25,6 @@ import com.google.common.collect.Sets;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
-import org.apache.kudu.util.Slice;
-import org.apache.kudu.util.Slices;
-
-
 /**
  * A Statistics belongs to a specific AsyncKuduClient. It stores client-level
  * statistics including number of operations, number of bytes written, number of
@@ -101,8 +96,7 @@ public class Statistics {
    * @return the value of the statistic
    */
   public long getTabletStatistic(String tabletId, Statistic statistic) {
-    Slice tabletIdAsSlice = Slices.copiedBuffer(tabletId, Charset.defaultCharset());
-    TabletStatistics tabletStatistics = stsMap.get(tabletIdAsSlice);
+    TabletStatistics tabletStatistics = stsMap.get(tabletId);
     if (tabletStatistics == null) {
       return 0;
     } else {
@@ -173,8 +167,7 @@ public class Statistics {
    * @return table name
    */
   public String getTableName(String tabletId) {
-    Slice tabletIdAsSlice = Slices.copiedBuffer(tabletId, Charset.defaultCharset());
-    TabletStatistics tabletStatistics = stsMap.get(tabletIdAsSlice);
+    TabletStatistics tabletStatistics = stsMap.get(tabletId);
     if (tabletStatistics == null) {
       return null;
     } else {

http://git-wip-us.apache.org/repos/asf/kudu/blob/332ebcd0/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java
index fadbfde..7afb524 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java
@@ -23,6 +23,9 @@ import org.junit.Test;
 
 import org.apache.kudu.client.Statistics.Statistic;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class TestStatistics extends BaseKuduTest {
 
   private static final String TABLE_NAME = TestStatistics.class.getName() + "-"
@@ -69,7 +72,13 @@ public class TestStatistics extends BaseKuduTest {
     assertEquals(rowCount, statistics.getClientStatistic(Statistic.OPS_ERRORS));
     assertEquals(byteSize * 2, statistics.getClientStatistic(Statistic.BYTES_WRITTEN));
 
-    assertEquals(1, statistics.getTableSet().size());
-    assertEquals(1, statistics.getTabletSet().size());
+    List<String> tableNames = new ArrayList<>(statistics.getTableSet());
+    assertEquals(1, tableNames.size());
+    assertEquals(TABLE_NAME, tableNames.get(0));
+    assertEquals(rowCount, statistics.getTableStatistic(TABLE_NAME, Statistic.WRITE_OPS));
+
+    List<String> tabletIds = new ArrayList<>(statistics.getTabletSet());
+    assertEquals(1, tabletIds.size());
+    assertEquals(rowCount, statistics.getTabletStatistic(tabletIds.get(0), Statistic.WRITE_OPS));
   }
 }