You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2019/02/20 18:29:38 UTC

[kudu] branch master updated (083ab13 -> 6278a2e)

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

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


    from 083ab13  [examples] updated README.md for basic-python-example
     new aca61da  Fix some compilation warnings
     new b216718  Fix RAT warnings
     new 6278a2e  KUDU-1900: add loopback check and test

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:
 build-support/release/rat_exclude_files.txt  |   7 ++
 src/kudu/common/row_changelist.cc            |   2 +-
 src/kudu/common/row_changelist.h             |   2 +-
 src/kudu/consensus/consensus_queue.cc        |   4 +
 src/kudu/consensus/log_util.cc               |   2 +-
 src/kudu/integration-tests/security-itest.cc | 150 ++++++++++++++++++++++-----
 src/kudu/tablet/deltafile.cc                 |   2 +-
 src/kudu/util/env_posix.cc                   |   4 +-
 src/kudu/util/logging.cc                     |   3 +-
 src/kudu/util/net/net_util.cc                |  19 ++++
 src/kudu/util/net/net_util.h                 |  12 +++
 src/kudu/util/net/sockaddr.cc                |   7 +-
 src/kudu/util/net/socket.cc                  |   7 +-
 13 files changed, 182 insertions(+), 39 deletions(-)


[kudu] 03/03: KUDU-1900: add loopback check and test

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

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

commit 6278a2e0590c17ff2c56d0f3d3070c9c4acd10db
Author: gsolovyev <gs...@ve0518.halxg.cloudera.com>
AuthorDate: Fri Feb 8 08:54:16 2019 -0800

    KUDU-1900: add loopback check and test
    
    This change modifies Socket::IsLoopbackConnection to check
    if remote IP is in 127.0.0.0/8 subnet. With this change
    all connections from 127.0.0.0/8 are treated as local.
    
    This change adds force_external_client_ip parameter to
    AuthTokenIssuingTest. Before this change, AuthTokenIssuingTest
    relied on Kudu treating connections between non-matching
    addresses in 127.0.0.0/8 subnet as external. With this change,
    the test forces Kudu client to use an non-loopback IP
    address for test cases that verify external connections.
    If an external IP address is not available, test cases that
    require an external IP will be skipped.
    
    For convenience, this change adds two static utility methods
    (HostPort::IsLoopback and HostPort::AddrToString), and refactors
    Sockaddr::host and Sockaddr::IsAnyLocalAddress to use these
    static methods.
    
    Change-Id: I3483a9729ddeeb7901e3738532a45b49e713208f
    Reviewed-on: http://gerrit.cloudera.org:8080/12474
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/integration-tests/security-itest.cc | 150 ++++++++++++++++++++++-----
 src/kudu/util/net/net_util.cc                |  19 ++++
 src/kudu/util/net/net_util.h                 |  12 +++
 src/kudu/util/net/sockaddr.cc                |   7 +-
 src/kudu/util/net/socket.cc                  |   7 +-
 5 files changed, 163 insertions(+), 32 deletions(-)

diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index acd6466..3b9b0a7 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -19,6 +19,7 @@
 
 #include <cstdio>
 #include <memory>
+#include <ostream>
 #include <string>
 #include <vector>
 
@@ -54,6 +55,7 @@
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/path_util.h"
+#include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 #include "kudu/util/subprocess.h"
 #include "kudu/util/test_macros.h"
@@ -314,36 +316,124 @@ TEST_F(SecurityITest, TestWorldReadablePrivateKey) {
       credentials_name));
 }
 
+Status AssignIPToClient(bool external) {
+  // If the test does not require an external IP address
+  // assign loopback IP to FLAGS_local_ip_for_outbound_sockets.
+  if (!external) {
+    FLAGS_local_ip_for_outbound_sockets = "127.0.0.1";
+    return Status::OK();
+  }
+
+  // Try finding an external IP address to assign to
+  // FLAGS_local_ip_for_outbound_sockets.
+  vector<Network> local_networks;
+  RETURN_NOT_OK(GetLocalNetworks(&local_networks));
+
+  for (const auto& network : local_networks) {
+    if (!network.IsLoopback()) {
+      FLAGS_local_ip_for_outbound_sockets = network.GetAddrAsString();
+      // Found and assigned an external IP.
+      return Status::OK();
+    }
+  }
+
+  // Could not find an external IP.
+  return Status::NotFound("Could not find an external IP.");
+}
+
+// Descriptive constants for test parameters.
+const bool LOOPBACK_ENCRYPTED = true;
+const bool LOOPBACK_PLAIN = false;
+const bool TOKEN_PRESENT = true;
+const bool TOKEN_MISSING = false;
+const bool LOOPBACK_CLIENT_IP = false;
+const bool EXTERNAL_CLIENT_IP = true;
+const char* const AUTH_REQUIRED = "required";
+const char* const AUTH_DISABLED = "disabled";
+const char* const ENCRYPTION_REQUIRED = "required";
+const char* const ENCRYPTION_DISABLED = "disabled";
+
 struct AuthTokenIssuingTestParams {
   const BindMode bind_mode;
   const string rpc_authentication;
   const string rpc_encryption;
   const bool rpc_encrypt_loopback_connections;
+  const bool force_external_client_ip;
   const bool authn_token_present;
 };
+
 class AuthTokenIssuingTest :
     public SecurityITest,
     public ::testing::WithParamInterface<AuthTokenIssuingTestParams> {
 };
+
 INSTANTIATE_TEST_CASE_P(, AuthTokenIssuingTest, ::testing::ValuesIn(
     vector<AuthTokenIssuingTestParams>{
-      { BindMode::LOOPBACK, "required", "required", true,  true,  },
-      { BindMode::LOOPBACK, "required", "required", false, true,  },
-      //BindMode::LOOPBACK, "required", "disabled": non-acceptable
-      //BindMode::LOOPBACK, "required", "disabled": non-acceptable
-      { BindMode::LOOPBACK, "disabled", "required", true,  true,  },
-      { BindMode::LOOPBACK, "disabled", "required", false, true,  },
-      { BindMode::LOOPBACK, "disabled", "disabled", true,  false, },
-      { BindMode::LOOPBACK, "disabled", "disabled", false, true,  },
+      // The following 3 test cases cover passing authn token over an
+      // encrypted loopback connection.
+      // Note: AUTH_REQUIRED with ENCRYPTION_DISABLED is not
+      // an acceptable configuration.
+      { BindMode::LOOPBACK, AUTH_REQUIRED, ENCRYPTION_REQUIRED,
+        LOOPBACK_ENCRYPTED, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::LOOPBACK, AUTH_DISABLED, ENCRYPTION_REQUIRED,
+        LOOPBACK_ENCRYPTED, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::LOOPBACK, AUTH_DISABLED, ENCRYPTION_DISABLED,
+        LOOPBACK_ENCRYPTED, LOOPBACK_CLIENT_IP, TOKEN_MISSING, },
+
+      // The following 3 test cases cover passing authn token over an
+      // unencrypted loopback connection.
+      { BindMode::LOOPBACK, AUTH_REQUIRED, ENCRYPTION_REQUIRED,
+        LOOPBACK_PLAIN, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::LOOPBACK, AUTH_DISABLED, ENCRYPTION_REQUIRED,
+        LOOPBACK_PLAIN, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::LOOPBACK, AUTH_DISABLED, ENCRYPTION_DISABLED,
+        LOOPBACK_PLAIN, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
+
+      // The following 3 test cases cover passing authn token over an
+      // external connection.
+      { BindMode::LOOPBACK, AUTH_REQUIRED, ENCRYPTION_REQUIRED,
+        LOOPBACK_PLAIN, EXTERNAL_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::LOOPBACK, AUTH_DISABLED, ENCRYPTION_REQUIRED,
+        LOOPBACK_PLAIN, EXTERNAL_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::LOOPBACK, AUTH_DISABLED, ENCRYPTION_DISABLED,
+        LOOPBACK_PLAIN, EXTERNAL_CLIENT_IP, TOKEN_MISSING, },
+
+      // The following 3 test cases verify that enforcement of encryption
+      // for loopback connections does not affect external connections.
+      { BindMode::LOOPBACK, AUTH_REQUIRED, ENCRYPTION_REQUIRED,
+        LOOPBACK_ENCRYPTED, EXTERNAL_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::LOOPBACK, AUTH_DISABLED, ENCRYPTION_REQUIRED,
+        LOOPBACK_ENCRYPTED, EXTERNAL_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::LOOPBACK, AUTH_DISABLED, ENCRYPTION_DISABLED,
+        LOOPBACK_ENCRYPTED, EXTERNAL_CLIENT_IP, TOKEN_MISSING, },
 #if defined(__linux__)
-      { BindMode::UNIQUE_LOOPBACK, "required", "required", true,  true,  },
-      { BindMode::UNIQUE_LOOPBACK, "required", "required", false, true,  },
-      //BindMode::UNIQUE_LOOPBACK, "required", "disabled": non-acceptable
-      //BindMode::UNIQUE_LOOPBACK, "required", "disabled": non-acceptable
-      { BindMode::UNIQUE_LOOPBACK, "disabled", "required", true,  true,  },
-      { BindMode::UNIQUE_LOOPBACK, "disabled", "required", false, true,  },
-      { BindMode::UNIQUE_LOOPBACK, "disabled", "disabled", true,  false, },
-      { BindMode::UNIQUE_LOOPBACK, "disabled", "disabled", false, false, },
+      // The following 6 test cases verify that a connection from 127.0.0.1
+      // to another loopback address is treated as a loopback connection.
+      { BindMode::UNIQUE_LOOPBACK, AUTH_REQUIRED, ENCRYPTION_REQUIRED,
+        LOOPBACK_ENCRYPTED, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::UNIQUE_LOOPBACK, AUTH_DISABLED, ENCRYPTION_REQUIRED,
+        LOOPBACK_ENCRYPTED, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::UNIQUE_LOOPBACK, AUTH_DISABLED, ENCRYPTION_DISABLED,
+        LOOPBACK_ENCRYPTED, LOOPBACK_CLIENT_IP, TOKEN_MISSING, },
+
+      { BindMode::UNIQUE_LOOPBACK, AUTH_REQUIRED, ENCRYPTION_REQUIRED,
+        LOOPBACK_PLAIN, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::UNIQUE_LOOPBACK, AUTH_DISABLED, ENCRYPTION_REQUIRED,
+        LOOPBACK_PLAIN, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
+
+      { BindMode::UNIQUE_LOOPBACK, AUTH_DISABLED, ENCRYPTION_DISABLED,
+        LOOPBACK_PLAIN, LOOPBACK_CLIENT_IP, TOKEN_PRESENT, },
 #endif
     }
 ));
@@ -355,25 +445,35 @@ TEST_P(AuthTokenIssuingTest, ChannelConfidentiality) {
   cluster_opts_.num_masters = 1;
   cluster_opts_.num_tablet_servers = 0;
   // --user-acl: just restoring back the default setting.
-  cluster_opts_.extra_master_flags.emplace_back("--user-acl=*");
+  cluster_opts_.extra_master_flags.emplace_back("--user_acl=*");
 
   const auto& params = GetParam();
+
+  // When testing external connections, make sure the client connects from
+  // an external IP, so that the connection is not considered to be local.
+  // When testing local connections, make sure that the client
+  // connects from standard loopback IP.
+  // Skip tests that require an external connection
+  // if the host does not have a non-loopback interface.
+  Status s = AssignIPToClient(params.force_external_client_ip);
+  if (s.IsNotFound()) {
+    LOG(WARNING) << s.message().ToString() << "\nSkipping external connection test.";
+    return;
+  }
+  // Fail if GetLocalNetworks failed to determine network interfaces.
+  ASSERT_OK(s);
+
+  // Set flags and start cluster.
   cluster_opts_.bind_mode = params.bind_mode;
   cluster_opts_.extra_master_flags.emplace_back(
-      Substitute("--rpc-authentication=$0", params.rpc_authentication));
+      Substitute("--rpc_authentication=$0", params.rpc_authentication));
   cluster_opts_.extra_master_flags.emplace_back(
-      Substitute("--rpc-encryption=$0", params.rpc_encryption));
+      Substitute("--rpc_encryption=$0", params.rpc_encryption));
   cluster_opts_.extra_master_flags.emplace_back(
       Substitute("--rpc_encrypt_loopback_connections=$0",
                  params.rpc_encrypt_loopback_connections));
   ASSERT_OK(StartCluster());
 
-  // Make sure the client always connects from the standard loopback address.
-  // This is crucial when the master is running with UNIQUE_LOOPBACK mode: the
-  // test scenario expects the client connects from other than 127.0.0.1 address
-  // so the connection is not considered a 'loopback' one.
-  FLAGS_local_ip_for_outbound_sockets = "127.0.0.1";
-
   // In current implementation, KuduClientBuilder calls ConnectToCluster() on
   // the newly created instance of the KuduClient.
   client::sp::shared_ptr<KuduClient> client;
diff --git a/src/kudu/util/net/net_util.cc b/src/kudu/util/net/net_util.cc
index d735616..6e719d0 100644
--- a/src/kudu/util/net/net_util.cc
+++ b/src/kudu/util/net/net_util.cc
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <arpa/inet.h>
 #include <sys/socket.h>
 #include <ifaddrs.h>
 #include <limits.h>
@@ -249,6 +250,16 @@ string HostPort::ToCommaSeparatedString(const vector<HostPort>& hostports) {
   return JoinStrings(hostport_strs, ",");
 }
 
+bool HostPort::IsLoopback(uint32_t addr) {
+    return (NetworkByteOrder::FromHost32(addr) >> 24) == 127;
+}
+
+string HostPort::AddrToString(uint32_t addr) {
+  char str[INET_ADDRSTRLEN];
+  ::inet_ntop(AF_INET, &addr, str, INET_ADDRSTRLEN);
+  return str;
+}
+
 Network::Network()
   : addr_(0),
     netmask_(0) {
@@ -293,6 +304,14 @@ Status Network::ParseCIDRStrings(const string& comma_sep_addrs,
   return Status::OK();
 }
 
+bool Network::IsLoopback() const {
+  return HostPort::IsLoopback(addr_);
+}
+
+string Network::GetAddrAsString() const {
+  return HostPort::AddrToString(addr_);
+}
+
 bool IsPrivilegedPort(uint16_t port) {
   return port <= 1024 && port != 0;
 }
diff --git a/src/kudu/util/net/net_util.h b/src/kudu/util/net/net_util.h
index 13fb512..b904b00 100644
--- a/src/kudu/util/net/net_util.h
+++ b/src/kudu/util/net/net_util.h
@@ -92,6 +92,12 @@ class HostPort {
   // "inverse" of ParseStrings().
   static std::string ToCommaSeparatedString(const std::vector<HostPort>& host_ports);
 
+  // Returns true if addr is within 127.0.0.0/8 range.
+  static bool IsLoopback(uint32_t addr);
+
+  // Returns dotted-decimal ('1.2.3.4') representation of IP address in addr.
+  static std::string AddrToString(uint32_t addr);
+
  private:
   std::string host_;
   uint16_t port_;
@@ -131,6 +137,12 @@ class Network {
   // Returns true if the address is within network.
   bool WithinNetwork(const Sockaddr& addr) const;
 
+  // Returns true if the network is within 127.0.0.0/8 range.
+  bool IsLoopback() const;
+
+  // Returns addr part of addr:mask pair as string.
+  std::string GetAddrAsString() const;
+
   // Parses a "addr/netmask" (CIDR notation) pair into this object.
   Status ParseCIDRString(const std::string& addr);
 
diff --git a/src/kudu/util/net/sockaddr.cc b/src/kudu/util/net/sockaddr.cc
index 6efc4d8..cc100c6 100644
--- a/src/kudu/util/net/sockaddr.cc
+++ b/src/kudu/util/net/sockaddr.cc
@@ -25,7 +25,6 @@
 #include <cstring>
 #include <string>
 
-#include "kudu/gutil/endian.h"
 #include "kudu/gutil/hash/builtin_type_hash.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -89,9 +88,7 @@ int Sockaddr::port() const {
 }
 
 std::string Sockaddr::host() const {
-  char str[INET_ADDRSTRLEN];
-  ::inet_ntop(AF_INET, &addr_.sin_addr, str, INET_ADDRSTRLEN);
-  return str;
+  return HostPort::AddrToString(addr_.sin_addr.s_addr);
 }
 
 const struct sockaddr_in& Sockaddr::addr() const {
@@ -107,7 +104,7 @@ bool Sockaddr::IsWildcard() const {
 }
 
 bool Sockaddr::IsAnyLocalAddress() const {
-  return (NetworkByteOrder::FromHost32(addr_.sin_addr.s_addr) >> 24) == 127;
+  return HostPort::IsLoopback(addr_.sin_addr.s_addr);
 }
 
 Status Sockaddr::LookupHostname(string* hostname) const {
diff --git a/src/kudu/util/net/socket.cc b/src/kudu/util/net/socket.cc
index bc3b525..2ffb5cf 100644
--- a/src/kudu/util/net/socket.cc
+++ b/src/kudu/util/net/socket.cc
@@ -299,8 +299,11 @@ bool Socket::IsLoopbackConnection() const {
   Sockaddr local, remote;
   if (!GetSocketAddress(&local).ok()) return false;
   if (!GetPeerAddress(&remote).ok()) return false;
-
-  // Compare without comparing ports.
+  // Check if remote address is in 127.0.0.0/8 subnet.
+  if (remote.IsAnyLocalAddress()) {
+    return true;
+  }
+  // Compare local and remote addresses without comparing ports.
   local.set_port(0);
   remote.set_port(0);
   return local == remote;


[kudu] 01/03: Fix some compilation warnings

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

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

commit aca61daa38f49be977315fd2dbb021f1b085f848
Author: Yingchun Lai <40...@qq.com>
AuthorDate: Tue Feb 19 10:04:26 2019 -0500

    Fix some compilation warnings
    
    Fix compilation warnings on CentOS 7.3 with
    gcc 4.8.5, and Ubuntu 18.04 with gcc 7.3.0
    
    Change-Id: Ia11e3a3ec1b81332aac0b1c841a78d9fb145c33d
    Reviewed-on: http://gerrit.cloudera.org:8080/12522
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/common/row_changelist.cc     | 2 +-
 src/kudu/common/row_changelist.h      | 2 +-
 src/kudu/consensus/consensus_queue.cc | 4 ++++
 src/kudu/consensus/log_util.cc        | 2 +-
 src/kudu/tablet/deltafile.cc          | 2 +-
 src/kudu/util/env_posix.cc            | 4 ++--
 src/kudu/util/logging.cc              | 3 ++-
 7 files changed, 12 insertions(+), 7 deletions(-)

diff --git a/src/kudu/common/row_changelist.cc b/src/kudu/common/row_changelist.cc
index b08ded1..5681439 100644
--- a/src/kudu/common/row_changelist.cc
+++ b/src/kudu/common/row_changelist.cc
@@ -223,7 +223,7 @@ Status RowChangeListDecoder::MutateRowAndCaptureChanges(RowBlockRow* dst_row,
     DecodedUpdate dec;
     RETURN_NOT_OK(DecodeNext(&dec));
     int col_idx;
-    const void* value;
+    const void* value = nullptr;
     RETURN_NOT_OK(dec.Validate(*dst_schema, &col_idx, &value));
     // Reinserts don't update keys so they shouldn't include the key columns.
     DCHECK(!dst_schema->is_key_column(col_idx));
diff --git a/src/kudu/common/row_changelist.h b/src/kudu/common/row_changelist.h
index a2b36e8..e415c27 100644
--- a/src/kudu/common/row_changelist.h
+++ b/src/kudu/common/row_changelist.h
@@ -380,7 +380,7 @@ class RowChangeListDecoder {
     ColumnId col_id;
 
     // If true, this update sets the given column to NULL.
-    bool null;
+    bool null = false;
 
     // The "raw" value of the updated column.
     //   - in the case of a fixed length type such as an integer,
diff --git a/src/kudu/consensus/consensus_queue.cc b/src/kudu/consensus/consensus_queue.cc
index 383a5cd..41815ad 100644
--- a/src/kudu/consensus/consensus_queue.cc
+++ b/src/kudu/consensus/consensus_queue.cc
@@ -1259,7 +1259,11 @@ bool PeerMessageQueue::ResponseFromPeer(const std::string& peer_uuid,
   }
 
   if (mode_copy == LEADER && updated_commit_index != boost::none) {
+  // Suppress false positive about 'updated_commit_index' used when uninitialized.
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
     NotifyObserversOfCommitIndexChange(*updated_commit_index);
+#pragma GCC diagnostic pop
   }
 
   return send_more_immediately;
diff --git a/src/kudu/consensus/log_util.cc b/src/kudu/consensus/log_util.cc
index 3680fd3..27c143b 100644
--- a/src/kudu/consensus/log_util.cc
+++ b/src/kudu/consensus/log_util.cc
@@ -486,7 +486,7 @@ Status ReadableLogSegment::ParseHeaderMagicAndHeaderLength(const Slice &data,
 }
 
 Status ReadableLogSegment::ReadFooter() {
-  uint32_t footer_size;
+  uint32_t footer_size = 0;
   RETURN_NOT_OK(ReadFooterMagicAndFooterLength(&footer_size));
 
   if (footer_size == 0 || footer_size > kLogSegmentMaxHeaderOrFooterSize) {
diff --git a/src/kudu/tablet/deltafile.cc b/src/kudu/tablet/deltafile.cc
index 9d5bc01..e6b7a0a 100644
--- a/src/kudu/tablet/deltafile.cc
+++ b/src/kudu/tablet/deltafile.cc
@@ -558,7 +558,7 @@ Status DeltaFileIterator<Type>::PrepareBatch(size_t nrows, int prepare_flags) {
   }
 
   while (!exhausted_) {
-    rowid_t next_block_rowidx;
+    rowid_t next_block_rowidx = 0;
     RETURN_NOT_OK(GetFirstRowIndexInCurrentBlock(&next_block_rowidx));
     VLOG(2) << "Current delta block starting at row " << next_block_rowidx;
 
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index b6215f8..5d1e55c 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -1126,7 +1126,7 @@ class PosixEnv : public Env {
                                      string* created_filename,
                                      unique_ptr<WritableFile>* result) OVERRIDE {
     TRACE_EVENT1("io", "PosixEnv::NewTempWritableFile", "template", name_template);
-    int fd;
+    int fd = 0;
     string tmp_filename;
     RETURN_NOT_OK(MkTmpFile(name_template, &fd, &tmp_filename));
     RETURN_NOT_OK(InstantiateNewWritableFile(tmp_filename, fd, opts, result));
@@ -1152,7 +1152,7 @@ class PosixEnv : public Env {
   virtual Status NewTempRWFile(const RWFileOptions& opts, const string& name_template,
                                string* created_filename, unique_ptr<RWFile>* res) OVERRIDE {
     TRACE_EVENT1("io", "PosixEnv::NewTempRWFile", "template", name_template);
-    int fd;
+    int fd = 0;
     RETURN_NOT_OK(MkTmpFile(name_template, &fd, created_filename));
     res->reset(new PosixRWFile(*created_filename, fd, opts.sync_on_close));
     return Status::OK();
diff --git a/src/kudu/util/logging.cc b/src/kudu/util/logging.cc
index fcf035f..cfda466 100644
--- a/src/kudu/util/logging.cc
+++ b/src/kudu/util/logging.cc
@@ -31,6 +31,7 @@
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 
+#include "kudu/gutil/basictypes.h"
 #include "kudu/gutil/callback.h"  // IWYU pragma: keep
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/spinlock.h"
@@ -174,7 +175,7 @@ void FlushCoverageOnExit() {
   static std::once_flag once;
   std::call_once(once, [] {
       static const char msg[] = "Flushing coverage data before crash...\n";
-      write(STDERR_FILENO, msg, arraysize(msg));
+      ignore_result(write(STDERR_FILENO, msg, arraysize(msg)));
       TryFlushCoverage();
     });
   in_call = false;


[kudu] 02/03: Fix RAT warnings

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

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

commit b21671890d36e7e858d631927acd7b16268f7f2a
Author: Andrew Wong <aw...@apache.org>
AuthorDate: Tue Feb 19 22:16:35 2019 -0800

    Fix RAT warnings
    
    Change-Id: Icd9f79f19312982e56ad9e4b5aff9c5fe1f39bec
    Reviewed-on: http://gerrit.cloudera.org:8080/12533
    Reviewed-by: Grant Henke <gr...@apache.org>
    Tested-by: Andrew Wong <aw...@cloudera.com>
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 build-support/release/rat_exclude_files.txt | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/build-support/release/rat_exclude_files.txt b/build-support/release/rat_exclude_files.txt
index 28a9132..5c5a47b 100644
--- a/build-support/release/rat_exclude_files.txt
+++ b/build-support/release/rat_exclude_files.txt
@@ -14,6 +14,10 @@ build-support/iwyu/fix_includes.py
 build-support/iwyu/iwyu_tool.py
 build-support/iwyu/mappings/boost-all-private.imp
 build-support/iwyu/mappings/boost-all.imp
+build-support/mini-cluster/LICENSE-BINARY-JAR-LINUX.txt
+build-support/mini-cluster/LICENSE-BINARY-JAR-OSX.txt
+build-support/mini-cluster/NOTICE-BINARY-JAR-LINUX.txt
+build-support/mini-cluster/NOTICE-BINARY-JAR-OSX.txt
 build-support/release/rat_exclude_files.txt
 docs/support/doxygen/client_api.footer.in
 examples/README.md
@@ -30,14 +34,17 @@ java/kudu-client/src/main/java/org/apache/kudu/client/NonRecoverableException.ja
 java/kudu-client/src/main/java/org/apache/kudu/client/PleaseThrottleException.java
 java/kudu-client/src/main/java/org/apache/kudu/client/RecoverableException.java
 java/kudu-client/src/main/java/org/apache/kudu/client/RpcProxy.java
+java/kudu-client/src/test/resources/assign-location.py
 java/kudu-client/src/test/resources/flags
 java/kudu-flume-sink/src/test/resources/testAvroKuduOperationsProducer.avsc
+java/kudu-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
 java/kudu-spark-tools/src/test/resources/TestImportExportFiles.csv
 python/MANIFEST.in
 python/README.md
 python/pytest.ini
 python/requirements.txt
 python/setup.cfg
+python/version.txt
 src/kudu/client/symbols.map
 src/kudu/gutil/arm_instruction_set_select.h
 src/kudu/gutil/atomic_refcount.h