[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-24 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica unsafe_change_config'.

This tool lets us replace a N-replica config on a tablet server with a
new config containing N or less replicas. This is particularly useful
when we have majority of the replicas down and for some reason we are not
able to bring the tablet back online using other recovery tools like
'kudu remote_replica copy'. We can use this tool to force a new config on the
surviving replica providing all the replica uuids of the new config from
the cli tool. As a result of the forced config change, the automatic leader
election kicks in via raft mechanisms and the re-replication is triggered
from master (if needed due to under-replicated tablet) to bring the replica
count of the tablet back upto N.

How does the tool bring tablet back online with new config:
a) The tool acts as a 'fake' leader and generates the consensus update with
   a bumped up term along with the new config. The surviving node (leader or
   follower) accepts the request and replicates the request and goes through
   a pre-election phase in which a leader is elected among the nodes provided
   in the config. If the new config provides enough VOTERs to win an election,
   the leader election succeeds and the new config will be committed.
   Master can eventually recognize this consensus state change and make sure
   tablet is re-replicated back to healthy count if it finds the tablet
   under-replicated.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   so master will very likely choose the new healthy live servers for
   re-replication if needed. If the dead nodes come back after master is
   updated with new unsafely forced config, master will delete the replicas
   on those dead nodes via DeleteTablet RPC because they are no longer part
   of the tablet config.

Also, the UnsafeChangeConfig() API adds a flag to append another change_config
op while there is one pending config in the log. This flag lifts the safety
net around pending configs which states that there can be only one pending
config at the max for a given replica.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect. The future revs of this patch should weaken this assumption and build
more safety guarantees around situations dead nodes coming back during the
unsafe change config operations on the cluster.

Tests associated with this patch:
- Unsafe config change when there is one follower survivor in the cluster.
- Unsafe config change when there is one leader survivor in the cluster.
- Unsafe config change when the unsafe config contains 2 replicas.
- Unsafe config change on a 5-replica config with 2 replicas in the new config.
- Unsafe config change when there is a pending config on the surviving leader.
- Unsafe config change when there is a pending config on a surviving follower.
- Unsafe config change when there are back to back pending configs on WAL,
  and verify that tablet bootstraps fine.
- Test back to back unsafe config changes when there are multiple pending
  configs present with the replica and the one with 'sane' new config will
  bring the tablet back to online state.

TODO:
1) Test exercising all the error cases in the UnsafeChangeConfig API.
2) Test the UnsafeChangeConfig RPC directly without going via external tool.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_meta.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/consensus/time_manager.h
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
17 files changed, 1,216 insertions(+), 111 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/66/6066/18
-- 
To view, visit http://gerrit.cloudera.org:8080/6066
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700

[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-24 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Patch Set 16:

(9 comments)

TFTR Mike again, updated now.

http://gerrit.cloudera.org:8080/#/c/6066/17//COMMIT_MSG
Commit Message:

PS17, Line 24: he s
> "fake"
Done


PS17, Line 40: 
> UnsafeChangeConfig
Done


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

PS17, Line 778: .
> Nit: please also add a space after this period (and all periods)
Done


PS17, Line 780: ,
> please add a space after this comma (and all commas)
Done


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

PS16, Line 1635: tablet $1
> Use SecureShortDebugString() from pb_util.h
Done


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

Line 1597: }
> Use SecureShortDebugString() from pb_util.h:
Done


PS17, Line 1694: 
> SecureShortDebugString(committed_config)
Done


PS17, Line 1695: 
> SecureShortDebugString(new_config)
Done


http://gerrit.cloudera.org:8080/#/c/6066/16/src/kudu/tools/tool_action_remote_replica.cc
File src/kudu/tools/tool_action_remote_replica.cc:

PS16, Line 366:   LOG(WARNING) << "NOTE: the new config may be replicated 
asynchronously "
  :<< "to other peers and it may take a while to 
bring the tablet "
  :<< "back to majority replica count depending 
upon how much time "
  :<< "the new peer replicas take to catch up with 
the tablet server "
  :<< "enforcing the new config.";
> It's still there in rev 17.
Done, removed now, yeah looks like I pressed 'done' before updating this code :)


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 16
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-24 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica unsafe_change_config'.

This tool lets us replace a N-replica config on a tablet server with a
new config containing N or less replicas. This is particularly useful
when we have majority of the replicas down and for some reason we are not
able to bring the tablet back online using other recovery tools like
'kudu remote_replica copy'. We can use this tool to force a new config on the
surviving replica providing all the replica uuids of the new config from
the cli tool. As a result of the forced config change, the automatic leader
election kicks in via raft mechanisms and the re-replication is triggered
from master (if needed due to under-replicated tablet) to bring the replica
count of the tablet back upto N.

How does the tool bring tablet back online with new config:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term along with the new config. The surviving node (leader or
   follower) accepts the request and replicates the request and goes through
   a pre-election phase in which a leader is elected among the nodes provided
   in the config. If the new config provides enough VOTERs to win en election,
   the leader election succeeds and the new config will be committed.
   Master can eventually recognize this consensus state change and make sure
   tablet is re-replicated back to healthy count if it finds the tablet
   under-replicated.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   so master will very likely choose the new healthy live servers for
   re-replication if needed. If the dead nodes come back after master is
   updated with new unsafely forced config, master will delete the replicas
   on those dead nodes via DeleteTablet RPC because they are no longer part
   of the tablet config.

Also, the UnasfeChangeConfig() API adds a flag to append another change_config
op while there is one pending config in the log. This flag lifts the safety
net around pending configs which states that there can be only one pending
config at the max for a given replica.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect. The future revs of this patch should weaken this assumption and build
more safety guarantees around situations dead nodes coming back during the
unsafe change config operations on the cluster.

Tests associated with this patch:
- Unsafe config change when there is one follower survivor in the cluster.
- Unsafe config change when there is one leader survivor in the cluster.
- Unsafe config change when the unsafe config contains 2 replicas.
- Unsafe config change on a 5-replica config with 2 replicas in the new config.
- Unsafe config change when there is a pending config on the surviving leader.
- Unsafe config change when there is a pending config on a surviving follower.
- Unsafe config change when there are back to back pending configs on WAL,
  and verify that tablet bootstraps fine.
- Test back to back unsafe config changes when there are multiple pending
  configs present with the replica and the one with 'sane' new config will
  bring the tablet back to online state.

TODO:
1) Test exercising all the error cases in the UnsafeChangeConfig API.
2) Test the UnsafeChangeConfig RPC directly without going via external tool.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_meta.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/consensus/time_manager.h
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
17 files changed, 1,222 insertions(+), 111 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/66/6066/17
-- 
To view, visit http://gerrit.cloudera.org:8080/6066
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700

[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-23 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Patch Set 16:

(34 comments)

http://gerrit.cloudera.org:8080/#/c/6066/16/src/kudu/consensus/consensus.proto
File src/kudu/consensus/consensus.proto:

Line 494:   // The raft config sent to destination server.
> Add: Only the 'permanent_uuid' of each peer in the config is required (addr
Done


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

PS16, Line 553: :
> nit: add space after colon
Done


PS16, Line 554: :
> and here
Done


PS16, Line 777: :
> nit: space after colon here and elsewhere in this log message
Done


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

PS16, Line 1167: :
> space after colon here and in the rest of this commit message
Done


PS16, Line 1330:  
> nit: Add a period before this space.
Done


PS16, Line 1594: Node
> Replica
Done


PS16, Line 1594: so new config will be appended.
> but the new config will be unsafely changed anyway.
Done


PS16, Line 1595:  :
> ": "
Done


Line 1596: << state_->GetPendingConfigUnlocked().DebugString();
> Add:
It may not be a good idea to display new config here especially when we know 
that what the tool passed in has just the list of uuids with default values for 
RaftConfigPB fields. new_config is going to be built later on looking at 
committed_config and we are displaying that at L1686, so its redundant too here.


PS16, Line 1596: ebugString
> use SecureShortDebugString() instead
Done


PS16, Line 1605:  and that node is a VOTER in the config
> remove this
Done


PS16, Line 1606: This is to prevent the API allowing an invalid uuid into the 
new config.
> Consider changing this to: This allows a manual recovery tool to only have 
Done


PS16, Line 1616: for tablet $1
> for tablet $1. Committed config: $2
Done


PS16, Line 1630: we don't want to enforce unsafe config that way
> Let's replace this part of the comment with: "it is rare and a replica with
Done


PS16, Line 1633: committed
> new
ok I will go with this, but the reason I was apprehensive of using 
new/committed is, VOTER/NON_VOTER field is read from committed config and user 
doesn't have a say in that. Hence the error message quoting "not a VOTER in new 
config" may be misleading.


PS16, Line 1635: tablet $1
> tablet $1. Rejected config: $2
Done


PS16, Line 1639: preceding_opid.index() + 1
> Define replicate_opid_index up here and use it here as well as down below.
Done


PS16, Line 1660: leader_term
> let's rename this variable 'new_term'
Done


PS16, Line 1680: DVLOG(1) << "Consensus request: "
> VLOG_WITH_PREFIX(3) << "UnsafeChangeConfig: Generated consensus request: "
Done


Line 1681:   DVLOG(1) << "Replicate Msg: " << 
SecureShortDebugString(*replicate);
> Remove this line since it's redundant with the line above
Done


PS16, Line 1684: ,
> nit: add space after comma
Done


PS16, Line 1685: DebugString()
> SecureShortDebugString() here and below
Done


PS16, Line 1685:  :
> space after colon, not before
Done


PS16, Line 1686:  :
> here also: space after colon, not before
Done


http://gerrit.cloudera.org:8080/#/c/6066/16/src/kudu/consensus/raft_consensus_state.cc
File src/kudu/consensus/raft_consensus_state.cc:

PS16, Line 241: (
> nit: Remove extra pair of parentheses here
Done


PS16, Line 245:   << Substitute("New committed config must equal pending 
config, but does not. "
  :   "Pending config: $0, committed config: $1",
  :   SecureShortDebugString(pending_config),
  :   SecureShortDebugString(config_to_commit));
> nit: it seems like the indentation is a little messed up on these 4 lines n
Done


http://gerrit.cloudera.org:8080/#/c/6066/16/src/kudu/tools/tool_action_remote_replica.cc
File src/kudu/tools/tool_action_remote_replica.cc:

Line 45: #include "kudu/tools/tool_test_util.h"
> Is this used? It looks like a test utility library?
yeah, actually I was initially using GetKuduCtlAbsolutePath("/bin/kudu") 
instead of random "kudu-tools" as caller id. I updated the code at L381 now.


PS16, Line 366:   LOG(WARNING) << "NOTE: the new config may be replicated 
asynchronously "
  :<< "to other peers and it may take a while to 
bring the tablet "
  :<< "back to majority replica count depending 
upon how much time "
  :<< "the new peer replicas take to catch up with 
the tablet server "
  :<< "enforcing the new config.";
> I think we should remove this message. I think it is confusing because it t
My concern about removing this message is that, user wouldn't know about these 
operations happening in the background and think that 

[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-23 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Patch Set 15:

(5 comments)

TFTR Mike/David again. We could follow up TimeManager friend method suggestion 
in an upcoming patch.

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

Line 1605:   if (!IsRaftConfigVoter(local_peer_uuid, committed_config)) {
> How about we just move this down to line 1649 and say:
Done


Line 1633:   if (!ContainsKey(retained_peer_uuids, local_peer_uuid)) {
> This seems redundant with the voter check if we do the voter check on the n
Done


PS15, Line 1646: continue;
> unnecessary
Done


http://gerrit.cloudera.org:8080/#/c/6066/15/src/kudu/tools/kudu-admin-test.cc
File src/kudu/tools/kudu-admin-test.cc:

PS15, Line 171: LOG(INFO)
> How about VLOG(1) here and below?
thanks for catching, some leftover debugging.


Line 920: 
> Let's keep the master dead until here, then run
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 15
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-23 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

Tests associated with this patch:
- Unsafe config change when there is one follower survivor in the cluster.
- Unsafe config change when there is one leader survivor in the cluster.
- Unsafe config change when the unsafe config contains 2 nodes.
- Unsafe config change on a 5-replica config with 2 nodes in the new config.
- Unsafe config change when there is a pending config on the surviving leader.
- Unsafe config change when there is a pending config on a surviving follower.
- Unsafe config change when there are back to back pending configs on WAL,
  and verify that tablet bootstraps fine.
- Test back to back unsafe config changes when there are multiple pending
  configs present on the node and the one with 'sane' new config will
  bring the tablet back to online state.

TODO:
1) Test exercising all the error cases in the UnsafeChangeConfig API.
2) Test the UnsafeChangeConfig RPC directly without going via external tool.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_meta.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/consensus/time_manager.h
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
17 files changed, 1,210 insertions(+), 111 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/66/6066/16
-- 
To view, visit http://gerrit.cloudera.org:8080/6066
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 16
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-23 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Patch Set 14:

(16 comments)

http://gerrit.cloudera.org:8080/#/c/6066/14/src/kudu/consensus/metadata.proto
File src/kudu/consensus/metadata.proto:

Line 93:   optional bool unsafe_config_change = 4;
> do we need to give this a default of false?
Done


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

Line 616: if (!new_config.unsafe_config_change()) {
> I think we need a default for this field or we first need to check has_unsa
added default, yeah agreed.


PS14, Line 1612: Peer uuid $0 is not found on original
> Peer with UUID $0 is not in the committed
Done


PS14, Line 1613: the node
> this replica
Done


Line 1617: if (!IsRaftConfigVoter(peer_uuid, committed_config)) {
> I don't think this voter check is necessary. What we should do here is chec
Done, moved that check up in the API since the VOTER type can be deciphered 
much earlier looking at committed config.


PS14, Line 1653: This ensures that current node replicates the new config 
proposed,
> This makes this request appear to come from a new leader that the local rep
Done


PS14, Line 1654: it would step down before replicating
> this will cause it to step down
Done


PS14, Line 1654: node
> local replica
Done


http://gerrit.cloudera.org:8080/#/c/6066/14/src/kudu/consensus/raft_consensus_state.cc
File src/kudu/consensus/raft_consensus_state.cc:

PS14, Line 257: FLAGS_fault_crash_before_append_commit
> Don't reuse this unrelated fault flag for this test. That flag is for the W
Done


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

Line 132:   friend class RaftConsensus;
> Intentional?
yeah, because we wanted to grab TimeManager::SerialTimeStamp as per David's 
last comment, and he also suggested friending would be better than making that 
routine public.


http://gerrit.cloudera.org:8080/#/c/6066/14/src/kudu/tools/kudu-admin-test.cc
File src/kudu/tools/kudu-admin-test.cc:

PS14, Line 523: 1
> Magic number. Why 1? Also, what if the election is churny? Since we know a 
I took this suggestion and even found that is unreliable. i.e there is a window 
between election result(which finds the leader_ts) and leader committing the 
last opid. So querying GetLastOpId may not help. Besides 
WaitUntilCommittedOpIdIndexIs() internally invokes GetLastOpId to compare with 
the value we pass in as argument, so calling GetLastOpId before that is 
redundant. What we essentially need here was to 
WaitUntilAtleastCommittedOpId(1,...) so that we know at least the replica 
committed the opid we expect (still doesn't guarantee it is the leader though).


PS14, Line 786: 8 
> 9
Done


PS14, Line 828: 1
> Why 1? Same question as above and for the other tests.
I took this logic from an existing test in raft_consensus, so there may be some 
room for improvement here in the event of election churn, but I believe we can 
expect the leader to be stable in the absence of any external events in these 
itests ?


PS14, Line 847: >
> If we use >= 0 here then we can delete lines 857-859.
indeed :), good find, done.


PS14, Line 880: 11
> Why 11? Shouldn't it be post_commit opid index from line 828 plus 5 for # o
no, this includes commit of config changes driven by re-replications from 
master, so 4 new nodes get added bumping it to 11. But, I have removed this 
WaitUntil..() from all the tests at this stage since I am just counting the 
config replication # on a new node added by re-replication now at this stage. 
i.e. Wait at line 881 alone is sufficient.


http://gerrit.cloudera.org:8080/#/c/6066/14/src/kudu/tools/tool_action_remote_replica.cc
File src/kudu/tools/tool_action_remote_replica.cc:

PS14, Line 431: Enforce raft config on the tablet server with a new config
> Using the word enforce in this context sounds strange to me. How about:
Done, filled the latter under ExtraDescription.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 14
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-21 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

Tests associated with this patch:
- Unsafe config change when there is one follower survivor in the cluster.
- Unsafe config change when there is one leader survivor in the cluster.
- Unsafe config change when the unsafe config contains 2 nodes.
- Unsafe config change on a 5-replica config with 2 nodes in the new config.
- Unsafe config change when there is a pending config on the surviving leader.
- Unsafe config change when there is a pending config on a surviving follower.
- Unsafe config change when there are back to back pending configs on WAL,
  and verify that tablet bootstraps fine.
- Test back to back unsafe config changes when there are multiple pending
  configs present on the node and the one with 'sane' new config will
  bring the tablet back to online state.

TODO:
1) Test exercising all the error cases in the UnsafeChangeConfig API.
2) Test the UnsafeChangeConfig RPC directly without going via external tool.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
15 files changed, 1,131 insertions(+), 111 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 13
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-20 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has abandoned this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Abandoned

Uploaded to new link by mistake, original patch is in 
http://gerrit.cloudera.org:8080/6066/

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

Gerrit-MessageType: abandon
Gerrit-Change-Id: Icdf7b962d404ab4a9e838695d60ac2cd9cf3df1a
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-20 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

Tests associated with this patch:
- Unsafe config change when there is one follower survivor in the cluster.
- Unsafe config change when there is one leader survivor in the cluster.
- Unsafe config change when the unsafe config contains 2 nodes.
- Unsafe config change on a 5-replica config with 2 nodes in the new config.
- Unsafe config change when there is a pending config on a surviving leader.
- Unsafe config change when there is a pending config on a surviving leader.
- Unsafe config change when there are back to back pending configs on logs.

TODO:
1) Test exercising all the error cases in the UnsafeChangeConfig API.
2) Test the UnsafeChangeConfig RPC directly without going via external tool.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_common.cc
M src/kudu/tools/tool_action_common.h
M src/kudu/tools/tool_action_local_replica.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
18 files changed, 1,149 insertions(+), 151 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/66/6066/12
-- 
To view, visit http://gerrit.cloudera.org:8080/6066
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 12
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-20 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy,

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

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

to review the following change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

Tests associated with this patch:
- Unsafe config change when there is one follower survivor in the cluster.
- Unsafe config change when there is one leader survivor in the cluster.
- Unsafe config change when the unsafe config contains 2 nodes.
- Unsafe config change on a 5-replica config with 2 nodes in the new config.
- Unsafe config change when there is a pending config on a surviving leader.
- Unsafe config change when there is a pending config on a surviving leader.
- Unsafe config change when there are back to back pending configs on logs.

TODO:
1) Test exercising all the error cases in the UnsafeChangeConfig API.
2) Test the UnsafeChangeConfig RPC directly without going via external tool.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700

ver 2

Change-Id: Icdf7b962d404ab4a9e838695d60ac2cd9cf3df1a
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_common.cc
M src/kudu/tools/tool_action_common.h
M src/kudu/tools/tool_action_local_replica.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
18 files changed, 1,149 insertions(+), 151 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Icdf7b962d404ab4a9e838695d60ac2cd9cf3df1a
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Mike Percy 


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-20 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Patch Set 11:

(22 comments)

http://gerrit.cloudera.org:8080/#/c/6066/11//COMMIT_MSG
Commit Message:

Line 41: 
> A description of the test cases you have implemented would be nice.
Done


http://gerrit.cloudera.org:8080/#/c/6066/11/src/kudu/consensus/consensus.proto
File src/kudu/consensus/consensus.proto:

Line 495:   required RaftConfigPB new_config = 4;
> On second thought, maybe we can defer that since right now we only support 
agreed, not changing for now.


PS11, Line 499: otherwise 'new_configuration' is set
> Update comment
Done


PS11, Line 522: Unsafe
> nit: Can we change this to Force instead of Unsafe?
I initially started with that naming convention, but changed it to Unsafe only 
because the ForceChangeConfig seemed bit vague to me since the actual 
ChangeConfig itself is a forced operation from a follower point of view. 
However, if we want to differentiate between automatic and manual, we could go 
with ManualChangeConfig perhaps ?


http://gerrit.cloudera.org:8080/#/c/6066/11/src/kudu/consensus/metadata.proto
File src/kudu/consensus/metadata.proto:

PS11, Line 93: allow_unsafe
> How about forced_config_change?
this can go along with the naming convention for API, since I was suggesting 
'manual' earlier, perhaps manual_config_change ?


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

PS7, Line 1667: pid_index);
> You are using safetime which is in the past. Don't we want the timestamp to
Lemme poke David to be sure, I am not sure if it makes much difference. One of 
the point David had was perhaps we shouldn't let the API to be exercised too 
soon by the user. i.e., we can have a check somewhere which says consensus on 
this node was stuck for say more than 10 minutes at the least to allow room for 
automatic recovery.


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

Line 1592:   // Check that passed replica uuids are part of the committed 
config on this node.
> Note: You can use the helper function IsRaftConfigVoter() from quorum_util.
Done, I didn't use that routine because we already can compare directly its 
member type here. Also I am explicitly not comparing with specific membership 
type, because thats something which can be enforced by the tool. This also 
means we could let the user specify membership type on the CLI.


Line 1601:   new_peer.last_known_addr().host() == 
committed_peer.last_known_addr().host() &&
> Do we expect last_known_addr to always be passed in the new config? I thoug
yeah we do. If you can recall, there was a reason why we chose RaftConfigPB as 
part of RPC, reason being in the case of 5-replica original config {ABCDE} and 
2 nodes are part of the enforced config {AB} being written on node A, we want 
to be able to provide both A and B's HostPort endpoints as part of config. The 
tool could query HostPort details from master, but tool wouldn't know whether 
master has a stale info or recent. So for now, pushing back on the user to 
provide the complete info instead of this API or the tool determining that info 
at run time.


Line 1623:   RaftConfigPB new_config = config;
> How about just use a copy of the committed config, but with the uuids we wa
That's a good idea, but given that we are supplying the entire config to the 
API this is not different than the current code wrt end result. Did you have 
any other concern in mind ?


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

Line 238:   // In the event of unsafe config change enforced from an external 
tool,
> I don't understand that logic. If there are multiple "pending" configs, whi
I will move this discussion to latest patch since this code/comments have 
changed a bit there, and I see that you have added sine comments there as well.


http://gerrit.cloudera.org:8080/#/c/6066/11/src/kudu/consensus/raft_consensus_state.cc
File src/kudu/consensus/raft_consensus_state.cc:

PS11, Line 212: Allowing config update
> How about: Allowing forced config change
Done


PS11, Line 239: may not have a pending config
> You mean that it may not match the currently-pending config, right?
thats correct, thanks for catching, updated.


PS11, Line 242: both
> I don't think this will work in all cases; see below
update comment here.


PS11, Line 247: !config_to_commit.allow_unsafe()
> As mentioned in the rev 7 thread, I don't think this condition is needed
thats true, updated.


PS11, Line 248: pending_config.allow_unsafe() && config_to_commit.allow_unsafe()
> I think this should be removed due to the following scenario:
great catch, yeah.. but how can we let the 

[kudu-CR] KUDU-1769: fs check action with rudimentary data block GC

2017-03-17 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1769: fs check action with rudimentary data block GC
..


Patch Set 2: Code-Review+1

(3 comments)

Thanks for adding this Adar, LGTM, couple of minor comments below.

http://gerrit.cloudera.org:8080/#/c/6361/2//COMMIT_MSG
Commit Message:

Line 24: I do expect that the block manager will take on some check or repair 
duties,
Thinking aloud, would it make sense to run the scan of orphan blocks every now 
and then as a mantainance manager background task and once the number of orphan 
blocks crosses certain threshold, perhaps trigger the repair ?


http://gerrit.cloudera.org:8080/#/c/6361/2/src/kudu/tools/tool_action_fs.cc
File src/kudu/tools/tool_action_fs.cc:

PS2, Line 50: repair
I wonder if  term 'repair' could give any wrong impression about the underlying 
operation being performed. We are cleaning up the orphan blocks and fixing 
inconsistency for sure, but functionality is not similar to fsck --repair kinda 
I guess, and I don't have any better alternative word suggestion either :(. may 
be --free_orphan_blocks ?


PS2, Line 72: /*context*/
Is this a way to silence unused warnings ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I82da8e234c338cd5a7540a22cebfa0a4958388ec
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-16 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
1) Test exercising all the error cases in the UnsafeChangeConfig API.
2) Test the UnsafeChangeConfig RPC directly without going via external tool.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_common.cc
M src/kudu/tools/tool_action_common.h
M src/kudu/tools/tool_action_local_replica.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
18 files changed, 1,073 insertions(+), 150 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 11
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-11 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
0) Test with 2 pending config changes on WAL one after another
   and crash and check that tablet bootstrap is fine.
2) Test with a 5-replica config replacing the old config {ABCDE}
   with a new config {AB} on A.
3) Test exercising all the error cases in the UnsafeChangeConfig API.
4) Test the UnsafeChangeConfig RPC directly without going via external tool.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_common.cc
M src/kudu/tools/tool_action_common.h
M src/kudu/tools/tool_action_local_replica.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
18 files changed, 908 insertions(+), 150 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/66/6066/10
-- 
To view, visit http://gerrit.cloudera.org:8080/6066
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 10
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-11 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
0) Test with 2 pending config changes on WAL one after another
   and crash and check that tablet bootstrap is fine.
2) Test with a 5-replica config replacing the old config {ABCDE}
   with a new config {AB} on A.
3) Test exercising all the error cases in the UnsafeChangeConfig API.
4) Test the UnsafeChangeConfig RPC directly without going via external tool.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_common.cc
M src/kudu/tools/tool_action_common.h
M src/kudu/tools/tool_action_local_replica.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
18 files changed, 897 insertions(+), 150 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 9
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-11 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Patch Set 7:

(27 comments)

http://gerrit.cloudera.org:8080/#/c/6066/7/src/kudu/consensus/consensus.proto
File src/kudu/consensus/consensus.proto:

PS7, Line 488: 4
> You can just number this starting from 1
Done.


PS7, Line 492: sender_id
> How about caller_uuid or caller_id
Done


PS7, Line 497: required RaftPeerPB server
> Comment needs update?
Done


PS7, Line 506: new_config
> Is this used for something? Regardless, see my comment above, it seems to m
Done


http://gerrit.cloudera.org:8080/#/c/6066/7/src/kudu/consensus/metadata.proto
File src/kudu/consensus/metadata.proto:

Line 93:   optional bool allow_unsafe = 4;
> I think this belongs in ChangeConfigRecordPB
I guess you changed your mind about this suggestion later if I am correct.


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

Line 1148: if (state_->GetActiveRoleUnlocked() != RaftPeerPB::LEADER) {
> This change can be reverted, right?
yeah, I was about to, missed in previous patch. Done.


PS7, Line 1328: .
> nit: missing space after period
Done


PS7, Line 1328: UNLOCKED
> why _UNLOCKED?
Done, yeah that was not intentional.


Line 1563:  
UnsafeChangeConfigResponsePB *resp,
> warning: parameter 'resp' is unused [misc-unused-parameters]
Done


Line 1598:   string replica_uuid = peer_pb.permanent_uuid();
> warning: the variable 'replica_uuid' is copy-constructed from a const refer
Done


Line 1598:   string replica_uuid = peer_pb.permanent_uuid();
> Based on my comments elsewhere I think this removal logic should probably h
I believe there are substantial changes here based on our offline discussions, 
so please re-review this portion.


Line 1630:   new_config.add_peers()->CopyFrom(new_peer);
> So... we are only allowing changing to a single-node config?
That has changed now, the CLI/API takes the new config itself.


PS7, Line 1651: current_term + 1
> please extract this into a new variable for use here and on line 1664
Done


PS7, Line 1665: preceding_opid.index() + 1
> Please extract a variable for this so it doesn't look like a magic number. 
Done


PS7, Line 1667: msg_timestamp
> Don't we need this to be higher
higher than ? can you explain bit more what you have in mind ?


Line 1672:   LOG_WITH_PREFIX(WARNING)
> I don't think it's necessary for all caps in the server log. If you want to
Done


Line 1680: s = StatusFromPB(consensus_resp.error().status());
> Is there a specific error type you are looking for here?
Oh yeah, definitely in my radar, updated the commit_msg accordingly.


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

PS8, Line 1612:   // Take the snapshot of the queue state and timestamp to 
stick them
  :   // in the consensus request later.
  :   int64_t all_replicated_index = 
queue_->GetAllReplicatedIndex();
  :   int64 last_committed_index = queue_->GetCommittedIndex();
  :   OpId preceding_opid = queue_->GetLastOpIdInLog();
  :   uint64 msg_timestamp = time_manager_->GetSafeTime().value();
> needs to be done under the lock.
David, sorry still not following. Did you mean lock for timestamp ?


PS8, Line 1672: LOG_WITH_PREFIX(WARNING)
  : << "REPLACING THE CONFIG ON THIS SERVER WITH A NEW 
CONFIG,"
  : << "THIS OPERATION IS IRREVERSIBLE !!\n"
  : << "COMMITTED CONFIG :" << 
committed_config.DebugString()
  : << "NEW CONFIG :" << new_config.DebugString();
> add one like this with the result of the actual change config.
Actually, the fact that we have written this new_config doesn't mean that 
cluster came back up and tablet became operational. Tool throws a warning today 
quoting this operation is asynchronous and lot can happen in the background, 
longer wait, etc. As such, there is nothing else to print here apart from 
config being overwritten(new_config). Or did you have something else in mind ?


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

PS7, Line 226: committed_config
> As we discussed offline, it would be an improvement to rename this argument
Done


Line 238:   if (!committed_config.allow_unsafe()) {
> This doesn't seem right to me. I think this check should only be skipped if
Actually, there is a chicken-and-egg kinda issue here; I have tried to make 
this clear in comments now. We should check for equality of config only when we 
know both pending and to-be-committed config match wrt their 'unsafe' flag. Let 
me know if you want the comments to be more clearer.



[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-09 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
0) Test with 2 pending config changes on WAL one after another
   and crash and check that tablet bootstrap is fine.
1) Accept more than one peers to be added from the command line
   to support multiple peers to be added in the replaced config.
2) Once 1) is in place, test with a 5-replica config replacing
   the old config {ABCDE} with new config {AB} on A.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
15 files changed, 657 insertions(+), 110 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/66/6066/7
-- 
To view, visit http://gerrit.cloudera.org:8080/6066
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-09 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
0) Test with 2 pending config changes on WAL one after another
   and crash and check that tablet bootstrap is fine.
1) Accept more than one peers to be added from the command line
   to support multiple peers to be added in the replaced config.
2) Once 1) is in place, test with a 5-replica config replacing
   the old config {ABCDE} with new config {AB} on A.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
15 files changed, 657 insertions(+), 110 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/66/6066/6
-- 
To view, visit http://gerrit.cloudera.org:8080/6066
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-09 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Patch Set 4:

(7 comments)

http://gerrit.cloudera.org:8080/#/c/6066/4/src/kudu/consensus/consensus.h
File src/kudu/consensus/consensus.h:

PS4, Line 255: ReplaceConfig
> there's only one implementation of consensus now, no need to do this.
Thanks, yeah, it should be pure virtual here.


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

PS4, Line 1561: ReplaceConfig
> this needs a LOG WARNING in big bold letters at several stages.
done.


PS4, Line 1573:   // ReplaceConfig() replaces the config irrespective of 
opid_index value,
  :   // hence there is no meaning for cas_config_opid_index in 
this API.
> wouldn't this be a great way to make sure that you're changing the actual c
yeah, it could be, for now tool doesn't read what's there in the config, I 
should add that.


PS4, Line 1614:   // in the consensus request later.
  :   int64_t all_replicated_index = 
queue_->GetAllReplicatedIndex();
  :   OpId last_opid = queue_->GetLastOpIdInLog();
  :   uint64 msg_timestamp = time_manager_->GetSafeTime().value();
> I mentioned in a previous rev that these need to be obtained under a lock s
The term is obtained from ReplicaState @L1588 and these fields are grabbed from 
queue state and each of these member functions grab a spinlock from inside. As 
such I didn't see a reason for queue state snapshot to collate here. Or may be 
I haven't understood your comment fully.


PS4, Line 1675: state_->GetActiveRoleUnlocked() != RaftPeerPB::LEADER
> isn't it likely that the lonely node will think he's the leader?
Actually this whole block of code was removed now because of couple of reasons: 
a) This was only aborting the in-memory cmeta config changes pending and was 
not addressing what to do with the WAL entries. b) It became moot after we 
faked leader uuid from the tool; i.e., whether this node was a follower or a 
leader, the pending config changes get aborted as part of 
Update()->AbortOpsAfter() based on what is present on log and what's in 
consensus req. If this node was the leader, he would step down looking at 
higher term and abort.


http://gerrit.cloudera.org:8080/#/c/6066/4/src/kudu/integration-tests/cluster_itest_util.h
File src/kudu/integration-tests/cluster_itest_util.h:

PS4, Line 160: NO_WAIT_FOR_LEADER
> nit: DONT_WAIT_FOR_LEADER
done


PS4, Line 166: WaitForReplicasReportedToMaster
> best to make this return a Status::TimeOut
Sounds good, tx


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-09 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.

The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The tool acts as a fake leader and generates the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once the new config update is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster
   bringing the tablet back to online state.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a flag to bypass the
'allow-max-one-pending-config-change' rule to append another change_config
op while there is one pending on the log.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
0) Test with 2 pending config changes on WAL one after another
   and crash and check that tablet bootstrap is fine.
1) Accept more than one peers to be added from the command line
   to support multiple peers to be added in the replaced config.
2) Once 1) is in place, test with a 5-replica config replacing
   the old config {ABCDE} with new config {AB} on A.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/metadata.proto
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_state.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
M src/kudu/tserver/tablet_service.h
15 files changed, 620 insertions(+), 97 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/66/6066/5
-- 
To view, visit http://gerrit.cloudera.org:8080/6066
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-08 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Patch Set 1:

(1 comment)

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

PS1, Line 1556: RaftConsensus::ReplaceConfig
> TFTR David, just a heads up here that latest patch doesn't reflect all the 
After discussion with Mike yesterday couple of things we want to revise further:
1) Instead of squeezing this functionality in leader-driven ChangeConfig RPC, 
we will have a new RPC which accepts ReplaceConfigRequestPB containing bunch of 
uuids to be removed from the config (as per your original suggestion of making 
the API less user-error-prone). We can change these semantics later if we want 
to.

2) For the pending config change which was replicated on WAL, we decided to 
relax "allow-max-one-pending-config-change" criteria for this API. This means 
we may observe 2 replicated CONFIG_CHANGE on the log after crash recovery first 
one was original one, second was appended by the tool.

3) Current patch uses last replicated opid to set the committed index field in 
consensus request instead of last committed opid, I have fixed that. I still 
need to see what value should we pass in 'all_replicated_index' in the 
consensus request or do we need that at all.

4) Some misc tooling changes to accommodate 1).

5) I have removed changes from consensus_queue.[hc], since they were no longer 
necessary after I updated the patch to use a fake leader_uuid.

6) Fake leader uuid string will be replaced by a static id string "/bin/kudu" 
after 1) is in place and that static id string is supplied by the tool itself. 
My concern here was if it goes through any uuid sanity checks somewhere in the 
core consensus, testing will confirm this.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-07 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..


Patch Set 1:

(1 comment)

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

PS1, Line 1556: RaftConsensus::ReplaceConfig
> I'll let mike address the pending/active config concerns of this method, si
TFTR David, just a heads up here that latest patch doesn't reflect all the 
suggestions made above, and some rationales are listed below.

1) I agree with adding checks about consensus being stuck for a while instead 
of letting the tool operate immediately without allowing automatic recovery. Is 
there a metrics we should rely on ? or should that check be a function of some 
peers in the config and they being unresponsive for a while ? The latter case 
falls under node eviction after certain timeouts, so we could wait for at least 
follower_unavailable_considered_failed_sec.

2) I am having second thoughts on making the API accept list of removed uuids, 
because we want to be able to enforce a config of {AB} on node A despite a 
committed config of {ACD} when C and D become dead nodes at some point. This 
also means that, tool also needs to input the Host and Port address endpoint of 
B and there are other commands to fetch that info. Currently uuid presence 
check is overly restrictive becz it checks if the passed uuid is in config or 
not and it accepts only one uuid at the moment. I need to add a unit test for 
the checks I have introduced in this API - i.e simulating user errors and see 
that they fail as expected.
Having the API accept a list of uuids to be added for the new config seems like 
a better idea. This requires either a new RPC taking RaftConfigPB in the 
request directly, or ChangeConfigRequestPB can take a list of RaftPeerPB and 
the API can construct the new config from that list. Usual 
AddServer/RemoveServer can expect the size of that list to be 1. Let me know 
your thoughts.

3) I refactored a bit grabbing consensus states and consensus queue states as 
one snapshot before building the new config and consensus request for Update(). 
The API assumes currently that dead nodes aren't interfering, so this snapshot 
of the state becomes relevant when we weaken those assumptions for future 
revisions.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-07 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.
The lonely survivor being operated by the tool tends to become the leader in
new config in majority of the situations because:
a) The API/tool acts as a fake leader and generaets the consensus update with
   a bumped up term. The surviving node accepts the request and holds
   a pre-election upon recognizing the leader heartbeat failure.
   This results in the surviving node electing himself as the leader
   and once this config change is reported to master, master can
   re-replicate the tablet to other healthy nodes in the cluster.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader elected in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a way to abort a pending config change
because pending config comes in the way of recovery tool trying to
replicate/commit the new config on the surviving tablet server. There is only
one pending config change allowed at a time for a given tablet, hence
aborting the pending config change seems safest bet.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
0) Test with unresponsive servers evicted from the config after the
   'follower_unavailable_considered_failed_sec' timeout.
1) Accept more than one peers to be added from the command line
   to support multiple peers to be added in the replaced config.
   This needs some change to ChangeConfigRequestPB to acept multiple
   RaftPeerPB in one single request.
2) Once 1) is in place, test with a 5-replica config replacing
   the old config {ABCDE} with new config {AB} on A.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
13 files changed, 577 insertions(+), 96 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-03 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: KUDU-1330: Add a tool to unsafely recover from loss of majority 
replicas
..

KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is particularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.
The lonely survivor talking to the tool tends to become the leader in
new config in majority of the use cases because:
a) The API/tool mimicks the actual leader the survivor node
   had voted for and replicate the new config with a higher term and
   bumped up opid_index. This ensures that 2 new nodes added as part of
   re-replication respect the term emitted by this node and accept
   this node as leader.
b) Assumption is that, the dead nodes are not coming back during this recovery,
   hence the leader chosen in step a) will still be the leader when we see
   the replication factor restored back to 3.

Also, the ReplaceConfig() API adds a way to abort a pending config change
because pending config comes in the way of recovery tool trying to
replicate/commit the new config on the surviving tablet server. There is only
one pending config change allowed at a time for a given tablet, hence
aborting the pending config change seems safest bet.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
0) Accept more replica_uuids from the command line to make support multiple
   peers to be added in the new config.
1) Test with a 5-replica config forcing the old {ABCDE} to new {AB} on A.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/time_manager-test.cc
M src/kudu/consensus/time_manager.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
15 files changed, 545 insertions(+), 103 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] WIP KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-03-03 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: WIP KUDU-1330: Add a tool to unsafely recover from loss of 
majority replicas
..


Patch Set 2:

(12 comments)

I am still addressing some refactoring/testing related to David's earlier 
feedbacks, so this update has addressed only Mike's review comments so far.

http://gerrit.cloudera.org:8080/#/c/6066/2//COMMIT_MSG
Commit Message:

PS2, Line 13: perticularly
> particularly
Done


Line 22: a) The API/tool acts as a fake leader mimicking the actual leader the 
node
> As mentioned in comments on raft_consensus.cc the tool should not impersona
I will probably post an update here after a quick testing, is the concern here 
that we may be picking an old leader if we rely on survivor's committed 
config's 'voted_for' field ? Even if we did, since we are bumping up the term, 
we know that this node is going to be elected as leader (assuming dead nodes 
not coming back).


Line 26: a) Assumption is that, the dead nodes are not coming back with a 
higher term,
> s/a/b/
Actually, by 'retained' I meant the the elected leader after election triggered 
as a result of config change in step a). I reworded this now.


Line 43: 2) Add a test case for when the node has a pending config change,
> +1 on this
I hadn't updated this list after my last commit, pls note that some of the test 
cases are already added, I updated the commit msg now to reflect current state.


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

PS2, Line 1559: client_cb
> No need to plumb in a callback if this is not an asynchronous method
Done


PS2, Line 1560: error_code
> Would be nice to use this
Done


Line 1600:   consensus_req.set_caller_uuid(leader_uuid);
> This should be the client UUID or a special value to indicate that this was
Do you have a concern on this approach ? Whether survivor is a single follower 
or a single leader, both of them would have had voted for the this leader. This 
approach works in both use cases.  I thought this is a simpler idea than using 
a new/fake uuid from tool. Note that whole change assumes that the dead nodes 
aren't coming back while this is in progress.


Line 1601:   consensus_req.set_caller_term(current_term);
> Use current term + 1
I didn't bump this to single leader use case, because the existing leader steps 
down and bails out from pre-election here without committing a new config: 
https://github.com/apache/kudu/blob/master/src/kudu/consensus/raft_consensus.cc#L401
However, I need to debug little more what other changes are needed if we were 
to bump up the term for single leader survivor. Do you think it's necessary to 
bump up the term for single leader survivor ?


Line 1603:   consensus_req.set_committed_index(last_op_id.index() + 1);
> Don't increment the committed index; leave it at whatever value it previous
Done


PS2, Line 1613: last_op_id.index() + 1
> This is repeated a few times in this function. Let's extract a variable to 
Done


PS2, Line 1615: time_manager_->GetSafeTime().value()
> Hmm. I would like David's input on this
I will poke David here, I don't exactly recall why I had to set here, perhaps 
due a CHECK failure down the line in UpdateReplica. It somewhere has a check 
for timestamp on a consensus request.


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

Line 121
> I don't think you need to remove this
In the case of single leader leader propgating the new config, he calls 
UpdateReplica on himself, hence I had to remove couple of DCHECKs as part of 
that. I also removed one on consensus_queue.cc


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] WIP KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-02-28 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: WIP KUDU-1330: Add a tool to unsafely recover from loss of 
majority replicas
..

WIP KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is perticularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.
The lonely survivor talking to the tool tends to become the leader in
new config in majority of the use cases because:
a) The API/tool acts as a fake leader mimicking the actual leader the node
   had voted for and replicate the new config with a higher term and
   bumped up op_index. This ensures that other 2 nodes added later on respect
   the term emitted by this node and accept this node as leader.
a) Assumption is that, the dead nodes are not coming back with a higher term,
   hence leadership is retained.

Also the ReplaceConfig() API adds a way to abort a pending config change
because pending config comes in the way of recovery tool trying to
replicate/commit the new config on the surviving tablet server. There is only
one pending config change allowed at a time for a given tablet, hence
aborting the pending config change seems safest bet.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
0) Accept more replica_uuids from the command line to make support multiple
   peers to be added in the new config.
1) Add a test case when 1 leader is alive.
2) Add a test case for when the node has a pending config change,
   covering the cases when the node is a leader or a follower.
3) Test with a 5-replica config forcing the old {ABCDE} to new {AB} on A.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/time_manager.cc
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
14 files changed, 488 insertions(+), 96 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] WIP KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

2017-02-17 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy,

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

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

to review the following change.

Change subject: WIP KUDU-1330: Add a tool to unsafely recover from loss of 
majority replicas
..

WIP KUDU-1330: Add a tool to unsafely recover from loss of majority replicas

This patch adds an API to allow unsafe config change via an external
recovery tool 'kudu remote_replica replace_config'.

This tool lets us replace a 3-node config on a tablet server with a
1-node config. This is perticularly useful when we have 2 out of 3
replicas down and we want to bring the tablet back to operational state.
We can use this tool to force a new config on the surviving node providing
all the details of the new config from the tool. As a result
of the forced config change, the automatic leader election kicks in via
raft mechanisms and the re-replication is triggered from master to bring
the replica count back upto 3-node config.
The lonely survivor talking to the tool tends to become the leader in
new config in majority of the use cases because:
a) The API/tool acts as a fake leader mimicking the actual leader the node
   had voted for and replicate the new config with a higher term and
   bumped up op_index. This ensures that other 2 nodes added later on respect
   the term emitted by this node and accept this node as leader.
a) Assumption is that, the dead nodes are not coming back with a higher term,
   hence leadership is retained.

Also the ReplaceConfig() API adds a way to abort a pending config change
because pending config comes in the way of recovery tool trying to
replicate/commit the new config on the surviving tablet server. There is only
one pending config change allowed at a time for a given tablet, hence
aborting the pending config change seems safest bet.

This patch is a first in series for unsafe config changes, and assumes that
the dead servers are not coming back while the new config change is taking
effect.

TODO:
0) Accept more replica_uuids from the command line to make support multiple
   peers to be added in the new config.
1) Add a test case when 1 leader is alive.
2) Add a test case for when the node has a pending config change,
   covering the cases when the node is a leader or a follower.
3) Test with a 5-replica config forcing the old {ABCDE} to new {AB} on A.

Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/integration-tests/cluster_itest_util.cc
M src/kudu/integration-tests/cluster_itest_util.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tools/kudu-admin-test.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tserver/tablet_service.cc
11 files changed, 310 insertions(+), 86 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I908d8c981df74d56dbd034e72001d379fb314700
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Mike Percy 


[kudu-CR] [tools] Add missing help text from few tools

2017-02-02 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo,

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

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

to review the following change.

Change subject: [tools] Add missing help text from few tools
..

[tools] Add missing help text from few tools

Change-Id: I49cb5d34fcd5153435d6ad4c34d496a8f58cc144
---
M src/kudu/tools/tool_action_cluster.cc
M src/kudu/tools/tool_action_remote_replica.cc
M src/kudu/tools/tool_action_test.cc
3 files changed, 3 insertions(+), 0 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I49cb5d34fcd5153435d6ad4c34d496a8f58cc144
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Mike Percy 


[kudu-CR] tool: remove dead code

2017-01-23 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: tool: remove dead code
..


Patch Set 1: Code-Review+1

It's good you noticed them Adar. Ship it from my side.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I93b173c4f2f91d1d7466f99ddc3264a8796e5b51
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: No


[kudu-CR] [util] fixed env-test on OS X

2017-01-19 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [util] fixed env-test on OS X
..


Patch Set 2:

(2 comments)

I am not seeing radio buttons on gerrit, perhaps some browser plugin issue I am 
facing at the moment, so consider +1 from my side.

http://gerrit.cloudera.org:8080/#/c/5738/2//COMMIT_MSG
Commit Message:

Line 14: 
Do you want to post the stacktrace of the failure ?


http://gerrit.cloudera.org:8080/#/c/5738/2/src/kudu/util/env_posix.cc
File src/kudu/util/env_posix.cc:

Line 1277:   l.rlim_max = limit;
> I'm not following: why would the rlimit struct be relevant here?  The probl
Calling setrlimit down below with astronomical value was the original cause; So 
if rlim_max was uninitialized, 'rlim_max(uint64_t) = limit(uint32_t)' here will 
end up in same issue. But ignore this, I just confirmed that structs come with 
default constructors too, so this is safe.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic355912b98b3fa592481e2457147e23de98447ea
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-HasComments: Yes


[kudu-CR] [consensus] KUDU-1613: Fix replica eviction failure for WRONG SERVER UUID

2017-01-17 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [consensus] KUDU-1613: Fix replica eviction failure for 
WRONG_SERVER_UUID
..


Patch Set 7:

(6 comments)

TFTR Mike, I will be adding one more test which changes just the UUID keeping 
the tablets intact (which means all tablets are reported behind a new server 
UUID but behind same RPC endpoints) after I figure out a anomaly with 
non_participant I observed in the new test.

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

Line 305: if (response_.error().code() != 
TabletServerErrorPB::WRONG_SERVER_UUID &&
> I think we should consider WRONG_SERVER_UUID as unresponsive (ERROR) instea
Done, please note that as per our offline discussions last week, the tablet 
copy is driven due to this error code deemed as tolerable until heartbeat 
timeout kicks out the replica and re-replication is triggered at taht point.


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

Line 816: // Verify that tablet copy is triggered when peer responds with
> Is this test still supposed to be here?
Not after we removed the WRONG_SERVER_UUID from queue->ResponseFromPeer(). 
Updated.


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

Line 597:   peer->needs_tablet_copy = true;
> Why this for WRONG_SERVER_UUID?
I suppose this isn't needed if we are driving the tablet-copy via replica 
eviction and then re-replication, right ?


http://gerrit.cloudera.org:8080/#/c/5111/7/src/kudu/integration-tests/raft_consensus-itest.cc
File src/kudu/integration-tests/raft_consensus-itest.cc:

Line 2789:   MonoDelta kTimeout = MonoDelta::FromSeconds(30);
> nit: const
Done


Line 2795:   int leader_idx = -1;
> nit: This is written in a C-like way. It would be a bit more readable to de
Done


Line 2816: 
> Please add a comment in here to explain what's going on:
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1741: Keep MiniCluster::Restart consistent with ExternalMiniCluster::Restart

2017-01-09 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Alexey Serbin, Kudu Jenkins,

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

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

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

Change subject: KUDU-1741: Keep MiniCluster::Restart consistent with 
ExternalMiniCluster::Restart
..

KUDU-1741: Keep MiniCluster::Restart consistent with 
ExternalMiniCluster::Restart

MiniCluster and ExternalMiniCluster utility classes follow opposite
semantics for restart. For eg, ExternalMiniCluster::Restart() expects
that all the nodes must be shutdown before we restart, whereas
MiniCluster::Restart() expects that all servers are up and running
so that it can internally execute shutdown and start in that order.

This patch changes the semantics of MiniCluster::Restart() to follow
that of ExternalMiniCluster::Restart() to keep them consistent and
the existing tests are modified to follow the new semantics of
MiniCluster::Restart().

Change-Id: Iad33b7c46bfca3f277ccbca7d0420272f06a6633
---
M src/kudu/client/client-test.cc
M src/kudu/integration-tests/alter_table-test.cc
M src/kudu/integration-tests/create-table-stress-test.cc
M src/kudu/integration-tests/fuzz-itest.cc
M src/kudu/integration-tests/registration-test.cc
M src/kudu/master/master-test.cc
M src/kudu/master/mini_master.cc
M src/kudu/master/mini_master.h
M src/kudu/tserver/mini_tablet_server.cc
9 files changed, 17 insertions(+), 10 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iad33b7c46bfca3f277ccbca7d0420272f06a6633
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 


[kudu-CR] Add some path / env related helper functions

2017-01-09 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: Add some path / env related helper functions
..


Patch Set 3: -Code-Review

(1 comment)

http://gerrit.cloudera.org:8080/#/c/5618/3/src/kudu/util/env_posix.cc
File src/kudu/util/env_posix.cc:

Line 896: unique_ptr wd(getcwd(NULL, 0));
> Please ask a Kudu macOS-based dev to build this before merging.
Done; compiles fine on macOS.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ia664708a09493923abbf2ff4e56e3d49c62cf97e
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Mike Percy 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Lars Volker 
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] Add Google Breakpad support to Kudu

2017-01-09 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: Add Google Breakpad support to Kudu
..


Patch Set 3:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/5620/3/src/kudu/util/minidump.cc
File src/kudu/util/minidump.cc:

PS3, Line 164: Attempting to delete it
> I disagree -- it's easier for grep purposes to keep individual events descr
ok that makes sense, perhaps we could just reorder the error as "failed to read 
minidump file. Read error: " << s.ToString() << "Attempting to delete it." so 
that read error message and reason string are adjacent to each other.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I495695cc38b75377f20b0497093a81ed5baa887f
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Mike Percy 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Lars Volker 
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1741: Make MiniCluster and ExternalMiniCluster follow one semantic for Restart

2017-01-09 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1741: Make MiniCluster and ExternalMiniCluster follow one 
semantic for Restart
..


Patch Set 1:

> Intuitively restart means "stop then start" for me, and as shown by
 > your patch there's definitely a need for a method that does both.
 > IMO if you want to keep restart to mean "start a stopped process"
 > then that's fine but we should also have something that stops then
 > stars.

TFTR JD, yeah I agree and current MiniCluster::Restart was more intuitive in 
that sense; I changed the MiniCluster(against the intuitive way) instead of 
ExternalMiniCluster mainly because ExternalMiniCluster was more widely used by 
pretty much all unit tests and integration tests. 

ExternalMiniCluster/ExternalMaster/ExternalTabletServer expect the 
cluster/nodes to have been shutdown before restart. We could :
a) change the ExternalMiniCluster::Restart to internally do a shutdown/start to 
make it intuitive (follow MiniCluster semantics).
b) Add a new API ShutDownAndStart, which would achieve what you are suggesting.

I think a) sounds like more reasonable approach. Let me know what you think.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iad33b7c46bfca3f277ccbca7d0420272f06a6633
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-HasComments: No


[kudu-CR] EMC: Don't reuse data dir for log dir

2017-01-09 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: EMC: Don't reuse data dir for log dir
..


Patch Set 3: Code-Review+1

(2 comments)

http://gerrit.cloudera.org:8080/#/c/5619/3//COMMIT_MSG
Commit Message:

PS3, Line 11: .
It would be nice if we could elaborate the problem with current directory 
structure here, looking at the COMMIT_MSG alone the intent of the patch wasn't 
totally clear to me.


http://gerrit.cloudera.org:8080/#/c/5619/3/src/kudu/integration-tests/external_mini_cluster.cc
File src/kudu/integration-tests/external_mini_cluster.cc:

PS3, Line 809: process
Do we want to note the addition of this in the COMMIT_MSG ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iba23c429989df524da51eb012a491766df06e955
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Mike Percy 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Lars Volker 
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] Add Google Breakpad support to Kudu

2017-01-09 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: Add Google Breakpad support to Kudu
..


Patch Set 3:

(4 comments)

I forgot to press send on this friday. 
Thinking out loud, I am trying to see what's the real advantage breakpad can 
give us over gdb corefile ? If we could squeeze in more info about kudu 
components - things like which tablet_id caused the crash - in the crash info, 
it might be one way of gaining control over how we want to present crash dump 
info in a useful manner for debugger.

http://gerrit.cloudera.org:8080/#/c/5620/3//COMMIT_MSG
Commit Message:

Line 36: 
It would be good to include a section on how-to go about debugging the minidump 
file. For eg, which tool to use, should we rely on another tool to convert 
minidump to corefile and run gdb etc.


http://gerrit.cloudera.org:8080/#/c/5620/3/src/kudu/util/minidump.cc
File src/kudu/util/minidump.cc:

PS3, Line 160: Slice
Do we explicitly want to include #include "kudu/util/slice.h" for this ?


PS3, Line 164: Attempting to delete it
Nit: would be nice to print this in next line separately.


PS3, Line 221: Impala
s/Impala/kudu ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I495695cc38b75377f20b0497093a81ed5baa887f
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Mike Percy 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Lars Volker 
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] Add some path / env related helper functions

2017-01-06 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: Add some path / env related helper functions
..


Patch Set 3: Code-Review+1

(5 comments)

http://gerrit.cloudera.org:8080/#/c/5618/3/src/kudu/util/env_util-test.cc
File src/kudu/util/env_util-test.cc:

PS3, Line 79: We have to clean this up manually
If this path is part of test directory, do we still have to clean this up 
manually ? or are we cleaning it up only to verify Deleterecursive routine 
after CreateRecursive ?


PS3, Line 112: real_dir
Nit: symlink could have been created to test_dir_ itself instead of a new 
real_dir.


http://gerrit.cloudera.org:8080/#/c/5618/3/src/kudu/util/env_util.cc
File src/kudu/util/env_util.cc:

PS3, Line 206: Canonicalize
If we move this line to above L200 we don't need to special case symlink here. 
i.e, Canonicalize does more than just resolving symlink, so it's sufficient to 
check is_dir and continue at L203.


http://gerrit.cloudera.org:8080/#/c/5618/3/src/kudu/util/env_util.h
File src/kudu/util/env_util.h:

PS3, Line 74: .
Nit: IMO it would be good to add in the comment that this is emulating mkdir -p.


http://gerrit.cloudera.org:8080/#/c/5618/3/src/kudu/util/path_util.cc
File src/kudu/util/path_util.cc:

PS3, Line 56: SplitPath
Do we want to the name of the routine to be bit more intuitive ? Perhaps 
SplitPathIntoSegments (inline with JoinPathSegments) ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ia664708a09493923abbf2ff4e56e3d49c62cf97e
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Mike Percy 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Lars Volker 
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1741: Make MiniCluster and ExternalMiniCluster follow one semantic for Restart

2017-01-04 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Alexey Serbin,

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

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

to review the following change.

Change subject: KUDU-1741: Make MiniCluster and ExternalMiniCluster follow one 
semantic for Restart
..

KUDU-1741: Make MiniCluster and ExternalMiniCluster follow one semantic for 
Restart

MiniCluster and ExternalMiniCluster utility classes follow opposite
semantics for restart. For eg, ExternalMiniCluster::Restart() expects
that all the nodes must be shutdown before we do restart, whereas
MiniCluster::Restart() expects that all servers are up and running
so that it can internally execute shutdown and start in that order.
This patch changes the semantics of MiniCluster::Restart() to follow
that of ExternalMiniCluster::Restart() to keep the consistent and
the existing tests are modified to follow the new semantics of
MiniCluster::Restart().

Change-Id: Iad33b7c46bfca3f277ccbca7d0420272f06a6633
---
M src/kudu/client/client-test.cc
M src/kudu/integration-tests/alter_table-test.cc
M src/kudu/integration-tests/create-table-stress-test.cc
M src/kudu/integration-tests/fuzz-itest.cc
M src/kudu/integration-tests/registration-test.cc
M src/kudu/master/master-test.cc
M src/kudu/master/mini_master.cc
M src/kudu/master/mini_master.h
M src/kudu/tserver/mini_tablet_server.cc
9 files changed, 17 insertions(+), 10 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Iad33b7c46bfca3f277ccbca7d0420272f06a6633
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: Mike Percy 


[kudu-CR] Fix flakiness in ToolTest.TestRemoteReplicaCopy

2017-01-03 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: Fix flakiness in ToolTest.TestRemoteReplicaCopy
..


Patch Set 1: Code-Review+1

Thanks Todd for fixing this, ship it from my side.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I740f6b469561a17792a315caf4a10c2627b55149
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Todd Lipcon 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-HasComments: No


[kudu-CR] KUDU-1776: Fix "kudu remote replica copy" connecting to wildcard address

2016-12-09 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1776: Fix "kudu remote_replica copy" connecting to 
wildcard address
..


Patch Set 6:

(9 comments)

http://gerrit.cloudera.org:8080/#/c/5378/6/src/kudu/integration-tests/external_mini_cluster.h
File src/kudu/integration-tests/external_mini_cluster.h:

PS6, Line 82: as binding ip 
> nit: "as the IP to bind to"
Done


PS6, Line 109: UNIQUE_LOOPBACK
> UNIQUE_LOOPBACK on Linux, LOOPBACK on macOS.
Done


Line 115:   BindMode bind_mode_;
> This is a struct, so per the style guide this member shouldn't have a taili
good catch, done.


Line 419:   virtual void set_rpc_bind_address(std::string address) = 0;
> Why make this a virtual public method when it is only ever called by constr
Good point,done.


Line 420:   virtual const std::string& get_rpc_bind_address() const = 0;
> Why make this a virtual public method? Why not just a protected variable?
Done


Line 503:   virtual void set_rpc_bind_address(std::string host) override {
> Why override these methods when they are verbatim the same for both impleme
yeah, defeats the the purpose altogether, i moved them to base class.


Line 536:   virtual void set_rpc_bind_address(std::string host) override {
> And these...
Done


http://gerrit.cloudera.org:8080/#/c/5378/6/src/kudu/tools/kudu-tool-test.cc
File src/kudu/tools/kudu-tool-test.cc:

Line 232:   ExternalMiniClusterOptions opts_;
> nit: maybe name this cluster_opts_
Done


Line 964:   opts_.bind_mode_ = ExternalMiniClusterOptions::WILDCARD;
> Please add a comment noting that this line is important in order to provide
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1608: Catalog manager can stop retrying DeleteTablet upon fatal errors

2016-12-09 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1608: Catalog manager can stop retrying DeleteTablet upon 
fatal errors
..


Patch Set 3:

> (1 comment)
 > 
 > Looks good. As we discussed, please try to build tests for this
 > using tserver-side metrics to verify that DeleteTablet() RPCs
 > aren't received once the master has figured out that it should
 > stop.
 > 
 > If anyone else has suggestions on how Dinesh could test this,
 > please chime in!

Hi Adar/Mike, TFTR here, things changed slightly over past couple of days 
around  other patches related to this, hence putting this on hold at the 
moment. I will be addressing comments on these once I figure out the other 
patches related to these error codes are in good shape, so please bear with me.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id45f07667b6e62ce4814acfdf931dea2af4332d1
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: No


[kudu-CR] KUDU-1776: Fix "kudu remote replica copy" connecting to wildcard address

2016-12-09 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Todd Lipcon, Kudu Jenkins,

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

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

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

Change subject: KUDU-1776: Fix "kudu remote_replica copy" connecting to 
wildcard address
..

KUDU-1776: Fix "kudu remote_replica copy" connecting to wildcard address

"remote_replica copy" asks the source server for its bound address,
but that address could be wildcard (0.0.0.0) in a real cluster,
and if same address is stuffed in RPC, it fails the validation
at the source tablet server:

tablet_id: "834e7673f32e4802bdf26f9fff9162ef"
copy_peer_uuid: "893679e1b1dd4342bf1fb8f058ec2789"
copy_peer_addr {
host: "0.0.0.0"
port: 7050
}
UNKNOWN_ERROR: Invalid wildcard address to tablet copy from: 0.0.0.0 (resolved 
to 0.0.0.0)

Fix: Tool remote_replica calls GetStatus RPC on a remote server which is
expected to return host/port information belonging to remote RPC server.
However, if the remote RPC was bound to wildcard ip address, the returned
host ip contains 0.0.0.0. This patch adds HostPortFromSockaddrReplaceWildcard
which helps to replace the wildcard with hostname of the remote server.

Testing: Patch makes few changes to external_mini_cluster such that it
supports binding to wildcard ip now. This is coupled with few consolidations
around bind_rpc_address_ fields which are shared between masters and
tablet servers, hence moved that up in the class hierarchy.

Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
---
M src/kudu/integration-tests/external_mini_cluster.cc
M src/kudu/integration-tests/external_mini_cluster.h
M src/kudu/integration-tests/log-rolling-itest.cc
M src/kudu/server/server_base.cc
M src/kudu/tools/kudu-tool-test.cc
5 files changed, 97 insertions(+), 41 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/78/5378/6
-- 
To view, visit http://gerrit.cloudera.org:8080/5378
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] KUDU-1798: [env] File manager broken on OS X

2016-12-09 Thread Dinesh Bhat (Code Review)
Hello Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: KUDU-1798: [env] File manager broken on OS X
..

KUDU-1798: [env] File manager broken on OS X

Env::IncreaseOpenFileLimit() relies upon few system calls
which behave slightly differently on OS X than on linux.

setrlimit() returns EINVAL if specified limit is RLIM_INFINITY.
getrlimit() happen to return rlim_max=RLIM_INFINITY on OS X
because apparently OS X doesn't enforce the open file limit.
So if we try to bump up the current limit to max limit,
it returns EINVAL on OS X. This patch addresses this issue
with a workaround aimed towards OS X only. Here we issue
sysctlbyname("kern.maxfilesperproc") to fetch the underlying
limit imposed by the OS X kernel to set that as rlim_max value.

OS X 10.x releases seem to support max 10240 open fds
per process. Also added a small test to verify this change.

Change-Id: Idb18b4e52ad80f89f61a92898c50c479643c12ec
---
M src/kudu/util/env-test.cc
M src/kudu/util/env_posix.cc
2 files changed, 31 insertions(+), 0 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Idb18b4e52ad80f89f61a92898c50c479643c12ec
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Kudu Jenkins


[kudu-CR] [env] File manager broken on OS X due to getrlimit returning RLIM INFINITY

2016-12-09 Thread Dinesh Bhat (Code Review)
Hello Adar Dembo,

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

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

to review the following change.

Change subject: [env] File manager broken on OS X due to getrlimit returning 
RLIM_INFINITY
..

[env] File manager broken on OS X due to getrlimit returning RLIM_INFINITY

Env::IncreaseOpenFileLimit() relies upon few system calls
which behave slightly differently on OS X than on linux.

setrlimit() returns EINVAL if specified limit is RLIM_INFINITY.
getrlimit() happen to return rlim_max=RLIM_INFINITY on OS X
because apparently OS X doesn't enforce the open file limit.
So if we try to bump up the current limit to max limit,
it returns EINVAL on OS X. This patch addresses this issue
with a workaround aimed towards OS X only. Here we issue
sysctlbyname("kern.maxfilesperproc") to fetch the underlying
limit imposed by the OS X kernel to set that as rlim_max value.

OS X 10.x releases seem to support max 10240 open fds
per process. Also added a small test to verify this change.

Change-Id: Idb18b4e52ad80f89f61a92898c50c479643c12ec
---
M src/kudu/util/env-test.cc
M src/kudu/util/env_posix.cc
2 files changed, 32 insertions(+), 0 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Idb18b4e52ad80f89f61a92898c50c479643c12ec
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 


[kudu-CR] KUDU-1776: Fix "kudu remote replica copy" connecting to wildcard address

2016-12-08 Thread Dinesh Bhat (Code Review)
Hello Todd Lipcon, Kudu Jenkins,

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

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

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

Change subject: KUDU-1776: Fix "kudu remote_replica copy" connecting to 
wildcard address
..

KUDU-1776: Fix "kudu remote_replica copy" connecting to wildcard address

"remote_replica copy" asks the source server for its bound address,
but that address could be wildcard (0.0.0.0) in a real cluster,
and if same address is stuffed in RPC, it fails the validation
at the source tablet server:

tablet_id: "834e7673f32e4802bdf26f9fff9162ef"
copy_peer_uuid: "893679e1b1dd4342bf1fb8f058ec2789"
copy_peer_addr {
host: "0.0.0.0"
port: 7050
}
UNKNOWN_ERROR: Invalid wildcard address to tablet copy from: 0.0.0.0 (resolved 
to 0.0.0.0)

Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
---
M src/kudu/integration-tests/external_mini_cluster.cc
M src/kudu/integration-tests/external_mini_cluster.h
M src/kudu/integration-tests/log-rolling-itest.cc
M src/kudu/server/server_base.cc
M src/kudu/tools/kudu-tool-test.cc
5 files changed, 97 insertions(+), 41 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/78/5378/5
-- 
To view, visit http://gerrit.cloudera.org:8080/5378
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] KUDU-1776: Fix "kudu remote replica copy" connecting to wildcard address

2016-12-08 Thread Dinesh Bhat (Code Review)
Hello Todd Lipcon, Kudu Jenkins,

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

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

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

Change subject: KUDU-1776: Fix "kudu remote_replica copy" connecting to 
wildcard address
..

KUDU-1776: Fix "kudu remote_replica copy" connecting to wildcard address

"remote_replica copy" asks the source server for its bound address,
but that address could be wildcard (0.0.0.0) in a real cluster,
and if same address is stuffed in RPC, it fails the validation
at the source tablet server:

tablet_id: "834e7673f32e4802bdf26f9fff9162ef"
copy_peer_uuid: "893679e1b1dd4342bf1fb8f058ec2789"
copy_peer_addr {
host: "0.0.0.0"
port: 7050
}
UNKNOWN_ERROR: Invalid wildcard address to tablet copy from: 0.0.0.0 (resolved 
to 0.0.0.0)

Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
---
M src/kudu/integration-tests/external_mini_cluster.cc
M src/kudu/integration-tests/external_mini_cluster.h
M src/kudu/server/server_base.cc
M src/kudu/tools/kudu-tool-test.cc
4 files changed, 98 insertions(+), 42 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] KUDU-1776: Fix "kudu remote replica copy" connecting to wildcard address

2016-12-08 Thread Dinesh Bhat (Code Review)
Hello Todd Lipcon, Kudu Jenkins,

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

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

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

Change subject: KUDU-1776: Fix "kudu remote_replica copy" connecting to 
wildcard address
..

KUDU-1776: Fix "kudu remote_replica copy" connecting to wildcard address

"remote_replica copy" asks the source server for its bound address,
but that address could be wildcard (0.0.0.0) in a real cluster,
and if same address is stuffed in RPC, it fails the validation
at the source tablet server:

tablet_id: "834e7673f32e4802bdf26f9fff9162ef"
copy_peer_uuid: "893679e1b1dd4342bf1fb8f058ec2789"
copy_peer_addr {
host: "0.0.0.0"
port: 7050
}
UNKNOWN_ERROR: Invalid wildcard address to tablet copy from: 0.0.0.0 (resolved 
to 0.0.0.0)

Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
---
M src/kudu/integration-tests/external_mini_cluster.cc
M src/kudu/integration-tests/external_mini_cluster.h
M src/kudu/server/server_base.cc
M src/kudu/tools/kudu-tool-test.cc
4 files changed, 90 insertions(+), 41 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] KUDU-1753 [tablet service] continue scan request on deleted tablet

2016-12-07 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1753 [tablet_service] continue scan request on deleted 
tablet
..


Patch Set 5:

> > Alexey, just thinking out loud here:  Tablet not found in the
 > JIRA
 > > corresponds to tablet tombstoned state in the tablet server,
 > which
 > > means any subsequent scan on that server is not expected to yield
 > > any data, right ? This fix aims to continue the scan until it
 > > fetches whatever is cached so far. So, I am curious if this
 > > guarantees to fix the underlying issue though, because the
 > > GetTabletRef can fail anywhere during the scan and not just under
 > > HandleContinueScanRequest, right ?
 > 
 > The original issue described in JIRA is 'Illegal state: Tablet is
 > not running'.  The hypothesis about the 'Tablet not found'  -- is
 > just a hypothesis that it's related to that.

I discovered recently that TABLET_NOT_RUNNING error code embodies tablets in 
bootstrapped state, and also the tablets failed to come up on the node for 
whatever reasons. There is also another distinct error code TABLET_NOT_FOUND 
for deleted tablets and this fix is aimed to address the latter situation, 
right ? The description in the JIRA seemed to indicate this situation may arise 
due to former error code too. I am just curious if the fix and the related test 
have taken former error code into account.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ica48c52a81862f47a9245003915d18be411bf8b1
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Matthew Jacobs 
Gerrit-Reviewer: Mike Percy 
Gerrit-HasComments: No


[kudu-CR] KUDU-1753 [delete table-test] deleted-while-scanned test

2016-12-07 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1753 [delete_table-test] deleted-while-scanned test
..


Patch Set 6: Code-Review+1

(5 comments)

I will defer +2 to David, LGTM, few nits.

http://gerrit.cloudera.org:8080/#/c/5345/2/src/kudu/integration-tests/delete_table-test.cc
File src/kudu/integration-tests/delete_table-test.cc:

Line 1320: // Setup batch size to be small enough to guarantee the scan 
will not fetch
> Another piece of information which helps to understand that impala was issu
I see, thank you for explaining.


PS2, Line 1324: ASSERT_TRUE(scanner.HasMoreRows());
  : KuduScanBatch batch;
  : ASSERT_OK(scanner.NextBatch());
  : size_t row_count = batch.NumRows();
  : 
> I didn't find better alternative for this simple check among those methods.
Since you are checking for existence of tablets on the cluster, and not 
specific TS, this seems fine. Personally, I would have preferred to go to 
master with RPC for eg, itest::GetTableLocations() or something like that 
because it fully reflects the table's state on the cluster. inspect_ reflects 
only on-disk contents on the servers, but the cleanup may not be complete by 
then.


http://gerrit.cloudera.org:8080/#/c/5345/6/src/kudu/integration-tests/delete_table-test.cc
File src/kudu/integration-tests/delete_table-test.cc:

PS6, Line 1316: shared_ptr
Nit: we could use sp::shared_ptr here to indicate that it's not C++ shared_ptr 
and keep "shared_ptr" reserved for C++ standard library one.


PS6, Line 1331: ON_ERROR_DO_NOT_DUMP_STACKS
Do we want to put a comment here as to why we wanted to avoid dumping stacks ?


PS6, Line 1357: StopCluster
Nit: Wasn't AssertNoCrashes above sufficient ? Or is this added as a good 
practice ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I983e68862b5535f2f95eedd41850a8a88e95e69c
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Matthew Jacobs 
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] [consensus] KUDU-1407 replica is not evcited when TABLET NOT RUNNING

2016-12-07 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [consensus] KUDU-1407 replica is not evcited when 
TABLET_NOT_RUNNING
..


Patch Set 3:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/5352/3//COMMIT_MSG
Commit Message:

PS3, Line 7: evcited
> haven't looked at this code yet, but I can't tell from the commit message w
Todd, yeah this fix aimed to evict the replicas returning TABLET_NOT_RUNNING. I 
had looked at your comments on the JIRA before attempting a fix on this. I may 
have misunderstood the bug little bit here. My understanding was that tablet 
server would return TABLET_NOT_RUNNING error once after tablet has reached a 
"steady" state of FAILED or some other error state. It appears like we return 
TABLET_NOT_RUNNING when tablet is anything other than RUNNING, so bootstrapping 
would fall under this too. We definitely don't want to evict when tablet is 
bootstrapping (until we hit a timeout ? ).

As a solution, a) would it be better to introduce another response code here ? 
something like TABLET_BOOTSTRAPPING which is an indication for consensus to not 
to evict such replicas from config until some timeout, and treat 
TABLET_NOT_RUNNING as a fatal error ? 
b) Or else we could treat TABLET_NOT_RUNNING as a transient error code and 
after 300 secs (sufficient window to copy large tablets given the improvements 
in upcoming tablet copy workflows ?) we could evict the replica from config.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I554ad61f25a7de78eda60fd50228e0f015c1b625
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] KUDU-921. tablet copy: Make the StartTabletCopy() RPC async

2016-12-06 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-921. tablet copy: Make the StartTabletCopy() RPC async
..


Patch Set 6:

(3 comments)

LGTM, couple of nits.

http://gerrit.cloudera.org:8080/#/c/5045/6/src/kudu/tserver/ts_tablet_manager.cc
File src/kudu/tserver/ts_tablet_manager.cc:

PS6, Line 62: 10
Was this number picked considering a balanced (not overloaded) node may not 
have more than 20-30 tablets ? Why not use NumCPUS() instead ?


Line 412: #define CALLBACK_AND_RETURN(status) \
> If we do that, we also need to include cb too. I just thought it looked mes
Yeah, my concern was a) it obligates the caller to follow inflexible naming 
conventions and also its non-intuitive b) this leaves the programmer making 
mistakes like not setting the error_code before this macro (well, that room 
exists even when we include it as macro arg). If this MACRO isn't used outside 
of this context, I don't feel strongly about changing it.


PS6, Line 824: Cancel all outstanding tablet copy tasks
Am I correct in understanding that current code will wait till all in-progress 
copy threads complete vs the cancelling will pre-empt those threads and help 
with eagerly cleanups ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I95c63f2bfd6762487862efbdba9cb3676112
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Mike Percy 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1785. Fix potential crash in TabletCopySourceSession

2016-12-06 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1785. Fix potential crash in TabletCopySourceSession
..


Patch Set 4: Code-Review+1

(4 comments)

http://gerrit.cloudera.org:8080/#/c/5363/4/src/kudu/integration-tests/tablet_copy_client_session-itest.cc
File src/kudu/integration-tests/tablet_copy_client_session-itest.cc:

Line 44:int num_tablets);
Nit: num_tablets defaults to 1 ?


Line 46:   const MonoDelta kTimeout = MonoDelta::FromSeconds(30);
Nit: Do we want to move the timeout to individual tests since future tests may 
have different timeout requirements ?


PS4, Line 121: WaitUntilTabletRunning
Do we want to wait for tablet copy or just wait till we get a 
Status::IllegalState("Tablet is still bootstrapping") ? If it's the former we 
might want to update the comment L89 which says we expect simple failure out of 
this test. Or perhaps restarting the source tserver while some lambda threads 
continuously attempting StartTabletCopy might give us more predictable results ?


PS4, Line 122: DeleteTablet
Why is this necessary for testing this fix ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I6f3ec35885dbf1a81c23ac10b1c9556dfddbd4b7
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Mike Percy 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] [consensus] KUDU-1613: Fix replica eviction failure for WRONG SERVER UUID

2016-12-06 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [consensus] KUDU-1613: Fix replica eviction failure for 
WRONG_SERVER_UUID
..

[consensus] KUDU-1613: Fix replica eviction failure for WRONG_SERVER_UUID

Fixes replica eviction from leader where a follower returning
WRONG_SERVER_UUID (perhaps due to a disks-removed situation) is never
evicted from the consensus config.
If a tablet server changes its uuid, it starts responding with
Status::InvalidArgument for most of the RPCs along with the error
code WRONG_SERVER_UUID in the response. Raft consensus can treat this
error code as fatal since this means that all the replicas on that
node has become unavailable and that situation is irrecoverable.
Consensus should evict such nodes from the tablet config and try
to re-replicate if needed. Same tablet server carrying a new uuid
could be one of the candidates considered for re-replication.

Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
---
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/integration-tests/raft_consensus-itest.cc
5 files changed, 191 insertions(+), 46 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 7
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [consensus] KUDU-1613: Fix replica eviction failure for WRONG SERVER UUID

2016-12-06 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [consensus] KUDU-1613: Fix replica eviction failure for 
WRONG_SERVER_UUID
..


Patch Set 5:

(7 comments)

http://gerrit.cloudera.org:8080/#/c/5111/5//COMMIT_MSG
Commit Message:

Line 9: Fixes replica eviction from leader where a follower returning
> Can you please explain your approach to fixing this issue in the commit mes
Done, could you see updated version would suffice ?


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

Line 238:   switch (ps.response) {
> Maybe add a default case that calls LOG(FATAL) on the (unknown) response ty
Done


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

PS5, Line 140: respond with appropriate response status.
> Nit: technically, this method does not "respond" (in the sense of sending a
yeah, good catch. fixed.


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

Line 816: // Verify that tablet copy is triggered when peer responds with
> Why do we want tablet copy to be triggered when the peer responds that it i
I copied this line-by-line from TestTriggerTabletCopyIfTabletNotFound above. In 
this case, WRONG_SERVER_UUID almost always means that for some reason tablet 
server went through reinitialization, which means  all the tablets become 
inaccessible due to uuid mismatch for all RPCs via CheckUuidMatchOrRespond 
after the reinitialization. In some way, this is the same situation as 
TABLET_NOT_FOUND but impact is more because it affected all replicas on that 
node. As for why tablet copy is necessary, I think it's probably more relevant 
when the tablet becomes under-replicated. I added this test really for that 
1-line change in consensus_queue.cc.


http://gerrit.cloudera.org:8080/#/c/5111/5/src/kudu/integration-tests/raft_consensus-itest.cc
File src/kudu/integration-tests/raft_consensus-itest.cc:

Line 2770
> Why was this comment removed? If it's no longer true, shouldn't the entire 
yeah, this line of code(and comment was supposed to be removed in next patch). 
I removed both comment+code in this patch itself initially, but added only code 
back realizing that this code is testing TABLET_NOT_RUNNING situation(forgot to 
add comment back). I think I am going to leave it without comments for now 
because my my next patch laid on top of this is removing both anyways, and it's 
a somewhat messy rebase situation otherwise.


PS5, Line 2782: // These tests exhibit the replica eviction behaviors when a 
follower
  : // returns WRONG_SERVER_UUID error code. Tests verify that 
followers
  : // returning these error codes are evicted from consensus after 
a
  : // specified --follower_unavailable_considered_failed_sec.
> Nit: this is talking about tests in the plural, but it's just a single test
done, yeah I think I forgot to change comments when I split one patch into 
multiple patches.


Line 2795:   int leader_indx = -1;
> Nit: conventionally, we use 'idx' as an infix for naming an index, not 'ind
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [consensus] KUDU-1613: Fix replica eviction failure for WRONG SERVER UUID

2016-12-06 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [consensus] KUDU-1613: Fix replica eviction failure for 
WRONG_SERVER_UUID
..

[consensus] KUDU-1613: Fix replica eviction failure for WRONG_SERVER_UUID

Fixes replica eviction from leader where a follower returning
WRONG_SERVER_UUID (perhaps due to a disks-removed situation) is never
evicted from the consensus config.
If a tablet server changes its uuid, it starts responding with
Status::InvalidArgument for most of the RPCs along with the error
code WRONG_SERVER_UUID in the response. Raft consensus can treat this
error code as fatal since this means that all the replicas on that
node has become unavailable and that situation is irrecoverable.
Consensus should evict such nodes from the tablet config and try
to re-replicate if needed. Same tablet server carrying a new uuid
could be one of the candidates considered for re-replication.

Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
---
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/integration-tests/raft_consensus-itest.cc
5 files changed, 189 insertions(+), 47 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] KUDU-1776: Fix "kudu remote replica copy" binding to wildcard address

2016-12-06 Thread Dinesh Bhat (Code Review)
Hello Todd Lipcon,

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

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

to review the following change.

Change subject: KUDU-1776: Fix "kudu remote_replica copy" binding to wildcard 
address
..

KUDU-1776: Fix "kudu remote_replica copy" binding to wildcard address

"remote_replica copy" asks the source server for its bound address,
but that address could be wildcard (0.0.0.0) in a real cluster,
and if same address is stuffed in RPC, it fails the validation
at the source tablet server:

tablet_id: "834e7673f32e4802bdf26f9fff9162ef"
copy_peer_uuid: "893679e1b1dd4342bf1fb8f058ec2789"
copy_peer_addr {
host: "0.0.0.0"
port: 7050
}
UNKNOWN_ERROR: Invalid wildcard address to tablet copy from: 0.0.0.0 (resolved 
to 0.0.0.0)

Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
---
M src/kudu/integration-tests/external_mini_cluster.cc
M src/kudu/integration-tests/external_mini_cluster.h
M src/kudu/server/server_base.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/util/net/net_util.cc
M src/kudu/util/net/net_util.h
6 files changed, 28 insertions(+), 8 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Ie5d0a37b39a3774caab5b5d8dba3d9750bf5f21f
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] KUDU-1608: Catalog manager can stop retrying DeleteTablet upon fatal errors

2016-12-04 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo,

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

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

to review the following change.

Change subject: KUDU-1608: Catalog manager can stop retrying DeleteTablet upon 
fatal errors
..

KUDU-1608: Catalog manager can stop retrying DeleteTablet upon fatal errors

Master continuously retries DeleteTablet RPC for few non-retriable
error codes like WRONG_SERVER_UUID or TABLET_NOT_RUNNING both of
which indicate that tablet ceases to exist on that tserver.
Such errors can be deemed fatal and DeleteTablet need not be retried.

Change-Id: Id45f07667b6e62ce4814acfdf931dea2af4332d1
---
M src/kudu/master/catalog_manager.cc
1 file changed, 22 insertions(+), 11 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Id45f07667b6e62ce4814acfdf931dea2af4332d1
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Mike Percy 


[kudu-CR] [consensus] KUDU-1407 replica is not evcited when TABLET NOT RUNNING

2016-12-04 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [consensus] KUDU-1407 replica is not evcited when 
TABLET_NOT_RUNNING
..

[consensus] KUDU-1407 replica is not evcited when TABLET_NOT_RUNNING

Fixes replica eviction failure where a follower returning
TABLET_NOT_RUNNING is not evicted from the consensus config.

Change-Id: I554ad61f25a7de78eda60fd50228e0f015c1b625
---
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tserver/tablet_server-test-base.h
M src/kudu/tserver/tablet_server_test_util.cc
M src/kudu/tserver/ts_tablet_manager.cc
M src/kudu/tserver/ts_tablet_manager.h
8 files changed, 97 insertions(+), 34 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I554ad61f25a7de78eda60fd50228e0f015c1b625
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] [consensus] KUDU-1407 replica is not evcited when TABLET NOT RUNNING

2016-12-04 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy,

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

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

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

Change subject: [consensus] KUDU-1407 replica is not evcited when 
TABLET_NOT_RUNNING
..

[consensus] KUDU-1407 replica is not evcited when TABLET_NOT_RUNNING

Fixes replica eviction failure where a follower returning
TABLET_NOT_RUNNING is not evicted from the consensus config.

Change-Id: I554ad61f25a7de78eda60fd50228e0f015c1b625
---
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tserver/tablet_server-test-base.h
M src/kudu/tserver/tablet_server_test_util.cc
M src/kudu/tserver/ts_tablet_manager.cc
M src/kudu/tserver/ts_tablet_manager.h
8 files changed, 95 insertions(+), 33 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I554ad61f25a7de78eda60fd50228e0f015c1b625
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot


[kudu-CR] [consensus] KUDU-1613: Fix replica eviction failure for WRONG SERVER UUID

2016-12-04 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [consensus] KUDU-1613: Fix replica eviction failure for 
WRONG_SERVER_UUID
..


Patch Set 5:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/5111/3/src/kudu/master/catalog_manager.cc
File src/kudu/master/catalog_manager.cc:

Line 2609:  const scoped_refptr& tablet,
> Would be better to use the standard prefix T  P  for 
This change is moved to different patch, addressing it there.


http://gerrit.cloudera.org:8080/#/c/5111/3/src/kudu/tserver/ts_tablet_manager.h
File src/kudu/tserver/ts_tablet_manager.h:

Line 213:   // Open a tablet meta from the local file system by loading its 
superblock.
> Why optional<>* ? New param needs doc
Since this change moved to a different patch, I have addressed it there.
http://gerrit.cloudera.org:8080/5352


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [consensus] KUDU-1407 replica is not evcited when TABLET NOT RUNNING

2016-12-04 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy,

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

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

to review the following change.

Change subject: [consensus] KUDU-1407 replica is not evcited when 
TABLET_NOT_RUNNING
..

[consensus] KUDU-1407 replica is not evcited when TABLET_NOT_RUNNING

Fixes replica eviction failure where a follower returning
TABLET_NOT_RUNNING is not evicted from the consensus config.

Change-Id: I554ad61f25a7de78eda60fd50228e0f015c1b625
---
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tserver/tablet_server-test-base.h
M src/kudu/tserver/tablet_server_test_util.cc
M src/kudu/tserver/ts_tablet_manager.cc
M src/kudu/tserver/ts_tablet_manager.h
8 files changed, 91 insertions(+), 31 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I554ad61f25a7de78eda60fd50228e0f015c1b625
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Mike Percy 


[kudu-CR] [consensus] KUDU-1613: Fix replica eviction failure for WRONG SERVER UUID

2016-12-03 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: [consensus] KUDU-1613: Fix replica eviction failure for 
WRONG_SERVER_UUID
..

[consensus] KUDU-1613: Fix replica eviction failure for WRONG_SERVER_UUID

Fixes replica eviction from leader where a follower returning
WRONG_SERVER_UUID perhaps due to a disks-removed situation and
the follower is never evicted from the consensus config.

Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
---
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/integration-tests/raft_consensus-itest.cc
5 files changed, 182 insertions(+), 46 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [consensus] KUDU-1613: Fix replica eviction failure for WRONG SERVER UUID

2016-12-03 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Kudu Jenkins,

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

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

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

Change subject: [consensus] KUDU-1613: Fix replica eviction failure for 
WRONG_SERVER_UUID
..

[consensus] KUDU-1613: Fix replica eviction failure for WRONG_SERVER_UUID

Fixes replica eviction from leader where a follower returning
WRONG_SERVER_UUID perhaps due to a disks-removed situation and
the follower is never evicted from the consensus config.

Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
---
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/integration-tests/raft_consensus-itest.cc
5 files changed, 182 insertions(+), 50 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [consensus] KUDU-1613: Fix replica eviction failure for WRONG SERVER UUID

2016-12-03 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [consensus] KUDU-1613: Fix replica eviction failure for 
WRONG_SERVER_UUID
..


Patch Set 3:

(4 comments)

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

Line 198: 
> nit: spurious line
Done


Line 241:   // Fallthrough intended.
> use FALLTHROUGH_INTENDED macro instead
Done


Line 256:   }
> better to break at the end of the case
Done


http://gerrit.cloudera.org:8080/#/c/5111/3/src/kudu/tserver/tablet_server_test_util.cc
File src/kudu/tserver/tablet_server_test_util.cc:

Line 27: DEFINE_int32(num_updater_threads, 1, "Number of updating threads to 
launch");
> Why put this stuff in this file? Maybe you need to create a new tablet_serv
yeah that was my initial thought, but I decided to chuck a new file for few 
variable definitions which need linking from other modules. This source file 
seemed like a good candidate to hold them since it's part of same library 
against which other modules link.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [i-tests] TestWorkload::set num tablets() accepts 1

2016-12-03 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [i-tests] TestWorkload::set_num_tablets() accepts 1
..


Patch Set 1: Code-Review+1

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaf9d0e06550fffdd6a04f6bfba53fe873fb33749
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-HasComments: No


[kudu-CR] KUDU-1753 [delete table-test] deleted-while-scanned test

2016-12-02 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1753 [delete_table-test] deleted-while-scanned test
..


Patch Set 2:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/5345/1//COMMIT_MSG
Commit Message:

PS1, Line 10: untill
typo


http://gerrit.cloudera.org:8080/#/c/5345/2/src/kudu/integration-tests/delete_table-test.cc
File src/kudu/integration-tests/delete_table-test.cc:

Line 1320: NO_FATALS(DeleteTable(w.table_name(), 
ON_ERROR_DO_NOT_DUMP_STACKS));
Hmmm, I haven't looked at scan codepaths, but at high level I would guess 
deleting a tablet while scan is in progress could be a different reaction from 
a client than deleting a table altogether, former one could go through retry on 
different tablet server, etc ? This may not give the intended test coverage for 
KUDU-1753 if that happened to be true.


PS2, Line 1324: vector tablets;
  : do {
  :   SleepFor(MonoDelta::FromMilliseconds(256));
  :   tablets = inspect_->ListTablets();
  : } while (!tablets.empty());
There are bunch of utils in cluster_itest_util.cc, you might want to use 
readily available routines like WaitForNumTabletsOnTS or 
WaitForTabletDataStateOnTS.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I983e68862b5535f2f95eedd41850a8a88e95e69c
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Matthew Jacobs 
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-HasComments: Yes


[kudu-CR] KUDU-1753 [tablet service] continue scan request on deleted tablet

2016-12-02 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-1753 [tablet_service] continue scan request on deleted 
tablet
..


Patch Set 2:

Alexey, just thinking out loud here:  Tablet not found in the JIRA corresponds 
to tablet tombstoned state in the tablet server, which means any subsequent 
scan on that server is not expected to yield any data, right ? This fix aims to 
continue the scan until it fetches whatever is cached so far. So, I am curious 
if this guarantees to fix the underlying issue though, because the GetTabletRef 
can fail anywhere during the scan and not just under HandleContinueScanRequest, 
right ?

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ica48c52a81862f47a9245003915d18be411bf8b1
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Matthew Jacobs 
Gerrit-Reviewer: Mike Percy 
Gerrit-HasComments: No


[kudu-CR] [tools] Tombstone the tablet with "local replica delete"

2016-12-02 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [tools] Tombstone the tablet with "local_replica delete"
..


Patch Set 4:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/5191/4/src/kudu/tools/tool_action_local_replica.cc
File src/kudu/tools/tool_action_local_replica.cc:

Line 789:   .Description("Tombstone the Kudu replica in the local 
filesystem")
> Sorry about this, but I changed the terminology to "tablet replica" in http
Done, didn't see those changes flying by. I must have been asleep. Manually 
rebased now.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [tools] Tombstone the tablet with "local replica delete"

2016-12-02 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [tools] Tombstone the tablet with "local_replica delete"
..

[tools] Tombstone the tablet with "local_replica delete"

This change makes the default action of 'local_replica delete'
tool to tombstone the tablet. Release 1.1 shipped the tool
with "--clean_unsafe" flag which let you delete the replica
from the disks. Deleting removes consensus metadata for the given
tablet and hence voilates Raft vote durability requirements.
Tombstoning on the other hand will preserve Raft votes for the
specfied tablet on the local node, hence is a safer choice
to be the default action for this tool.

This adds the support for tombstone action for this tool
which was unsupported in previous release.

Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
---
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_local_replica.cc
2 files changed, 145 insertions(+), 54 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/91/5191/6
-- 
To view, visit http://gerrit.cloudera.org:8080/5191
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [tools] Tombstone the tablet with "local replica delete"

2016-12-01 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [tools] Tombstone the tablet with "local_replica delete"
..

[tools] Tombstone the tablet with "local_replica delete"

This change makes the default action of 'local_replica delete'
tool to tombstone the tablet. Release 1.1 shipped the tool
with "--clean_unsafe" flag which let you delete the replica
from the disks. Deleting removes consensus metadata for the given
tablet and hence voilates Raft vote durability requirements.
Tombstoning on the other hand will preserve Raft votes for the
specfied tablet on the local node, hence is a safer choice
to be the default action for this tool.

This adds the support for tombstone action for this tool
which was unsupported in previous release.

Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
---
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_local_replica.cc
2 files changed, 145 insertions(+), 54 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/91/5191/5
-- 
To view, visit http://gerrit.cloudera.org:8080/5191
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [tools] Tombstone the tablet with "local replica delete"

2016-12-01 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [tools] Tombstone the tablet with "local_replica delete"
..


Patch Set 4:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/5191/3/src/kudu/tools/tool_action_local_replica.cc
File src/kudu/tools/tool_action_local_replica.cc:

PS3, Line 185: ")
> The way to appease tidy-bot is like this:
yeah, unless you feel strongly about it, I decided not to special case 
Status::Corruption upon second thoughts. Current prefix of 'Error in log 
segment' should cover any present/future error returned from ReadNextEntry().


http://gerrit.cloudera.org:8080/#/c/5191/4/src/kudu/tools/tool_action_local_replica.cc
File src/kudu/tools/tool_action_local_replica.cc:

Line 789:   .Description("Tombstone the Kudu replica in the local 
filesystem")
> How about: "Delete a tablet replica from the local system. By default, leav
Done, to be consistent with other help strings, I haven't used 'tablet 
replica', but sticking to 'Kudu replica'.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [ts tablet manager] simplified WaitForAllBootstrapsToFinish

2016-12-01 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [ts_tablet_manager] simplified WaitForAllBootstrapsToFinish
..


Patch Set 1: Code-Review+1

(1 comment)

http://gerrit.cloudera.org:8080/#/c/5301/1/src/kudu/tablet/tablet_peer.cc
File src/kudu/tablet/tablet_peer.cc:

Line 390:   CHECK(!error.ok());
> That's fine.  This check just enforces passing non-OK status code as a para
Sorry, I think I misread the code earlier, got confused between state_ and 
status here. +1 from me.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3e49b7741f48e619a3d1b93d6bba65f0eb16d849
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-HasComments: Yes


[kudu-CR] KUDU-921. tablet copy: Make the StartTabletCopy() RPC async

2016-12-01 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: KUDU-921. tablet copy: Make the StartTabletCopy() RPC async
..


Patch Set 6: Code-Review+1

(3 comments)

http://gerrit.cloudera.org:8080/#/c/5045/6/src/kudu/integration-tests/tablet_copy-itest.cc
File src/kudu/integration-tests/tablet_copy-itest.cc:

PS6, Line 949: StartTabletCopy
I wonder if multithreading this and running in parallel will make the 
probability of making num_service_unavailable > 0 high.


http://gerrit.cloudera.org:8080/#/c/5045/6/src/kudu/tserver/ts_tablet_manager.cc
File src/kudu/tserver/ts_tablet_manager.cc:

Line 412: #define CALLBACK_AND_RETURN(status) \
Do we want to include error_code as a macro argument ? Otherwise, IMHO it's bit 
confusing to restrict the caller to have the variable named 'error_code' from 
each callsite and could be bug-prone too.


PS6, Line 529: cb(Status::OK(), TabletServerErrorPB::UNKNOWN_ERROR)
Could we introduce a TabletServerErrorPB::OK to go along with such calls ? This 
being the success case, does this expect the caller to ignore the error_code ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I95c63f2bfd6762487862efbdba9cb3676112
Gerrit-PatchSet: 6
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Mike Percy 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [tools] Tombstone the tablet with "local replica delete"

2016-12-01 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [tools] Tombstone the tablet with "local_replica delete"
..

[tools] Tombstone the tablet with "local_replica delete"

This change makes the default action of 'local_replica delete'
tool to tombstone the tablet. Release 1.1 shipped the tool
with "--clean_unsafe" flag which let you delete the replica
from the disks. Deleting removes consensus metadata for the given
tablet and hence voilates Raft vote durability requirements.
Tombstoning on the other hand will preserve Raft votes for the
specfied tablet on the local node, hence is a safer choice
to be the default action for this tool.

This adds the support for tombstone action for this tool
which was unsupported in previous release.

Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
---
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_local_replica.cc
2 files changed, 144 insertions(+), 54 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [tools] Tombstone the tablet with "local replica delete"

2016-11-30 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [tools] Tombstone the tablet with "local_replica delete"
..

[tools] Tombstone the tablet with "local_replica delete"

This change makes the default action of 'local_replica delete'
tool to tombstone the tablet. Release 1.1 shipped the tool
with "--clean_unsafe" flag which let you delete the replica
from the disks. Deleting removes consensus metadata for the given
tablet and hence voilates Raft vote durability requirements.
Tombstoning on the other hand will preserve Raft votes for the
specfied tablet on the local node, hence is a safer choice
to be the default action for this tool.

This adds the support for tombstone action for this tool
which was unsupported in previous release.

Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
---
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_local_replica.cc
2 files changed, 148 insertions(+), 54 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [tools] Tombstone the tablet with "local replica delete"

2016-11-30 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [tools] Tombstone the tablet with "local_replica delete"
..


Patch Set 2:

(12 comments)

TFTR Adar/Mike, addressed comments, one response inline below:

http://gerrit.cloudera.org:8080/#/c/5191/1//COMMIT_MSG
Commit Message:

PS1, Line 9: This change makes the default action of 'local_replica delete'
   : tool to tombstone the tablet.
> I see. Can you reword this paragraph then, to make the (lack of a semantic)
Done


http://gerrit.cloudera.org:8080/#/c/5191/2/src/kudu/tools/kudu-tool-test.cc
File src/kudu/tools/kudu-tool-test.cc:

Line 1072: ASSERT_EQ(tablet_peers.size(), 1);
> nit: expected value comes first in the ASSERT_EQ macro
Done


Line 1117:   {
> nit: These extra braces seem superfluous here.
Done


Line 1120: ASSERT_EQ(tablet_peers.size(), 0);
> nit: reversed order of ASSERT_EQ arguments
Done


Line 1154: ASSERT_EQ(tablet_peers.size(), 1);
> reverse args
Done


Line 1156: ASSERT_TRUE(tablet_peers[0]->log() != nullptr);
> remove this line? seems over-defensive
Done


Line 1158: ASSERT_TRUE(last_logged_opid.IsInitialized());
> why this line? Maybe you mean
Naah, being extra defensive here too, it's not equal to MinimumOpID btw because 
we have generated some workload so OpId index is set to some sane integer value 
with last_logged_opid.


Line 1190: 
ASSERT_EQ(tablet_peers[0]->tablet_metadata()->tablet_data_state(),
> nit: order of ASSERT_EQ args here and below
Done


http://gerrit.cloudera.org:8080/#/c/5191/2/src/kudu/tools/tool_action_local_replica.cc
File src/kudu/tools/tool_action_local_replica.cc:

Line 165: Status FindLastCommittedOpId(FsManager* fs, const string& tablet_id,
> doc
Done


Line 166:  boost::optional* 
last_committed_opid) {
> I don't think boost::optional* should be used as an out-parameter her
Hmmm, couple of things I want to mention before I take your suggestion:

0) DeleteTabletData() at L327 takes an optional, and that was the 
original motive to use optional here.
1) This routine could return NotFound as well which we treat as success in the 
caller and proceed with Tombstoning.
2) This may fall under the second use case for 'optional' you mentioned above, 
no ? i.e in a situation where output may be set or unset.
3) If we don't want to keep 'optional' out param, we could either copy the OpId 
returned here in the caller or have a if/else combined with boost::none while 
calling DeleteTabletData().

So, I am kinda still favoring the current approach we have. Let me know.


PS2, Line 180: s = Status::Corruption(
 : Substitute("Corruption detected in log segment: $0", 
s.ToString()));
> I agree with Adar unless the existing error message isn't understandable. I
Done, yeah I wanted to be bit more verbose than the error string returned by 
ReadNextEntry().


Line 189: if (last_committed_opid->is_initialized()) return Status::OK();
> Just use a separate bool for this
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [ts tablet manager] simplified WaitForAllBootstrapsToFinish

2016-11-30 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [ts_tablet_manager] simplified WaitForAllBootstrapsToFinish
..


Patch Set 1:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/5301/1/src/kudu/tablet/tablet_peer.cc
File src/kudu/tablet/tablet_peer.cc:

Line 390:   CHECK(!error.ok());
At times we may want to use TabletPeer::SetFailed() from a MiniCluster test on 
a healthy peer. I would think that state_ is independent of the error so it's 
ok not to CHECK for error.ok() in this routine.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3e49b7741f48e619a3d1b93d6bba65f0eb16d849
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-HasComments: Yes


[kudu-CR] block manager: start using the file cache

2016-11-30 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: block manager: start using the file cache
..


Patch Set 4:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/5147/4/src/kudu/fs/log_block_manager.cc
File src/kudu/fs/log_block_manager.cc:

PS4, Line 366: OpenExistingRWFile
I am curious to know what's the challenge in driving the new file creation 
through the FileCache(assuming we add a support for that in FileCache) @L350 
instead of creating first and then opening through cache. I saw the COMMIT_MSG 
mentioning it too but didn't understand the reasons behind it.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ieeefd31eca340111bc535eac1f982290e7703a88
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Adar Dembo 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] util: add file cache

2016-11-30 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: util: add file cache
..


Patch Set 5:

(11 comments)

Excellent stuff Adar, took a first pass, still to see tests and deletion path. 
There are some minor comments and questions below..

http://gerrit.cloudera.org:8080/#/c/5146/5/src/kudu/util/cache.cc
File src/kudu/util/cache.cc:

PS5, Line 34: Override all cache implementations to use just one shard
Can we put a comment describing the motive for this flag ?


http://gerrit.cloudera.org:8080/#/c/5146/5/src/kudu/util/file_cache.cc
File src/kudu/util/file_cache.cc:

PS5, Line 57: > 3
nit: could we compare 'size == 4' here  ?


PS5, Line 62: *reinterpret_cast(s.mutable_data())" ?


PS5, Line 67: *reinterpret_cast
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: Dan Burkert 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [consensus] KUDU-1718: Fix few bugs around replica eviction failures

2016-11-29 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Mike Percy, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [consensus] KUDU-1718: Fix few bugs around replica eviction 
failures
..

[consensus] KUDU-1718: Fix few bugs around replica eviction failures

KUDU-1613: Tablet server returning WRONG_SERVER_UUID perhaps
due to a disks-wiped-out situation and the server was never evicted
from the consensus config.
KUDU-1407: Tablet under TABLET_NOT_RUNNING state due to a large
tablet eventually failed to bootstrap should be evicted and replicated
either on same node or somewhere else.
KUDU-1608: Catalog manager retries DeleteTablet RPC forever upon
replica eviction in some of the above scenarios. We can treat some of
these errors as fatal, and stop retrying on them.

Added bunch of tests to repro the bugs and verify these fixes.

Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
---
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/integration-tests/raft_consensus-itest.cc
M src/kudu/master/catalog_manager.cc
M src/kudu/tserver/tablet_server-test-base.h
M src/kudu/tserver/tablet_server_test_util.cc
M src/kudu/tserver/ts_tablet_manager.cc
M src/kudu/tserver/ts_tablet_manager.h
10 files changed, 299 insertions(+), 74 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [consensus] KUDU-1718: Fix few bugs around replica eviction failures

2016-11-29 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [consensus] KUDU-1718: Fix few bugs around replica eviction 
failures
..


Patch Set 2:

(7 comments)

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

PS2, Line 235: // Analyze the response from peer.
 :   PeerResponseStatus ps = AnalyzePeerResponse();
 :   if (PREDICT_FALSE(ps.response != PeerResponseStatus::OK)) {
 : VLOG_WITH_PREFIX_UNLOCKED(1) << "Error response from peer: " 
<< ps.status.ToString()
 :  << ": " << 
response_.ShortDebugString();
 : if (ps.response == PeerResponseStatus::ERROR_BUT_ALIVE) {
 :   
queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
 : }
 : ProcessResponseError(ps.status);
> can you change this for a switch case where the error cases get a fall thro
Good idea, Done.


http://gerrit.cloudera.org:8080/#/c/5111/2/src/kudu/integration-tests/raft_consensus-itest.cc
File src/kudu/integration-tests/raft_consensus-itest.cc:

PS2, Line 2695: // Tests KUDU-1613 and KUDU-1407 fixes when followers aren't 
evicted.
> please be a little more discriptive
Done


http://gerrit.cloudera.org:8080/#/c/5111/2/src/kudu/master/catalog_manager.cc
File src/kudu/master/catalog_manager.cc:

Line 2586:   virtual void HandleResponse(int attempt) OVERRIDE {
> warning: parameter 'attempt' is unused [misc-unused-parameters]
We are better off tackling this in a different patch since this will make other 
interfaces inconsistent. This variable is being unused at other places too 
AFAICT.


PS2, Line 2589:  VLOG(4) << "Error deleting the tablet " << tablet_id() << ": "
  :   << resp_.DebugString();
> should we use a higher log level here or is there enough information in the
I think the warnings below carry enough info for logging purposes, I added this 
if we want to be more verbose with "vmodule" option. I reduced it down to 1 
looking at other VLOG levels in this file.


http://gerrit.cloudera.org:8080/#/c/5111/2/src/kudu/tserver/tablet_server-test-base.h
File src/kudu/tserver/tablet_server-test-base.h:

Line 476: const char* TabletServerTestBase::kTableId = "TestTable";
> warning: variable 'kTableId' defined in a header file; variable definitions
Done


Line 477: const char* TabletServerTestBase::kTabletId = "TestTablet";
> warning: variable 'kTabletId' defined in a header file; variable definition
Done


Line 478: const string TabletServerTestBase::kTestDir = 
"TabletServerTest-fsroot";
> warning: variable 'kTestDir' defined in a header file; variable definitions
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [tools] Tombstone the tablet with "local replica delete"

2016-11-28 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [tools] Tombstone the tablet with "local_replica delete"
..


Patch Set 1:

(9 comments)

http://gerrit.cloudera.org:8080/#/c/5191/1//COMMIT_MSG
Commit Message:

PS1, Line 9: This change makes the default action of 'local_replica delete'
   : tool to tombstone the tablet.
> Are we comfortable with this semantic change given that 'local_replica dele
Adar, fyi what we shipped in 1.1 was a tool allowing the deletion via 
'--clean_unsafe' flag. i.e If I had used the tool without the flag, the 1.1 
threw an unsupported error. Now we are adding the tombstone support post 1.1 
while retaining old behavior for clean_unsafe. So the semantic really didn't 
change as such.


http://gerrit.cloudera.org:8080/#/c/5191/1/src/kudu/tools/kudu-tool-test.cc
File src/kudu/tools/kudu-tool-test.cc:

Line 1089
> Nit: why remove this? Stylistically I think preceding a comment with an emp
agreed, and I prefer that too. This wasn't intentional removal, thanks for 
catching it. fixed.


Line 1156:   // Grab the tablet_id to delete
> Nit: terminate with a period.
Done


PS1, Line 1157:   ListTabletsRequestPB req;
  :   ListTabletsResponsePB resp;
  :   RpcController rpc;
  :   rpc.set_timeout(kTimeout);
  :   {
  : unique_ptr ts_proxy;
  : ASSERT_OK(BuildProxy(ts->bound_rpc_addr().ToString(),
  :  tserver::TabletServer::kDefaultPort, 
_proxy));
  : ASSERT_OK(ts_proxy->ListTablets(req, , ));
  :   }
  :   ASSERT_FALSE(resp.has_error());
  :   ASSERT_EQ(resp.status_and_schema_size(), 1);
  :   const string& tablet_id = 
resp.status_and_schema(0).tablet_status().tablet_id();
> It's a little weird to see a test that treats the mini cluster as both "ext
Yeah good points, fixed at both places.


Line 1185: if (tablet_peer->log()) {
> Why isn't this guaranteed to be true? As written, it's unclear when the tes
yeah, 'if (tablet_peer->log())' check here was moot and not needed. What was 
not clear to me at the time was whether we have any other situation I am 
unaware of where last_entry_op_id_ may not be initialized for a tablet. So 
wanted to be on the safer side. I think that situation doesn't exist for 2 
reasons:
1) We are generating some workload and waiting for them to finish. So we expect 
the last_entry_op_id_ to be initialized at this stage.
2) Between this and L1231 the last_logged_opid isn't expected to change, so we 
can remove the check at L1230 as well.

I also added asserts at both places to check opids are initialized.


http://gerrit.cloudera.org:8080/#/c/5191/1/src/kudu/tools/tool_action_local_replica.cc
File src/kudu/tools/tool_action_local_replica.cc:

Line 97: using consensus::ReplicateMsg;
> warning: using decl 'ReplicateMsg' is unused [misc-unused-using-decls]
Done


Line 167:  boost::optional& 
last_committed_opid) {
> warning: non-const reference parameter 'last_committed_opid', make it const
Done


Line 183:   } else if (s.IsEndOfFile()) {
> warning: don't use else after return [readability-else-after-return]
Done


PS1, Line 315:<< "Can not delete (tombstone) the tablet, 
use --clean_unsafe to delete"
 :<< " the tablet permanently from the node.";
> Nit: use a new LOG(WARNING) for this part, to allow s.ToString() to be the 
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [tools] Tombstone the tablet with "local replica delete"

2016-11-28 Thread Dinesh Bhat (Code Review)
Hello Mike Percy, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [tools] Tombstone the tablet with "local_replica delete"
..

[tools] Tombstone the tablet with "local_replica delete"

This change makes the default action of 'local_replica delete'
tool to tombstone the tablet. To fully delete the tablet from
the node, user can use --clean_unsafe option.

Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
---
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tools/tool_action_local_replica.cc
2 files changed, 146 insertions(+), 53 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ia773de475431eb85fb0dbe724d524e8dd59b1b12
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [consensus] KUDU-1718: Fix few bugs around replica eviction failures

2016-11-18 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [consensus] KUDU-1718: Fix few bugs around replica eviction 
failures
..

[consensus] KUDU-1718: Fix few bugs around replica eviction failures

KUDU-1613: Tablet server returning WRONG_SERVER_UUID perhaps
due to a disks-wiped-out situation and the server was never evicted
from the consensus config.
KUDU-1407: Tablet under TABLET_NOT_RUNNING state due to a large
tablet eventually failed to bootstrap should be evicted and replicated
either on same node or somewhere else.
KUDU-1608: Catalog manager retries DeleteTablet RPC forever upon
replica eviction in some of the above scenarios. We can treat some of
these errors as fatal, and stop retrying on them.

Added bunch of tests to repro the bugs and verify these fixes.

Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
---
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/integration-tests/raft_consensus-itest.cc
M src/kudu/master/catalog_manager.cc
M src/kudu/tserver/tablet_server-test-base.h
M src/kudu/tserver/ts_tablet_manager.cc
M src/kudu/tserver/ts_tablet_manager.h
9 files changed, 271 insertions(+), 56 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] tablet copy: Make the StartTabletCopy() RPC async

2016-11-17 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: tablet copy: Make the StartTabletCopy() RPC async
..


Patch Set 2:

(5 comments)

http://gerrit.cloudera.org:8080/#/c/5045/2/src/kudu/tserver/ts_tablet_manager.cc
File src/kudu/tserver/ts_tablet_manager.cc:

Line 320: 
spurious \n ?


PS2, Line 436: success_callback
I am curious why we needed a different callback for success ? One callback can 
have 
if (status == success ) 
  OpenTablet()
else 
  Tombstone()

?


PS2, Line 444: open_tablet_pool_
Interesting, just thinking out loud here about possible outcomes of using the 
same threadpool as that of OpenTablet(): Given that open_tablet_pool_ contains 
as many max threads as there are tablets to open during server bring up, tablet 
copy could contend with OpenTablet() threads if consensus for these tablets 
starts up in the context of OpenTablet(). i.e, leader could get to know about 
follower before OpenTablet thread finishes and starts a tablet copy session 
(because it is tombstoned or whatever reason) and now there is a possibility 
that there are no threads available to start async tablet copy ? Or am I just 
hallucinating ?


PS2, Line 446: TOMBSTONE_NOT_OK
There is nothing to tombstone at this point right ? We could as well leave the 
tablet as it is since it hasn't changed its state yet.


http://gerrit.cloudera.org:8080/#/c/5045/2/src/kudu/tserver/ts_tablet_manager.h
File src/kudu/tserver/ts_tablet_manager.h:

PS2, Line 328: tablet_copy_sessions_
Good idea, we could eventually connect this to metrics or tools to show the 
number of tablets being tablet copied, right ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I95c63f2bfd6762487862efbdba9cb3676112
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Mike Percy 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [Timestamp] use 'operator<' instead of ComesBefore

2016-11-17 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [Timestamp] use 'operator<' instead of ComesBefore
..


Patch Set 3: Code-Review+1

(1 comment)

http://gerrit.cloudera.org:8080/#/c/5096/3/src/kudu/common/timestamp.h
File src/kudu/common/timestamp.h:

Line 109:   return rhs < lhs;
> Nope, this is how you define '>' operator using '<': you just swap argument
oh yeah, I missed that :), thanks.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4a5b0f90e92f6de40822cfe872b417cf0c53121a
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [consensus] KUDU-1718: Fix few bugs around replica eviction failures

2016-11-16 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [consensus] KUDU-1718: Fix few bugs around replica eviction 
failures
..


Patch Set 1:

(1 comment)

TFTR David,
 
> (1 comment)
 > 
 > would it be possible to split this patch into several ones that
 > take care of each ticker individually or would that be too hard?

I can do that if it helps for easier review, but clubbing them in one patch was 
intentional since they were all related to one set of problem space and it was 
easier to test them together. Also, the actual fixes are in few lines under 
consensus_peers.cc and catalog_manager.cc, majority of the diffs are in tests I 
think. 
PS: I need to move the kudu-tool-test diffs to a correct location, because I 
used kudu-tool-test as a placeholder and it is a terrible idea to place this 
test there.

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

PS1, Line 235:   if (!controller_.status().ok()) {
 : if (controller_.status().IsRemoteError()) {
 :   // Most controller errors are caused by network issues or 
corner cases
 :   // like shutdown and failure to serialize a protobuf. 
Therefore, we
 :   // generally consider these errors to indicate an 
unreachable peer.
 :   // However, a RemoteError wraps some other error 
propagated from the
 :   // remote peer, so we know the remote is alive. Therefore, 
we will let
 :   // the queue know that the remote is responsive.
 :   
queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
 : }
 : ProcessResponseError(controller_.status());
 : return;
 :   }
 :   // Again, let the queue know that the remote is still 
responsive, since we
 :   // will not be sending this error response through to the 
queue.
 :   // For certain error codes, we want the queue to treat the 
remote as
 :   // unresponsive and take necessary actions, hence bypassing 
the notification
 :   // for those error codes.
 :   if (response_.has_error()) {
 : if (response_.error().code() != 
TabletServerErrorPB::TABLET_NOT_FOUND &&
 : response_.error().code() != 
TabletServerErrorPB::WRONG_SERVER_UUID &&
 : response_.error().code() != 
TabletServerErrorPB::TABLET_NOT_RUNNING) {
 :   
queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
 :   
ProcessResponseError(StatusFromPB(response_.error().status()));
 :   return;
 : }
 :   } else if (response_.status().has_error()) {
 : if (response_.status().error().code() == 
consensus::ConsensusErrorPB::CANNOT_PREPARE) {
 :   
queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
 :   
ProcessResponseError(StatusFromPB(response_.error().status()));
 :   return;
 : }
 :   }
> It would be good if we could have something like we have in the c++ client 
Yeah, that's a good idea, I will look into that. Also slightly related to this 
topic: I wanted to group these errors into 2 buckets,  fatal and retriable. 
That way we can consolidate all of the error codes in one place and less window 
for programmers to leave some error codes behind. I will attempt that in a 
different patch though as it is not clear to me whether such grouping could 
work in all situations. For eg, TABLET_NOT_FOUND may be a fatal error for 
consensus, but may not be a fatal for some other workflow like DeleteTablet.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [consensus] KUDU-1718: Fix few bugs around replica eviction failures

2016-11-16 Thread Dinesh Bhat (Code Review)
Hello David Ribeiro Alves, Adar Dembo, Todd Lipcon,

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

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

to review the following change.

Change subject: [consensus] KUDU-1718: Fix few bugs around replica eviction 
failures
..

[consensus] KUDU-1718: Fix few bugs around replica eviction failures

KUDU-1613: Tablet server returning WRONG_SERVER_UUID perhaps
due to a disks-wiped-out situation and the server was never evicted
from the consensus config.
KUDU-1407: Tablet under TABLET_NOT_RUNNING state due to a large
tablet eventually failed to bootstrap should be evicted and replicated
either on same node or somewhere else.
KUDU-1608: Catalog manager retries DeleteTablet RPC forever upon
replica eviction in some of the above scenarios. We can treat some of
these errors as fatal, and stop retrying on them.

Added bunch of tests to repro the bugs and verify these fixes.

Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
---
M src/kudu/consensus/consensus_peers.cc
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/integration-tests/mini_cluster.cc
M src/kudu/integration-tests/mini_cluster.h
M src/kudu/master/catalog_manager.cc
M src/kudu/tools/kudu-tool-test.cc
M src/kudu/tserver/mini_tablet_server.cc
M src/kudu/tserver/mini_tablet_server.h
M src/kudu/tserver/tablet_server-test-base.h
M src/kudu/tserver/ts_tablet_manager.cc
M src/kudu/tserver/ts_tablet_manager.h
12 files changed, 295 insertions(+), 36 deletions(-)


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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I0d3f84fd297a8e4760208a213c1ee393e92499a3
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Todd Lipcon 


[kudu-CR] [Timestamp] use 'operator<' instead of ComesBefore

2016-11-16 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [Timestamp] use 'operator<' instead of ComesBefore
..


Patch Set 3:

(4 comments)

http://gerrit.cloudera.org:8080/#/c/5096/3/src/kudu/common/timestamp.h
File src/kudu/common/timestamp.h:

Line 87: inline int Timestamp::CompareTo(const Timestamp& other) const {
Do we still need this routine or is this leftover ?


Line 109:   return rhs < lhs;
Should this be rhs.v < lhs.v ?


Line 113:   return !(lhs > rhs);
same as above.


Line 117:   return !(lhs < rhs);
same as above.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4a5b0f90e92f6de40822cfe872b417cf0c53121a
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Tidy Bot
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR](gh-pages) Add 11/15 weekly update

2016-11-15 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: Add 11/15 weekly update
..


Patch Set 3: Code-Review+1

(1 comment)

http://gerrit.cloudera.org:8080/#/c/5098/3/_posts/2016-11-15-weekly-update.md
File _posts/2016-11-15-weekly-update.md:

PS3, Line 70: client
s/client/manual recovery/ ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I487b092739b3f921e3ab0b186f82c7f368d84f4f
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: gh-pages
Gerrit-Owner: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] Reject CREATE TABLE ops with even replication factor

2016-11-10 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: Reject CREATE TABLE ops with even replication factor
..


Patch Set 5: Code-Review+1

(1 comment)

LGTM, one nit.

http://gerrit.cloudera.org:8080/#/c/4945/5//COMMIT_MSG
Commit Message:

Line 7: Reject CREATE TABLE ops with even replication factor
If we have a JIRA filed for this issue/task, it's a good practice to start the 
COMMIT_MSG with jira number.
"KUDU-1658: Reject CREATE"


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I1674cc59cdfc2955d42fc5e4d8c0d962d9cc8e8e
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Hao Hao 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Hao Hao 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] [doc] mention the loadgen tool in the release notes

2016-11-09 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [doc] mention the loadgen tool in the release notes
..


Patch Set 1: Code-Review+1

(3 comments)

http://gerrit.cloudera.org:8080/#/c/5020/1/docs/release_notes.adoc
File docs/release_notes.adoc:

PS1, Line 64: * The `insert-generated-rows` tool has been removed.
Was this a part of CLI earlier ? if not, I think the mentioning of this being 
obsoleted @L67 is sufficient, instead of explicitly calling this out under CLI 
section.


PS1, Line 65: new
probably redundant.


PS1, Line 66: temporary
may be s/temporary/new ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I1a227c475b05cb394718812ae9c825022f67994d
Gerrit-PatchSet: 1
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Alexey Serbin 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-HasComments: Yes


[kudu-CR] Consolidate Row/CompactionInputRow printing on compaction

2016-11-08 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: Consolidate Row/CompactionInputRow printing on compaction
..


Patch Set 3: Code-Review+1

(1 comment)

http://gerrit.cloudera.org:8080/#/c/4988/3/src/kudu/tablet/compaction.cc
File src/kudu/tablet/compaction.cc:

PS3, Line 823: 3
Not a review comment per se, was it intentional to keep Input row loglevel 
different than output row loglevel (below) in this function ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ifd9f59094ac1f5f9c6343c1d5770dcc089f9
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: David Ribeiro Alves 
Gerrit-Reviewer: Alexey Serbin 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: Yes


[kudu-CR] Rename LogicalClock::NowForMetrics() to GetCurrentTime()

2016-11-08 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: Rename LogicalClock::NowForMetrics() to GetCurrentTime()
..


Patch Set 3:

> Dinesh: Yeah I saw that and I pondered what you are suggesting.
 > Ended up not doing it because in the HybridClock::NowForMetrics is
 > really just used for metrics and does actually update the clocks's
 > logical value if needed. For a clock with a physical component this
 > makes sense: we want it to read the latest time. However for a
 > logical clock just getting the current time shouldn't change it.
 > 
 > It bummed me a little bit to break the symmetry, but since these
 > are not virtual methods I resigned to living with it :)

Sounds good, thank you for explaining, I was lazy enough to not to look into 
the definition of HybridClock::Now() earlier.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iac6f8dd9e79657b81409127e539e509da4b2b8de
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: David Ribeiro Alves 
Gerrit-Reviewer: David Ribeiro Alves 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy 
Gerrit-Reviewer: Todd Lipcon 
Gerrit-HasComments: No


[kudu-CR] [doc] 1.1.0 release notes for tools

2016-11-08 Thread Dinesh Bhat (Code Review)
Hello Jean-Daniel Cryans, Adar Dembo, Kudu Jenkins,

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

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

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

Change subject: [doc] 1.1.0 release notes for tools
..

[doc] 1.1.0 release notes for tools

Change-Id: I6dc80b356d4cec6d0065a515dce5fdd94581346a
---
M docs/release_notes.adoc
1 file changed, 7 insertions(+), 0 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I6dc80b356d4cec6d0065a515dce5fdd94581346a
Gerrit-PatchSet: 4
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins


[kudu-CR] [doc] 1.1.0 release notes for tools

2016-11-08 Thread Dinesh Bhat (Code Review)
Dinesh Bhat has posted comments on this change.

Change subject: [doc] 1.1.0 release notes for tools
..


Patch Set 3:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/5012/3/docs/release_notes.adoc
File docs/release_notes.adoc:

PS3, Line 57: tablet
> replica
Done


PS3, Line 59: delete the
:   replica of a tablet
> "delete a replica"
Done


PS3, Line 60: when the tablet server is unable to restart due to bad state of 
the replica.
> Is this part really necessary? "kudu local_replica delete" can be used for 
Yeah, I wanted to emphasize implicitly that this tool is mainly used when 
tablet server is offline for so and so reasons. Come to think of it, the fact 
that it's an action under 'local_replica' makes it obvious that this tool can 
be used only when the server is offline, isn't it ? Removed it now.

Also, this tool only works with --clean_unsafe flag for the deletion to happen. 
otherwise it throws 'currently unsupported' and that support for 'tombstoning 
the tablet when server is offline' will be part of upcoming release. Although 
all this has clear help description under the tool, should we put that in 
release notes as well ?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I6dc80b356d4cec6d0065a515dce5fdd94581346a
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Adar Dembo 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins
Gerrit-HasComments: Yes


[kudu-CR] [doc] 1.1.0 release notes for tools

2016-11-08 Thread Dinesh Bhat (Code Review)
Hello Jean-Daniel Cryans, Kudu Jenkins,

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

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

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

Change subject: [doc] 1.1.0 release notes for tools
..

[doc] 1.1.0 release notes for tools

Change-Id: I6dc80b356d4cec6d0065a515dce5fdd94581346a
---
M docs/release_notes.adoc
1 file changed, 8 insertions(+), 0 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I6dc80b356d4cec6d0065a515dce5fdd94581346a
Gerrit-PatchSet: 3
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins


[kudu-CR] [doc] 1.1.0 release notes for tools

2016-11-08 Thread Dinesh Bhat (Code Review)
Hello Jean-Daniel Cryans, Kudu Jenkins,

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

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

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

Change subject: [doc] 1.1.0 release notes for tools
..

[doc] 1.1.0 release notes for tools

Change-Id: I6dc80b356d4cec6d0065a515dce5fdd94581346a
---
M docs/release_notes.adoc
1 file changed, 8 insertions(+), 0 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I6dc80b356d4cec6d0065a515dce5fdd94581346a
Gerrit-PatchSet: 2
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Dinesh Bhat 
Gerrit-Reviewer: Jean-Daniel Cryans 
Gerrit-Reviewer: Kudu Jenkins


  1   2   3   >