[ 
https://issues.apache.org/jira/browse/KUDU-1548?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dinesh Bhat updated KUDU-1548:
------------------------------
    Assignee:     (was: Dinesh Bhat)

> Looping raft_consensus-itest hitting some failures
> --------------------------------------------------
>
>                 Key: KUDU-1548
>                 URL: https://issues.apache.org/jira/browse/KUDU-1548
>             Project: Kudu
>          Issue Type: Bug
>    Affects Versions: 0.10.0
>            Reporter: Dinesh Bhat
>         Attachments: dist-test-results-1548.tar.gz
>
>
> Find the detailed test results/logs under the attached tarfile 
> dist-test-results-1548.tar.gz
> [~mpercy] aready is addressing one test flakiness here: 
> https://gerrit.cloudera.org/#/c/3819
> {noformat}
> The test log looked something like this:
> I0729 18:59:47.834403 11544 raft_consensus.cc:370] T 
> e3503c47a21649ca931234999cd0bb45 P d4f64819170a4cf78fe4c9e9a72ec4b9 [term 1 
> FOLLOWER]: No leader contacted us within the election timeout. Triggering 
> leader election
> I0729 18:59:47.834686 11544 raft_consensus.cc:2019] T 
> e3503c47a21649ca931234999cd0bb45 P d4f64819170a4cf78fe4c9e9a72ec4b9 [term 1 
> FOLLOWER]: Advancing to term 2
> I0729 18:59:47.840427 11544 leader_election.cc:223] T 
> e3503c47a21649ca931234999cd0bb45 P d4f64819170a4cf78fe4c9e9a72ec4b9 
> [CANDIDATE]: Term 2 election: Requesting vote from peer 
> 54197053abab4b6cb1b1632c9d1062dc
> I0729 18:59:47.840860 11544 leader_election.cc:223] T 
> e3503c47a21649ca931234999cd0bb45 P d4f64819170a4cf78fe4c9e9a72ec4b9 
> [CANDIDATE]: Term 2 election: Requesting vote from peer 
> 3522a8de8170476dba0beb58cb2150d4
> I0729 18:59:47.872720 11669 raft_consensus.cc:869] T 
> e3503c47a21649ca931234999cd0bb45 P 3522a8de8170476dba0beb58cb2150d4 [term 1 
> FOLLOWER]: Refusing update from remote peer 54197053abab4b6cb1b1632c9d1062dc: 
> Log matching property violated. Preceding OpId in replica: term: 1 index: 1. 
> Preceding OpId from leader: term: 1 index: 2. (index mismatch)
> I0729 18:59:47.874522 11454 consensus_queue.cc:578] T 
> e3503c47a21649ca931234999cd0bb45 P 54197053abab4b6cb1b1632c9d1062dc [LEADER]: 
> Connected to new peer: Peer: 3522a8de8170476dba0beb58cb2150d4, Is new: false, 
> Last received: 1.1, Next index: 2, Last known committed idx: 1, Last exchange 
> result: ERROR, Needs remote bootstrap: false
> I0729 18:59:47.878105 11150 raft_consensus.cc:1324] T 
> e3503c47a21649ca931234999cd0bb45 P 54197053abab4b6cb1b1632c9d1062dc [term 1 
> LEADER]: Handling vote request from an unknown peer 
> d4f64819170a4cf78fe4c9e9a72ec4b9
> I0729 18:59:47.878290 11150 raft_consensus.cc:2014] T 
> e3503c47a21649ca931234999cd0bb45 P 54197053abab4b6cb1b1632c9d1062dc [term 1 
> LEADER]: Stepping down as leader of term 1
> I0729 18:59:47.878451 11150 raft_consensus.cc:499] T 
> e3503c47a21649ca931234999cd0bb45 P 54197053abab4b6cb1b1632c9d1062dc [term 1 
> LEADER]: Becoming Follower/Learner. State: Replica: 
> 54197053abab4b6cb1b1632c9d1062dc, State: 1, Role: LEADER
> Watermarks: {Received: term: 1 index: 2 Committed: term: 1 index: 1}
> I0729 18:59:47.878968 11150 consensus_queue.cc:162] T 
> e3503c47a21649ca931234999cd0bb45 P 54197053abab4b6cb1b1632c9d1062dc 
> [NON_LEADER]: Queue going to NON_LEADER mode. State: All replicated op: 0.0, 
> Majority replicated op: 1.1, Committed index: 1.1, Last appended: 1.2, 
> Current term: 1, Majority size: -1, State: 1, Mode: NON_LEADER
> I0729 18:59:47.879871 11150 consensus_peers.cc:358] T 
> e3503c47a21649ca931234999cd0bb45 P 54197053abab4b6cb1b1632c9d1062dc -> Peer 
> 3522a8de8170476dba0beb58cb2150d4 (127.37.56.2:53243): Closing peer: 
> 3522a8de8170476dba0beb58cb2150d4
> I0729 18:59:47.882057 11150 raft_consensus.cc:2019] T 
> e3503c47a21649ca931234999cd0bb45 P 54197053abab4b6cb1b1632c9d1062dc [term 1 
> FOLLOWER]: Advancing to term 2
> I0729 18:59:47.885711 11150 raft_consensus.cc:1626] T 
> e3503c47a21649ca931234999cd0bb45 P 54197053abab4b6cb1b1632c9d1062dc [term 2 
> FOLLOWER]: Leader election vote request: Denying vote to candidate 
> d4f64819170a4cf78fe4c9e9a72ec4b9 for term 2 because replica has last-logged 
> OpId of term: 1 index: 2, which is greater than that of the candidate, which 
> has last-logged OpId of term: 1 index: 1.
> I0729 18:59:47.892060 11477 leader_election.cc:361] T 
> e3503c47a21649ca931234999cd0bb45 P d4f64819170a4cf78fe4c9e9a72ec4b9 
> [CANDIDATE]: Term 2 election: Vote denied by peer 
> 54197053abab4b6cb1b1632c9d1062dc. Message: Invalid argument: T 
> e3503c47a21649ca931234999cd0bb45 P 54197053abab4b6cb1b1632c9d1062dc [term 2 
> FOLLOWER]: Leader election vote request: Denying vote to candidate 
> d4f64819170a4cf78fe4c9e9a72ec4b9 for term 2 because replica has last-logged 
> OpId of term: 1 index: 2, which is greater than that of the candidate, which 
> has last-logged OpId of term: 1 index: 1.
> I0729 18:59:47.894548 11669 raft_consensus.cc:1324] T 
> e3503c47a21649ca931234999cd0bb45 P 3522a8de8170476dba0beb58cb2150d4 [term 1 
> FOLLOWER]: Handling vote request from an unknown peer 
> d4f64819170a4cf78fe4c9e9a72ec4b9
> I0729 18:59:47.894688 11669 raft_consensus.cc:2019] T 
> e3503c47a21649ca931234999cd0bb45 P 3522a8de8170476dba0beb58cb2150d4 [term 1 
> FOLLOWER]: Advancing to term 2
> I0729 18:59:47.896904 11669 raft_consensus.cc:1626] T 
> e3503c47a21649ca931234999cd0bb45 P 3522a8de8170476dba0beb58cb2150d4 [term 2 
> FOLLOWER]: Leader election vote request: Denying vote to candidate 
> d4f64819170a4cf78fe4c9e9a72ec4b9 for term 2 because replica has last-logged 
> OpId of term: 1 index: 2, which is greater than that of the candidate, which 
> has last-logged OpId of term: 1 index: 1.
> I0729 18:59:47.898056 11477 leader_election.cc:361] T 
> e3503c47a21649ca931234999cd0bb45 P d4f64819170a4cf78fe4c9e9a72ec4b9 
> [CANDIDATE]: Term 2 election: Vote denied by peer 
> 3522a8de8170476dba0beb58cb2150d4. Message: Invalid argument: T 
> e3503c47a21649ca931234999cd0bb45 P 3522a8de8170476dba0beb58cb2150d4 [term 2 
> FOLLOWER]: Leader election vote request: Denying vote to candidate 
> d4f64819170a4cf78fe4c9e9a72ec4b9 for term 2 because replica has last-logged 
> OpId of term: 1 index: 2, which is greater than that of the candidate, which 
> has last-logged OpId of term: 1 index: 1.
> I0729 18:59:47.898530 11477 leader_election.cc:248] T 
> e3503c47a21649ca931234999cd0bb45 P d4f64819170a4cf78fe4c9e9a72ec4b9 
> [CANDIDATE]: Term 2 election: Election decided. Result: candidate lost.
> I0729 18:59:47.899269 11598 raft_consensus.cc:1821] T 
> e3503c47a21649ca931234999cd0bb45 P d4f64819170a4cf78fe4c9e9a72ec4b9 [term 2 
> FOLLOWER]: Leader election lost for term 2. Reason: None given
> W0729 19:00:48.184541 11433 batcher.cc:315] Timed out: Failed to write batch 
> of 10 ops to tablet e3503c47a21649ca931234999cd0bb45 after 340 attempt(s): 
> Failed to write to server: (no server available): Write(tablet: 
> e3503c47a21649ca931234999cd0bb45, num_ops: 10, num_attempts: 340) passed its 
> deadline: Illegal state: Replica 54197053abab4b6cb1b1632c9d1062dc is not 
> leader of this config. Role: FOLLOWER. Consensus state: current_term: 2 
> leader_uuid: "" config { peers { permanent_uuid: 
> "54197053abab4b6cb1b1632c9d1062dc" member_type: VOTER last_known_addr { host: 
> "127.37.56.0" port: 33531 } } peers { permanent_uuid: 
> "3522a8de8170476dba0beb58cb2150d4" member_type: VOTER last_known_addr { host: 
> "127.37.56.2" port: 53243 } } }
> E0729 19:00:48.185212  9528 raft_consensus-itest.cc:246] Unexpected error: 
> Timed out: Failed to write batch of 10 ops to tablet 
> e3503c47a21649ca931234999cd0bb45 after 340 attempt(s): Failed to write to 
> server: (no server available): Write(tablet: 
> e3503c47a21649ca931234999cd0bb45, num_ops: 10, num_attempts: 340) passed its 
> deadline: Illegal state: Replica 54197053abab4b6cb1b1632c9d1062dc is not 
> leader of this config. Role: FOLLOWER. Consensus state: current_term: 2 
> leader_uuid: "" config { peers { permanent_uuid: 
> "54197053abab4b6cb1b1632c9d1062dc" member_type: VOTER last_known_addr { host: 
> "127.37.56.0" port: 33531 } } peers { permanent_uuid: 
> "3522a8de8170476dba0beb58cb2150d4" member_type: VOTER last_known_addr { host: 
> "127.37.56.2" port: 53243 } } }
> {noformat}
> Apart from above, I am also observing few other interesting issues from the 
> logs from various tests. I was hoping to upload the tarfile of logs, but not 
> able to find the upload button.
> ========================> this looks like a genuine catch by the test:
> {noformat}
> INFO: Connected to all 7 Tablet Servers
> INFO: Table TestTable is HEALTHY
> INFO: The metadata for 1 tables is HEALTHY
> I0729 19:04:55.100972 30908 tablet.cc:1317] T 
> b6a2f16af1054af5ac2135d875fd16b9 Flush: entering phase 2 (starting to 
> duplicate updates in new rowsets)
> I0729 19:04:55.102052 30908 tablet.cc:1372] T 
> b6a2f16af1054af5ac2135d875fd16b9 Flush Phase 2: carrying over any updates 
> which arrived during Phase 1
> I0729 19:04:55.102242 30908 tablet.cc:1374] T 
> b6a2f16af1054af5ac2135d875fd16b9 Phase 2 snapshot: 
> MvccSnapshot[committed={T|T < 6020378860105490432 or (T in 
> {6020378860105490432})}]
> I0729 19:04:57.870882 30908 tablet.cc:1415] T 
> b6a2f16af1054af5ac2135d875fd16b9 Flush successful on 18820 rows (457274 bytes)
> I0729 19:04:57.906981 30908 maintenance_manager.cc:353] Time spent running 
> FlushMRSOp(b6a2f16af1054af5ac2135d875fd16b9): real 9.390s    user 5.560s     
> sys 0.108s
> I0729 19:04:58.039110 30908 tablet.cc:1449] T 
> b6a2f16af1054af5ac2135d875fd16b9 Compaction: stage 1 complete, picked 2 
> rowsets to compact
> I0729 19:04:58.039314 30908 compaction.cc:588] Selected 2 rowsets to compact:
> I0729 19:04:58.039435 30908 compaction.cc:591] RowSet(0)(current size on 
> disk: ~424615 bytes)
> I0729 19:04:58.039618 30908 compaction.cc:591] RowSet(1)(current size on 
> disk: ~408012 bytes)
> I0729 19:04:58.039865 30908 tablet.cc:1237] T 
> b6a2f16af1054af5ac2135d875fd16b9 Compaction: entering phase 1 (flushing 
> snapshot). Phase 1 snapshot: MvccSnapshot[committed={T|T < 
> 6020378860105490432 or (T in {6020378860105490432})}]
> I0729 19:04:58.042769 30908 multi_column_writer.cc:85] Opened CFile writer 
> for column key[int32 NOT NULL]
> I0729 19:04:58.043309 30908 multi_column_writer.cc:85] Opened CFile writer 
> for column int_val[int32 NOT NULL]
> I0729 19:04:58.043864 30908 multi_column_writer.cc:85] Opened CFile writer 
> for column string_val[string NULLABLE]
> -----------------------
> TestTable
> -----------------------
> T b6a2f16af1054af5ac2135d875fd16b9 P 7f1a9aa5093941a48014539dac242908 
> (127.107.54.1:50824): Checksum: 82472867647901
> T b6a2f16af1054af5ac2135d875fd16b9 P cccf457b5acd470d8014c032439134e5 
> (127.107.54.6:59813): Checksum: 82472867647901
> T b6a2f16af1054af5ac2135d875fd16b9 P f342febecdae405c835dd8942dc65b7c 
> (127.107.54.2:49802): Checksum: 82472867647901
> T b6a2f16af1054af5ac2135d875fd16b9 P 8021d519fa2b431f97965b4e2ebd417f 
> (127.107.54.3:48160): Checksum: 82472867647901
> T b6a2f16af1054af5ac2135d875fd16b9 P f5c4abc758674f03947c4dd3e85cc97a 
> (127.107.54.4:53658): Checksum: 82472867647901
> T b6a2f16af1054af5ac2135d875fd16b9 P 9cce3c528e7c4d539d2e553f31bf0496 
> (127.107.54.0:48194): Checksum: 82472867647901
> T b6a2f16af1054af5ac2135d875fd16b9 P c54f418ad2d245488f429c826c8ab64f 
> (127.107.54.5:50458): Checksum: 82472867647901
> /data/10/dinesh/kudu/src/kudu/integration-tests/cluster_verifier.cc:103: 
> Failure
> Failed
> Bad status: Corruption: row count 38470 is not exactly expected value 40000
> /data/10/dinesh/kudu/src/kudu/integration-tests/ts_itest-base.h:473: Failure
> Expected: v.CheckRowCount(kTableId, ClusterVerifier::EXACTLY, 
> expected_result_count) doesn't generate new fatal failures in the current 
> thread.
>   Actual: it does.
> /data/10/dinesh/kudu/src/kudu/integration-tests/raft_consensus-itest.cc:1000: 
> Failure
> Expected: AssertAllReplicasAgree(FLAGS_client_inserts_per_thread * 
> FLAGS_num_client_threads) doesn't generate new fatal failures in the current 
> thread.
>   Actual: it does.
> I0729 19:04:58.774780 27446 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testfyBxna/build/tsan/bin/kudu-master with pid 27935
> I0729 19:04:58.860906 27446 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testfyBxna/build/tsan/bin/kudu-tserver with pid 31766
> W0729 19:04:58.924994 28455 heartbeater.cc:405] Failed to heartbeat to 
> 127.0.0.1:38591: Network error: Failed to send heartbeat: Client connection 
> negotiation failed: client connection to 127.0.0.1:38591: connect: Connection 
> refused (error 111)
> I0729 19:04:58.927733 27446 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testfyBxna/build/tsan/bin/kudu-tserver with pid 31960
> W0729 19:04:58.956255 30910 consensus_peers.cc:333] T 
> b6a2f16af1054af5ac2135d875fd16b9 P f342febecdae405c835dd8942dc65b7c -> Peer 
> 9cce3c528e7c4d539d2e553f31bf0496 (127.107.54.0:48194): Couldn't send request 
> to peer 9cce3c528e7c4d539d2e553f31bf0496 for tablet 
> b6a2f16af1054af5ac2135d875fd16b9. Status: Network error: Client connection 
> negotiation failed: client connection to 127.107.54.0:48194: connect: 
> Connection refused (error 111). Retrying in the next heartbeat period. 
> Already tried 1 times.
> I0729 19:04:58.974784 27446 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testfyBxna/build/tsan/bin/kudu-tserver with pid 30905
> W0729 19:04:59.052007 31340 heartbeater.cc:405] Failed to heartbeat to 
> 127.0.0.1:38591: Network error: Failed to send heartbeat: Client connection 
> negotiation failed: client connection to 127.0.0.1:38591: connect: Connection 
> refused (error 111)
> I0729 19:04:59.055640 27446 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testfyBxna/build/tsan/bin/kudu-tserver with pid 28341
> W0729 19:04:59.083106 30664 heartbeater.cc:405] Failed to heartbeat to 
> 127.0.0.1:38591: Network error: Failed to send heartbeat: Client connection 
> negotiation failed: client connection to 127.0.0.1:38591: connect: Connection 
> refused (error 111)
> I0729 19:04:59.119417 27446 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testfyBxna/build/tsan/bin/kudu-tserver with pid 31174
> W0729 19:04:59.153098 28806 heartbeater.cc:405] Failed to heartbeat to 
> 127.0.0.1:38591: Network error: Failed to send heartbeat: Client connection 
> negotiation failed: client connection to 127.0.0.1:38591: connect: Connection 
> refused (error 111)
> I0729 19:04:59.193274 27446 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testfyBxna/build/tsan/bin/kudu-tserver with pid 30504
> I0729 19:04:59.258599 27446 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testfyBxna/build/tsan/bin/kudu-tserver with pid 28691
> I0729 19:04:59.320971 27446 test_util.cc:70] 
> -----------------------------------------------
> I0729 19:04:59.321200 27446 test_util.cc:71] Had fatal failures, leaving test 
> files at 
> /tmp/kudutest-1000/raft_consensus-itest.RaftConsensusITest.MultiThreadedInsertWithFailovers.1469818744551962-27446
> [  FAILED  ] RaftConsensusITest.MultiThreadedInsertWithFailovers (351497 ms)
> {noformat}
> ================> watchdog kicking in, although this doesn't seem to be 
> failing the test.
> {noformat}
> Related trace 'txn':
> Metrics: 
> {"raft.queue_time_us":307,"raft.run_cpu_time_us":185,"raft.run_wall_time_us":185,"child_traces":[["txn",{"num_ops":50,"prepare.queue_time_us":273,"prepare.run_cpu_time_us":7414,"prepare.run_wall_time_us":7442}],["txn",{"num_ops":50,"prepare.queue_time_us":1430,"prepare.run_cpu_time_us":7585,"prepare.run_wall_time_us":24823,"spinlock_wait_cycles":39318272}],["txn",{"prepare.queue_time_us":26022}]]}
> I0729 18:58:06.279924 22734 raft_consensus.cc:815] T 
> 50c839cb46394fdab0b77fca859f4060 P aeaeb005d6c143eba8fd2e76ae4194d0 [term 1 
> FOLLOWER]: Deduplicated request from leader. Original: 0.0->[1.1-1.5]   
> Dedup: 1.5->[]
> W0729 18:58:06.340052 22430 kernel_stack_watchdog.cc:130] Thread 22810 stuck 
> at /data/10/dinesh/kudu/src/kudu/util/thread.cc:508 for 306ms:
> Kernel stack:
> [<ffffffff810dcf96>] futex_wait_queue_me+0xd6/0x150
> [<ffffffff81156f55>] __alloc_pages_nodemask+0x165/0xbb0
> [<ffffffff810ddfc6>] futex_wait+0x186/0x280
> [<ffffffff81094cba>] atomic_notifier_call_chain+0x1a/0x20
> [<ffffffff8109d272>] set_task_cpu+0xa2/0x1c0
> [<ffffffff810a645b>] update_curr+0x1ab/0x1b0
> [<ffffffff810dfac8>] do_futex+0x368/0xb10
> [<ffffffff810a68d3>] check_preempt_wakeup+0x103/0x1d0
> [<ffffffff81056caf>] kvm_clock_read+0x1f/0x30
> [<ffffffff8109cfb4>] check_preempt_curr+0x84/0xa0
> [<ffffffff810a0274>] wake_up_new_task+0xf4/0x1b0
> [<ffffffff8118525d>] mprotect_fixup+0x15d/0x250
> [<ffffffff810e03c0>] SyS_futex+0x150/0x1c0
> [<ffffffff8154c279>] stub_clone+0x69/0x90
> [<ffffffff8154bf0d>] system_call_fast_compare_end+0x10/0x15
> [<ffffffffffffffff>] 0xffffffffffffffff
> User stack:
>     @           0x425fc9  __tsan::ProcessPendingSignals() at ??:0
>     @           0x44c54f  pthread_create at ??:0
>     @     0x7fa2cee92301  kudu::Thread::StartThread() at ??:0
>     @     0x7fa2ceea5ee9  kudu::Thread::Create<>() at ??:0
>     @     0x7fa2ceea278c  kudu::ThreadPool::CreateThreadUnlocked() at ??:0
>     @     0x7fa2ceea31bb  kudu::ThreadPool::Submit() at ??:0
>     @     0x7fa2ceea2e13  kudu::ThreadPool::SubmitFunc() at ??:0
>     @     0x7fa2ceea2cd8  kudu::ThreadPool::SubmitClosure() at ??:0
>     @     0x7fa2d4b7ef6d  kudu::tablet::TransactionDriver::ApplyAsync() at 
> ??:0
>     @     0x7fa2d4b7a7db  
> kudu::tablet::TransactionDriver::ReplicationFinished() at ??:0
>     @     0x7fa2d4b71dbc  kudu::internal::RunnableAdapter<>::Run() at ??:0
>     @     0x7fa2d4b71bfc  kudu::internal::InvokeHelper<>::MakeItSo() at ??:0
>     @     0x7fa2d4b71b16  kudu::internal::Invoker<>::Run() at ??:0
>     @     0x7fa2d587b689  kudu::Callback<>::Run() at ??:0
>     @     0x7fa2d45eec79  
> kudu::consensus::ConsensusRound::NotifyReplicationFinished() at ??:0
>     @     0x7fa2d469942e  
> kudu::consensus::ReplicaState::AdvanceCommittedIndexUnlocked() at ??:0
> I0729 18:58:06.546285 22827 rpcz_store.cc:248] Call 
> kudu.tserver.TabletServerService.Write from 127.0.0.1:35348
> {noformat}
> ============================>   Some random data race detection by 
> raft_consensus-itest.RaftConsensusITest.TestAutomaticLeaderElection. Although 
> this didn't seem like test specific.
> {noformat}
> W0729 18:57:58.055794 21432 kernel_stack_watchdog.cc:130] Thread 21503 stuck 
> at /data/10/dinesh/kudu/src/kudu/util/thread.cc:508 for 321ms:
> Kernel stack:
> [<ffffffff8109f961>] sys_sched_yield+0x41/0x50
> [<ffffffff8154bf0d>] system_call_fast_compare_end+0x10/0x15
> [<ffffffffffffffff>] 0xffffffffffffffff
> User stack:
>     @           0x425fc9  __tsan::ProcessPendingSignals() at ??:0
>     @           0x44c54f  pthread_create at ??:0
>     @     0x7f1823011301  kudu::Thread::StartThread() at ??:0
>     @     0x7f1823024ee9  kudu::Thread::Create<>() at ??:0
>     @     0x7f182302178c  kudu::ThreadPool::CreateThreadUnlocked() at ??:0
>     @     0x7f18230221bb  kudu::ThreadPool::Submit() at ??:0
>     @     0x7f1823021e13  kudu::ThreadPool::SubmitFunc() at ??:0
>     @     0x7f1823021cd8  kudu::ThreadPool::SubmitClosure() at ??:0
>     @     0x7f18287e0a50  kudu::consensus::RaftConsensus::MarkDirty() at ??:0
>     @     0x7f18287f4ef6  
> kudu::consensus::RaftConsensus::MarkDirtyOnSuccess() at ??:0
>     @     0x7f182880b3ce  kudu::internal::RunnableAdapter<>::Run() at ??:0
>     @     0x7f182880b289  kudu::internal::InvokeHelper<>::MakeItSo() at ??:0
>     @     0x7f182880b1b2  kudu::internal::Invoker<>::Run() at ??:0
>     @     0x7f18299fa689  kudu::Callback<>::Run() at ??:0
>     @     0x7f18287fa9f0  
> kudu::consensus::RaftConsensus::NonTxRoundReplicationFinished() at ??:0
>     @     0x7f18288093b7  kudu::internal::RunnableAdapter<>::Run() at ??:0
> I0729 18:57:58.117558 19951 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testkyvCLE/build/tsan/bin/kudu-master with pid 20520
> I0729 18:57:58.171807 19951 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testkyvCLE/build/tsan/bin/kudu-tserver with pid 21309
> I0729 18:57:58.214403 19951 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testkyvCLE/build/tsan/bin/kudu-tserver with pid 20692
> W0729 18:57:58.254817 20934 consensus_peers.cc:333] T 
> 19cbc0882f244718af30f1c87c4fdc03 P 2f45c3dad4354e71ab1ed176fe1df1f8 -> Peer 
> 131c71317237499fb49fa7d174861974 (127.77.239.0:33063): Couldn't send request 
> to peer 131c71317237499fb49fa7d174861974 for tablet 
> 19cbc0882f244718af30f1c87c4fdc03. Status: Network error: Client connection 
> negotiation failed: client connection to 127.77.239.0:33063: connect: 
> Connection refused (error 111). Retrying in the next heartbeat period. 
> Already tried 1 times.
> I0729 18:57:58.270571 19951 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testkyvCLE/build/tsan/bin/kudu-tserver with pid 21428
> I0729 18:57:58.304476 19951 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testkyvCLE/build/tsan/bin/kudu-tserver with pid 20926
> I0729 18:57:58.332816 19951 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_testkyvCLE/build/tsan/bin/kudu-tserver with pid 21043
> ==================
> WARNING: ThreadSanitizer: data race (pid=19951)
>   Write of size 8 at 0x7d4c0000b1a0 by main thread:
>     #0 pthread_cond_destroy 
> /data/10/dinesh/kudu/thirdparty/llvm-3.8.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:1157
>  (raft_consensus-itest+0x0000004b8bdc)
>     #1 kudu::ConditionVariable::~ConditionVariable() 
> /data/10/dinesh/kudu/src/kudu/util/condition_variable.cc:57:12 
> (libkudu_util.so+0x00000018f2d8)
>     #2 kudu::ThreadPool::~ThreadPool() 
> /data/10/dinesh/kudu/src/kudu/util/threadpool.cc:132:1 
> (libkudu_util.so+0x0000002f031d)
>     #3 kudu::DefaultDeleter<kudu::ThreadPool>::operator()(kudu::ThreadPool*) 
> const /data/10/dinesh/kudu/src/kudu/gutil/gscoped_ptr.h:145:5 
> (libmaster.so+0x0000001141c6)
>     #4 kudu::internal::gscoped_ptr_impl<kudu::ThreadPool, 
> kudu::DefaultDeleter<kudu::ThreadPool> >::~gscoped_ptr_impl() 
> /data/10/dinesh/kudu/src/kudu/gutil/gscoped_ptr.h:228:7 
> (libmaster.so+0x000000114155)
>     #5 gscoped_ptr<kudu::ThreadPool, kudu::DefaultDeleter<kudu::ThreadPool> 
> >::~gscoped_ptr() /data/10/dinesh/kudu/src/kudu/gutil/gscoped_ptr.h:318:7 
> (libmaster.so+0x0000000fbe58)
>     #6 kudu::DnsResolver::~DnsResolver() 
> /data/10/dinesh/kudu/src/kudu/util/net/dns_resolver.cc:45:1 
> (libkudu_util.so+0x00000028c5fc)
>     #7 
> kudu::DefaultDeleter<kudu::DnsResolver>::operator()(kudu::DnsResolver*) const 
> /data/10/dinesh/kudu/src/kudu/gutil/gscoped_ptr.h:145:5 
> (libkudu_client.so+0x0000001575a6)
>     #8 kudu::internal::gscoped_ptr_impl<kudu::DnsResolver, 
> kudu::DefaultDeleter<kudu::DnsResolver> >::reset(kudu::DnsResolver*) 
> /data/10/dinesh/kudu/src/kudu/gutil/gscoped_ptr.h:254:7 
> (libkudu_client.so+0x00000015752d)
>     #9 gscoped_ptr<kudu::DnsResolver, kudu::DefaultDeleter<kudu::DnsResolver> 
> >::reset(kudu::DnsResolver*) 
> /data/10/dinesh/kudu/src/kudu/gutil/gscoped_ptr.h:375:40 
> (libkudu_client.so+0x000000140228)
>     #10 kudu::client::KuduClient::Data::~Data() 
> /data/10/dinesh/kudu/src/kudu/client/client-internal.cc:270:3 
> (libkudu_client.so+0x000000165f05)
>     #11 kudu::client::KuduClient::~KuduClient() 
> /data/10/dinesh/kudu/src/kudu/client/client.cc:257:3 
> (libkudu_client.so+0x000000132b55)
>     #12 
> std::tr1::_Sp_deleter<kudu::client::KuduClient>::operator()(kudu::client::KuduClient*)
>  const 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/tr1/shared_ptr.h:285:41
>  (libkudu_client.so+0x000000155e56)
>     #13 std::tr1::_Sp_counted_base_impl<kudu::client::KuduClient*, 
> std::tr1::_Sp_deleter<kudu::client::KuduClient>, 
> (__gnu_cxx::_Lock_policy)2>::_M_dispose() 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/tr1/shared_ptr.h:257:9
>  (libkudu_client.so+0x000000155c92)
>     #14 std::tr1::_Sp_counted_base<(__gnu_cxx::_Lock_policy)2>::_M_release() 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/tr1/shared_ptr.h:141:6
>  (raft_consensus-itest+0x00000059f745)
>     #15 
> std::tr1::__shared_count<(__gnu_cxx::_Lock_policy)2>::~__shared_count() 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/tr1/shared_ptr.h:341:4
>  (raft_consensus-itest+0x00000059f6b5)
>     #16 std::tr1::__shared_ptr<kudu::client::KuduClient, 
> (__gnu_cxx::_Lock_policy)2>::~__shared_ptr() 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/tr1/shared_ptr.h:541:11
>  (raft_consensus-itest+0x00000059f64c)
>     #17 std::tr1::shared_ptr<kudu::client::KuduClient>::~shared_ptr() 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/tr1/shared_ptr.h:985:11
>  (raft_consensus-itest+0x00000058f028)
>     #18 
> kudu::tserver::TabletServerIntegrationTestBase::~TabletServerIntegrationTestBase()
>  /data/10/dinesh/kudu/src/kudu/integration-tests/ts_itest-base.h:67:7 
> (raft_consensus-itest+0x00000058f185)
>     #19 kudu::tserver::RaftConsensusITest::~RaftConsensusITest() 
> /data/10/dinesh/kudu/src/kudu/integration-tests/raft_consensus-itest.cc:97:7 
> (raft_consensus-itest+0x00000058e981)
>     #20 
> kudu::tserver::RaftConsensusITest_TestAutomaticLeaderElection_Test::~RaftConsensusITest_TestAutomaticLeaderElection_Test()
>  
> /data/10/dinesh/kudu/src/kudu/integration-tests/raft_consensus-itest.cc:1005:1
>  (raft_consensus-itest+0x00000058d598)
>     #21 
> kudu::tserver::RaftConsensusITest_TestAutomaticLeaderElection_Test::~RaftConsensusITest_TestAutomaticLeaderElection_Test()
>  
> /data/10/dinesh/kudu/src/kudu/integration-tests/raft_consensus-itest.cc:1005:1
>  (raft_consensus-itest+0x00000058d5dc)
>     #22 void 
> testing::internal::HandleSehExceptionsInMethodIfSupported<testing::Test, 
> void>(testing::Test*, void (testing::Test::*)(), char const*) 
> /data/10/dinesh/kudu/thirdparty/gmock-1.7.0/gtest/src/gtest.cc:2078:10 
> (libgmock.so+0x000000044939)
>     #23 void 
> testing::internal::HandleExceptionsInMethodIfSupported<testing::Test, 
> void>(testing::Test*, void (testing::Test::*)(), char const*) 
> /data/10/dinesh/kudu/thirdparty/gmock-1.7.0/gtest/src/gtest.cc:2114 
> (libgmock.so+0x000000044939)
>     #24 main /data/10/dinesh/kudu/src/kudu/util/test_main.cc:48:13 
> (libkudu_test_main.so+0x000000001a34)
>   Previous read of size 8 at 0x7d4c0000b1a0 by thread T30:
>     #0 pthread_cond_signal 
> /data/10/dinesh/kudu/thirdparty/llvm-3.8.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:1143
>  (raft_consensus-itest+0x0000004b8970)
>     #1 kudu::ConditionVariable::Signal() 
> /data/10/dinesh/kudu/src/kudu/util/condition_variable.cc:136:12 
> (libkudu_util.so+0x00000018f998)
>     #2 kudu::ThreadPool::Submit(std::shared_ptr<kudu::Runnable> const&) 
> /data/10/dinesh/kudu/src/kudu/util/threadpool.cc:239:3 
> (libkudu_util.so+0x0000002f1498)
>     #3 kudu::ThreadPool::SubmitFunc(boost::function<void ()> const&) 
> /data/10/dinesh/kudu/src/kudu/util/threadpool.cc:182:10 
> (libkudu_util.so+0x0000002f0e12)
>     #4 kudu::DnsResolver::ResolveAddresses(kudu::HostPort const&, 
> std::vector<kudu::Sockaddr, std::allocator<kudu::Sockaddr> >*, 
> kudu::Callback<void (kudu::Status const&)> const&) 
> /data/10/dinesh/kudu/src/kudu/util/net/dns_resolver.cc:57:14 
> (libkudu_util.so+0x00000028c720)
>     #5 
> kudu::client::internal::RemoteTabletServer::InitProxy(kudu::client::KuduClient*,
>  kudu::Callback<void (kudu::Status const&)> const&) 
> /data/10/dinesh/kudu/src/kudu/client/meta_cache.cc:118:3 
> (libkudu_client.so+0x00000019b7d3)
>     #6 
> kudu::client::internal::MetaCacheServerPicker::PickLeader(kudu::Callback<void 
> (kudu::Status const&, kudu::client::internal::RemoteTabletServer*)> const&, 
> kudu::MonoTime const&) 
> /data/10/dinesh/kudu/src/kudu/client/meta_cache.cc:389:3 
> (libkudu_client.so+0x00000019e6d8)
>     #7 kudu::rpc::RetriableRpc<kudu::client::internal::RemoteTabletServer, 
> kudu::tserver::WriteRequestPB, kudu::tserver::WriteResponsePB>::SendRpc() 
> /data/10/dinesh/kudu/src/kudu/rpc/retriable_rpc.h:129:3 
> (libkudu_client.so+0x00000011bc76)
>     #8 kudu::rpc::RpcRetrier::DelayedRetryCb(kudu::rpc::Rpc*, kudu::Status 
> const&) /data/10/dinesh/kudu/src/kudu/rpc/rpc.cc:88:5 
> (libkrpc.so+0x00000016290c)
>     #9 boost::_mfi::mf2<void, kudu::rpc::RpcRetrier, kudu::rpc::Rpc*, 
> kudu::Status const&>::operator()(kudu::rpc::RpcRetrier*, kudu::rpc::Rpc*, 
> kudu::Status const&) const 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/bind/mem_fn_template.hpp:280:29
>  (libkrpc.so+0x000000163ddb)
>     #10 void boost::_bi::list3<boost::_bi::value<kudu::rpc::RpcRetrier*>, 
> boost::_bi::value<kudu::rpc::Rpc*>, boost::arg<1> 
> >::operator()<boost::_mfi::mf2<void, kudu::rpc::RpcRetrier, kudu::rpc::Rpc*, 
> kudu::Status const&>, boost::_bi::rrlist1<kudu::Status const&> 
> >(boost::_bi::type<void>, boost::_mfi::mf2<void, kudu::rpc::RpcRetrier, 
> kudu::rpc::Rpc*, kudu::Status const&>&, boost::_bi::rrlist1<kudu::Status 
> const&>&, int) 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/bind/bind.hpp:398:9 
> (libkrpc.so+0x000000163c98)
>     #11 void boost::_bi::bind_t<void, boost::_mfi::mf2<void, 
> kudu::rpc::RpcRetrier, kudu::rpc::Rpc*, kudu::Status const&>, 
> boost::_bi::list3<boost::_bi::value<kudu::rpc::RpcRetrier*>, 
> boost::_bi::value<kudu::rpc::Rpc*>, boost::arg<1> > 
> >::operator()<kudu::Status const&>(kudu::Status const&) 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/bind/bind.hpp:1234:16 
> (libkrpc.so+0x000000163b4a)
>     #12 
> boost::detail::function::void_function_obj_invoker1<boost::_bi::bind_t<void, 
> boost::_mfi::mf2<void, kudu::rpc::RpcRetrier, kudu::rpc::Rpc*, kudu::Status 
> const&>, boost::_bi::list3<boost::_bi::value<kudu::rpc::RpcRetrier*>, 
> boost::_bi::value<kudu::rpc::Rpc*>, boost::arg<1> > >, void, kudu::Status 
> const&>::invoke(boost::detail::function::function_buffer&, kudu::Status 
> const&) 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/function/function_template.hpp:159:11
>  (libkrpc.so+0x0000001636b9)
>     #13 boost::function1<void, kudu::Status const&>::operator()(kudu::Status 
> const&) const 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/function/function_template.hpp:770:14
>  (libkrpc.so+0x00000013f08a)
>     #14 kudu::rpc::DelayedTask::TimerHandler(ev::timer&, int) 
> /data/10/dinesh/kudu/src/kudu/rpc/reactor.cc:473:5 (libkrpc.so+0x00000013b6fa)
>     #15 void ev::base<ev_timer, 
> ev::timer>::method_thunk<kudu::rpc::DelayedTask, 
> &kudu::rpc::DelayedTask::TimerHandler>(ev_loop*, ev_timer*, int) 
> /data/10/dinesh/kudu/thirdparty/installed/include/ev++.h:479:7 
> (libkrpc.so+0x0000001495bb)
>     #16 ev_invoke_pending 
> /data/10/dinesh/kudu/thirdparty/libev-4.20/ev.c:3155 
> (libev.so.4+0x0000000074fc)
>     #17 kudu::rpc::ReactorThread::RunThread() 
> /data/10/dinesh/kudu/src/kudu/rpc/reactor.cc:306:3 (libkrpc.so+0x0000001359dc)
>     #18 boost::_mfi::mf0<void, 
> kudu::rpc::ReactorThread>::operator()(kudu::rpc::ReactorThread*) const 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/bind/mem_fn_template.hpp:49:29
>  (libkrpc.so+0x00000014513b)
>     #19 void boost::_bi::list1<boost::_bi::value<kudu::rpc::ReactorThread*> 
> >::operator()<boost::_mfi::mf0<void, kudu::rpc::ReactorThread>, 
> boost::_bi::list0>(boost::_bi::type<void>, boost::_mfi::mf0<void, 
> kudu::rpc::ReactorThread>&, boost::_bi::list0&, int) 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/bind/bind.hpp:259:9 
> (libkrpc.so+0x000000145018)
>     #20 boost::_bi::bind_t<void, boost::_mfi::mf0<void, 
> kudu::rpc::ReactorThread>, 
> boost::_bi::list1<boost::_bi::value<kudu::rpc::ReactorThread*> > 
> >::operator()() 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/bind/bind.hpp:1222:16 
> (libkrpc.so+0x000000144f7a)
>     #21 
> boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, 
> boost::_mfi::mf0<void, kudu::rpc::ReactorThread>, 
> boost::_bi::list1<boost::_bi::value<kudu::rpc::ReactorThread*> > >, 
> void>::invoke(boost::detail::function::function_buffer&) 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/function/function_template.hpp:159:11
>  (libkrpc.so+0x000000144c10)
>     #22 boost::function0<void>::operator()() const 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/function/function_template.hpp:770:14
>  (libkrpc.so+0x0000000ee8ea)
>     #23 kudu::Thread::SuperviseThread(void*) 
> /data/10/dinesh/kudu/src/kudu/util/thread.cc:586:3 
> (libkudu_util.so+0x0000002e0f99)
>   As if synchronized via sleep:
>     #0 nanosleep 
> /data/10/dinesh/kudu/thirdparty/llvm-3.8.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:371
>  (raft_consensus-itest+0x0000004b3957)
>     #1 boost::detail::yield(unsigned int) 
> /data/10/dinesh/kudu/thirdparty/installed/include/boost/smart_ptr/detail/yield_k.hpp:151:9
>  (libmaster.so+0x000000139a76)
>     #2 kudu::Thread::StartThread(std::string const&, std::string const&, 
> boost::function<void ()> const&, unsigned long, scoped_refptr<kudu::Thread>*) 
> /data/10/dinesh/kudu/src/kudu/util/thread.cc:540:7 
> (libkudu_util.so+0x0000002e06d5)
>     #3 kudu::Status kudu::Thread::Create<void (kudu::ThreadPool::*)(bool), 
> kudu::ThreadPool*, bool>(std::string const&, std::string const&, void 
> (kudu::ThreadPool::* const&)(bool), kudu::ThreadPool* const&, bool const&, 
> scoped_refptr<kudu::Thread>*) 
> /data/10/dinesh/kudu/src/kudu/util/thread.h:162:12 
> (libkudu_util.so+0x0000002f3ee8)
>     #4 kudu::ThreadPool::CreateThreadUnlocked() 
> /data/10/dinesh/kudu/src/kudu/util/threadpool.cc:381:14 
> (libkudu_util.so+0x0000002f078b)
>     #5 kudu::ThreadPool::Submit(std::shared_ptr<kudu::Runnable> const&) 
> /data/10/dinesh/kudu/src/kudu/util/threadpool.cc:211:21 
> (libkudu_util.so+0x0000002f11ba)
>     #6 kudu::ThreadPool::SubmitFunc(boost::function<void ()> const&) 
> /data/10/dinesh/kudu/src/kudu/util/threadpool.cc:182:10 
> (libkudu_util.so+0x0000002f0e12)
>     #7 kudu::DnsResolver::ResolveAddresses(kudu::HostPort const&, 
> std::vector<kudu::Sockaddr, std::allocator<kudu::Sockaddr> >*, 
> kudu::Callback<void (kudu::Status const&)> const&) 
> /data/10/dinesh/kudu/src/kudu/util/net/dns_resolver.cc:57:14 
> (libkudu_util.so+0x00000028c720)
>     #8 
> kudu::client::internal::RemoteTabletServer::InitProxy(kudu::client::KuduClient*,
>  kudu::Callback<void (kudu::Status const&)> const&) 
> /data/10/dinesh/kudu/src/kudu/client/meta_cache.cc:118:3 
> (libkudu_client.so+0x00000019b7d3)
>     #9 
> kudu::client::KuduClient::Data::GetTabletServer(kudu::client::KuduClient*, 
> scoped_refptr<kudu::client::internal::RemoteTablet> const&, 
> kudu::client::KuduClient::ReplicaSelection, std::set<std::string, 
> std::less<std::string>, std::allocator<std::string> > const&, 
> std::vector<kudu::client::internal::RemoteTabletServer*, 
> std::allocator<kudu::client::internal::RemoteTabletServer*> >*, 
> kudu::client::internal::RemoteTabletServer**) 
> /data/10/dinesh/kudu/src/kudu/client/client-internal.cc:351:3 
> (libkudu_client.so+0x000000166ed3)
>     #10 kudu::client::KuduScanner::Data::OpenTablet(std::string const&, 
> kudu::MonoTime const&, std::set<std::string, std::less<std::string>, 
> std::allocator<std::string> >*) 
> /data/10/dinesh/kudu/src/kudu/client/scanner-internal.cc:345:28 
> (libkudu_client.so+0x0000001dd81e)
>     #11 kudu::client::KuduScanner::Data::OpenNextTablet(kudu::MonoTime 
> const&, std::set<std::string, std::less<std::string>, 
> std::allocator<std::string> >*) 
> /data/10/dinesh/kudu/src/kudu/client/scanner-internal.cc:213:10 
> (libkudu_client.so+0x0000001dc761)
>     #12 kudu::client::KuduScanner::Open() 
> /data/10/dinesh/kudu/src/kudu/client/client.cc:1072:3 
> (libkudu_client.so+0x00000013ac98)
>     #13 kudu::ClusterVerifier::DoCheckRowCount(std::string const&, 
> kudu::ClusterVerifier::ComparisonMode, int) 
> /data/10/dinesh/kudu/src/kudu/integration-tests/cluster_verifier.cc:119:3 
> (libintegration-tests.so+0x000000091a95)
>     #14 kudu::ClusterVerifier::CheckRowCount(std::string const&, 
> kudu::ClusterVerifier::ComparisonMode, int) 
> /data/10/dinesh/kudu/src/kudu/integration-tests/cluster_verifier.cc:103:3 
> (libintegration-tests.so+0x000000091370)
>     #15 
> kudu::tserver::TabletServerIntegrationTestBase::AssertAllReplicasAgree(int) 
> /data/10/dinesh/kudu/src/kudu/integration-tests/ts_itest-base.h:473:5 
> (raft_consensus-itest+0x00000057a8b6)
>     #16 
> kudu::tserver::RaftConsensusITest_TestAutomaticLeaderElection_Test::TestBody()
>  
> /data/10/dinesh/kudu/src/kudu/integration-tests/raft_consensus-itest.cc:1046:3
>  (raft_consensus-itest+0x00000052c6f7)
>     #17 void 
> testing::internal::HandleSehExceptionsInMethodIfSupported<testing::Test, 
> void>(testing::Test*, void (testing::Test::*)(), char const*) 
> /data/10/dinesh/kudu/thirdparty/gmock-1.7.0/gtest/src/gtest.cc:2078:10 
> (libgmock.so+0x000000044939)
>     #18 void 
> testing::internal::HandleExceptionsInMethodIfSupported<testing::Test, 
> void>(testing::Test*, void (testing::Test::*)(), char const*) 
> /data/10/dinesh/kudu/thirdparty/gmock-1.7.0/gtest/src/gtest.cc:2114 
> (libgmock.so+0x000000044939)
>     #19 main /data/10/dinesh/kudu/src/kudu/util/test_main.cc:48:13 
> (libkudu_test_main.so+0x000000001a34)
> {noformat}
> ================== Mmmmm, may be not so interesting one:
> {noformat}
> W0729 19:05:02.906085  8280 consensus_peers.cc:333] T 
> 711f402ed471458692610fdabce256a8 P a07659f1ae2444c5b7b6fbc9040f22ab -> Peer 
> 08f3a26cd73e4546a012b26cbf494c7c (127.26.135.1:48804): Couldn't send request 
> to peer 08f3a26cd73e4546a012b26cbf494c7c for tablet 
> 711f402ed471458692610fdabce256a8. Error code: TABLET_NOT_RUNNING (12). 
> Status: Illegal state: Tablet not RUNNING: FAILED: Corruption: Failed log 
> replay. Reason: Debug Info: Error playing entry 2 of segment 6 of tablet 
> 711f402ed471458692610fdabce256a8. Segment path: 
> /tmp/kudutest-1000/raft_consensus-itest.RaftConsensusITest.TestMasterReplacesEvictedFollowers.1469818927330810-6791/raft_consensus-itest-cluster/ts-1/wals/711f402ed471458692610fdabce256a8.recovery/wal-000000006.
>  Entry: type: COMMIT commit { op_type: WRITE_OP commited_op_id { term: 2 
> index: 35 } result { ops { mutated_stores { mrs_id: 3 } } } }: CommitMsg was 
> orphaned but it referred to stores which need replay. Commit: op_type: 
> WRITE_OP commited_op_id { term: 2 index: 35 } result { ops { mutated_stores { 
> mrs_id: 3 } } }. TabletMetadata: table_id: "3cc2bf30b612412f85f30331d371168d" 
> tablet_id: "711f402ed471458692610fdabce256a8" last_durable_mrs_id: 2 rowsets 
> { id: 3 last_durable_dms_id: -1 columns { block { id: 4023299910747499511 } 
> column_id: 10 } columns { block { id: 1707928975668245347 } column_id: 11 } 
> columns { block { id: 3632507944189818889 } column_id: 12 } undo_deltas { 
> block { id: 341880068839001445 } } bloom_block { id: 4206696818985822323 } } 
> table_name: "TestTable" schema { columns { id: 10 name: "key" type: INT32 
> is_key: true is_nullable: false encoding: AUTO_ENCODING compression: 
> DEFAULT_COMPRESSION cfile_block_size: 0 } columns { id: 11 name: "int_val" 
> type: INT32 is_key: false is_nullable: false encoding: AUTO_ENCODING 
> compression: DEFAULT_COMPRESSION cfile_block_size: 0 } columns { id: 12 name: 
> "string_val" type: STRING is_key: false is_nullable: true encoding: 
> AUTO_ENCODING compression: DEFAULT_COMPRESSION cfile_block_size: 0 } } 
> schema_version: 0 tablet_data_state: TABLET_DATA_READY partition { 
> partition_key_start: "" partition_key_end: "" } partition_schema { 
> range_schema { columns { id: 10 } } }. Retrying in the next heartbeat period. 
> Already tried 60 times.
> /data/10/dinesh/kudu/src/kudu/integration-tests/raft_consensus-itest.cc:2484: 
> Failure
> Failed
> Bad status: Timed out: Index 2 not available on all replicas after 30.000s.
> I0729 19:05:03.325995  6791 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_test8pHEMJ/build/tsan/bin/kudu-master with pid 8218
> I0729 19:05:03.351394  6791 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_test8pHEMJ/build/tsan/bin/kudu-tserver with pid 8262
> I0729 19:05:03.382586  6791 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_test8pHEMJ/build/tsan/bin/kudu-tserver with pid 8390
> I0729 19:05:03.413770  6791 external_mini_cluster.cc:658] Killing 
> /tmp/run_tha_test8pHEMJ/build/tsan/bin/kudu-tserver with pid 8507
> I0729 19:05:03.447187  6791 test_util.cc:70] 
> -----------------------------------------------
> I0729 19:05:03.447741  6791 test_util.cc:71] Had fatal failures, leaving test 
> files at 
> /tmp/kudutest-1000/raft_consensus-itest.RaftConsensusITest.TestMasterReplacesEvictedFollowers.1469818927330810-6791
> [  FAILED  ] RaftConsensusITest.TestMasterReplacesEvictedFollowers (39224 ms)
> {noformat}
> ============ Perhaps some more test flakiness:
> {noformat}
> ==================
> WARNING: ThreadSanitizer: signal-unsafe call inside of a signal (pid=21881)
>     #0 operator new(unsigned long) 
> /data/10/dinesh/kudu/thirdparty/llvm-3.8.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_new_delete.cc:45
>  (raft_consensus-itest+0x000000516803)
>     #1 __gnu_cxx::new_allocator<char>::allocate(unsigned long, void const*) 
> <null> (libstdc++.so.6+0x000000160f2a)
>     #2 std::string::_Rep::_S_create(unsigned long, unsigned long, 
> std::allocator<char> const&) <null> (libstdc++.so.6+0x000000160e0f)
>     #3 char* std::string::_S_construct<char const*>(char const*, char const*, 
> std::allocator<char> const&, std::forward_iterator_tag) 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/bits/basic_string.tcc:138:14
>  (libgutil.so+0x0000000c7b32)
>     #4 char* std::string::_S_construct_aux<char const*>(char const*, char 
> const*, std::allocator<char> const&, std::__false_type) 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/bits/basic_string.h:1772:18
>  (libgutil.so+0x0000000c79b8)
>     #5 char* std::string::_S_construct<char const*>(char const*, char const*, 
> std::allocator<char> const&) 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/bits/basic_string.h:1793:11
>  (libgutil.so+0x0000000c7958)
>     #6 std::basic_string<char, std::char_traits<char>, std::allocator<char> 
> >::basic_string(char const*, std::allocator<char> const&) <null> 
> (libstdc++.so.6+0x000000162333)
>     #7 kudu::PstackWatcher::HasProgram(char const*) 
> /data/10/dinesh/kudu/src/kudu/util/pstack_watcher.cc:86:10 
> (libkudu_util.so+0x000000217603)
>     #8 kudu::PstackWatcher::DumpPidStacks(int, int) 
> /data/10/dinesh/kudu/src/kudu/util/pstack_watcher.cc:113:7 
> (libkudu_util.so+0x000000217c7d)
>     #9 kudu::PstackWatcher::DumpStacks(int) 
> /data/10/dinesh/kudu/src/kudu/util/pstack_watcher.cc:107:10 
> (libkudu_util.so+0x00000021758a)
>     #10 KillTestOnTimeout(int) 
> /data/10/dinesh/kudu/src/kudu/util/test_main.cc:72:3 
> (libkudu_test_main.so+0x000000001dd1)
>     #11 __tsan::CallUserSignalHandler(__tsan::ThreadState*, bool, bool, bool, 
> int, my_siginfo_t*, void*) 
> /data/10/dinesh/kudu/thirdparty/llvm-3.8.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:1953
>  (raft_consensus-itest+0x00000048d1b0)
>     #12 kudu::ConditionVariable::Wait() const 
> /data/10/dinesh/kudu/src/kudu/util/condition_variable.cc:66:12 
> (libkudu_util.so+0x00000018f409)
>     #13 kudu::CountDownLatch::Wait() const 
> /data/10/dinesh/kudu/src/kudu/util/countdown_latch.h:74:7 
> (raft_consensus-itest+0x000000581388)
>     #14 
> kudu::tserver::RaftConsensusITest_MultiThreadedInsertWithFailovers_Test::TestBody()
>  
> /data/10/dinesh/kudu/src/kudu/integration-tests/raft_consensus-itest.cc:992:5 
> (raft_consensus-itest+0x00000052ac58)
>     #15 void 
> testing::internal::HandleSehExceptionsInMethodIfSupported<testing::Test, 
> void>(testing::Test*, void (testing::Test::*)(), char const*) 
> /data/10/dinesh/kudu/thirdparty/gmock-1.7.0/gtest/src/gtest.cc:2078:10 
> (libgmock.so+0x000000044939)
>     #16 void 
> testing::internal::HandleExceptionsInMethodIfSupported<testing::Test, 
> void>(testing::Test*, void (testing::Test::*)(), char const*) 
> /data/10/dinesh/kudu/thirdparty/gmock-1.7.0/gtest/src/gtest.cc:2114 
> (libgmock.so+0x000000044939)
>     #17 main /data/10/dinesh/kudu/src/kudu/util/test_main.cc:48:13 
> (libkudu_test_main.so+0x000000001a34)
> SUMMARY: ThreadSanitizer: signal-unsafe call inside of a signal 
> /data/10/dinesh/kudu/thirdparty/llvm-3.8.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_new_delete.cc:45
>  in operator new(unsigned long)
> ==================
> ==================
> WARNING: ThreadSanitizer: signal-unsafe call inside of a signal (pid=21881)
>     #0 operator new(unsigned long) 
> /data/10/dinesh/kudu/thirdparty/llvm-3.8.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_new_delete.cc:45
>  (raft_consensus-itest+0x000000516803)
>     #1 __gnu_cxx::new_allocator<std::string>::allocate(unsigned long, void 
> const*) 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/ext/new_allocator.h:104:27
>  (raft_consensus-itest+0x0000005af6ea)
>     #2 std::allocator_traits<std::allocator<std::string> 
> >::allocate(std::allocator<std::string>&, unsigned long) 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/bits/alloc_traits.h:357:16
>  (raft_consensus-itest+0x0000005af66c)
>     #3 std::_Vector_base<std::string, std::allocator<std::string> 
> >::_M_allocate(unsigned long) 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/bits/stl_vector.h:170:20
>  (raft_consensus-itest+0x0000005af54e)
>     #4 void std::vector<std::string, std::allocator<std::string> 
> >::_M_emplace_back_aux<std::string const&>(std::string const&) 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/bits/vector.tcc:412:22
>  (raft_consensus-itest+0x0000005c96a9)
>     #5 std::vector<std::string, std::allocator<std::string> 
> >::push_back(std::string const&) 
> /data/10/dinesh/kudu/thirdparty/installed-deps-tsan/gcc/include/c++/4.9.3/bits/stl_vector.h:923:4
>  (raft_consensus-itest+0x000000587607)
>     #6 kudu::PstackWatcher::HasProgram(char const*) 
> /data/10/dinesh/kudu/src/kudu/util/pstack_watcher.cc:88:3 
> (libkudu_util.so+0x000000217631)
>     #7 kudu::PstackWatcher::DumpPidStacks(int, int) 
> /data/10/dinesh/kudu/src/kudu/util/pstack_watcher.cc:113:7 
> (libkudu_util.so+0x000000217c7d)
>     #8 kudu::PstackWatcher::DumpStacks(int) 
> /data/10/dinesh/kudu/src/kudu/util/pstack_watcher.cc:107:10 
> (libkudu_util.so+0x00000021758a)
>     #9 KillTestOnTimeout(int) 
> /data/10/dinesh/kudu/src/kudu/util/test_main.cc:72:3 
> (libkudu_test_main.so+0x000000001dd1)
>     #10 __tsan::CallUserSignalHandler(__tsan::ThreadState*, bool, bool, bool, 
> int, my_siginfo_t*, void*) 
> /data/10/dinesh/kudu/thirdparty/llvm-3.8.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_interceptors.cc:1953
>  (raft_consensus-itest+0x00000048d1b0)
>     #11 kudu::ConditionVariable::Wait() const 
> /data/10/dinesh/kudu/src/kudu/util/condition_variable.cc:66:12 
> (libkudu_util.so+0x00000018f409)
>     #12 kudu::CountDownLatch::Wait() const 
> /data/10/dinesh/kudu/src/kudu/util/countdown_latch.h:74:7 
> (raft_consensus-itest+0x000000581388)
>     #13 
> kudu::tserver::RaftConsensusITest_MultiThreadedInsertWithFailovers_Test::TestBody()
>  
> /data/10/dinesh/kudu/src/kudu/integration-tests/raft_consensus-itest.cc:992:5 
> (raft_consensus-itest+0x00000052ac58)
>     #14 void 
> testing::internal::HandleSehExceptionsInMethodIfSupported<testing::Test, 
> void>(testing::Test*, void (testing::Test::*)(), char const*) 
> /data/10/dinesh/kudu/thirdparty/gmock-1.7.0/gtest/src/gtest.cc:2078:10 
> (libgmock.so+0x000000044939)
>     #15 void 
> testing::internal::HandleExceptionsInMethodIfSupported<testing::Test, 
> void>(testing::Test*, void (testing::Test::*)(), char const*) 
> /data/10/dinesh/kudu/thirdparty/gmock-1.7.0/gtest/src/gtest.cc:2114 
> (libgmock.so+0x000000044939)
>     #16 main /data/10/dinesh/kudu/src/kudu/util/test_main.cc:48:13 
> (libkudu_test_main.so+0x000000001a34)
> SUMMARY: ThreadSanitizer: signal-unsafe call inside of a signal 
> /data/10/dinesh/kudu/thirdparty/llvm-3.8.0.src/projects/compiler-rt/lib/tsan/rtl/tsan_new_delete.cc:45
>  in operator new(unsigned long)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to