You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@kudu.apache.org by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org> on 2019/10/02 17:46:43 UTC

[kudu-CR] KUDU-2800 draft 2

Volodymyr Verovkin has uploaded this change for review. ( http://gerrit.cloudera.org:8080/14300


Change subject: KUDU-2800 draft 2
......................................................................

KUDU-2800 draft 2

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 202 insertions(+), 0 deletions(-)



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 7
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 10:

(9 comments)

http://gerrit.cloudera.org:8080/#/c/14300/10//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/14300/10//COMMIT_MSG@11
PS10, Line 11: 
             : 1) Tablet replica bootstraps long time. No data modications happen during these time. Consensus state does not change.
             : 
             : 2) Tablet replica is shutdown for some time. During this time a lot of data modifications happen, log rolls, replica is replaced (with re-replication). Replica restarts and find out the it fell behind.
             : 
             : 3) Tablet replica bootstraps long time. During this time a lot of data modifications happen, log rolls, replica is replaced (with re-replication). Replica restarts and find out the it fell behind.
nit: could you wrap these lines at 72 characters? per #6 of https://chris.beams.io/posts/git-commit/


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@464
PS10, Line 464: ValidateConsensusState
nit: maybe rename this so it's clearer from the callsite what it's trying to achieve? Something like ValidateReplicaMoved() or something?


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@484
PS10, Line 484: 
              :   FLAGS_num_tablet_servers = 4;
              :   FLAGS_num_replicas = 3;
              :   NO_FATALS(BuildAndStart(
              :       {"--follower_unavailable_considered_failed_sec=3"},
              :       {"--tserver_unresponsive_timeout_ms=3000"}));
              : 
              :   ASSERT_EQ(4, tablet_servers_.size());
              : 
              :   // Extra sanity checks.
              :   vector<string> replica_servers = GetServersWithReplica(tablet_id_);
              :   ASSERT_EQ(3, replica_servers.size());
              :   vector<string> no_replica_servers = GetServersWithoutReplica(tablet_id_);
              :   ASSERT_EQ(1, no_replica_servers.size());
nit: is it important for the flags here to be different than the flags in the cases below? If not, maybe consider encapsulating in a function along the lines of SetupClusterForUnavailability() and reusing it in the below tests?


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@508
PS10, Line 508: ASSERT_EVENTUALLY([&] {
Is this ASSERT_EVENTUALLY necessary? If the replicas have been already moved in the first place, won't this never converge?


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@516
PS10, Line 516: // Check that the replica at the tablet servers that was restarted is not removed.
              :     ASSERT_TRUE(IsRaftConfigMember(replica_servers.front(), cstate.committed_config()));
Should this check be done on all 3 of the replica_servers?


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@597
PS10, Line 597: replica->uuid()
Isn't this just follower_uuid?


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@627
PS10, Line 627: 
              :   LOG(INFO) << "Waiting for log GC on " << leader->uuid();
              :   // Some WAL segments must exist, but wal segment 1 must not exist.
              :   ASSERT_OK(inspect_->WaitForFilePatternInTabletWalDirOnTs(
              :       leader_index, tablet_id_, { "wal-" }, { "wal-000000001" }));
              :   LOG(INFO) << "Log GC complete on " << leader->uuid();
How long does it take for this test to finish? Would it make sense to flush more frequently via --flush_threshold_secs so the WALs are not anchored for as long?


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@635
PS10, Line 635:   SleepFor(MonoDelta::FromSeconds(3));
nit: this probably isn't needed, right? Since we're going to retry the below check for up to 30 seconds


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

http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/tserver/ts_tablet_manager.cc@1077
PS10, Line 1077: << "Injecting " << FLAGS_tablet_open_bootstrap_inject_latency_ms
               :                      << "ms delay in tablet bootstrapping";
nit: just FYI Substitute could be used here, e.g.

 LOG(WARNING) << Substitute("Injecting $0 ms delay in in tablet bootstrapping", FLAGS_tablet_open_bootstrap_inject_latency_ms);



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 10
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Fri, 04 Oct 2019 00:54:50 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Adar Dembo, Greg Solovyev, 

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

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

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

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long
bootstrapping of tablet replica.
There case are covered:

1) Tablet replica bootstraps long time. No data modications happen
during these time. Consensus state does not change.

2) Tablet replica is shutdown for some time. During this time a lot of
data modifications happen, log rolls, replica is replaced (with
re-replication). Replica restarts and find out the it fell behind.

3) Tablet replica bootstraps long time. During this time a lot of data
modifications happen, log rolls, replica is replaced (with
re-replication). Replica restarts and find out the it fell behind.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 163 insertions(+), 0 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 14
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Adar Dembo, Greg Solovyev, 

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

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

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

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long
bootstrapping of a tablet replica.
The following cases are covered:

1) A tablet replica bootstraps long time. No data modications happen
during that time. The replica is not evicted from the tablet Raft
configuration and joins the quorum after the bootstrapping process is
finished.

2) A tablet replica is shut down for some time. During this time a lot
of data modifications happen, log rolls, the replica is replaced (with
re-replication). The replica restarts and finds out that is has been
evicted from the tablet Raft configuration.

3) A tablet replica bootstraps long time. During this time a lot
of data modifications happen, log rolls, the replica is replaced (with
re-replication). The replica eventually finishes boostrap and finds out
that is has been evicted from the tablet Raft configuration.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 163 insertions(+), 0 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 17
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 draft 2

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 draft 2
......................................................................


Patch Set 7:

(4 comments)

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

http://gerrit.cloudera.org:8080/#/c/14300/7/src/kudu/integration-tests/raft_config_change-itest.cc@461
PS7, Line 461: SlowBootstrapTest
nit: maybe, SlowTabletBootstrapTest would be more precise?


http://gerrit.cloudera.org:8080/#/c/14300/7/src/kudu/integration-tests/raft_config_change-itest.cc@508
PS7, Line 508:     TServerDetails* leader = nullptr;
             :     ASSERT_OK(GetLeaderReplicaWithRetries(tablet_id_, &leader));
             :     consensus::ConsensusStatePB cstate;
             :     ASSERT_OK(GetConsensusState(leader, tablet_id_, kTimeout,
             :                                 EXCLUDE_HEALTH_REPORT, &cstate));
             :     // Check that new replica has not been added.
             :     ASSERT_FALSE(IsRaftConfigMember(no_replica_servers.front(), cstate.committed_config()));
             :     // Check that the replica at the tablet servers that was restarted is not removed.
             :     ASSERT_TRUE(IsRaftConfigMember(replica_servers.front(), cstate.committed_config()));
Does it make sense to make the newly introduced ValidateConsensusState() to take care of this case as well?


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

http://gerrit.cloudera.org:8080/#/c/14300/7/src/kudu/tserver/ts_tablet_manager.cc@144
PS7, Line 144: hidden
Change to 'unsafe'.  That way it's automatically hidden from the documentation and requires `--unlock_unsafe_flags` to use.  Test-only flags which introduce such stuff are tagged 'unsafe' usually.


http://gerrit.cloudera.org:8080/#/c/14300/7/src/kudu/tserver/ts_tablet_manager.cc@1077
PS7, Line 1077: "Injecting " << FLAGS_tablet_open_bootstrap_inject_latency_ms
              :                      << "ms delay in tablet bootstrapping"
nit: maybe, use strings::Substitute() to format the message for easier readability in the code



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 7
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Comment-Date: Wed, 02 Oct 2019 19:32:39 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Adar Dembo, Greg Solovyev, 

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

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

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

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long
bootstrapping of tablet replica.
There case are covered:

1) Tablet replica bootstraps long time. No data modications happen
during these time. Consensus state does not change.

2) Tablet replica is shutdown for some time. During this time a lot of
data modifications happen, log rolls, replica is replaced (with
re-replication). Replica restarts and find out the it fell behind.

3) Tablet replica bootstraps long time. During this time a lot of data
modifications happen, log rolls, replica is replaced (with
re-replication). Replica restarts and find out the it fell behind.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 170 insertions(+), 0 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 12
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 draft 2

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 draft 2
......................................................................


Patch Set 7:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14300/7//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/14300/7//COMMIT_MSG@7
PS7, Line 7: KUDU-2800 draft 2
Add proper description



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 7
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Comment-Date: Wed, 02 Oct 2019 19:33:42 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Adar Dembo, Greg Solovyev, 

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

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

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

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long
bootstrapping of tablet replica.
There case are covered:

1) Tablet replica bootstraps long time. No data modications happen
during these time. Consensus state does not change.

2) Tablet replica is shutdown for some time. During this time a lot of
data modifications happen, log rolls, replica is replaced (with
re-replication). Replica restarts and find out the it fell behind.

3) Tablet replica bootstraps long time. During this time a lot of data
modifications happen, log rolls, replica is replaced (with
re-replication). Replica restarts and find out the it fell behind.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 175 insertions(+), 0 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 11
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 13:

(4 comments)

http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@635
PS10, Line 635: 
> This duration is equal to  "tablet_open_bootstrap_inject_latency_ms". The i
Makes sense, thanks for the explanation!


http://gerrit.cloudera.org:8080/#/c/14300/13/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/13/src/kudu/integration-tests/raft_config_change-itest.cc@484
PS13, Line 484: ever
server


http://gerrit.cloudera.org:8080/#/c/14300/13/src/kudu/integration-tests/raft_config_change-itest.cc@518
PS13, Line 518: false, 
nit: While this helps emphasize that we're not falling behind in this test, we're not writing either. This means that we should be able to tolerate frequent log rolling and still not move the replica, right? If so, we could probably get rid of this boolean entirely and roll logs frequently, since there's no risk of falling behind.


http://gerrit.cloudera.org:8080/#/c/14300/13/src/kudu/integration-tests/raft_config_change-itest.cc@571
PS13, Line 571:   auto replica_ets = cluster_->tablet_server_by_uuid(any_replica_server);
nit: Isn't this just `ts`?



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 13
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Wed, 09 Oct 2019 05:11:51 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Volodymyr Verovkin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 16:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/14300/15/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/15/src/kudu/integration-tests/raft_config_change-itest.cc@519
PS15, Line 519: tablet_open_bootstrap_inject_latency_ms
> tablet_bootstrap_inject_latency_ms
Done


http://gerrit.cloudera.org:8080/#/c/14300/15/src/kudu/integration-tests/raft_config_change-itest.cc@563
PS15, Line 563: tablet_open_bootstrap_inject_latency_ms
> tablet_bootstrap_inject_latency_ms
Done


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

http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/tserver/ts_tablet_manager.cc@142
PS11, Line 142:              "Injects latency into the tablet bootstrapping."
> I think the space is missing after the period in the first sentence:
Done



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 16
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Mon, 14 Oct 2019 17:36:51 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has removed a vote on this change.

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Removed Verified-1 by Kudu Jenkins (120)
-- 
To view, visit http://gerrit.cloudera.org:8080/14300
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: deleteVote
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 13
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Volodymyr Verovkin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 11:

(8 comments)

http://gerrit.cloudera.org:8080/#/c/14300/10//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/14300/10//COMMIT_MSG@11
PS10, Line 11: There case are covered:
             : 
             : 1) Tablet replica bootstraps long time. No data modications happen
             : during these time. Consensus state does not change.
             : 
             : 2) Tablet replica is shutdown for some time. During this time a lot of
> nit: could you wrap these lines at 72 characters? per #6 of https://chris.b
Done


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@464
PS10, Line 464: ted:
> nit: maybe rename this so it's clearer from the callsite what it's trying t
Done


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@484
PS10, Line 484:       "--consensus_rpc_timeout_ms=2000",
              :       "--follower_unavailable_considered_failed_sec=3"
              :     };
              : 
              :     if (add_flags_for_log_rolls) {
              :       AddFlagsForLogRolls(&ts_flags);
              :     }
              :     FLAGS_num_tablet_servers = 4;
              :     FLAGS_num_replicas = 3;
              :     NO_FATALS(BuildAndStart(ts_flags, kMasterFlags));
              : 
              :     ASSERT_EQ(4, tablet_servers_.size());
              : 
              :     // Extra sanity checks.
> nit: is it important for the flags here to be different than the flags in t
Done


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@508
PS10, Line 508:  as long as there are n
> Is this ASSERT_EVENTUALLY necessary? If the replicas have been already move
Until restarted replica finishes bootstrap, we may not see desirable state of consensus.


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@516
PS10, Line 516:  Shutdown any tablet server with tablet's replica.
              :   auto ts = cluster_->tablet_server_by_uuid(any_replica_server);
> Should this check be done on all 3 of the replica_servers?
We just want to see that restarted replica has not been evicted and spare server has not been added to consensus.


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@597
PS10, Line 597: 
> Isn't this just follower_uuid?
Done


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@627
PS10, Line 627: 
              : 
              : 
              : 
              : 
              : 
> How long does it take for this test to finish? Would it make sense to flush
We make log rolling frequently via options set in AddFlagsForLogRolls()


http://gerrit.cloudera.org:8080/#/c/14300/10/src/kudu/integration-tests/raft_config_change-itest.cc@635
PS10, Line 635: 
> nit: this probably isn't needed, right? Since we're going to retry the belo
This duration is equal to  "tablet_open_bootstrap_inject_latency_ms". The idea is that replica finishes bootstrap and tries to join consensus and finds out that it is not part of the consensus.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 11
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Mon, 07 Oct 2019 06:43:04 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 14:

(16 comments)

http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@10
PS14, Line 10: tablet replica
nit: a tablet replica


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@11
PS14, Line 11: There case
nit: The following cases


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@13
PS14, Line 13: Tablet replica
A tablet replica


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@14
PS14, Line 14: these time
nit: during this time


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@14
PS14, Line 14: Consensus state does not change
It might be a bit of overstatement: leader replica might change during that period, etc.  To be specific, maybe rephrase with:

The replica is not evicted from the tablet Raft configuration and joins the quorum after the bootstrapping process is finished.


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@16
PS14, Line 16: Tablet replica
A tablet replica


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@17
PS14, Line 17: replica
the replica


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@18
PS14, Line 18: Replica
The replica


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@18
PS14, Line 18: the it fell behind
that is has been evicted from the tablet Raft configuration


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@18
PS14, Line 18: find
finds


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@20
PS14, Line 20: Tablet replica
A tablet replica


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@22
PS14, Line 22: the it fell behind
that it has been evicted from the tablet Raft configuration


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@22
PS14, Line 22: find
finds


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@22
PS14, Line 22: Replica restarts
The replica eventually starts


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

http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/tserver/ts_tablet_manager.cc@141
PS11, Line 141: tablet_open_bootstrap_inject_latency_ms
> nit: why not simply tablet_bootstrap_inject_latency_ms ?
It seems you didn't address (missed?) this comment.


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/tserver/ts_tablet_manager.cc@142
PS11, Line 142:              "Injects latency into the tablet bootstrapping."
> nit: add a space
It seems you didn't address (missed?) this comment.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 14
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Fri, 11 Oct 2019 21:47:27 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Volodymyr Verovkin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 15:

(16 comments)

http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@10
PS14, Line 10: a tablet repli
> nit: a tablet replica
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@11
PS14, Line 11: The follow
> nit: The following cases
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@13
PS14, Line 13: A tablet repli
> A tablet replica
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@14
PS14, Line 14: he replica is not evicted from 
> It might be a bit of overstatement: leader replica might change during that
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@14
PS14, Line 14: that time.
> nit: during this time
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@16
PS14, Line 16: ished.
> A tablet replica
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@17
PS14, Line 17: 
> the replica
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@18
PS14, Line 18: ca is s
> The replica
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@18
PS14, Line 18: ome 
> finds
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@18
PS14, Line 18:  During this time 
> that is has been evicted from the tablet Raft configuration
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@20
PS14, Line 20: replication). 
> A tablet replica
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@22
PS14, Line 22: 
> The replica eventually starts
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@22
PS14, Line 22: 
> that it has been evicted from the tablet Raft configuration
Done


http://gerrit.cloudera.org:8080/#/c/14300/14//COMMIT_MSG@22
PS14, Line 22: 
> finds
Done


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

http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/tserver/ts_tablet_manager.cc@141
PS11, Line 141: tablet_bootstrap_inject_latency_ms, 0,
> It seems you didn't address (missed?) this comment.
Sorry, missed that comment.


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/tserver/ts_tablet_manager.cc@142
PS11, Line 142:              "Injects latency into the tablet bootstrapping."
> It seems you didn't address (missed?) this comment.
Sorry, missed that comment.
Where to add a space ?



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 15
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Fri, 11 Oct 2019 22:19:46 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 9:

> (1 comment)
 > 
 > Ran
 > ninja ilint
 > ninja iwyu

Intersting: most likely IWYU suggests different set of include files on your build machine.  It's known to be funky.  Anyway, this is its output for now: http://jenkins.kudu.apache.org/job/kudu-gerrit/18974/BUILD_TYPE=IWYU/artifact/build/latest/test-logs/iwyu.log


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 9
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Thu, 03 Oct 2019 01:36:10 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 15:

(1 comment)

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

http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/tserver/ts_tablet_manager.cc@142
PS11, Line 142:              "Injects latency into the tablet bootstrapping."
> Sorry, missed that comment.
I think the space is missing after the period in the first sentence:

  "Injects latency into the tablet bootstrapping."
  "For use in tests only."

Once those two lines are merged, there will be no space in the description of the flag.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 15
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Sat, 12 Oct 2019 02:08:24 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Adar Dembo, Greg Solovyev, 

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

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

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

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long
bootstrapping of a tablet replica.
The following cases are covered:

1) A tablet replica bootstraps long time. No data modications happen
during that time. The replica is not evicted from the tablet Raft
configuration and joins the quorum after the bootstrapping process is
finished.

2) A tablet replica is shut down for some time. During this time a lot
of data modifications happen, log rolls, the replica is replaced (with
re-replication). The replica restarts and finds out that is has been
evicted from the tablet Raft configuration.

3) A tablet replica bootstraps long time. During this time a lot
of data modifications happen, log rolls, the replica is replaced (with
re-replication). The replica eventually finishes boostrap and finds out
that is has been evicted from the tablet Raft configuration.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 163 insertions(+), 0 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/00/14300/15
-- 
To view, visit http://gerrit.cloudera.org:8080/14300
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 15
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Andrew Wong (Code Review)" <ge...@cloudera.org>.
Andrew Wong has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 13: Verified+1

The test failure is unrelated.


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 13
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Wed, 09 Oct 2019 04:48:03 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 11:

(12 comments)

http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@597
PS6, Line 597:   }
             :   workload.StopAndJoin();
             : 
             :   LOG(INFO) << "Waiting for log GC on " << leader->uuid();
             :   // Some WAL segments must exist, but wal segment 1 must not exist.
             :   ASSERT_OK(inspect_->WaitForFilePatternInTabletWalDirOnTs(
             :    
> I think if we shutdown leader, it may not quickly propagate across the clus
I think GetLeaderReplicaWithRetries() handles those cases as well.  Or you have some evidence it fails doing so?


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@628
PS6, Line 628: 
             : 
> We give that replica time equal to tablet_open_bootstrap_inject_latency_ms 
OK, but given the exact timings, might it be a source for flakiness?  Or, put it another way: what might be different in the ASSERT_EVENTUALLY below if this delay is not in place?


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@71
PS11, Line 71: using std::pair;
> warning: using decl 'pair' is unused [misc-unused-using-decls]
It makes sense to address the issues reported by the Tidy bot.


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@465
PS11, Line 465:   void ValidateConsensusStateChanged(const string& added_replica_uuid,
Add short doc for the parameters.


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@465
PS11, Line 465:   void ValidateConsensusStateChanged(const string& added_replica_uuid,
              :                               const string& removed_replica_uuid,
              :                               const MonoDelta& timeout) {
nit: align the parameters

This might be helpful https://google.github.io/styleguide/cppguide.html#Function_Declarations_and_Definitions for details.


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@476
PS11, Line 476:   void SetUpCluster(bool add_flags_for_log_rolls,
Document the parameters.


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@477
PS11, Line 477: any_replica_sever
any_replica_server ?


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@480
PS11, Line 480:     const vector<string> kMasterFlags {
              :         "--tserver_unresponsive_timeout_ms=3000"
              :     };
              :     vector<string> ts_flags {
              :       "--consensus_rpc_timeout_ms=2000",
              :       "--follower_unavailable_considered_failed_sec=3"
              :     };
nit: document the reasoning behind adding these extra flags and relationship between their values.


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@511
PS11, Line 511: const MonoDelta kTimeout = MonoDelta::FromSeconds(30);
nit: it seems this also can be factored out into a member of  the SlowTabletBootstrapTest class.


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@531
PS11, Line 531: // it falls behind and is removed from consensus
nit: add period


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

http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/tserver/ts_tablet_manager.cc@141
PS11, Line 141: tablet_open_bootstrap_inject_latency_ms
nit: why not simply tablet_bootstrap_inject_latency_ms ?


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/tserver/ts_tablet_manager.cc@142
PS11, Line 142:              "Injects latency into the tablet bootstrapping."
nit: add a space



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 11
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Mon, 07 Oct 2019 14:05:30 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 draft 2

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Volodymyr Verovkin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 draft 2
......................................................................


Patch Set 7:

(16 comments)

http://gerrit.cloudera.org:8080/#/c/14300/7//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/14300/7//COMMIT_MSG@7
PS7, Line 7: KUDU-2800 draft 2
> Add proper description
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@460
PS6, Line 460: // Check re-replication during slow tablet replica bootstrap.
> nit: add a period in the end of the sentence.
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@464
PS6, Line 464:                               const string& evicted_replica_uuid,
             :                               const MonoDelta& kTimeout) 
> I think it's better to explicitly say that replica is not evicted.
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@467
PS6, Line 467:     ASSERT_OK(GetLeaderReplicaWithRetries(tablet_id_, &leader));
> Here and in other new test scenarios: add SKIP_IF_SLOW_NOT_ALLOWED() since 
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@483
PS6, Line 483: 
> nit: the tablet's replica
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@492
PS6, Line 492: // Extra sanity checks.
> nit: if it's used only once, maybe put the lambda's code inside ASSERT_EVEN
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@500
PS6, Line 500: 
> server
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@585
PS6, Line 585: _servers.front();
> Drop this: I think it's too vague to say that the replica  removes its data
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@590
PS6, Line 590:   // Inject delay in
> I think it's quite obvious what happens in two lines below; maybe drop this
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@597
PS6, Line 597:   replica_ets->Shutdown();
             :   ASSERT_OK(ts->Restart());
             : 
             :   // Find a leader. In case we paused the leader above, this will wait until
             :   // we have elected a new one.
             :   TServerDetails* leader = nullptr;
             :   w
> I think GetLeaderReplicaWithRetries() takes care of retries already.  Why i
I think if we shutdown leader, it may not quickly propagate across the cluster and we can see it during GetLeaderReplicaWithRetries() till consensus detect loss of leader and elect new one.


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@628
PS6, Line 628:   // Some WAL segments must exist, but wal segment 1 must not exist.
             :   ASSERT_OK(inspect_->WaitForFilePatte
> Given that there is ASSERT_EVENTUALLY() below, is it necessary at all?
We give that replica time equal to tablet_open_bootstrap_inject_latency_ms to finish bootstrap in order to make sure that, once replica boot up and try to re-join consensus, it will fail doing that.


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@632
PS6, Line 632: 
             :   //Let the replica finish slow bootstrap
             :   SleepFor(MonoDelta::FromSeconds(3));
             : 
             :   ASSERT_EVENTUALLY([&]() {
             :     ValidateConsensusState(no_replica_servers.front(), follower_uuid, kTimeout);
             :   });
             : }
             : 
> This is a repeating pattern.  Maybe, factor it out into a class method?
Done


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

http://gerrit.cloudera.org:8080/#/c/14300/7/src/kudu/integration-tests/raft_config_change-itest.cc@461
PS7, Line 461: SlowBootstrapTest
> nit: maybe, SlowTabletBootstrapTest would be more precise?
Done


http://gerrit.cloudera.org:8080/#/c/14300/7/src/kudu/integration-tests/raft_config_change-itest.cc@508
PS7, Line 508:     TServerDetails* leader = nullptr;
             :     ASSERT_OK(GetLeaderReplicaWithRetries(tablet_id_, &leader));
             :     consensus::ConsensusStatePB cstate;
             :     ASSERT_OK(GetConsensusState(leader, tablet_id_, kTimeout,
             :                                 EXCLUDE_HEALTH_REPORT, &cstate));
             :     // Check that new replica has not been added.
             :     ASSERT_FALSE(IsRaftConfigMember(no_replica_servers.front(), cstate.committed_config()));
             :     // Check that the replica at the tablet servers that was restarted is not removed.
             :     ASSERT_TRUE(IsRaftConfigMember(replica_servers.front(), cstate.committed_config()));
> Does it make sense to make the newly introduced ValidateConsensusState() to
This case is slightly different.
In ValidateConsensusState() we test
ASSERT_TRUE(IsRaftConfigMember(no_replica_servers.front(), cstate.committed_config()));
but here
ASSERT_FALSE(IsRaftConfigMember(no_replica_servers.front(), cstate.committed_config()));


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

http://gerrit.cloudera.org:8080/#/c/14300/7/src/kudu/tserver/ts_tablet_manager.cc@144
PS7, Line 144: hidden
> Change to 'unsafe'.  That way it's automatically hidden from the documentat
Done


http://gerrit.cloudera.org:8080/#/c/14300/7/src/kudu/tserver/ts_tablet_manager.cc@1077
PS7, Line 1077: "Injecting " << FLAGS_tablet_open_bootstrap_inject_latency_ms
              :                      << "ms delay in tablet bootstrapping"
> nit: maybe, use strings::Substitute() to format the message for easier read
It's a copy-paste of FLAGS_delete_tablet_inject_latency_ms code. I found 4 code entries with exactly same code.



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 7
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Wed, 02 Oct 2019 20:51:22 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has removed a vote on this change.

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Removed Verified-1 by Kudu Jenkins (120)
-- 
To view, visit http://gerrit.cloudera.org:8080/14300
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: deleteVote
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 17
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 15:

> Build Failed
 > 
 > http://jenkins.kudu.apache.org/job/kudu-gerrit/19081/ : FAILURE

It seems something went wrong: the newly introduced tests reported an error:

  http://jenkins.kudu.apache.org/job/kudu-gerrit/19081/

One of the failures:
  http://dist-test.cloudera.org/job?job_id=jenkins-slave.1570832890.32521


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 15
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Sat, 12 Oct 2019 02:11:39 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Volodymyr Verovkin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 14:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/14300/13/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/13/src/kudu/integration-tests/raft_config_change-itest.cc@484
PS13, Line 484: ery 
> server
Done


http://gerrit.cloudera.org:8080/#/c/14300/13/src/kudu/integration-tests/raft_config_change-itest.cc@518
PS13, Line 518: 
> nit: While this helps emphasize that we're not falling behind in this test,
Done


http://gerrit.cloudera.org:8080/#/c/14300/13/src/kudu/integration-tests/raft_config_change-itest.cc@571
PS13, Line 571:   ASSERT_OK(GetLeaderReplicaWithRetries(tablet_id_, &leader));
> nit: Isn't this just `ts`?
Done



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 14
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Fri, 11 Oct 2019 21:05:16 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 17: Verified+1

Unrelated tests failures in:
  *  	org.apache.kudu.spark.kudu.DefaultSourceTest


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 17
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Mon, 14 Oct 2019 19:18:36 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Hello Alexey Serbin, 

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

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

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

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long bootstrapping of tablet replica.
There case are covered:

1) Tablet replica bootstraps long time. No data modications happen during these time. Consensus state does not change.

2) Tablet replica is shutdown for some time. During this time a lot of data modifications happen, log rolls, replica is replaced (with re-replication). Replica restarts and find out the it fell behind.

3) Tablet replica bootstraps long time. During this time a lot of data modifications happen, log rolls, replica is replaced (with re-replication). Replica restarts and find out the it fell behind.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 202 insertions(+), 0 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 8
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Volodymyr Verovkin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 12:

(10 comments)

http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@597
PS6, Line 597: 
             :   LOG(INFO) << "Waiting for log GC on " << leader->uuid();
             :   // Some WAL segments must exist, but wal segment 1 must not exist.
             :   ASSERT_OK(inspect_->WaitForFilePatternInTabletWalDirOnTs(
             :       leader_index, tablet_id_, { "wal-" }, { "wal-000000001" }));
             :   LOG(INFO) << "Log GC complete on " << leader->uuid();
             : 
> I think GetLeaderReplicaWithRetries() handles those cases as well.  Or you 
Done


http://gerrit.cloudera.org:8080/#/c/14300/6/src/kudu/integration-tests/raft_config_change-itest.cc@628
PS6, Line 628: 
             : 
> OK, but given the exact timings, might it be a source for flakiness?  Or, p
Done


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@71
PS11, Line 71: using std::string;
> It makes sense to address the issues reported by the Tidy bot.
Done


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@465
PS11, Line 465:   // Check that 'added_replica_uuid' was added to the consensus
> Add short doc for the parameters.
Done


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@465
PS11, Line 465:   // Check that 'added_replica_uuid' was added to the consensus
              :   // and 'removed_replica_uuid' was removed from consensus.
              :   void ValidateConsensusStateChanged(const string& added_
> nit: align the parameters
Done


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@476
PS11, Line 476:     ASSERT_FALSE(IsRaftConfigMember(removed_replica_uuid, cstate.committed_config()));
> Document the parameters.
Done


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@477
PS11, Line 477: 
> any_replica_server ?
Done


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@480
PS11, Line 480:   // to quickly fall behind log GC.
              :   // Returns 'any_replica_sever' - UUID of first server with tablet replica,
              :   // and 'no_replica_server' - UUID of server without tablet replica.
              :   void SetUpCluster(bool add_flags_for_log_rolls,
              :                     string* any_replica_sever,
              :                     string* no_replica_server)
              :   {
> nit: document the reasoning behind adding these extra flags and relationshi
Done


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@511
PS11, Line 511: 
> nit: it seems this also can be factored out into a member of  the SlowTable
Done


http://gerrit.cloudera.org:8080/#/c/14300/11/src/kudu/integration-tests/raft_config_change-itest.cc@531
PS11, Line 531:     ValidateConsensusStateChanged(any_replica_server, no_replica_server, kTimeout);
> nit: add period
Done



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 12
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Tue, 08 Oct 2019 01:51:57 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Adar Dembo, Greg Solovyev, 

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

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

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

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long bootstrapping of tablet replica.
There case are covered:

1) Tablet replica bootstraps long time. No data modications happen during these time. Consensus state does not change.

2) Tablet replica is shutdown for some time. During this time a lot of data modifications happen, log rolls, replica is replaced (with re-replication). Replica restarts and find out the it fell behind.

3) Tablet replica bootstraps long time. During this time a lot of data modifications happen, log rolls, replica is replaced (with re-replication). Replica restarts and find out the it fell behind.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 203 insertions(+), 0 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 10
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 15:

> > Build Failed
 > >
 > > http://jenkins.kudu.apache.org/job/kudu-gerrit/19081/ : FAILURE
 > 
 > It seems something went wrong: the newly introduced tests reported
 > an error:
 > 
 > http://jenkins.kudu.apache.org/job/kudu-gerrit/19081/
 > 
 > One of the failures:
 > http://dist-test.cloudera.org/job?job_id=jenkins-slave.1570832890.32521

Ah, the problem is with --tablet_open_bootstrap_inject_latency_ms flag: the flag has been renamed, and the tests failed because of that.  It's necessary to update the tests as well to reflect the new name of the flag.


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 15
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Sat, 12 Oct 2019 02:14:08 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Adar Dembo, Greg Solovyev, 

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

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

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

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long
bootstrapping of tablet replica.
There case are covered:

1) Tablet replica bootstraps long time. No data modications happen
during these time. Consensus state does not change.

2) Tablet replica is shutdown for some time. During this time a lot of
data modifications happen, log rolls, replica is replaced (with
re-replication). Replica restarts and find out the it fell behind.

3) Tablet replica bootstraps long time. During this time a lot of data
modifications happen, log rolls, replica is replaced (with
re-replication). Replica restarts and find out the it fell behind.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 169 insertions(+), 0 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 13
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has submitted this change and it was merged. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long
bootstrapping of a tablet replica.
The following cases are covered:

1) A tablet replica bootstraps long time. No data modications happen
during that time. The replica is not evicted from the tablet Raft
configuration and joins the quorum after the bootstrapping process is
finished.

2) A tablet replica is shut down for some time. During this time a lot
of data modifications happen, log rolls, the replica is replaced (with
re-replication). The replica restarts and finds out that is has been
evicted from the tablet Raft configuration.

3) A tablet replica bootstraps long time. During this time a lot
of data modifications happen, log rolls, the replica is replaced (with
re-replication). The replica eventually finishes boostrap and finds out
that is has been evicted from the tablet Raft configuration.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Reviewed-on: http://gerrit.cloudera.org:8080/14300
Tested-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 163 insertions(+), 0 deletions(-)

Approvals:
  Alexey Serbin: Looks good to me, approved; Verified

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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 18
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 15:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/14300/15/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/15/src/kudu/integration-tests/raft_config_change-itest.cc@519
PS15, Line 519: tablet_open_bootstrap_inject_latency_ms
tablet_bootstrap_inject_latency_ms


http://gerrit.cloudera.org:8080/#/c/14300/15/src/kudu/integration-tests/raft_config_change-itest.cc@563
PS15, Line 563: tablet_open_bootstrap_inject_latency_ms
tablet_bootstrap_inject_latency_ms



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 15
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Sat, 12 Oct 2019 02:15:28 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 8:

> Build Failed
 > 
 > http://jenkins.kudu.apache.org/job/kudu-gerrit/18967/ : FAILURE

IWYU and Lint are not happy:

http://jenkins.kudu.apache.org/job/kudu-gerrit/18967/BUILD_TYPE=IWYU/artifact/build/latest/test-logs/iwyu.log

http://jenkins.kudu.apache.org/job/kudu-gerrit/18967/BUILD_TYPE=LINT/artifact/build/latest/test-logs/lint.log

You can run lint target on your machine before submitting patches (both on Linux and macOS):

  make ilint

To check against IWYU, you'll need a Linux machine:

  make iwyu


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 8
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Wed, 02 Oct 2019 23:39:37 +0000
Gerrit-HasComments: No

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Hello Tidy Bot, Alexey Serbin, Kudu Jenkins, Andrew Wong, Adar Dembo, Greg Solovyev, 

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

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

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

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................

KUDU-2800 Test long bootstrapping tablet replicas

These tests cover change of consensus state in case of long bootstrapping of tablet replica.
There case are covered:

1) Tablet replica bootstraps long time. No data modications happen during these time. Consensus state does not change.

2) Tablet replica is shutdown for some time. During this time a lot of data modifications happen, log rolls, replica is replaced (with re-replication). Replica restarts and find out the it fell behind.

3) Tablet replica bootstraps long time. During this time a lot of data modifications happen, log rolls, replica is replaced (with re-replication). Replica restarts and find out the it fell behind.

Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
---
M src/kudu/integration-tests/raft_config_change-itest.cc
M src/kudu/tserver/ts_tablet_manager.cc
2 files changed, 206 insertions(+), 0 deletions(-)


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 9
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Volodymyr Verovkin (Code Review)" <ge...@cloudera.org>.
Volodymyr Verovkin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 9:

(1 comment)

Ran
ninja ilint
ninja iwyu

http://gerrit.cloudera.org:8080/#/c/14300/8/src/kudu/integration-tests/raft_config_change-itest.cc
File src/kudu/integration-tests/raft_config_change-itest.cc:

http://gerrit.cloudera.org:8080/#/c/14300/8/src/kudu/integration-tests/raft_config_change-itest.cc@41
PS8, Line 41: #include "kudu/integration-tests/cluster_itest_util.h"
> warning: #includes are not sorted properly [llvm-include-order]
Done



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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 9
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Thu, 03 Oct 2019 00:54:00 +0000
Gerrit-HasComments: Yes

[kudu-CR] KUDU-2800 Test long bootstrapping tablet replicas

Posted by "Alexey Serbin (Code Review)" <ge...@cloudera.org>.
Alexey Serbin has posted comments on this change. ( http://gerrit.cloudera.org:8080/14300 )

Change subject: KUDU-2800 Test long bootstrapping tablet replicas
......................................................................


Patch Set 17: Code-Review+2


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

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ie1fee42053194f51d7a869ce14788095d6627ed9
Gerrit-Change-Number: 14300
Gerrit-PatchSet: 17
Gerrit-Owner: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Reviewer: Adar Dembo <ad...@cloudera.com>
Gerrit-Reviewer: Alexey Serbin <as...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Greg Solovyev <gs...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Reviewer: Tidy Bot (241)
Gerrit-Reviewer: Volodymyr Verovkin <ve...@cloudera.com>
Gerrit-Comment-Date: Mon, 14 Oct 2019 19:18:48 +0000
Gerrit-HasComments: No