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/11/12 16:51:08 UTC

[kudu] branch branch-1.11.x updated (3b84cf7 -> ac16dcf)

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

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


    from 3b84cf7  KUDU-2989. Work around SASL bug when FQDN is >=64 characters
     new 4f21110  docs: additional instructions for installing memkind from source
     new 4b88dde  KUDU-2994: add NO_CHRONY flag
     new ac16dcf  [docs] update 1.11.1 release notes

The 3 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:
 CMakeLists.txt                                     |  4 ++
 .../mini-cluster/build_mini_cluster_binaries.sh    |  8 ++-
 docs/installation.adoc                             | 60 ++++++++++++++++++++++
 docs/release_notes.adoc                            | 16 ++++++
 src/kudu/cfile/block_cache.cc                      |  2 +-
 src/kudu/clock/CMakeLists.txt                      | 52 ++++++++++---------
 src/kudu/clock/ntp-test.cc                         | 10 ++++
 src/kudu/mini-cluster/CMakeLists.txt               | 14 +++--
 .../mini-cluster/external_mini_cluster-test.cc     | 18 +++++--
 src/kudu/mini-cluster/external_mini_cluster.cc     | 24 ++++++++-
 src/kudu/mini-cluster/external_mini_cluster.h      | 12 +++++
 11 files changed, 184 insertions(+), 36 deletions(-)


[kudu] 01/03: docs: additional instructions for installing memkind from source

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

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

commit 4f21110d57391e452e9cde787cfc091b608b1347
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Sat Nov 9 14:59:01 2019 -0800

    docs: additional instructions for installing memkind from source
    
    This is often necessary as most of our supported distros include a memkind
    that's too old. There are some complications, however:
    - Using the default prefix (/usr/local) means Kudu won't find the library.
    - Even after installing to /usr, ldconfig must be run to regenerate the
      cache of dynamic libraries in "trusted locations" like /usr/lib.
    
    Change-Id: I7d52d02ad87a27e5fef5ba5c0ab8932128a29762
    Reviewed-on: http://gerrit.cloudera.org:8080/14683
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    (cherry picked from commit b97c2f6ca78097186c3c77e759cf43bab966cc7b)
      Conflicts:
        docs/installation.adoc
    Reviewed-on: http://gerrit.cloudera.org:8080/14696
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 docs/installation.adoc        | 60 +++++++++++++++++++++++++++++++++++++++++++
 src/kudu/cfile/block_cache.cc |  2 +-
 2 files changed, 61 insertions(+), 1 deletion(-)

diff --git a/docs/installation.adoc b/docs/installation.adoc
index f7dc39f..3497501 100644
--- a/docs/installation.adoc
+++ b/docs/installation.adoc
@@ -109,6 +109,26 @@ $ sudo yum install -y scl-utils ${DTLS_RPM}
 $ sudo yum install -y devtoolset-3-toolchain
 ----
 
+. Optional: If support for Kudu's NVM (non-volatile memory) block cache is
+desired, install the memkind library.
++
+----
+$ sudo yum install memkind
+----
++
+If the memkind package provided with the Linux distribution is too old (1.8.0 or
+newer is required), build and install it from source.
++
+----
+$ sudo yum install numactl-libs numactl-devel
+$ git clone https://github.com/memkind/memkind.git
+$ cd memkind
+$ ./build.sh --prefix=/usr
+$ sudo yum remove memkind
+$ sudo make install
+$ sudo ldconfig
+----
+
 . Optional: Install some additional packages, including ruby, if you plan to build documentation.
 +
 ----
@@ -239,6 +259,26 @@ $ sudo apt-get install autoconf automake curl flex g++ gcc gdb git \
   openjdk-8-jdk openssl patch pkg-config python rsync unzip vim-common
 ----
 
+. Optional: If support for Kudu's NVM (non-volatile memory) block cache is
+desired, install the memkind library.
++
+----
+$ sudo apt-get install libmemkind0
+----
++
+If the memkind package provided with the Linux distribution is too old (1.8.0 or
+newer is required), build and install it from source.
++
+----
+$ sudo apt-get install libnuma1 libnuma-dev
+$ git clone https://github.com/memkind/memkind.git
+$ cd memkind
+$ ./build.sh --prefix=/usr
+$ sudo apt-get remove memkind
+$ sudo make install
+$ sudo ldconfig
+----
+
 . Optional: Install some additional packages, including ruby, if you plan to build documentation.
 +
 ----
@@ -356,6 +396,26 @@ $ sudo zypper install autoconf automake curl cyrus-sasl-devel \
   pkg-config python rsync unzip vim
 ----
 
+. Optional: If support for Kudu's NVM (non-volatile memory) block cache is
+desired, install the memkind library.
++
+----
+$ sudo zypper install memkind
+----
++
+If the memkind package provided with the Linux distribution is too old (1.8.0 or
+newer is required), build and install it from source.
++
+----
+$ sudo zypper install numactl-libs numactl-devel
+$ git clone https://github.com/memkind/memkind.git
+$ cd memkind
+$ ./build.sh --prefix=/usr
+$ sudo zypper remove memkind
+$ sudo make install
+$ sudo ldconfig
+----
+
 . Clone the Git repository and change to the new `kudu` directory.
 +
 [source,bash]
diff --git a/src/kudu/cfile/block_cache.cc b/src/kudu/cfile/block_cache.cc
index c00bee1..bf93703 100644
--- a/src/kudu/cfile/block_cache.cc
+++ b/src/kudu/cfile/block_cache.cc
@@ -55,7 +55,7 @@ DEFINE_string(block_cache_type, "DRAM",
               "Valid choices are 'DRAM' or 'NVM'. DRAM, the default, "
               "caches data in regular memory. 'NVM' caches data "
               "in a memory-mapped file using the memkind library. To use 'NVM', "
-              "libmemkind 1.6.0 or newer must be available on the system; "
+              "libmemkind 1.8.0 or newer must be available on the system; "
               "otherwise Kudu will crash.");
 
 using strings::Substitute;


[kudu] 02/03: KUDU-2994: add NO_CHRONY flag

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

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

commit 4b88dde117fc6166b485767f1e846bcc0f4ca745
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Mon Nov 11 20:30:31 2019 -0800

    KUDU-2994: add NO_CHRONY flag
    
    This patch addresses KUDU-2994, adding NO_CHRONY cmake/make flag.
    If -DNO_CHRONY=1 is specified with cmake options, then the Kudu testing
    framework does not rely on chronyc/chronyd, and no corresponding
    binaries are deployed under $BUILD_DIR/bin directory.  Also, the
    libmini_chronyd and libmini_chronyd_test_util libraries are not built
    and references to MiniChronyd are removed from ExternalMiniCluster
    in that case.
    
    The build_mini_cluster_binaries.sh script was updated to include
    -DNO_CHRONY=1 option correspondingly.
    
    I verified that the result set of tests passes using dist-test:
      http://dist-test.cloudera.org/job?job_id=aserbin.1573546681.122233
    
    Change-Id: I95495f8a2418546048e2ea28c4f44cb399f08fd3
    Reviewed-on: http://gerrit.cloudera.org:8080/14691
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Kudu Jenkins
    (cherry picked from commit 74bf0859c01f20de777e499805ddc256b5e9a6a1)
    Reviewed-on: http://gerrit.cloudera.org:8080/14695
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 CMakeLists.txt                                     |  4 ++
 .../mini-cluster/build_mini_cluster_binaries.sh    |  8 +++-
 src/kudu/clock/CMakeLists.txt                      | 52 ++++++++++++----------
 src/kudu/clock/ntp-test.cc                         | 10 +++++
 src/kudu/mini-cluster/CMakeLists.txt               | 14 +++---
 .../mini-cluster/external_mini_cluster-test.cc     | 18 ++++++--
 src/kudu/mini-cluster/external_mini_cluster.cc     | 24 +++++++++-
 src/kudu/mini-cluster/external_mini_cluster.h      | 12 +++++
 8 files changed, 107 insertions(+), 35 deletions(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index 2b3f2f9..ec3d8ff 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -1120,6 +1120,10 @@ include_directories(${KERBEROS_INCLUDE_DIR})
 ADD_THIRDPARTY_LIB(krb5
   SHARED_LIB "${KERBEROS_LIBRARY}")
 
+if (NO_CHRONY)
+  add_definitions("-DNO_CHRONY")
+endif()
+
 ## Google PerfTools
 ##
 ## Disabled with TSAN/ASAN as well as with gold+dynamic linking (see
diff --git a/build-support/mini-cluster/build_mini_cluster_binaries.sh b/build-support/mini-cluster/build_mini_cluster_binaries.sh
index 61efe34..2f9e9ee 100755
--- a/build-support/mini-cluster/build_mini_cluster_binaries.sh
+++ b/build-support/mini-cluster/build_mini_cluster_binaries.sh
@@ -103,11 +103,15 @@ rm -rf CMakeCache.txt CMakeFiles
 
 # We want a fast build with a small total output size, so we need to build in
 # release mode with dynamic linking so that all of the target executables can
-# use the same shared objects for their dependencies.
+# use the same shared objects for their dependencies. Since we may not
+# distribute chronyc/chronyd in kudu-binaries JAR due to licensing restrictions,
+# the test harness is built to not rely on chronyd as NTP server for tests run
+# with the mini-cluster.
 echo Configuring Kudu... >&2
 $SOURCE_ROOT/build-support/enable_devtoolset.sh \
   $SOURCE_ROOT/thirdparty/installed/common/bin/cmake ../.. \
-  -DNO_TESTS=1 -DCMAKE_BUILD_TYPE=RELEASE -DKUDU_LINK=dynamic $EXTRA_CMAKE_FLAGS
+  -DNO_TESTS=1 -DNO_CHRONY=1 \
+  -DCMAKE_BUILD_TYPE=RELEASE -DKUDU_LINK=dynamic $EXTRA_CMAKE_FLAGS
 
 echo Building Kudu... >&2
 NUM_PROCS=$(getconf _NPROCESSORS_ONLN)
diff --git a/src/kudu/clock/CMakeLists.txt b/src/kudu/clock/CMakeLists.txt
index 52204a4..ddb09f4 100644
--- a/src/kudu/clock/CMakeLists.txt
+++ b/src/kudu/clock/CMakeLists.txt
@@ -37,35 +37,41 @@ target_link_libraries(clock
 # mini_chronyd
 ##############################
 
-# These are copied/installed instead of linking because:
-#  * symlinks would not work with dist-test
-#  * hardlinks would not work if the target directory is at different
-#    filesystem than thirdparty
-file(COPY "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/bin/chronyc"
-  DESTINATION "${EXECUTABLE_OUTPUT_PATH}")
-file(COPY "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/sbin/chronyd"
-  DESTINATION "${EXECUTABLE_OUTPUT_PATH}")
+if (NOT NO_CHRONY)
+  # These are copied/installed instead of linking because:
+  #  * symlinks would not work with dist-test
+  #  * hardlinks would not work if the target directory is at different
+  #    filesystem than thirdparty
+  file(COPY "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/bin/chronyc"
+    DESTINATION "${EXECUTABLE_OUTPUT_PATH}")
+  file(COPY "${CMAKE_SOURCE_DIR}/thirdparty/installed/common/sbin/chronyd"
+    DESTINATION "${EXECUTABLE_OUTPUT_PATH}")
 
-set(MINI_CHRONYD_SRCS test/mini_chronyd.cc)
+  set(MINI_CHRONYD_SRCS test/mini_chronyd.cc)
 
-add_library(mini_chronyd ${MINI_CHRONYD_SRCS})
-target_link_libraries(mini_chronyd
-  gutil
-  kudu_test_util
-  kudu_util)
+  add_library(mini_chronyd ${MINI_CHRONYD_SRCS})
+  target_link_libraries(mini_chronyd
+    gutil
+    kudu_test_util
+    kudu_util)
 
-set(MINI_CHRONYD_TEST_UTIL_SRCS test/mini_chronyd_test_util.cc)
-add_library(mini_chronyd_test_util ${MINI_CHRONYD_TEST_UTIL_SRCS})
-target_link_libraries(mini_chronyd_test_util
-  gutil
-  kudu_test_util
-  kudu_util
-  mini_cluster)
+  set(MINI_CHRONYD_TEST_UTIL_SRCS test/mini_chronyd_test_util.cc)
+  add_library(mini_chronyd_test_util ${MINI_CHRONYD_TEST_UTIL_SRCS})
+  target_link_libraries(mini_chronyd_test_util
+    gutil
+    kudu_test_util
+    kudu_util
+    mini_cluster)
+endif() # (NOT NO_CHRONY)
 
 SET_KUDU_TEST_LINK_LIBS(clock)
 ADD_KUDU_TEST(hybrid_clock-test PROCESSORS 3)
 ADD_KUDU_TEST(logical_clock-test)
 
-SET_KUDU_TEST_LINK_LIBS(clock mini_chronyd mini_chronyd_test_util)
-ADD_KUDU_TEST(test/mini_chronyd-test)
+if (NO_CHRONY)
+  SET_KUDU_TEST_LINK_LIBS(clock)
+else()
+  SET_KUDU_TEST_LINK_LIBS(clock mini_chronyd mini_chronyd_test_util)
+  ADD_KUDU_TEST(test/mini_chronyd-test)
+endif()
 ADD_KUDU_TEST(ntp-test)
diff --git a/src/kudu/clock/ntp-test.cc b/src/kudu/clock/ntp-test.cc
index deb9b8e..50232f6 100644
--- a/src/kudu/clock/ntp-test.cc
+++ b/src/kudu/clock/ntp-test.cc
@@ -15,10 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#if !defined(NO_CHRONY)
 #include <algorithm>
 #include <cinttypes>
 #include <cstdint>
 #include <ctime>
+#endif
 #include <iterator>
 #include <memory>
 #include <string>
@@ -26,10 +28,13 @@
 #include <vector>
 
 #include <gflags/gflags_declare.h>
+#if !defined(NO_CHRONY)
 #include <glog/logging.h>
+#endif
 #include <gtest/gtest.h>
 
 #include "kudu/clock/builtin_ntp-internal.h"
+#if !defined(NO_CHRONY)
 #include "kudu/clock/builtin_ntp.h"
 #include "kudu/clock/test/mini_chronyd.h"
 #include "kudu/clock/test/mini_chronyd_test_util.h"
@@ -39,6 +44,7 @@
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
+#endif
 
 DECLARE_int32(ntp_initial_sync_wait_secs);
 DECLARE_string(builtin_ntp_servers);
@@ -396,6 +402,8 @@ TEST(TimeIntervalsTest, ThreeResponses) {
   }
 }
 
+#if !defined(NO_CHRONY)
+
 #define WALLTIME_DIAG_FMT   "%" PRId64 " +/- %8" PRId64 " us"
 
 // Test to verify functionality of the built-in NTP client by communicating
@@ -673,5 +681,7 @@ TEST_F(BuiltinNtpWithMiniChronydTest, SyncAndUnsyncReferenceServers) {
 #endif // #ifndef __APPLE__
 }
 
+#endif // #if !defined(NO_CHRONY)
+
 } // namespace clock
 } // namespace kudu
diff --git a/src/kudu/mini-cluster/CMakeLists.txt b/src/kudu/mini-cluster/CMakeLists.txt
index c1fe5bd..9b4e55b 100644
--- a/src/kudu/mini-cluster/CMakeLists.txt
+++ b/src/kudu/mini-cluster/CMakeLists.txt
@@ -21,8 +21,7 @@ set(MINI_CLUSTER_SRCS
   mini_cluster.cc
 )
 
-add_library(mini_cluster ${MINI_CLUSTER_SRCS})
-target_link_libraries(mini_cluster
+set(MINI_CLUSTER_LIBS
   gflags
   glog
   gmock
@@ -34,7 +33,6 @@ target_link_libraries(mini_cluster
   kudu_util
   master
   master_proto
-  mini_chronyd
   mini_hms
   mini_kdc
   mini_sentry
@@ -43,8 +41,14 @@ target_link_libraries(mini_cluster
   tserver
   tserver_proto
   tserver_service_proto
-  wire_protocol_proto
-)
+  wire_protocol_proto)
+
+if (NOT NO_CHRONY)
+  set(MINI_CLUSTER_LIBS ${MINI_CLUSTER_LIBS} mini_chronyd)
+endif()
+
+add_library(mini_cluster ${MINI_CLUSTER_SRCS})
+target_link_libraries(mini_cluster ${MINI_CLUSTER_LIBS})
 add_dependencies(mini_cluster
   kudu-tserver
   kudu-master)
diff --git a/src/kudu/mini-cluster/external_mini_cluster-test.cc b/src/kudu/mini-cluster/external_mini_cluster-test.cc
index 66faa32..70d81bf 100644
--- a/src/kudu/mini-cluster/external_mini_cluster-test.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster-test.cc
@@ -101,17 +101,27 @@ std::ostream& operator<<(std::ostream& o, BuiltInNtp opt) {
 
 class ExternalMiniClusterTest :
     public KuduTest,
-    public testing::WithParamInterface<tuple<Kerberos, HiveMetastore, BuiltInNtp>> {
+#if !defined(NO_CHRONY)
+    public testing::WithParamInterface<tuple<Kerberos, HiveMetastore, BuiltInNtp>>
+#else
+    public testing::WithParamInterface<tuple<Kerberos, HiveMetastore>>
+#endif
+{
 };
 
 INSTANTIATE_TEST_CASE_P(,
     ExternalMiniClusterTest,
     ::testing::Combine(
         ::testing::Values(Kerberos::DISABLED, Kerberos::ENABLED),
-        ::testing::Values(HiveMetastore::DISABLED, HiveMetastore::ENABLED),
+        ::testing::Values(HiveMetastore::DISABLED, HiveMetastore::ENABLED)
+#if !defined(NO_CHRONY)
+        ,
         ::testing::Values(BuiltInNtp::DISABLED,
                           BuiltInNtp::ENABLED_SINGLE_SERVER,
-                          BuiltInNtp::ENABLED_MULTIPLE_SERVERS)));
+                          BuiltInNtp::ENABLED_MULTIPLE_SERVERS,
+                          BuiltInNtp::DISABLED)
+#endif // #if !defined(NO_CHRONY) ...
+                          ));
 
 void SmokeTestKerberizedCluster(ExternalMiniClusterOptions opts) {
   ASSERT_TRUE(opts.enable_kerberos);
@@ -263,7 +273,9 @@ TEST_P(ExternalMiniClusterTest, TestBasicOperation) {
   if (std::get<1>(param) == HiveMetastore::ENABLED) {
     opts.hms_mode = HmsMode::ENABLE_HIVE_METASTORE;
   }
+#if !defined(NO_CHRONY)
   opts.num_ntp_servers = std::get<2>(param);
+#endif
 
   opts.num_masters = 3;
   opts.num_tablet_servers = 3;
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index b2f01bc..4cd450f 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -34,7 +34,9 @@
 
 #include "kudu/client/client.h"
 #include "kudu/client/master_rpc.h"
+#if !defined(NO_CHRONY)
 #include "kudu/clock/test/mini_chronyd.h"
+#endif
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/gutil/basictypes.h"
@@ -72,7 +74,9 @@
 #include "kudu/util/test_util.h"
 
 using kudu::client::internal::ConnectToClusterRpc;
+#if !defined(NO_CHRONY)
 using kudu::clock::MiniChronyd;
+#endif
 using kudu::master::ListTablesRequestPB;
 using kudu::master::ListTablesResponsePB;
 using kudu::master::MasterServiceProxy;
@@ -115,8 +119,12 @@ ExternalMiniClusterOptions::ExternalMiniClusterOptions()
       enable_sentry(false),
       logtostderr(true),
       start_process_timeout(MonoDelta::FromSeconds(70)),
-      rpc_negotiation_timeout(MonoDelta::FromSeconds(3)),
-      num_ntp_servers(UseSystemNtp() ? 0 : 1) {
+      rpc_negotiation_timeout(MonoDelta::FromSeconds(3))
+#if !defined(NO_CHRONY)
+      ,
+      num_ntp_servers(UseSystemNtp() ? 0 : 1)
+#endif
+    {
 }
 
 ExternalMiniCluster::ExternalMiniCluster()
@@ -159,6 +167,7 @@ Status ExternalMiniCluster::HandleOptions() {
   return Status::OK();
 }
 
+#if !defined(NO_CHRONY)
 Status ExternalMiniCluster::AddNtpFlags(std::vector<std::string>* flags) {
   DCHECK(flags);
   if (opts_.num_ntp_servers > 0) {
@@ -184,6 +193,7 @@ Status ExternalMiniCluster::AddNtpFlags(std::vector<std::string>* flags) {
   }
   return Status::OK();
 }
+#endif // #if !defined(NO_CHRONY) ...
 
 Status ExternalMiniCluster::StartSentry() {
   sentry_->SetDataRoot(opts_.cluster_root);
@@ -249,6 +259,7 @@ Status ExternalMiniCluster::Start() {
                           "could not set krb5 client env");
   }
 
+#if !defined(NO_CHRONY)
   // Start NTP servers, if requested.
   if (opts_.num_ntp_servers > 0) {
     // Collect and keep alive the set of sockets bound with SO_REUSEPORT option
@@ -272,6 +283,7 @@ Status ExternalMiniCluster::Start() {
                             Substitute("failed to start NTP server $0", i));
     }
   }
+#endif // #if !defined(NO_CHRONY) ...
 
   // Start the Sentry service and the HMS in the following steps, in order
   // to deal with the circular dependency in terms of configuring each
@@ -532,7 +544,9 @@ Status ExternalMiniCluster::StartMasters() {
     flags.emplace_back("--location_mapping_by_uuid");
 #   endif
   }
+#if !defined(NO_CHRONY)
   RETURN_NOT_OK(AddNtpFlags(&flags));
+#endif
 
   // Start the masters.
   const string& exe = GetBinaryPath(kMasterBinaryName);
@@ -616,7 +630,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
   auto flags = SubstituteInFlags(opts_.extra_tserver_flags, idx);
   std::copy(flags.begin(), flags.end(), std::back_inserter(extra_flags));
   opts.extra_flags = extra_flags;
@@ -636,6 +652,7 @@ Status ExternalMiniCluster::AddTabletServer() {
   return Status::OK();
 }
 
+#if !defined(NO_CHRONY)
 Status ExternalMiniCluster::AddNtpServer(const Sockaddr& addr) {
   clock::MiniChronydOptions options;
   options.index = ntp_servers_.size();
@@ -648,6 +665,7 @@ Status ExternalMiniCluster::AddNtpServer(const Sockaddr& addr) {
   ntp_servers_.emplace_back(std::move(chrony));
   return Status::OK();
 }
+#endif // #if !defined(NO_CHRONY) ...
 
 Status ExternalMiniCluster::WaitForTabletServerCount(int count, const MonoDelta& timeout) {
   MonoTime deadline = MonoTime::Now() + timeout;
@@ -826,6 +844,7 @@ vector<ExternalDaemon*> ExternalMiniCluster::daemons() const {
   return results;
 }
 
+#if !defined(NO_CHRONY)
 vector<MiniChronyd*> ExternalMiniCluster::ntp_servers() const {
   vector<MiniChronyd*> servers;
   servers.reserve(ntp_servers_.size());
@@ -835,6 +854,7 @@ vector<MiniChronyd*> ExternalMiniCluster::ntp_servers() const {
   }
   return servers;
 }
+#endif // #if !defined(NO_CHRONY) ...
 
 vector<HostPort> ExternalMiniCluster::master_rpc_addrs() const {
   vector<HostPort> master_hostports;
diff --git a/src/kudu/mini-cluster/external_mini_cluster.h b/src/kudu/mini-cluster/external_mini_cluster.h
index 79eca1d..f02c7ac 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.h
+++ b/src/kudu/mini-cluster/external_mini_cluster.h
@@ -53,9 +53,11 @@ class KuduClient;
 class KuduClientBuilder;
 } // namespace client
 
+#if !defined(NO_CHRONY)
 namespace clock {
 class MiniChronyd;
 } // namespace clock
+#endif
 
 namespace hms {
 class MiniHms;
@@ -187,6 +189,7 @@ struct ExternalMiniClusterOptions {
   // Default: empty
   LocationInfo location_info;
 
+#if !defined(NO_CHRONY)
   // Number of NTP servers to start as part of the cluster. The NTP servers are
   // used as true time references for the NTP client built into masters and
   // tablet servers. Specifying a value greater than 0 automatically enables
@@ -195,6 +198,7 @@ struct ExternalMiniClusterOptions {
   //
   // Default: 0
   int num_ntp_servers;
+#endif // #if !defined(NO_CHRONY) ...
 };
 
 // A mini-cluster made up of subprocesses running each of the daemons
@@ -224,9 +228,11 @@ class ExternalMiniCluster : public MiniCluster {
   // Requires that the master is already running.
   Status AddTabletServer();
 
+#if !defined(NO_CHRONY)
   // Add a new NTP server to the cluster. The new NTP server is started upon
   // adding, bind to the address and port specified by 'addr'.
   Status AddNtpServer(const Sockaddr& addr);
+#endif
 
   // Currently, this uses SIGKILL on each daemon for a non-graceful shutdown.
   void ShutdownNodes(ClusterNodes nodes) override;
@@ -292,7 +298,9 @@ class ExternalMiniCluster : public MiniCluster {
 
   // Return all configured NTP servers used for the synchronisation of the
   // built-in NTP client.
+#if !defined(NO_CHRONY)
   std::vector<clock::MiniChronyd*> ntp_servers() const;
+#endif
 
   MiniKdc* kdc() const {
     return kdc_.get();
@@ -424,13 +432,17 @@ class ExternalMiniCluster : public MiniCluster {
 
   Status DeduceBinRoot(std::string* ret);
   Status HandleOptions();
+#if !defined(NO_CHRONY)
   Status AddNtpFlags(std::vector<std::string>* flags);
+#endif
 
   ExternalMiniClusterOptions opts_;
 
   std::vector<scoped_refptr<ExternalMaster>> masters_;
   std::vector<scoped_refptr<ExternalTabletServer>> tablet_servers_;
+#if !defined(NO_CHRONY)
   std::vector<std::unique_ptr<clock::MiniChronyd>> ntp_servers_;
+#endif
   std::unique_ptr<MiniKdc> kdc_;
   std::unique_ptr<hms::MiniHms> hms_;
   std::unique_ptr<sentry::MiniSentry> sentry_;


[kudu] 03/03: [docs] update 1.11.1 release notes

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

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

commit ac16dcff8d84344712d16329d15fbabbb7c34c94
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Mon Nov 11 22:05:09 2019 -0800

    [docs] update 1.11.1 release notes
    
    The release notes for Kudu v1.11.1 updated to include recent fixes
    cherry-picked into the 1.11.1 branch.
    
    Change-Id: I40d7f93b094b330b3ef64ea0ff28d7834048626c
    Reviewed-on: http://gerrit.cloudera.org:8080/14692
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 docs/release_notes.adoc | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)

diff --git a/docs/release_notes.adoc b/docs/release_notes.adoc
index 2bb3672..7f08cb8 100644
--- a/docs/release_notes.adoc
+++ b/docs/release_notes.adoc
@@ -53,6 +53,22 @@ their Kudu processes will crash at startup.
   resolving required symbols via `dlsym()`
   (see link:https://issues.apache.org/jira/browse/KUDU-2990[KUDU-2990]).
 
+* Fixed an issue with `kudu cluster rebalancer` CLI tool crashing when running
+  against a location-aware cluster if a tablet server in one location doesn't
+  contain a single tablet replica
+  (see link:https://issues.apache.org/jira/browse/KUDU-2987[KUDU-2987]).
+
+* Fixed an issue with connection negotiation using SASL mechanism when server
+  FQDN is longer than 64 characters
+  (see link:https://issues.apache.org/jira/browse/KUDU-2989[KUDU-2989]).
+
+* Fixed an issue in the test harness of the kudu-binary JAR artifact. With this
+  fix, `kudu-master` and `kudu-tserver` processes of the mini-cluster's test
+  harness no longer rely on the test NTP server to synchronize their built-in
+  NTP client. Instead, the test harness relies on the local machine clock
+  synchronized by the system NTP daemon
+  (see link:https://issues.apache.org/jira/browse/KUDU-2994[KUDU-2994]).
+
 [[rn_1.11.0]]
 = Apache Kudu 1.11.0 Release Notes