[kudu-CR] consensus: consolidate Raft thread pools

2017-06-27 Thread Todd Lipcon (Code Review)
Todd Lipcon has submitted this change and it was merged.

Change subject: consensus: consolidate Raft thread pools
..


consensus: consolidate Raft thread pools

This patch consolidates the two thread pools used in Raft consensus: the
observer and "Raft" (for lack of a better name) pools. The former runs tasks
for infrequent events such as term and config changes while the latter is
used for periodic events such as processing heartbeat responses.

Both per-replica pools are consolidated into a single per-server pool.
Tokens are used to ensure that tasks belonging to a single replica are
logically grouped together. One token is shared by RaftConsensus (primary)
and PeerManager (secondary); this mirrors the existing sharing behavior and
is safe because token operations are thread-safe. A second serial token is
dedicated for the observer submissions, so that its (potentially)
long-running tasks don't starve any periodic Raft events.

The non-default max_threads may come as a surprise, but David showed me how
tasks submitted to either pool may sometimes lead to an fsync (mostly via
cmeta flushes). As such, it isn't safe to use the default num_cpus upper
bound, as that may cause such IO-based tasks to block other CPU-only tasks
(or worse, periodic tasks like heartbeat response processing).

What per-replica threads are not addressed by this patch?
- Failure detection thread: a threadpool isn't the right model
  for these. Something like a hash wheel timer would be ideal.
- Prepare thread pool (max_threads=1): these could be consolidated too, but
  their metrics are per-replica, and tokens don't (yet) support that.
  Meaning, if they were consolidated now, the metrics would also consolidate
  and would be less useful as a result.

Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Reviewed-on: http://gerrit.cloudera.org:8080/6946
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon 
---
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_peers.h
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/peer_manager.cc
M src/kudu/consensus/peer_manager.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/kserver/kserver.cc
M src/kudu/kserver/kserver.h
M src/kudu/master/sys_catalog.cc
M src/kudu/tablet/tablet_replica-test.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/tablet_copy_source_session-test.cc
M src/kudu/tserver/ts_tablet_manager.cc
19 files changed, 193 insertions(+), 123 deletions(-)

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



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

Gerrit-MessageType: merged
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 16
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-27 Thread Todd Lipcon (Code Review)
Todd Lipcon has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 15: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 15
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: No


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-26 Thread Adar Dembo (Code Review)
Hello Kudu Jenkins,

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

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

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

Change subject: consensus: consolidate Raft thread pools
..

consensus: consolidate Raft thread pools

This patch consolidates the two thread pools used in Raft consensus: the
observer and "Raft" (for lack of a better name) pools. The former runs tasks
for infrequent events such as term and config changes while the latter is
used for periodic events such as processing heartbeat responses.

Both per-replica pools are consolidated into a single per-server pool.
Tokens are used to ensure that tasks belonging to a single replica are
logically grouped together. One token is shared by RaftConsensus (primary)
and PeerManager (secondary); this mirrors the existing sharing behavior and
is safe because token operations are thread-safe. A second serial token is
dedicated for the observer submissions, so that its (potentially)
long-running tasks don't starve any periodic Raft events.

The non-default max_threads may come as a surprise, but David showed me how
tasks submitted to either pool may sometimes lead to an fsync (mostly via
cmeta flushes). As such, it isn't safe to use the default num_cpus upper
bound, as that may cause such IO-based tasks to block other CPU-only tasks
(or worse, periodic tasks like heartbeat response processing).

What per-replica threads are not addressed by this patch?
- Failure detection thread: a threadpool isn't the right model
  for these. Something like a hash wheel timer would be ideal.
- Prepare thread pool (max_threads=1): these could be consolidated too, but
  their metrics are per-replica, and tokens don't (yet) support that.
  Meaning, if they were consolidated now, the metrics would also consolidate
  and would be less useful as a result.

Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
---
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_peers.h
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/peer_manager.cc
M src/kudu/consensus/peer_manager.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/kserver/kserver.cc
M src/kudu/kserver/kserver.h
M src/kudu/master/sys_catalog.cc
M src/kudu/tablet/tablet_replica-test.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/tablet_copy_source_session-test.cc
M src/kudu/tserver/ts_tablet_manager.cc
19 files changed, 193 insertions(+), 123 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/46/6946/14
-- 
To view, visit http://gerrit.cloudera.org:8080/6946
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 14
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-26 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 13:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/6946/13/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

PS13, Line 191: raft_pool->NewToken(ThreadPool::ExecutionMode::SERIAL)
> it's misleading to name both tokens (the queue and the raft proper one) as 
Alright, I'll add "for_observers" or some such to this one.


http://gerrit.cloudera.org:8080/#/c/6946/13/src/kudu/consensus/raft_consensus.h
File src/kudu/consensus/raft_consensus.h:

PS13, Line 121: ConsensusOptions options,
  : std::unique_ptr cmeta,
  : const RaftPeerPB& local_peer_pb,
  : const scoped_refptr& metric_entity,
  : scoped_refptr time_manager,
  : ReplicaTransactionFactory* txn_factory,
  : const std::shared_ptr& messenger,
  : const scoped_refptr& log,
  : const std::shared_ptr& parent_mem_tracker,
  : const Callback& 
mark_dirty_clbk,
  : ThreadPool* raft_pool);
> heads up, this is going to conflict with mike's in flight patches
It's a race to the finish line!


http://gerrit.cloudera.org:8080/#/c/6946/13/src/kudu/kserver/kserver.cc
File src/kudu/kserver/kserver.cc:

PS13, Line 106:   if (raft_pool_) {
  : raft_pool_->Shutdown();
  :   }
> is ordering important here? I would think that shutting down the apply pool
I don't think it actually matters, since shutting down the messenger first 
ensures that any remaining RaftConsensus objects are destroyed.

But I'll reorder it to put raft_pool shutdown first just in case.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 13
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-22 Thread David Ribeiro Alves (Code Review)
David Ribeiro Alves has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 13:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/6946/13/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

PS13, Line 191: raft_pool->NewToken(ThreadPool::ExecutionMode::SERIAL)
it's misleading to name both tokens (the queue and the raft proper one) as 
"raft_pool_tokens" since it leads to thinking they are the same


http://gerrit.cloudera.org:8080/#/c/6946/13/src/kudu/consensus/raft_consensus.h
File src/kudu/consensus/raft_consensus.h:

PS13, Line 121: ConsensusOptions options,
  : std::unique_ptr cmeta,
  : const RaftPeerPB& local_peer_pb,
  : const scoped_refptr& metric_entity,
  : scoped_refptr time_manager,
  : ReplicaTransactionFactory* txn_factory,
  : const std::shared_ptr& messenger,
  : const scoped_refptr& log,
  : const std::shared_ptr& parent_mem_tracker,
  : const Callback& 
mark_dirty_clbk,
  : ThreadPool* raft_pool);
heads up, this is going to conflict with mike's in flight patches


http://gerrit.cloudera.org:8080/#/c/6946/13/src/kudu/kserver/kserver.cc
File src/kudu/kserver/kserver.cc:

PS13, Line 106:   if (raft_pool_) {
  : raft_pool_->Shutdown();
  :   }
is ordering important here? I would think that shutting down the apply pool 
first would be problematic since there might be stuff in flight to still be 
applied as we close the other pools first. am I missing something?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 13
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-20 Thread Adar Dembo (Code Review)
Hello David Ribeiro Alves, Todd Lipcon, Kudu Jenkins,

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

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

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

Change subject: consensus: consolidate Raft thread pools
..

consensus: consolidate Raft thread pools

This patch consolidates the two thread pools used in Raft consensus: the
observer and "Raft" (for lack of a better name) pools. The former runs tasks
for infrequent events such as term and config changes while the latter is
used for periodic events such as processing heartbeat responses.

Both per-replica pools are consolidated into a single per-server pool.
Tokens are used to ensure that tasks belonging to a single replica are
logically grouped together. One token is shared by RaftConsensus (primary)
and PeerManager (secondary); this mirrors the existing sharing behavior and
is safe because token operations are thread-safe. A second serial token is
dedicated for the observer submissions, so that its (potentially)
long-running tasks don't starve any periodic Raft events.

The non-default max_threads may come as a surprise, but David showed me how
tasks submitted to either pool may sometimes lead to an fsync (mostly via
cmeta flushes). As such, it isn't safe to use the default num_cpus upper
bound, as that may cause such IO-based tasks to block other CPU-only tasks
(or worse, periodic tasks like heartbeat response processing).

What per-replica threads are not addressed by this patch?
- Failure detection thread: a threadpool isn't the right model
  for these. Something like a hash wheel timer would be ideal.
- Prepare thread pool (max_threads=1): these could be consolidated too, but
  their metrics are per-replica, and tokens don't (yet) support that.
  Meaning, if they were consolidated now, the metrics would also consolidate
  and would be less useful as a result.

Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
---
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_peers.h
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/peer_manager.cc
M src/kudu/consensus/peer_manager.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/kserver/kserver.cc
M src/kudu/kserver/kserver.h
M src/kudu/master/sys_catalog.cc
M src/kudu/tablet/tablet_replica-test.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/tablet_copy_source_session-test.cc
M src/kudu/tserver/ts_tablet_manager.cc
19 files changed, 189 insertions(+), 123 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/46/6946/13
-- 
To view, visit http://gerrit.cloudera.org:8080/6946
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 13
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-20 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 12:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/6946/12/src/kudu/consensus/consensus_peers-test.cc
File src/kudu/consensus/consensus_peers-test.cc:

PS12, Line 57: CONCURRENT
> shouldn't this be SERIAL since it's replacing a pool with max_threads=1?
Nope, the "Raft pool" (as opposed to the observer pool) was never actually 
capped at one thread. And this token is acting as a stand-in for the Raft pool; 
it's fed into calls to Peer::NewRemotePeer().


http://gerrit.cloudera.org:8080/#/c/6946/12/src/kudu/server/server_base.h
File src/kudu/server/server_base.h:

PS12, Line 181:   // Thread pool for Raft-related operations, shared between 
all tablets.
  :   gscoped_ptr raft_pool_;
> not a huge fan of this "kudu-specific" stuff in server/ -- the hope was to 
I weighed the options and decided to add to the server hierarchy. Patch 
forthcoming.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 12
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-12 Thread Adar Dembo (Code Review)
Hello David Ribeiro Alves, Todd Lipcon, Kudu Jenkins,

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

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

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

Change subject: consensus: consolidate Raft thread pools
..

consensus: consolidate Raft thread pools

This patch consolidates the two thread pools used in Raft consensus: the
observer and "Raft" (for lack of a better name) pools. The former runs tasks
for infrequent events such as term and config changes while the latter is
used for periodic events such as processing heartbeat responses.

Both per-replica pools are consolidated into a single per-server pool.
Tokens are used to ensure that tasks belonging to a single replica are
logically grouped together. One token is shared by RaftConsensus (primary)
and PeerManager (secondary); this mirrors the existing sharing behavior and
is safe because token operations are thread-safe. A second serial token is
dedicated for the observer submissions, so that its (potentially)
long-running tasks don't starve any periodic Raft events.

The non-default max_threads may come as a surprise, but David showed me how
tasks submitted to either pool may sometimes lead to an fsync (mostly via
cmeta flushes). As such, it isn't safe to use the default num_cpus upper
bound, as that may cause such IO-based tasks to block other CPU-only tasks
(or worse, periodic tasks like heartbeat response processing).

What per-replica threads are not addressed by this patch?
- Failure detection thread: a threadpool isn't the right model
  for these. Something like a hash wheel timer would be ideal.
- Prepare thread pool (max_threads=1): these could be consolidated too, but
  their metrics are per-replica, and tokens don't (yet) support that.
  Meaning, if they were consolidated now, the metrics would also consolidate
  and would be less useful as a result.

Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
---
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_peers.h
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/peer_manager.cc
M src/kudu/consensus/peer_manager.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/master/sys_catalog.cc
M src/kudu/server/server_base.cc
M src/kudu/server/server_base.h
M src/kudu/tablet/tablet_replica-test.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/tablet_copy_source_session-test.cc
M src/kudu/tserver/ts_tablet_manager.cc
19 files changed, 192 insertions(+), 123 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/46/6946/11
-- 
To view, visit http://gerrit.cloudera.org:8080/6946
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 11
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-11 Thread Adar Dembo (Code Review)
Hello Kudu Jenkins,

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

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

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

Change subject: consensus: consolidate Raft thread pools
..

consensus: consolidate Raft thread pools

This patch consolidates the two thread pools used in Raft consensus: the
observer and "Raft" (for lack of a better name) pools. The former runs tasks
for infrequent events such as term and config changes while the latter is
used for periodic events such as processing heartbeat responses.

Both per-replica pools are consolidated into a single per-server pool.
Tokens are used to ensure that tasks belonging to a single replica are
logically grouped together. One token is shared by RaftConsensus (primary)
and PeerManager (secondary); this mirrors the existing sharing behavior and
is safe because token operations are thread-safe. A second serial token is
dedicated for the observer submissions, so that its (potentially)
long-running tasks don't starve any periodic Raft events.

The non-default max_threads may come as a surprise, but David showed me how
tasks submitted to either pool may sometimes lead to an fsync (mostly via
cmeta flushes). As such, it isn't safe to use the default num_cpus upper
bound, as that may cause such IO-based tasks to block other CPU-only tasks
(or worse, periodic tasks like heartbeat response processing).

What per-replica threads are not addressed by this patch?
- Failure detection thread: a threadpool isn't the right model
  for these. Something like a hash wheel timer would be ideal.
- Prepare thread pool (max_threads=1): these could be consolidated too, but
  their metrics are per-replica, and tokens don't (yet) support that.
  Meaning, if they were consolidated now, the metrics would also consolidate
  and would be less useful as a result.

Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
---
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_peers.h
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/peer_manager.cc
M src/kudu/consensus/peer_manager.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/master/sys_catalog.cc
M src/kudu/server/server_base.cc
M src/kudu/server/server_base.h
M src/kudu/tablet/tablet_replica-test.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/tablet_copy_source_session-test.cc
M src/kudu/tserver/ts_tablet_manager.cc
19 files changed, 192 insertions(+), 123 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/46/6946/9
-- 
To view, visit http://gerrit.cloudera.org:8080/6946
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 9
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-11 Thread Adar Dembo (Code Review)
Hello Kudu Jenkins,

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

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

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

Change subject: consensus: consolidate Raft thread pools
..

consensus: consolidate Raft thread pools

This patch consolidates the two thread pools used in Raft consensus: the
observer and "Raft" (for lack of a better name) pools. The former runs tasks
for infrequent events such as term and config changes while the latter is
used for periodic events such as processing heartbeat responses.

Both per-replica pools are consolidated into a single per-server pool.
Tokens are used to ensure that tasks belonging to a single replica are
logically grouped together. One token is shared by RaftConsensus (primary)
and PeerManager (secondary); this mirrors the existing sharing behavior and
is safe because token operations are thread-safe. A second serial token is
dedicated for the observer submissions, so that its (potentially)
long-running tasks don't starve any periodic Raft events.

The non-default max_threads may come as a surprise, but David showed me how
tasks submitted to either pool may sometimes lead to an fsync (mostly via
cmeta flushes). As such, it isn't safe to use the default num_cpus upper
bound, as that may cause such IO-based tasks to block other CPU-only tasks
(or worse, periodic tasks like heartbeat response processing).

What per-replica threads are not addressed by this patch?
- Failure detection thread: a threadpool isn't the right model
  for these. Something like a hash wheel timer would be ideal.
- Prepare thread pool (max_threads=1): these could be consolidated too, but
  their metrics are per-replica, and tokens don't (yet) support that.
  Meaning, if they were consolidated now, the metrics would also consolidate
  and would be less useful as a result.

Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
---
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_peers.h
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/peer_manager.cc
M src/kudu/consensus/peer_manager.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/master/sys_catalog.cc
M src/kudu/server/server_base.cc
M src/kudu/server/server_base.h
M src/kudu/tablet/tablet_replica-test.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/tablet_copy_source_session-test.cc
M src/kudu/tserver/ts_tablet_manager.cc
19 files changed, 187 insertions(+), 119 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/46/6946/8
-- 
To view, visit http://gerrit.cloudera.org:8080/6946
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 8
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-06 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 7:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/consensus_peers.cc
File src/kudu/consensus/consensus_peers.cc:

PS7, Line 133:   shared_ptr locked_token = 
raft_pool_token_.lock();
 :   if (!locked_token) {
 : return Status::ServiceUnavailable("raft thread pool has been 
shutdown");
 :   }
> I have mixed opinions on this.
We discussed this further over Slack. I'll try to summarize the main points:
1. It's true that, as far as the Kudu codebase is concerned, shared_ptr 
connotes shared ownership. Meaning, even if there was only one use of 
shared_ptr and the rest were weak_ptr, it wouldn't be obvious that this is an 
attempt at "single ownership + dependent sharing".
2. There is one clear-cut advantage to shared_ptr+weak_ptr: you can have more 
control if there's a bug in the lifecycle. For example, a dependent can CHECK() 
that lock() succeeded and reliably crash if not (if the dependent were using a 
raw pointer, dereferencing it could yield one of several outcomes). Without 
weak_ptr you have to rely instead on ASAN coverage of the affected code.
3. The use of shared_ptr+weak_ptr can hide real lifecycle issues. For example, 
suppose there was no synchronization between deleting and accessing the 
pointee. If using a raw pointer, TSAN will flag the access as a data race. If 
using a weak_ptr, the access is safe, TSAN doesn't flag anything, and the 
underlying lifecycle issue is never surfaced.

Given all this, I'm going to revert this back to unique_ptr for the owner 
(RaftConsensus) and raw pointers for dependents (Peers).


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-05 Thread Todd Lipcon (Code Review)
Todd Lipcon has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 7:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/consensus_peers.cc
File src/kudu/consensus/consensus_peers.cc:

PS7, Line 133:   shared_ptr locked_token = 
raft_pool_token_.lock();
 :   if (!locked_token) {
 : return Status::ServiceUnavailable("raft thread pool has been 
shutdown");
 :   }
> +1
I have mixed opinions on this.

On the one hand, I see your point that it's safer. However, it's sort of like 
saying "shared ptrs are safer than unique_ptrs, so we should use them instead."

I think in both cases it can be indicative of unclear lifetimes. If you use 
shared_ptr it's a strong signal that you don't know who will destruct something 
or who might retain a reference to it. A shared_ptr plus weak_ptr invites the 
question every time you access the weak_ptr: "is there any guarantee that this 
is still alive?" If you are 100% sure that it is still alive, then using a raw 
pointer illustrates that fact, whereas a weak_ptr says "I'm not really sure". 
There are certainly some cases where "not really sure" is accurate, but I would 
rather that be an explicit choice due to complex lifetime issues rather than a 
default which we take due to lack of analysis (aka laziness)


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-05 Thread Dan Burkert (Code Review)
Dan Burkert has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 7:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/consensus_peers.cc
File src/kudu/consensus/consensus_peers.cc:

PS7, Line 133:   shared_ptr locked_token = 
raft_pool_token_.lock();
 :   if (!locked_token) {
 : return Status::ServiceUnavailable("raft thread pool has been 
shutdown");
 :   }
> Yes, this token is shared between RaftConsensus and the peers, and this is 
+1


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-05 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 7:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

PS7, Line 197: shared_ptr 
raft_pool_token(std::make_shared(
 :   raft_pool->AllocateTokenSlot()));
> We need the token shutdown logic to be able to clear the queue's observers,
Quick correction: the observers are on their own dedicated token (see L188). 
That use case _does_ need Shutdown, as you described.

But, I don't think the RaftConsensus token needs Shutdown(), because it's got 
that funky lifecycle wherein it binds (and refs) itself into each submitted 
task. Thus, I don't believe either ThreadPool::Shutdown() (before my patch) or 
SequenceToken::Shutdown() (after my patch) are needed in 
RaftConsensus::Shutdown(). At best they may prevent tasks from being submitted 
after RaftConsensus has shut down, but eventually no more tasks will be 
submitted, and the last task will destroy RaftConsensus when it finishes 
running.

Having said all that, I gave it some more thought and option 1 doesn't actually 
solve the problem. Yes, it does away with the call to 
SequenceToken::Shutdown(), but both RaftConsensus and Peer tasks are still 
being submitted through one token and thus being serialized.

I'll investigate the feasibility of "non-serialized" tokens and report back 
with my findings.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-05 Thread David Ribeiro Alves (Code Review)
David Ribeiro Alves has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 7:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

PS7, Line 197: shared_ptr 
raft_pool_token(std::make_shared(
 :   raft_pool->AllocateTokenSlot()));
> Right, the use of a token here is so that we can call SequenceToken::Shutdo
We need the token shutdown logic to be able to clear the queue's observers, 
that much we had discovered when we last discussed this. Not sure whether the 
final Wait() that you had on consensus shutdown was safe.

For raft itself and the peers if feels forced to have to choose total 
serialization to be able to shutdown the tasks. In this case it's not too bad, 
raft actually serializes most calls under a single lock, as does the queue 
meaning there could be at most two things happening at the same time, at least 
for the "real work" paths. However I would expect this not to be the common 
case, so Ideally, I think your generalization of the token logic to multiple 
threads. I.e. basically have a "context" within a TP that you can shutdown and 
dictate how many threads it can have and grow to within a larger TP from which 
you pull threads for efficient would be preferable and the most powerful 
solution. If you feel like the TP saga has been going long enough another, 
likely easier, thing we could do it to just have two sequence tokens, one for 
the peers and one for raft.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-05 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 7:

(4 comments)

I filed KUDU-2029 for the unrelated build failure.

http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/consensus_peers.cc
File src/kudu/consensus/consensus_peers.cc:

PS7, Line 133:   shared_ptr locked_token = 
raft_pool_token_.lock();
 :   if (!locked_token) {
 : return Status::ServiceUnavailable("raft thread pool has been 
shutdown");
 :   }
> not sure I'm fully understanding this ownership model. Raft owns the token,
Yes, this token is shared between RaftConsensus and the peers, and this is the 
token used by RaftConsensus itself. However, RaftConsensus is the owner: the 
token should be destroyed when RaftConsensus is destroyed. Using weak_ptr here 
instead of a shared_ptr enforces that.

Given the RaftConsensus lifecycle I could have shared the token as a raw 
pointer, but I think "shared_ptr for the owner, weak_ptr for dependents" better 
describes the relationship, and it's safer too.

In general I think we should use this pattern instead of the "unique_ptr for 
the owner, raw pointers for dependents" style that's more prevalent in Kudu 
today. The status quo is from a time before we had access to C++11's weak 
pointers, and it's less expressive and less safe. The only exception should be 
if creating/destroying dependents is a hot path and we don't want to incur the 
overhead of weak ref count add/dec.


http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/consensus_queue-test.cc
File src/kudu/consensus/consensus_queue-test.cc:

PS7, Line 20: #include 
> not your fault, but mind swapping this include with the one below?
Done


PS7, Line 45: using std::shared_ptr;
> remove?
Done


http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

PS7, Line 197: shared_ptr 
raft_pool_token(std::make_shared(
 :   raft_pool->AllocateTokenSlot()));
> hum, before this wasn't a single threaded TP and you're making it one, any 
Right, the use of a token here is so that we can call SequenceToken::Shutdown() 
during RaftConsensus::Shutdown() and thus ensure that there are no outstanding 
tasks when RaftConsensus is finished shutting down. Those are the semantics we 
had when we used a dedicated threadpool.

If you think the serialization aspect of the token is bad, I can think of a 
couple options:
1. Remove the call to SequenceToken::Shutdown() from RaftConsensus::Shutdown() 
and restore the "last task can destroy its own token" logic that I removed 
(didn't need it if we shutdown tokens).
2. Expand the token logic to allow for the allocation of "non-serialized" 
tokens. They would still allow for the logical grouping of tasks (and, as a 
result, token shutdown), but their tasks could be processed in parallel by 
worker threads. I think this can be done but I haven't really thought through 
the complexity.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-05 Thread David Ribeiro Alves (Code Review)
David Ribeiro Alves has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 7:

(4 comments)

http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/consensus_peers.cc
File src/kudu/consensus/consensus_peers.cc:

PS7, Line 133:   shared_ptr locked_token = 
raft_pool_token_.lock();
 :   if (!locked_token) {
 : return Status::ServiceUnavailable("raft thread pool has been 
shutdown");
 :   }
not sure I'm fully understanding this ownership model. Raft owns the token, 
right? , which isn't destroyed but on RaftConsensus dctor, at which point the 
peers should no longer exist. Thus it should be safe to pass the token by 
pointer or reference. Did you find any cases where that wasn't safe?

EDIT: Oh I see, this is not the same token as in RaftConsensus. This token is 
shared between the queue and the peers.


http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/consensus_queue-test.cc
File src/kudu/consensus/consensus_queue-test.cc:

PS7, Line 20: #include 
not your fault, but mind swapping this include with the one below?


PS7, Line 45: using std::shared_ptr;
remove?


http://gerrit.cloudera.org:8080/#/c/6946/7/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

PS7, Line 197: shared_ptr 
raft_pool_token(std::make_shared(
 :   raft_pool->AllocateTokenSlot()));
hum, before this wasn't a single threaded TP and you're making it one, any 
reason for that? In particular, since this is shared between the peers and raft 
consensus there could be simultaneous things happening and now that isn't 
possible.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-06-03 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 6:

(7 comments)

http://gerrit.cloudera.org:8080/#/c/6946/6//COMMIT_MSG
Commit Message:

PS6, Line 8: 
   : This patch consolidates the two thread pools used in Raft 
consensus: the
   : observer and "Raft" (for lack of a better name) pools. The former 
runs tasks
   : for infrequent events such as term and config changes while the 
latter is
   : used for periodic events such as processing heartbeat responses.
   : 
   : In this patch, each per-replica pool is consolidated into a single
   : per-server pool. Sequence tokens are used to ensure that tasks 
belonging to
   : a single replica are executed serially and in order. For the 
"Raft" pool, a
   : single sequence token is shared by the PeerManager and 
RaftConsensus; this
   : mirrors the existing sharing behavior and is safe because token 
operations
   : are thread-safe.
   : 
   : The non-default max_threads may come as a surprise, but David 
showed me how
   : tasks submitted to either pool may sometimes lead to an fsync 
(mostly via
   : cmeta flushes). As such, it isn't safe to use the default num_cpus 
upper
   : bound, as that may cause such IO-based tasks to block other 
CPU-only tasks
   : (or worse, periodic tasks like heartbeat response processing).
   : 
   : What per-replica threads are not addressed by this patch?
   : - Failure detection thread: a threadpool isn't the right model
   :   for these. Something like a hash wheel timer would be ideal.
   : - Prepare thread pool (max_threads=1): these could be consolidated 
too, but
   :   their metrics are per-replica, and sequence tokens don't (yet) 
support
   :   that. Meaning, if they were consolidated now, the metrics would 
also
   :   consolidate and would be less useful as a result.
> would be good to have a few runs of raft_consensus-itest on dist-test with 
I made a bunch of changes to the threadpool slot implementation and to this 
consolidation patch. I ran raft_consensus-itest on dist-test in DEBUG and slow 
mode.

Out of 1000 runs, only 1 failed:
  F0603 02:23:28.156342 15190 test_workload.cc:220] Check failed: row_count >= 
expected_row_count (7382 vs. 7384)

The full log is available here: https://filebin.ca/3OloKjkUOdaK


http://gerrit.cloudera.org:8080/#/c/6946/6/src/kudu/consensus/consensus_queue.cc
File src/kudu/consensus/consensus_queue.cc:

PS6, Line 114: ThreadPool* raft_observers_pool
> noticed that, for consensus peers, you're passing the token whereas here yo
Done


PS6, Line 917: raft_observers_pool_token_.Wait()
> shutdown on a tp prevents more ops from being submitted while this doesn't,
Yes, this was the race we discussed last week.

I've since added a proper token shutdown routine and used it here.


http://gerrit.cloudera.org:8080/#/c/6946/2/src/kudu/consensus/consensus_queue.h
File src/kudu/consensus/consensus_queue.h:

PS2, Line 434:   // The pool token which executes observe
> remove? also maybe add some info about the experiments you made where you f
I've removed the TODO.

I don't think it's safe to conclude that running notifications in parallel is 
unsafe, at least not based on my dist-test runs. All of the failures were row 
count related, and I the runs took place before you merged your fix for the 
semi-obscure bug that was producing bad row counts.


http://gerrit.cloudera.org:8080/#/c/6946/6/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

PS6, Line 192: raft_pool
> on the peer manager side you call this request_* something. mind using the 
Done


PS6, Line 1743: raft_pool_token_->Wait();
> again, theres a change in semantics from "close the tp and wait for the ong
As we discussed last week, this particular semantic change wasn't dangerous 
because the "last task in a slot may destroy its token" exception I added to 
ThreadPool ensured that when RaftConsensus was destroyed, it was via its last 
task. RaftConsensus could still submit useless tasks to the token after 
Shutdown(), but they'd no-op and eventually lead to the object's destruction.

In any case this is now moot because I added proper token shutdown behavior, so 
token-based submission should now have identical semantics.


http://gerrit.cloudera.org:8080/#/c/6946/6/src/kudu/server/server_base.cc
File src/kudu/server/server_base.cc:

PS6, Line 255: queue-observers-pool
> use "raft_observers_pool" or something
Now moot: I further consolidated the two pools into one.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54

[kudu-CR] consensus: consolidate Raft thread pools

2017-05-26 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 6:

David and I looked at an interesting TSAN failure together 
(https://gist.github.com/adembo/dc2122e99468523bc8dafec8ad5a62d8) and I wanted 
to publish our findings.

The change in semantics to PeerMessageQueue::Close() is unsafe. Previously, the 
observers pool was exclusively owned by RaftConsensus, and shutting it down 
before closing the queue ensured that any threads handling peer responses 
concurrently would be unable to submit tasks to the observers pool. Replacing 
ThreadPool::Shutdown() with SequenceToken::Wait() does not prevent future 
submissions to the pool, which is a problem because submitted tasks do not take 
refs on PeerMessageQueue or RaftConsensus (nor should they; after all, 
PeerMessageQueue is designed to be outlived by RaftConsensus), and so by the 
time they run, RaftConsensus may have been destroyed.

David and I considered various ways to solve this problem from within 
PeerMessageQueue itself, but they all seemed complex. One approach is to 
reacquire the queue's lock before submitting to the token, checking if the 
queue has been closed first. Unfortunately this would require holding the queue 
spinlock while grabbing the pool's mutex, which doesn't seem like a good idea.

Instead, I think I'm going to bite the bullet and provide a 
SequenceToken::Shutdown() method that would atomically stop future submissions 
on the token and wait for current submissions to finish. We worked out a rough 
way to implement this using an additional SequenceSlot state, without the need 
for an extra lock.

I'll give it a shot next week.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: No


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-26 Thread David Ribeiro Alves (Code Review)
David Ribeiro Alves has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 6:

(7 comments)

http://gerrit.cloudera.org:8080/#/c/6946/6//COMMIT_MSG
Commit Message:

PS6, Line 8: 
   : This patch consolidates the two thread pools used in Raft 
consensus: the
   : observer and "Raft" (for lack of a better name) pools. The former 
runs tasks
   : for infrequent events such as term and config changes while the 
latter is
   : used for periodic events such as processing heartbeat responses.
   : 
   : In this patch, each per-replica pool is consolidated into a single
   : per-server pool. Sequence tokens are used to ensure that tasks 
belonging to
   : a single replica are executed serially and in order. For the 
"Raft" pool, a
   : single sequence token is shared by the PeerManager and 
RaftConsensus; this
   : mirrors the existing sharing behavior and is safe because token 
operations
   : are thread-safe.
   : 
   : The non-default max_threads may come as a surprise, but David 
showed me how
   : tasks submitted to either pool may sometimes lead to an fsync 
(mostly via
   : cmeta flushes). As such, it isn't safe to use the default num_cpus 
upper
   : bound, as that may cause such IO-based tasks to block other 
CPU-only tasks
   : (or worse, periodic tasks like heartbeat response processing).
   : 
   : What per-replica threads are not addressed by this patch?
   : - Failure detection thread: a threadpool isn't the right model
   :   for these. Something like a hash wheel timer would be ideal.
   : - Prepare thread pool (max_threads=1): these could be consolidated 
too, but
   :   their metrics are per-replica, and sequence tokens don't (yet) 
support
   :   that. Meaning, if they were consolidated now, the metrics would 
also
   :   consolidate and would be less useful as a result.
would be good to have a few runs of raft_consensus-itest on dist-test with this 
change. iirc you did that at some point, mind re-running those and posting the 
results here? particularly the *crashynodes* and *churnyelections* tests


http://gerrit.cloudera.org:8080/#/c/6946/6/src/kudu/consensus/consensus_queue.cc
File src/kudu/consensus/consensus_queue.cc:

PS6, Line 114: ThreadPool* raft_observers_pool
noticed that, for consensus peers, you're passing the token whereas here you're 
passing the threadpool? any particular reason for that? if not ming making that 
consistent?


PS6, Line 917: raft_observers_pool_token_.Wait()
shutdown on a tp prevents more ops from being submitted while this doesn't, is 
that a problem?


http://gerrit.cloudera.org:8080/#/c/6946/2/src/kudu/consensus/consensus_queue.h
File src/kudu/consensus/consensus_queue.h:

PS2, Line 434:   // The pool token which executes observe
remove? also maybe add some info about the experiments you made where you found 
out it wasn't safe to run these in parallel (you did those iirc, right?)


http://gerrit.cloudera.org:8080/#/c/6946/6/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

PS6, Line 192: raft_pool
on the peer manager side you call this request_* something. mind using the same 
name as the pool (like raft_pool_token) so that it's easier to track where it 
runs?


PS6, Line 1743: raft_pool_token_->Wait();
again, theres a change in semantics from "close the tp and wait for the ongoing 
stuff to happen" to "wait for the ongoing stuff to happen even if more is 
added" seems like we should have the same semantics in the token slots?


http://gerrit.cloudera.org:8080/#/c/6946/6/src/kudu/server/server_base.cc
File src/kudu/server/server_base.cc:

PS6, Line 255: queue-observers-pool
use "raft_observers_pool" or something


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-25 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 5:

PS5 yielded the following crash: 
https://gist.github.com/adembo/bb30754825ec0bf10cba783d07f3ac7d

Looks like the last deref to the token shared between RaftConsensus and Peer 
happened _after_ RaftConsensus was destroyed, so when the token was destructed, 
its pool was already gone.

Is this a sign that shared ownership of the token is a bad idea, and instead 
RaftConsensus should be the sole owner and share out its token via raw pointer 
to Peer? Does that make sense?

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: No


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-24 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 4:

I ran raft_consensus-itest in slow mode 1000 times. All of them passed.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: No


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-24 Thread Adar Dembo (Code Review)
Hello Dan Burkert, David Ribeiro Alves, Todd Lipcon, Kudu Jenkins,

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

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

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

Change subject: consensus: consolidate Raft thread pools
..

consensus: consolidate Raft thread pools

This patch consolidates the two thread pools used in Raft consensus: the
observer and "Raft" (for lack of a better name) pools. The former runs tasks
for infrequent events such as term and config changes while the latter is
used for periodic events such as processing heartbeat responses.

In this patch, each per-replica pool is consolidated into a single
per-server pool. Sequence tokens are used to ensure that tasks belonging to
a single replica are executed serially and in order. For the "Raft" pool, a
single sequence token is shared by the PeerManager and RaftConsensus; this
mirrors the existing sharing behavior and is safe because token operations
are thread-safe.

The non-default max_threads may come as a surprise, but David showed me how
tasks submitted to either pool may sometimes lead to an fsync (mostly via
cmeta flushes). As such, it isn't safe to use the default num_cpus upper
bound, as that may cause such IO-based tasks to block other CPU-only tasks
(or worse, periodic tasks like heartbeat response processing).

What per-replica threads are not addressed by this patch?
- Failure detection thread: a threadpool isn't the right model
  for these. Something like a hash wheel timer would be ideal.
- Prepare thread pool (max_threads=1): these could be consolidated too, but
  their metrics are per-replica, and sequence tokens don't (yet) support
  that. Meaning, if they were consolidated now, the metrics would also
  consolidate and would be less useful as a result.

Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
---
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_peers.h
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/peer_manager.cc
M src/kudu/consensus/peer_manager.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/master/sys_catalog.cc
M src/kudu/server/server_base.cc
M src/kudu/server/server_base.h
M src/kudu/tablet/tablet_replica-test.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/tablet_copy_source_session-test.cc
M src/kudu/tserver/ts_tablet_manager.cc
19 files changed, 189 insertions(+), 117 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-24 Thread Adar Dembo (Code Review)
Hello Dan Burkert, David Ribeiro Alves, Todd Lipcon, Kudu Jenkins,

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

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

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

Change subject: consensus: consolidate Raft thread pools
..

consensus: consolidate Raft thread pools

This patch consolidates the two thread pools used in Raft consensus: the
observer and "Raft" (for lack of a better name) pools. The former runs tasks
for infrequent events such as term and config changes while the latter is
used for periodic events such as processing heartbeat responses.

In this patch, each per-replica pool is consolidated into a single
per-server pool. Sequence tokens are used to ensure that tasks belonging to
a single replica are executed serially and in order. For the "Raft" pool, a
single sequence token is shared by the PeerManager and RaftConsensus; this
mirrors the existing sharing behavior and is safe because token operations
are thread-safe.

The non-default max_threads may come as a surprise, but David showed me how
tasks submitted to either pool may sometimes lead to an fsync (mostly via
cmeta flushes). As such, it isn't safe to use the default num_cpus upper
bound, as that may cause such IO-based tasks to block other CPU-only tasks
(or worse, periodic tasks like heartbeat response processing).

What per-replica threads are not addressed by this patch?
- Failure detection thread: a threadpool isn't the right model
  for these. Something like a hash wheel timer would be ideal.
- Prepare thread pool (max_threads=1): these could be consolidated too, but
  their metrics are per-replica, and sequence tokens don't (yet) support
  that. Meaning, if they were consolidated now, the metrics would also
  consolidate and would be less useful as a result.

Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
---
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_peers.h
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/peer_manager.cc
M src/kudu/consensus/peer_manager.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/master/sys_catalog.cc
M src/kudu/server/server_base.cc
M src/kudu/server/server_base.h
M src/kudu/tablet/tablet_replica-test.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/tablet_copy_source_session-test.cc
M src/kudu/tserver/ts_tablet_manager.cc
19 files changed, 189 insertions(+), 117 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-24 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 2:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/6946/2/src/kudu/consensus/consensus_peers.h
File src/kudu/consensus/consensus_peers.h:

PS2, Line 130: )
> extra parenthesis?
Done


http://gerrit.cloudera.org:8080/#/c/6946/2/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

Line 191:   shared_ptr token(new 
SequenceToken(raft_pool->AllocateTokenSlot()));
> I think this should be
Done


http://gerrit.cloudera.org:8080/#/c/6946/2/src/kudu/consensus/raft_consensus_quorum-test.cc
File src/kudu/consensus/raft_consensus_quorum-test.cc:

Line 168:   new SequenceToken(raft_pool_->AllocateTokenSlot()));
> likewise
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-23 Thread Dan Burkert (Code Review)
Dan Burkert has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 2:

I was curious so I tried it; pretty good result: https://imgur.com/a/vBr9d

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: No


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-23 Thread Dan Burkert (Code Review)
Dan Burkert has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 2:

(3 comments)

Nice.  Would be great to load this on to the flash cluster and see what the 
resulting difference in threads is for an otherwise idle cluster.  Each tserver 
on that cluster has ~5k tablets, so the savings could be substantial.

http://gerrit.cloudera.org:8080/#/c/6946/2/src/kudu/consensus/consensus_peers.h
File src/kudu/consensus/consensus_peers.h:

PS2, Line 130: )
extra parenthesis?


http://gerrit.cloudera.org:8080/#/c/6946/2/src/kudu/consensus/raft_consensus.cc
File src/kudu/consensus/raft_consensus.cc:

Line 191:   shared_ptr token(new 
SequenceToken(raft_pool->AllocateTokenSlot()));
I think this should be

shared_ptr token = make_shared(raft_pool->AllocateTokenSlot());


http://gerrit.cloudera.org:8080/#/c/6946/2/src/kudu/consensus/raft_consensus_quorum-test.cc
File src/kudu/consensus/raft_consensus_quorum-test.cc:

Line 168:   new SequenceToken(raft_pool_->AllocateTokenSlot()));
likewise


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-22 Thread Adar Dembo (Code Review)
Adar Dembo has posted comments on this change.

Change subject: consensus: consolidate Raft thread pools
..


Patch Set 1:

I'm having trouble reconciling the SequenceToken lifecycle requirements with 
the complicated RaftConsensus lifecycle.

The issue is that while RaftConsensus::ShutDown() is called in a normal context 
(typically a service pool thread handling the DeleteTablet() RPC), the  object 
itself may be destroyed later, by its thread pool's worker thread. This happens 
because the RaftConsensus object is bound and retained by several callbacks 
that run on the thread pool. When this happens, the process crashes because 
~RaftConsensus tries to release its SequenceToken, but the token's slot is 
still active because it's running this destructor.

I'm not sure how to fix this because I still don't fully understand 
RaftConsensus's locking. Is it sufficient for RaftConsensus::Shutdown() to 
destroy the SequenceToken and for the various callbacks to check if the token 
is live before submitting to it? Or does that require additional 
synchronization?

Here's a sample failure: 
https://gist.github.com/adembo/9a3c8e91807ba41bb4aceef67ec1dc0a with some 
additional logging. Note how thread 12205 initiates the crash, and the last 
thing it logged was "Destroying runnable" but not "Runnable destroyed"; this is 
a sign that the crash was in the worker thread's runnable release code.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: No


[kudu-CR] consensus: consolidate Raft thread pools

2017-05-22 Thread Adar Dembo (Code Review)
Hello Dan Burkert, David Ribeiro Alves, Todd Lipcon,

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

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

to review the following change.

Change subject: consensus: consolidate Raft thread pools
..

consensus: consolidate Raft thread pools

This patch consolidates the two thread pools used in Raft consensus: the
observer and "Raft" (for lack of a better name) pools. The former runs tasks
for infrequent events such as term and config changes while the latter is
used for periodic events such as processing heartbeat responses.

In this patch, each per-replica pool is consolidated into a single
per-server pool. Sequence tokens are used to ensure that tasks belonging to
a single replica are executed serially and in order. For the "Raft" pool, a
single sequence token is shared by the PeerManager and RaftConsensus; this
mirrors the existing sharing behavior and is safe because token operations
are thread-safe.

The non-default max_threads may come as a surprise, but David showed me how
tasks submitted to either pool may sometimes lead to an fsync (mostly via
cmeta flushes). As such, it isn't safe to use the default num_cpus upper
bound, as that may cause such IO-based tasks to block other CPU-only tasks
(or worse, periodic tasks like heartbeat response processing).

What per-replica threads are not addressed by this patch?
- Failure detection thread: a threadpool isn't the right model
  for these. Something like a hash wheel timer would be ideal.
- Prepare thread pool (max_threads=1): these could be consolidated too, but
  their metrics are per-replica, and sequence tokens don't (yet) support
  that. Meaning, if they were consolidated now, the metrics would also
  consolidate and would be less useful as a result.

Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
---
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_peers.h
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/peer_manager.cc
M src/kudu/consensus/peer_manager.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/master/sys_catalog.cc
M src/kudu/server/server_base.cc
M src/kudu/server/server_base.h
M src/kudu/tablet/tablet_replica-test.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/tablet_copy_source_session-test.cc
M src/kudu/tserver/ts_tablet_manager.cc
19 files changed, 188 insertions(+), 117 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I8958c607d11ac2e94908670c985059bef0ff5f54
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Todd Lipcon