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 2017/02/15 20:59:53 UTC

kudu git commit: KUDU-1855. Restrict file permissions on all created files

Repository: kudu
Updated Branches:
  refs/heads/master 56402bd1a -> daf289d6b


KUDU-1855. Restrict file permissions on all created files

Prior to this patch, the permissions of data files created by Kudu were
quite inconsistent:

- directories were created with 0755 & ~umask
- files created using NewWritableFile or NewRWFile (eg logs and data
  files) were created with 0644 ~ umask
- log index files were created with 0666 ~ umask
- files created as tmp and moved into place (eg PBC files) were created
  as 0600 regardless of umask

With the case of common management tools like CM, the process umask was
getting set to 022, resulting in the following permissions:

  /data0/kudu/master:
  total 20
  drwxr-xr-x 2 kudu kudu 4096 Feb 13 23:16 consensus-meta
  drwxr-xr-x 2 kudu kudu 4096 Feb 13 23:00 data
  -rw------- 1 kudu kudu  659 Feb 13 22:58 instance
  drwxr-xr-x 2 kudu kudu 4096 Feb 13 23:18 tablet-meta
  drwxr-xr-x 3 kudu kudu 4096 Feb 13 23:16 wals

  /data0/kudu/master/consensus-meta:
  total 8
  -rw------- 1 kudu kudu 8143 Feb 13 23:16 00000000000000000000000000000000

  /data0/kudu/master/data:
  total 229400
  -rw-r--r-- 1 kudu kudu 33554432 Feb 13 23:13 37e244afdae94877aea9cf9cd6a1300d.data
  -rw-r--r-- 1 kudu kudu      633 Feb 13 23:13 37e244afdae94877aea9cf9cd6a1300d.metadata
  -rw------- 1 kudu kudu      654 Feb 13 22:58 block_manager_instance

  /data0/kudu/master/tablet-meta:
  total 12
  -rw------- 1 kudu kudu 9494 Feb 13 23:18 00000000000000000000000000000000

  /data0/kudu/master/wals:
  total 4
  drwxr-xr-x 2 kudu kudu 4096 Feb 13 23:16 00000000000000000000000000000000

  /data0/kudu/master/wals/00000000000000000000000000000000:
  total 65540
  -rw-r--r-- 1 kudu kudu 24000000 Feb 13 23:16 index.000000000
  -rw------- 1 kudu kudu 67108864 Feb 13 23:17 wal-000000001

Obviously the above is not only inconsistent, but also rather insecure since
any local user could read any data file.

This patch addresses the issue as follows:

* introduces a new flag --umask, which defaults to "077". this is parsed
  at startup and the process umask is changed appropriately. Although
  we could have just used the environment-provided process umask, it
  seems more prudent to be "secure by default" here, given that the
  default umask in so many environments is permissive.

  For the purpose of comparison, other database engines such as MySQL
  and Postgres seem to do the same.

* all files are created with mode 0666, so that they take the widest
  permissions allowed by the umask. Given the default umask, this
  creates files with permissions 0600.

* all directories are created with mode 0777, subject to umask.
  Given the default, this creates directories with permissions 0700.

* temporary files are manually chmodded to 0666 & ~umask, so they end up
  with the same permissions as files created by other methods.

Additionally, since users may be upgrading from older versions with the
lax permissions, this checks at startup that the root data directory
permissions are subjected to the configured umask. For example, a data
directory with permissions 0755 would change to 0700. This does not
attempt to recursively chmod the contents of data directories, since
that might be expensive with the file block manager, and because setting
the top-level permissions to prevent path traversal should be
sufficiently secure.

A new unit test verifies the permissions on a few key files. I also ran
a master locally and checked that no files had wider permissions than
expected.

Change-Id: I5879f4a631e68a18161f723be18bcde1cf6c056d
Reviewed-on: http://gerrit.cloudera.org:8080/6004
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/daf289d6
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/daf289d6
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/daf289d6

Branch: refs/heads/master
Commit: daf289d6b20376f7207445c5d270e73eb1995126
Parents: 56402bd
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Feb 14 13:54:39 2017 -0800
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Wed Feb 15 20:59:20 2017 +0000

----------------------------------------------------------------------
 src/kudu/fs/fs_manager-test.cc | 51 +++++++++++++++++++++++++++++++++++--
 src/kudu/fs/fs_manager.cc      | 17 +++++++++++--
 src/kudu/fs/fs_manager.h       |  4 +++
 src/kudu/util/env.h            |  9 +++++++
 src/kudu/util/env_posix.cc     | 36 +++++++++++++++++++++++---
 src/kudu/util/flags.cc         | 45 ++++++++++++++++++++++++++++++++
 src/kudu/util/flags.h          |  4 +++
 7 files changed, 159 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/daf289d6/src/kudu/fs/fs_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager-test.cc b/src/kudu/fs/fs_manager-test.cc
index dcaf286..0c99e53 100644
--- a/src/kudu/fs/fs_manager-test.cc
+++ b/src/kudu/fs/fs_manager-test.cc
@@ -15,19 +15,23 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <sys/stat.h>
+#include <unistd.h>
+
 #include <memory>
+#include <unordered_set>
 
 #include <glog/logging.h>
 #include <glog/stl_logging.h>
 #include <gtest/gtest.h>
-#include <unistd.h>
-#include <unordered_set>
 
 #include "kudu/fs/block_manager.h"
 #include "kudu/fs/fs_manager.h"
+#include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/util/env_util.h"
+#include "kudu/util/flags.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/oid_generator.h"
 #include "kudu/util/test_macros.h"
@@ -40,6 +44,8 @@ using std::unordered_set;
 using std::vector;
 using strings::Substitute;
 
+DECLARE_string(umask);
+
 namespace kudu {
 
 class FsManagerTestBase : public KuduTest {
@@ -280,4 +286,45 @@ TEST_F(FsManagerTestBase, TestTmpFilesCleanup) {
   ASSERT_EQ(0, n_tmp_files);
 }
 
+namespace {
+
+string FilePermsAsString(const string& path) {
+  struct stat s;
+  CHECK_ERR(stat(path.c_str(), &s));
+  return StringPrintf("%03o", s.st_mode & ACCESSPERMS);
+}
+
+} // anonymous namespace
+
+TEST_F(FsManagerTestBase, TestUmask) {
+  // With the default umask, we should create files with permissions 600
+  // and directories with permissions 700.
+  ASSERT_EQ(077, g_parsed_umask) << "unexpected default value";
+  string root = GetTestPath("fs_root");
+  EXPECT_EQ("700", FilePermsAsString(root));
+  EXPECT_EQ("700", FilePermsAsString(fs_manager()->GetConsensusMetadataDir()));
+  EXPECT_EQ("600", FilePermsAsString(fs_manager()->GetInstanceMetadataPath(root)));
+
+  // With umask 007, we should create files with permissions 660
+  // and directories with 770.
+  FLAGS_umask = "007";
+  HandleCommonFlags();
+  ASSERT_EQ(007, g_parsed_umask);
+  root = GetTestPath("new_root");
+  ReinitFsManager({ root }, { root });
+  ASSERT_OK(fs_manager()->CreateInitialFileSystemLayout());
+  EXPECT_EQ("770", FilePermsAsString(root));
+  EXPECT_EQ("770", FilePermsAsString(fs_manager()->GetConsensusMetadataDir()));
+  EXPECT_EQ("660", FilePermsAsString(fs_manager()->GetInstanceMetadataPath(root)));
+
+  // If we change the umask back to being restrictive and re-open the filesystem,
+  // the permissions on the root dir should be fixed accordingly.
+  FLAGS_umask = "077";
+  HandleCommonFlags();
+  ASSERT_EQ(077, g_parsed_umask);
+  ReinitFsManager({ root }, { root });
+  ASSERT_OK(fs_manager()->Open());
+  EXPECT_EQ("700", FilePermsAsString(root));
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/daf289d6/src/kudu/fs/fs_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager.cc b/src/kudu/fs/fs_manager.cc
index 2fa6e65..c3d1a55 100644
--- a/src/kudu/fs/fs_manager.cc
+++ b/src/kudu/fs/fs_manager.cc
@@ -33,6 +33,7 @@
 #include "kudu/fs/fs.pb.h"
 #include "kudu/fs/log_block_manager.h"
 #include "kudu/gutil/map-util.h"
+#include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/numbers.h"
 #include "kudu/gutil/strings/split.h"
@@ -42,6 +43,8 @@
 #include "kudu/gutil/strtoint.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/env_util.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/flags.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/net/net_util.h"
@@ -74,7 +77,8 @@ DEFINE_string(fs_data_dirs, "",
               "block directory.");
 TAG_FLAG(fs_data_dirs, stable);
 
-using google::protobuf::Message;
+DECLARE_string(umask);
+
 using kudu::env_util::ScopedFileDeleter;
 using kudu::fs::BlockManagerOptions;
 using kudu::fs::CreateBlockOptions;
@@ -230,9 +234,10 @@ void FsManager::InitBlockManager() {
 Status FsManager::Open() {
   RETURN_NOT_OK(Init());
 
-  // Remove leftover tmp files
+  // Remove leftover tmp files and fix permissions.
   if (!read_only_) {
     CleanTmpFiles();
+    CheckAndFixPermissions();
   }
 
   for (const string& root : canonicalized_all_fs_roots_) {
@@ -513,6 +518,14 @@ void FsManager::CleanTmpFiles() {
   }
 }
 
+void FsManager::CheckAndFixPermissions() {
+  for (const string& root : canonicalized_all_fs_roots_) {
+    WARN_NOT_OK(env_->EnsureFileModeAdheresToUmask(root),
+                Substitute("could not check and fix permissions for path: $0",
+                           root));
+  }
+}
+
 // ==========================================================================
 //  Dump/Debug utils
 // ==========================================================================

http://git-wip-us.apache.org/repos/asf/kudu/blob/daf289d6/src/kudu/fs/fs_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager.h b/src/kudu/fs/fs_manager.h
index b86edf1..438832e 100644
--- a/src/kudu/fs/fs_manager.h
+++ b/src/kudu/fs/fs_manager.h
@@ -242,6 +242,10 @@ class FsManager {
   // Logs warnings in case of errors.
   void CleanTmpFiles();
 
+  // Checks that the permissions of the root data directories conform to the
+  // configured umask, and tightens them as necessary if they do not.
+  void CheckAndFixPermissions();
+
   static const char *kDataDirName;
   static const char *kTabletMetadataDirName;
   static const char *kWalDirName;

http://git-wip-us.apache.org/repos/asf/kudu/blob/daf289d6/src/kudu/util/env.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env.h b/src/kudu/util/env.h
index c14d979..442a451 100644
--- a/src/kudu/util/env.h
+++ b/src/kudu/util/env.h
@@ -312,6 +312,15 @@ class Env {
   // Gets the kernel release string for this machine.
   virtual std::string GetKernelRelease() = 0;
 
+  // Ensure that the file with the given path has permissions which adhere
+  // to the current configured umask (from flags.h). If the permissions are
+  // wider than the current umask, then a warning is logged and the permissions
+  // are fixed.
+  //
+  // Returns a bad Status if the file does not exist or the permissions cannot
+  // be changed.
+  virtual Status EnsureFileModeAdheresToUmask(const std::string& path) = 0;
+
   // Special string injected into file-growing operations' random failures
   // (if enabled).
   //

http://git-wip-us.apache.org/repos/asf/kudu/blob/daf289d6/src/kudu/util/env_posix.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index 76f0804..6158bdc 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -39,6 +39,7 @@
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/env.h"
 #include "kudu/util/errno.h"
+#include "kudu/util/flags.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/fault_injection.h"
 #include "kudu/util/logging.h"
@@ -239,7 +240,7 @@ static Status DoOpen(const string& filename, Env::CreateMode mode, int* fd) {
     default:
       return Status::NotSupported(Substitute("Unknown create mode $0", mode));
   }
-  const int f = open(filename.c_str(), flags, 0644);
+  const int f = open(filename.c_str(), flags, 0666);
   if (f < 0) {
     return IOError(filename, errno);
   }
@@ -880,7 +881,7 @@ class PosixEnv : public Env {
     TRACE_EVENT1("io", "PosixEnv::CreateDir", "path", name);
     ThreadRestrictions::AssertIOAllowed();
     Status result;
-    if (mkdir(name.c_str(), 0755) != 0) {
+    if (mkdir(name.c_str(), 0777) != 0) {
       result = IOError(name, errno);
     }
     return result;
@@ -1046,7 +1047,7 @@ class PosixEnv : public Env {
     ThreadRestrictions::AssertIOAllowed();
     *lock = nullptr;
     Status result;
-    int fd = open(fname.c_str(), O_RDWR | O_CREAT, 0644);
+    int fd = open(fname.c_str(), O_RDWR | O_CREAT, 0666);
     if (fd < 0) {
       result = IOError(fname, errno);
     } else if (LockOrUnlock(fd, true) == -1) {
@@ -1342,6 +1343,28 @@ class PosixEnv : public Env {
     return string(u.release);
   }
 
+  Status EnsureFileModeAdheresToUmask(const string& path) override {
+    struct stat s;
+    if (stat(path.c_str(), &s) != 0) {
+      return IOError("stat", errno);
+    }
+    CHECK_NE(g_parsed_umask, -1);
+    if (s.st_mode & g_parsed_umask) {
+      uint32_t old_perms = s.st_mode & ACCESSPERMS;
+      uint32_t new_perms = old_perms & ~g_parsed_umask;
+      LOG(WARNING) << "Path " << path << " has permissions "
+                   << StringPrintf("%03o", old_perms)
+                   << " which are less restrictive than current umask value "
+                   << StringPrintf("%03o", g_parsed_umask)
+                   << ": resetting permissions to "
+                   << StringPrintf("%03o", new_perms);
+      if (chmod(path.c_str(), new_perms) != 0) {
+        return IOError("chmod", errno);
+      }
+    }
+    return Status::OK();
+  }
+
  private:
   // unique_ptr Deleter implementation for fts_close
   struct FtsCloser {
@@ -1359,6 +1382,13 @@ class PosixEnv : public Env {
       return IOError(Substitute("Call to mkstemp() failed on name template $0", name_template),
                      errno);
     }
+    // mkstemp defaults to making files with permissions 0600. But, if the
+    // user configured a more permissive umask, then we ensure that the
+    // resulting file gets the desired (wider) permissions.
+    uint32_t new_perms = 0666 & ~g_parsed_umask;
+    if (new_perms != 0600) {
+      CHECK_ERR(fchmod(created_fd, new_perms));
+    }
     *fd = created_fd;
     *created_filename = fname.get();
     return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/daf289d6/src/kudu/util/flags.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/flags.cc b/src/kudu/util/flags.cc
index 1d8580b..458951b 100644
--- a/src/kudu/util/flags.cc
+++ b/src/kudu/util/flags.cc
@@ -23,6 +23,9 @@
 #include <unordered_set>
 #include <vector>
 
+#include <sys/stat.h>
+#include <sys/types.h>
+
 #include <gflags/gflags.h>
 #include <gperftools/heap-profiler.h>
 
@@ -65,6 +68,31 @@ DEFINE_bool(disable_core_dumps, false, "Disable core dumps when this process cra
 TAG_FLAG(disable_core_dumps, advanced);
 TAG_FLAG(disable_core_dumps, evolving);
 
+DEFINE_string(umask, "077",
+              "The umask that will be used when creating files and directories. "
+              "Permissions of top-level data directories will also be modified at "
+              "start-up to conform to the given umask. Changing this value may "
+              "enable unauthorized local users to read or modify data stored by Kudu.");
+TAG_FLAG(umask, advanced);
+
+static bool ValidateUmask(const char* /*flagname*/, const string& value) {
+  uint32_t parsed;
+  if (!safe_strtou32_base(value.c_str(), &parsed, 8)) {
+    LOG(ERROR) << "Invalid umask: must be an octal string";
+    return false;
+  }
+
+  // Verify that the umask doesn't restrict the permissions of the owner.
+  // If it did, we'd end up creating files that we can't read.
+  if ((parsed & 0700) != 0) {
+    LOG(ERROR) << "Invalid umask value: must not restrict owner permissions";
+    return false;
+  }
+  return true;
+}
+static bool dummy = google::RegisterFlagValidator(&FLAGS_umask, &ValidateUmask);
+
+
 DEFINE_bool(unlock_experimental_flags, false,
             "Unlock flags marked as 'experimental'. These flags are not guaranteed to "
             "be maintained across releases of Kudu, and may enable features or behavior "
@@ -228,6 +256,10 @@ DECLARE_bool(version);
 TAG_FLAG(version, stable);
 
 namespace kudu {
+
+// After flags have been parsed, the umask value is filled in here.
+uint32_t g_parsed_umask = -1;
+
 namespace {
 
 void AppendXMLTag(const char* tag, const string& txt, string* r) {
@@ -320,6 +352,17 @@ void CheckFlagsAllowed() {
   }
 }
 
+void SetUmask() {
+  // We already validated with a nice error message using the ValidateUmask
+  // FlagValidator above.
+  CHECK(safe_strtou32_base(FLAGS_umask.c_str(), &g_parsed_umask, 8));
+  uint32_t old_mask = umask(g_parsed_umask);
+  if (old_mask != g_parsed_umask) {
+    VLOG(2) << "Changed umask from " << StringPrintf("%03o", old_mask) << " to "
+            << StringPrintf("%03o", g_parsed_umask);
+  }
+}
+
 } // anonymous namespace
 
 int ParseCommandLineFlags(int* argc, char*** argv, bool remove_flags) {
@@ -350,6 +393,8 @@ void HandleCommonFlags() {
     DisableCoreDumps();
   }
 
+  SetUmask();
+
 #ifdef TCMALLOC_ENABLED
   if (FLAGS_enable_process_lifetime_heap_profiling) {
     HeapProfilerStart(FLAGS_heap_profile_path.c_str());

http://git-wip-us.apache.org/repos/asf/kudu/blob/daf289d6/src/kudu/util/flags.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/flags.h b/src/kudu/util/flags.h
index 8c29367..f87e3b2 100644
--- a/src/kudu/util/flags.h
+++ b/src/kudu/util/flags.h
@@ -25,6 +25,10 @@
 
 namespace kudu {
 
+// The umask of the process, set based on the --umask flag during
+// HandleCommonFlags().
+extern uint32_t g_parsed_umask;
+
 // Looks for flags in argv and parses them.  Rearranges argv to put
 // flags first, or removes them entirely if remove_flags is true.
 // If a flag is defined more than once in the command line or flag