You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2019/10/03 17:03:37 UTC

[kudu] branch master updated (708f250 -> f9adfd1)

This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


    from 708f250  [clock] remove 'experimental' tag from some clock-related flags
     new d119d52  create-table-itest: deflake TestCreateWhenMajorityOfReplicasFailCreation
     new f9adfd1  [mini_chronyd] introduce MiniChronyd::address()

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/clock/ntp-test.cc                       |  5 +++--
 src/kudu/clock/test/mini_chronyd-test.cc         | 20 ++++++++------------
 src/kudu/clock/test/mini_chronyd.cc              | 12 ++++++++++--
 src/kudu/clock/test/mini_chronyd.h               |  6 +++++-
 src/kudu/integration-tests/create-table-itest.cc | 16 +++++++++-------
 src/kudu/mini-cluster/external_mini_cluster.cc   |  3 +--
 6 files changed, 36 insertions(+), 26 deletions(-)


[kudu] 02/02: [mini_chronyd] introduce MiniChronyd::address()

Posted by al...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit f9adfd16b8d03b350b8dacf7bbb95fc1975dca14
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Wed Oct 2 16:21:32 2019 -0700

    [mini_chronyd] introduce MiniChronyd::address()
    
    Introduced MiniChronyd::address() method to find IP address and port
    at which the underlying NTP server is listening for incoming requests.
    In case if the NTP server is bound to the wildcard address, use the
    loopback address for NTP communication with the server.
    
    Also, updated corresponding call sites to use the new method instead of
    accessing MiniChronyd's options.
    
    Prior to this patch, the ToolTest.TestRemoteReplicaCopy test scenario
    would fail if running with built-in NTP client enabled.
    
    Change-Id: I62a326a47160aefe9a5806162e6df7c4d6060ba5
    Reviewed-on: http://gerrit.cloudera.org:8080/14355
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/clock/ntp-test.cc                     |  5 +++--
 src/kudu/clock/test/mini_chronyd-test.cc       | 20 ++++++++------------
 src/kudu/clock/test/mini_chronyd.cc            | 12 ++++++++++--
 src/kudu/clock/test/mini_chronyd.h             |  6 +++++-
 src/kudu/mini-cluster/external_mini_cluster.cc |  3 +--
 5 files changed, 27 insertions(+), 19 deletions(-)

diff --git a/src/kudu/clock/ntp-test.cc b/src/kudu/clock/ntp-test.cc
index 8cf7ee8..5985882 100644
--- a/src/kudu/clock/ntp-test.cc
+++ b/src/kudu/clock/ntp-test.cc
@@ -583,9 +583,10 @@ TEST_F(BuiltinNtpWithMiniChronydTest, SyncAndUnsyncReferenceServers) {
     options.index = 3;
     options.local = false;
     for (const auto& server : sync_servers) {
+      const auto addr = server->address();
       MiniChronydServerOptions server_options;
-      server_options.address = server->options().bindaddress;
-      server_options.port = server->options().port;
+      server_options.address = addr.host();
+      server_options.port = addr.port();
       options.servers.emplace_back(std::move(server_options));
     }
 
diff --git a/src/kudu/clock/test/mini_chronyd-test.cc b/src/kudu/clock/test/mini_chronyd-test.cc
index feba53f..a7776d5 100644
--- a/src/kudu/clock/test/mini_chronyd-test.cc
+++ b/src/kudu/clock/test/mini_chronyd-test.cc
@@ -59,8 +59,7 @@ TEST_F(MiniChronydTest, UnsynchronizedServer) {
     ASSERT_EQ(0, stats.ntp_packets_received);
   }
 
-  auto s = MiniChronyd::CheckNtpSource(
-      { HostPort(chrony->options().bindaddress, chrony->options().port) });
+  auto s = MiniChronyd::CheckNtpSource({ chrony->address() });
   ASSERT_TRUE(s.IsRuntimeError()) << s.ToString();
   ASSERT_STR_CONTAINS(s.ToString(),
                       "failed measure clock offset from reference NTP servers");
@@ -82,8 +81,7 @@ TEST_F(MiniChronydTest, BasicSingleServerInstance) {
 
   // A chronyd that uses the system clock as a reference lock should present
   // itself as reliable NTP server.
-  const HostPort ntp_endpoint(chrony->options().bindaddress,
-                              chrony->options().port);
+  const HostPort ntp_endpoint(chrony->address());
   {
     // Make sure the server opens ports to listen and serve requests
     // from NTP clients.
@@ -129,8 +127,7 @@ TEST_F(MiniChronydTest, BasicMultipleServerInstances) {
     options.index = idx;
     unique_ptr<MiniChronyd> chrony;
     ASSERT_OK(StartChronydAtAutoReservedPort(&chrony, &options));
-    ntp_endpoints.emplace_back(chrony->options().bindaddress,
-                               chrony->options().port);
+    ntp_endpoints.emplace_back(chrony->address());
     servers.emplace_back(std::move(chrony));
   }
 
@@ -193,8 +190,7 @@ TEST_F(MiniChronydTest, MultiTierBasic) {
     options.index = idx;
     unique_ptr<MiniChronyd> chrony;
     ASSERT_OK(StartChronydAtAutoReservedPort(&chrony, &options));
-    ntp_endpoints_0.emplace_back(chrony->options().bindaddress,
-                                 chrony->options().port);
+    ntp_endpoints_0.emplace_back(chrony->address());
     servers_0.emplace_back(std::move(chrony));
   }
 
@@ -205,15 +201,15 @@ TEST_F(MiniChronydTest, MultiTierBasic) {
     options.index = idx;
     options.local = false;
     for (const auto& ref : servers_0) {
+      const auto addr = ref->address();
       MiniChronydServerOptions server_options;
-      server_options.port = ref->options().port;
-      server_options.address = ref->options().bindaddress;
+      server_options.address = addr.host();
+      server_options.port = addr.port();
       options.servers.emplace_back(std::move(server_options));
     }
     unique_ptr<MiniChronyd> chrony;
     ASSERT_OK(StartChronydAtAutoReservedPort(&chrony, &options));
-    ntp_endpoints_1.emplace_back(chrony->options().bindaddress,
-                                 chrony->options().port);
+    ntp_endpoints_1.emplace_back(chrony->address());
     servers_1.emplace_back(std::move(chrony));
   }
 
diff --git a/src/kudu/clock/test/mini_chronyd.cc b/src/kudu/clock/test/mini_chronyd.cc
index 183f2b2..ecb9624 100644
--- a/src/kudu/clock/test/mini_chronyd.cc
+++ b/src/kudu/clock/test/mini_chronyd.cc
@@ -161,15 +161,23 @@ MiniChronyd::~MiniChronyd() {
 }
 
 const MiniChronydOptions& MiniChronyd::options() const {
-  CHECK(process_) << "must start the chronyd process first";
   return options_;
 }
 
 pid_t MiniChronyd::pid() const {
-  CHECK(process_) << "must start the chronyd process first";
+  CHECK(process_) << "must start chronyd process first";
   return process_->pid();
 }
 
+HostPort MiniChronyd::address() const {
+  CHECK(process_) << "must start chronyd process first";
+  // If the test NTP server is bound to the wildcard IP address,
+  // use the loopback IP address to communicate with the server.
+  return HostPort(options_.bindaddress == kWildcardIpAddr ? kLoopbackIpAddr
+                                                          : options_.bindaddress,
+                  options_.port);
+}
+
 Status MiniChronyd::Start() {
   SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, "starting chronyd");
   CHECK(!process_);
diff --git a/src/kudu/clock/test/mini_chronyd.h b/src/kudu/clock/test/mini_chronyd.h
index 5cc5e4f..b7bd45e 100644
--- a/src/kudu/clock/test/mini_chronyd.h
+++ b/src/kudu/clock/test/mini_chronyd.h
@@ -188,12 +188,16 @@ class MiniChronyd {
 
   ~MiniChronyd();
 
-  // Return the options the underlying chronyd has been started with.
+  // Return the options which the underlying chronyd is given to start with.
   const MiniChronydOptions& options() const;
 
   // Get the PID of the chronyd process.
   pid_t pid() const;
 
+  // Get the IP address and port at which the underlying NTP server is listening
+  // for incoming requests. Should be called only when NTP server is started.
+  HostPort address() const;
+
   // Start the mini chronyd in server-only mode.
   Status Start() WARN_UNUSED_RESULT;
 
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index cd33faf..23814be 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -165,8 +165,7 @@ Status ExternalMiniCluster::AddNtpFlags(std::vector<std::string>* flags) {
     vector<string> ntp_endpoints;
     CHECK_EQ(opts_.num_ntp_servers, ntp_servers_.size());
     for (const auto& server : ntp_servers_) {
-      const auto& opt = server->options();
-      ntp_endpoints.emplace_back(HostPort(opt.bindaddress, opt.port).ToString());
+      ntp_endpoints.emplace_back(server->address().ToString());
     }
     // Point the built-in NTP client to the test NTP server running as a part
     // of the cluster.


[kudu] 01/02: create-table-itest: deflake TestCreateWhenMajorityOfReplicasFailCreation

Posted by al...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit d119d529beb691a84134d02e33ecdce6102a7a35
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Wed Oct 2 23:06:36 2019 -0700

    create-table-itest: deflake TestCreateWhenMajorityOfReplicasFailCreation
    
    The last check in this test only had 10s to complete, which isn't enough in
    TSAN builds due to KUDU-2963. Let's replace it with an ASSERT_EVENTUALLY
    which, besides being cleaner, also gives it more time to finish.
    
    Change-Id: I37da7ea12f5432b947200fe8f08540e7b1ecbcec
    Reviewed-on: http://gerrit.cloudera.org:8080/14358
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/integration-tests/create-table-itest.cc | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)

diff --git a/src/kudu/integration-tests/create-table-itest.cc b/src/kudu/integration-tests/create-table-itest.cc
index ba7fc39..6d41b57 100644
--- a/src/kudu/integration-tests/create-table-itest.cc
+++ b/src/kudu/integration-tests/create-table-itest.cc
@@ -140,15 +140,17 @@ TEST_F(CreateTableITest, TestCreateWhenMajorityOfReplicasFailCreation) {
   // The server that was up from the beginning should be left with only
   // one tablet, eventually, since the tablets which failed to get created
   // properly should get deleted.
-  vector<string> tablets;
-  int wait_iter = 0;
-  while (tablets.size() != 1 && wait_iter++ < 100) {
+  //
+  // Note that the tablet count can rise before it falls due to the server
+  // handling a mix of obsolete CreateTablet RPCs (see KUDU-2963 for more
+  // details). This test isn't terribly precise; we may catch the server with
+  // just one tablet during the "upswing", before reaching the steady state.
+  ASSERT_EVENTUALLY([&] {
+    vector<string> tablets = inspect_->ListTabletsWithDataOnTS(0);
     LOG(INFO) << "Waiting for only one tablet to be left on TS 0. Currently have: "
               << tablets;
-    SleepFor(MonoDelta::FromMilliseconds(100));
-    tablets = inspect_->ListTabletsWithDataOnTS(0);
-  }
-  ASSERT_EQ(1, tablets.size()) << "Tablets on TS0: " << tablets;
+    ASSERT_EQ(1, tablets.size()) << "Tablets on TS0: " << tablets;
+  });
 }
 
 // Regression test for KUDU-1317. Ensure that, when a table is created,