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/12/04 05:58:35 UTC

[kudu] branch master updated (e45f893 -> eb2b70d)

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 e45f893  KUDU-1938 Make UTF-8 truncation faster pt 2
     new 7ea1f53  [test] fix HybridClockTest.TestWaitUntilAfter_TestCase2
     new eb2b70d  [tests] use system_unsync time source for tests by default

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/client/client_examples-test.sh        |  2 ++
 src/kudu/clock/hybrid_clock-test.cc            |  8 ++++--
 src/kudu/clock/hybrid_clock.cc                 | 17 +++++++------
 src/kudu/mini-cluster/external_mini_cluster.cc | 34 ++++++++++++++------------
 src/kudu/mini-cluster/external_mini_cluster.h  |  4 +--
 src/kudu/scripts/start_kudu.sh                 | 11 ++++++++-
 src/kudu/util/test_util.cc                     | 22 ++++++++++-------
 src/kudu/util/test_util.h                      |  7 ------
 8 files changed, 61 insertions(+), 44 deletions(-)


[kudu] 02/02: [tests] use system_unsync time source for tests by default

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 eb2b70d4b96be2fc2fdd6b3625acc284ac5774be
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Sat Oct 5 12:17:46 2019 -0700

    [tests] use system_unsync time source for tests by default
    
    This patch switches all KuduTest-derived tests to use 'system_unsync'
    time source by default (note that becomes so for test based on both
    internal and external mini-clusters).  The rationale is explained below.
    
    Essentially, there is no need to require NTP-synchronized clock for
    a generic test derived from KuduTest.  All the participating Kudu
    masters and tablet servers (and their sub-components in case of unit
    tests) are run at the same node using the same local wallclock.  Using
    the same wallclock makes the synchronisation of clocks unnecessary.
    
    The scenarios that verify the functioning of particular Kudu subsystems
    when using the 'system' and the 'builtin' clock sources should be put
    into a set of dedicated scenarios.  That's partially so even now: the
    scenarios which require verification of the built-in NTP client
    functionality are separated into a dedicated test set.  However, it's
    necessary to add dedicated test scenarios for the 'system' clock source
    and add more coverage for the 'built-in' clock source.
    
    That's exactly what follow-up changelists will do, i.e. add dedicated
    test scenarios to cover the specific functionality attributed to the use
    of the following clock sources:
      * system  : local machine's wallclock clock synchronized with NTP
      * builtin : independently tracked true time by the built-in NTP client
    
    Change-Id: If7edbf884afaa19121aa92a4ce93c8a7eeb2d937
    Reviewed-on: http://gerrit.cloudera.org:8080/14379
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/client/client_examples-test.sh        |  2 ++
 src/kudu/clock/hybrid_clock-test.cc            |  4 +++
 src/kudu/clock/hybrid_clock.cc                 | 17 +++++++------
 src/kudu/mini-cluster/external_mini_cluster.cc | 34 ++++++++++++++------------
 src/kudu/mini-cluster/external_mini_cluster.h  |  4 +--
 src/kudu/scripts/start_kudu.sh                 | 11 ++++++++-
 src/kudu/util/test_util.cc                     | 22 ++++++++++-------
 src/kudu/util/test_util.h                      |  7 ------
 8 files changed, 59 insertions(+), 42 deletions(-)

diff --git a/src/kudu/client/client_examples-test.sh b/src/kudu/client/client_examples-test.sh
index 75d5ac7..a495e08 100755
--- a/src/kudu/client/client_examples-test.sh
+++ b/src/kudu/client/client_examples-test.sh
@@ -162,6 +162,7 @@ mkdir -p "$BASE_DIR/master/logs"
   --log_dir="$BASE_DIR/master/logs" \
   --fs_wal_dir="$BASE_DIR/master/wals" \
   --fs_data_dirs="$BASE_DIR/master/data" \
+  --time_source=system_unsync \
   --webserver_interface=$LOCALHOST_IP \
   --webserver_port=0 \
   --rpc_bind_addresses=$LOCALHOST_IP:$MASTER_RPC_PORT &
@@ -178,6 +179,7 @@ mkdir -p "$BASE_DIR/ts/logs"
   --fs_data_dirs="$BASE_DIR/ts/data" \
   --rpc_bind_addresses=$LOCALHOST_IP:$TSERVER_RPC_PORT \
   --local_ip_for_outbound_sockets=$LOCALHOST_IP \
+  --time_source=system_unsync \
   --webserver_interface=$LOCALHOST_IP \
   --webserver_port=0 \
   --tserver_master_addrs=$LOCALHOST_IP:$MASTER_RPC_PORT &
diff --git a/src/kudu/clock/hybrid_clock-test.cc b/src/kudu/clock/hybrid_clock-test.cc
index b4aed17..d5e5641 100644
--- a/src/kudu/clock/hybrid_clock-test.cc
+++ b/src/kudu/clock/hybrid_clock-test.cc
@@ -353,6 +353,10 @@ TEST_F(HybridClockTest, TestRideOverNtpInterruption) {
 
 #ifndef __APPLE__
 TEST_F(HybridClockTest, TestNtpDiagnostics) {
+  FLAGS_time_source = "system";
+  clock_.reset(new HybridClock);
+  ASSERT_OK(clock_->Init());
+
   vector<string> log;
   clock_->time_service()->DumpDiagnostics(&log);
   string s = JoinStrings(log, "\n");
diff --git a/src/kudu/clock/hybrid_clock.cc b/src/kudu/clock/hybrid_clock.cc
index b15bc2a..0ea1408 100644
--- a/src/kudu/clock/hybrid_clock.cc
+++ b/src/kudu/clock/hybrid_clock.cc
@@ -29,6 +29,7 @@
 #include "kudu/clock/builtin_ntp.h"
 #include "kudu/clock/mock_ntp.h"
 #include "kudu/clock/system_ntp.h"
+#include "kudu/clock/system_unsync_time.h"
 #include "kudu/gutil/bind.h"
 #include "kudu/gutil/bind_helpers.h"
 #include "kudu/gutil/macros.h"
@@ -41,10 +42,6 @@
 #include "kudu/util/monotime.h"
 #include "kudu/util/status.h"
 
-#ifdef __APPLE__
-#include "kudu/clock/system_unsync_time.h"
-#endif
-
 using kudu::Status;
 using std::string;
 using strings::Substitute;
@@ -60,13 +57,17 @@ DEFINE_bool(use_hybrid_clock, true,
             "implementation. This should be disabled for testing purposes only.");
 TAG_FLAG(use_hybrid_clock, hidden);
 
+// Use the 'system' time source by default in standard (non-test) environment.
+// This requires local machine clock to be NTP-synchronized.
 DEFINE_string(time_source, "system",
               "The time source that HybridClock should use. Must be one of "
-              "'system', 'builtin', or 'mock' ('mock' is for tests only)");
+              "'builtin', 'system', 'system_unsync', or 'mock' "
+              "('system_unsync' and 'mock' are for tests only)");
 TAG_FLAG(time_source, evolving);
 DEFINE_validator(time_source, [](const char* flag_name, const string& value) {
-  if (boost::iequals(value, "system") ||
-      boost::iequals(value, "builtin") ||
+  if (boost::iequals(value, "builtin") ||
+      boost::iequals(value, "system") ||
+      boost::iequals(value, "system_unsync") ||
       boost::iequals(value, "mock")) {
     return true;
   }
@@ -137,6 +138,8 @@ Status HybridClock::Init() {
 #else
     time_service_.reset(new clock::SystemUnsyncTime);
 #endif
+  } else if (boost::iequals(FLAGS_time_source, "system_unsync")) {
+    time_service_.reset(new clock::SystemUnsyncTime);
   } else if (boost::iequals(FLAGS_time_source, "builtin")) {
     time_service_.reset(new clock::BuiltInNtp);
   } else {
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index 4cd450f..0120dc9 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -87,6 +87,7 @@ using kudu::server::ServerStatusPB;
 using kudu::tserver::ListTabletsRequestPB;
 using kudu::tserver::ListTabletsResponsePB;
 using kudu::tserver::TabletServerServiceProxy;
+using std::copy;
 using std::pair;
 using std::string;
 using std::unique_ptr;
@@ -121,10 +122,9 @@ ExternalMiniClusterOptions::ExternalMiniClusterOptions()
       start_process_timeout(MonoDelta::FromSeconds(70)),
       rpc_negotiation_timeout(MonoDelta::FromSeconds(3))
 #if !defined(NO_CHRONY)
-      ,
-      num_ntp_servers(UseSystemNtp() ? 0 : 1)
-#endif
-    {
+      , num_ntp_servers(1)
+#endif // #if !defined(NO_CHRONY) ...
+{
 }
 
 ExternalMiniCluster::ExternalMiniCluster()
@@ -167,9 +167,11 @@ Status ExternalMiniCluster::HandleOptions() {
   return Status::OK();
 }
 
-#if !defined(NO_CHRONY)
-Status ExternalMiniCluster::AddNtpFlags(std::vector<std::string>* flags) {
+Status ExternalMiniCluster::AddTimeSourceFlags(std::vector<std::string>* flags) {
   DCHECK(flags);
+#if defined(NO_CHRONY)
+  flags->emplace_back("--time_source=system_unsync");
+#else
   if (opts_.num_ntp_servers > 0) {
     vector<string> ntp_endpoints;
     CHECK_EQ(opts_.num_ntp_servers, ntp_servers_.size());
@@ -190,10 +192,12 @@ Status ExternalMiniCluster::AddNtpFlags(std::vector<std::string>* flags) {
     // Switch the clock to use the built-in NTP client which clock is
     // synchronized with the test NTP server.
     flags->emplace_back("--time_source=builtin");
+  } else {
+    flags->emplace_back("--time_source=system_unsync");
   }
+#endif // #if defined(NO_CHRONY) ... else ...
   return Status::OK();
 }
-#endif // #if !defined(NO_CHRONY) ...
 
 Status ExternalMiniCluster::StartSentry() {
   sentry_->SetDataRoot(opts_.cluster_root);
@@ -515,7 +519,7 @@ Status ExternalMiniCluster::StartMasters() {
     }
   }
 
-  vector<string> flags = opts_.extra_master_flags;
+  vector<string> flags;
   flags.emplace_back("--rpc_reuseport=true");
   if (num_masters > 1) {
     flags.emplace_back(Substitute("--master_addresses=$0",
@@ -544,9 +548,11 @@ Status ExternalMiniCluster::StartMasters() {
     flags.emplace_back("--location_mapping_by_uuid");
 #   endif
   }
-#if !defined(NO_CHRONY)
-  RETURN_NOT_OK(AddNtpFlags(&flags));
-#endif
+  RETURN_NOT_OK(AddTimeSourceFlags(&flags));
+
+  // Add custom master flags.
+  copy(opts_.extra_master_flags.begin(), opts_.extra_master_flags.end(),
+       std::back_inserter(flags));
 
   // Start the masters.
   const string& exe = GetBinaryPath(kMasterBinaryName);
@@ -630,11 +636,9 @@ Status ExternalMiniCluster::AddTabletServer() {
         Substitute("$0/perf-$1.data", opts.log_dir, daemon_id);
   }
   vector<string> extra_flags;
-#if !defined(NO_CHRONY)
-  RETURN_NOT_OK(AddNtpFlags(&extra_flags));
-#endif
+  RETURN_NOT_OK(AddTimeSourceFlags(&extra_flags));
   auto flags = SubstituteInFlags(opts_.extra_tserver_flags, idx);
-  std::copy(flags.begin(), flags.end(), std::back_inserter(extra_flags));
+  copy(flags.begin(), flags.end(), std::back_inserter(extra_flags));
   opts.extra_flags = extra_flags;
   opts.start_process_timeout = opts_.start_process_timeout;
   opts.rpc_bind_address = HostPort(bind_host, 0);
diff --git a/src/kudu/mini-cluster/external_mini_cluster.h b/src/kudu/mini-cluster/external_mini_cluster.h
index f02c7ac..428ec3a 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.h
+++ b/src/kudu/mini-cluster/external_mini_cluster.h
@@ -432,9 +432,7 @@ class ExternalMiniCluster : public MiniCluster {
 
   Status DeduceBinRoot(std::string* ret);
   Status HandleOptions();
-#if !defined(NO_CHRONY)
-  Status AddNtpFlags(std::vector<std::string>* flags);
-#endif
+  Status AddTimeSourceFlags(std::vector<std::string>* flags);
 
   ExternalMiniClusterOptions opts_;
 
diff --git a/src/kudu/scripts/start_kudu.sh b/src/kudu/scripts/start_kudu.sh
index 23e9a18..31fd2d2 100755
--- a/src/kudu/scripts/start_kudu.sh
+++ b/src/kudu/scripts/start_kudu.sh
@@ -29,12 +29,14 @@ cat << EOF
 Usage:
 start_kudu.sh [flags]
 -h, --help         Print help
--m, --num-masters  Number of Kudu Master servers to start (default: 1)
+-m, --num-masters  Number of Kudu Masters to start (default: 1)
 -t, --num-tservers Number of Kudu Tablet Servers to start (default: 3)
 --rpc-master       RPC port of first Kudu Master; HTTP port is the next number.
                    Subsequent Masters will have following numbers
 --rpc-tserver      RPC port of first Kudu Tablet Server; HTTP port is the next
                    number. Subsequent Tablet Servers will have following numbers
+--time_source      Time source for Kudu Masters and Tablet Servers
+                   (default: system_unsync)
 -b, --builddir     Path to the Kudu build directory
 EOF
 }
@@ -43,6 +45,7 @@ NUM_MASTERS=1
 NUM_TSERVERS=3
 MASTER_RPC_PORT_BASE=8764
 TSERVER_RPC_PORT_BASE=9870
+TIME_SOURCE=system_unsync
 BUILDDIR="$PWD"
 echo $(readlink -f $(dirname $0))
 while (( "$#" )); do
@@ -67,6 +70,10 @@ while (( "$#" )); do
       TSERVER_RPC_PORT_BASE=$2
       shift 2
       ;;
+    --time_source)
+      TIME_SOURCE=$2
+      shift 2
+      ;;
     -b|--builddir)
       BUILDDIR="$2"
       shift 2
@@ -138,6 +145,7 @@ function start_master() {
   ARGS="$ARGS --fs_wal_dir=$dir_wal"
   ARGS="$ARGS --log_dir=$dir_log"
   ARGS="$ARGS --rpc_bind_addresses=$IP:$RPC_PORT"
+  ARGS="$ARGS --time_source=$TIME_SOURCE"
   ARGS="$ARGS --webserver_port=$HTTP_PORT"
   ARGS="$ARGS --webserver_interface=$IP"
   ARGS="$ARGS --webserver_doc_root=$WEBSERVER_DOC_ROOT"
@@ -155,6 +163,7 @@ function start_tserver() {
   ARGS="$ARGS --fs_wal_dir=$dir_wal"
   ARGS="$ARGS --log_dir=$dir_log"
   ARGS="$ARGS --rpc_bind_addresses=$IP:$RPC_PORT"
+  ARGS="$ARGS --time_source=$TIME_SOURCE"
   ARGS="$ARGS --webserver_port=$HTTP_PORT"
   ARGS="$ARGS --webserver_interface=$IP"
   ARGS="$ARGS --webserver_doc_root=$WEBSERVER_DOC_ROOT"
diff --git a/src/kudu/util/test_util.cc b/src/kudu/util/test_util.cc
index c41d1d1..ec9e23b 100644
--- a/src/kudu/util/test_util.cc
+++ b/src/kudu/util/test_util.cc
@@ -38,6 +38,7 @@
 
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <gtest/gtest-spi.h>
 
@@ -65,6 +66,8 @@ DEFINE_string(test_leave_files, "on_failure",
 
 DEFINE_int32(test_random_seed, 0, "Random seed to use for randomized tests");
 
+DECLARE_string(time_source);
+
 using boost::optional;
 using std::string;
 using std::vector;
@@ -74,7 +77,6 @@ namespace kudu {
 
 const char* kInvalidPath = "/dev/invalid-path-for-kudu-tests";
 static const char* const kSlowTestsEnvVar = "KUDU_ALLOW_SLOW_TESTS";
-static const char* const kUseSystemNtpEnvVar = "KUDU_USE_SYSTEM_NTP";
 
 static const uint64_t kTestBeganAtMicros = Env::Default()->NowMicros();
 
@@ -90,9 +92,9 @@ bool g_is_gtest = true;
 ///////////////////////////////////////////////////
 
 KuduTest::KuduTest()
-  : env_(Env::Default()),
-    flag_saver_(new google::FlagSaver()),
-    test_dir_(GetTestDataDirectory()) {
+    : env_(Env::Default()),
+      flag_saver_(new google::FlagSaver()),
+      test_dir_(GetTestDataDirectory()) {
   std::map<const char*, const char*> flags_for_tests = {
     // Disabling fsync() speeds up tests dramatically, and it's safe to do as no
     // tests rely on cutting power to a machine or equivalent.
@@ -109,10 +111,16 @@ KuduTest::KuduTest()
     {"ipki_server_key_size", "1024"},
     {"ipki_ca_key_size", "1024"},
     {"tsk_num_rsa_bits", "512"},
+    // For a generic Kudu test, the local wall-clock time is good enough even
+    // if it's not synchronized by NTP. All test components are run at the same
+    // node, so there aren't multiple time sources to synchronize.
+    {"time_source", "system_unsync"},
   };
   for (const auto& e : flags_for_tests) {
     // We don't check for errors here, because we have some default flags that
-    // only apply to certain tests.
+    // only apply to certain tests. If a flag is defined in a library which
+    // the test binary isn't linked with, then SetCommandLineOptionWithMode()
+    // reports an error since the flag is unknown to the gflags runtime.
     google::SetCommandLineOptionWithMode(e.first, e.second, google::SET_FLAGS_DEFAULT);
   }
   // If the TEST_TMPDIR variable has been set, then glog will automatically use that
@@ -196,10 +204,6 @@ bool AllowSlowTests() {
   return GetBooleanEnvironmentVariable(kSlowTestsEnvVar);
 }
 
-bool UseSystemNtp() {
-  return GetBooleanEnvironmentVariable(kUseSystemNtpEnvVar);
-}
-
 void OverrideFlagForSlowTests(const std::string& flag_name,
                               const std::string& new_value) {
   // Ensure that the flag is valid.
diff --git a/src/kudu/util/test_util.h b/src/kudu/util/test_util.h
index 7e9764d..d320e3e 100644
--- a/src/kudu/util/test_util.h
+++ b/src/kudu/util/test_util.h
@@ -87,13 +87,6 @@ class KuduTest : public ::testing::Test {
 // Returns true if slow tests are runtime-enabled.
 bool AllowSlowTests();
 
-// Returns true if tests should rely on the system clock synchronized by the
-// kernel NTP discipline. By default, test clusters should run their own
-// test NTP server and configure Kudu masters and tablet servers to use the
-// built-in NTP client as the clock source for HybridTime, where the built-in
-// NTP clients are pointed to the test cluster's dedicated NTP server.
-bool UseSystemNtp();
-
 // Override the given gflag to the new value, only in the case that
 // slow tests are enabled and the user hasn't otherwise overridden
 // it on the command line.


[kudu] 01/02: [test] fix HybridClockTest.TestWaitUntilAfter_TestCase2

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 7ea1f53f8b97226e9edd744add1ea2ae40e96781
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Tue Dec 3 18:27:29 2019 -0800

    [test] fix HybridClockTest.TestWaitUntilAfter_TestCase2
    
    In case of slow execution (e.g., a TSAN build) and very low estimated
    clock error (e.g., 0 as in case of 'system_unsync' clock), calling
    HybridClock::WaitUntilAfter() for a fresh past hybrid timestamp returns
    Status::OK() instead of Status::TimedOut() regardless of deadline
    because the supplied timestamp is seen clearly in the past.
    
    This patch addresses the issue by moving the target timestamp further
    into the future.  Now it's 2 milliseconds in the future, but it was
    2 microseconds in the future prior to this patch.
    
    Change-Id: I34108359a04b5bd3f2a957faf6d2e9a6dc022eb1
    Reviewed-on: http://gerrit.cloudera.org:8080/14823
    Tested-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/clock/hybrid_clock-test.cc | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/kudu/clock/hybrid_clock-test.cc b/src/kudu/clock/hybrid_clock-test.cc
index 09505e7..b4aed17 100644
--- a/src/kudu/clock/hybrid_clock-test.cc
+++ b/src/kudu/clock/hybrid_clock-test.cc
@@ -205,7 +205,7 @@ TEST_F(HybridClockTest, TestWaitUntilAfter_TestCase1) {
 // The normal case for transactions. Obtain a timestamp and then wait until
 // we're sure that tx_latest < now_earliest.
 TEST_F(HybridClockTest, TestWaitUntilAfter_TestCase2) {
-  MonoTime before = MonoTime::Now();
+  const MonoTime before = MonoTime::Now();
 
   // we do no time adjustment, this event should fall right within the possible
   // error interval
@@ -214,7 +214,7 @@ TEST_F(HybridClockTest, TestWaitUntilAfter_TestCase2) {
   clock_->NowWithError(&past_ts, &past_max_error);
   // Make sure the error is at least a small number of microseconds, to ensure
   // that we always have to wait.
-  past_max_error = std::max(past_max_error, static_cast<uint64_t>(20));
+  past_max_error = std::max(past_max_error, static_cast<uint64_t>(2000));
   Timestamp wait_until = HybridClock::AddPhysicalTimeToTimestamp(
       past_ts,
       MonoDelta::FromMicroseconds(past_max_error));