You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by jd...@apache.org on 2017/10/18 20:56:07 UTC

[1/3] kudu git commit: env_util: remove ScopedFileDeleter

Repository: kudu
Updated Branches:
  refs/heads/master f96ab36ad -> 67382e9a4


env_util: remove ScopedFileDeleter

It's a holdover from before ScopedCleanup existed.

The intermingling of files and directories is a neat feature, but it's only
used in two places and isn't worth the complexity of a dedicated class.

Change-Id: I245fec4fc07caffad40b1bf1ba9b16839b08136c
Reviewed-on: http://gerrit.cloudera.org:8080/8288
Reviewed-by: Andrew Wong <aw...@cloudera.com>
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Tested-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/27a62b0c
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/27a62b0c
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/27a62b0c

Branch: refs/heads/master
Commit: 27a62b0ca510bc682703b4c5da0b0160c4d4e5b3
Parents: f96ab36
Author: Adar Dembo <ad...@cloudera.com>
Authored: Thu Oct 12 19:05:37 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Oct 18 19:33:35 2017 +0000

----------------------------------------------------------------------
 src/kudu/fs/data_dirs.cc          | 33 +++++++++++++++++++++------------
 src/kudu/fs/fs_manager.cc         | 30 ++++++++++++++++++------------
 src/kudu/fs/log_block_manager.cc  |  9 ++++++---
 src/kudu/tools/tool_action_pbc.cc | 14 ++++++++++----
 src/kudu/util/env_util.cc         | 23 +----------------------
 src/kudu/util/env_util.h          | 24 ------------------------
 src/kudu/util/pb_util.cc          | 13 +++++++++----
 7 files changed, 65 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/27a62b0c/src/kudu/fs/data_dirs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index 1143514..9ece90d 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -20,7 +20,6 @@
 #include <algorithm>
 #include <cerrno>
 #include <cstdint>
-#include <deque>
 #include <iterator>
 #include <memory>
 #include <mutex>
@@ -45,7 +44,6 @@
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
-#include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/env.h"
@@ -56,6 +54,7 @@
 #include "kudu/util/oid_generator.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/random_util.h"
+#include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/status.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_util_prod.h"
@@ -111,10 +110,8 @@ namespace kudu {
 
 namespace fs {
 
-using env_util::ScopedFileDeleter;
 using internal::DataDirGroup;
 using std::default_random_engine;
-using std::deque;
 using std::iota;
 using std::set;
 using std::shuffle;
@@ -149,7 +146,10 @@ Status CheckHolePunch(Env* env, const string& path) {
   RETURN_NOT_OK(env->NewRWFile(opts, filename, &file));
 
   // The file has been created; delete it on exit no matter what happens.
-  ScopedFileDeleter file_deleter(env, filename);
+  auto file_deleter = MakeScopedCleanup([&]() {
+    WARN_NOT_OK(env->DeleteFile(filename),
+                "Could not delete file " + filename);
+  });
 
   // Preallocate it, making sure the file's size is what we'd expect.
   uint64_t sz;
@@ -370,8 +370,19 @@ Status DataDirManager::CreateNew(Env* env, CanonicalizedRootsList data_fs_roots,
 Status DataDirManager::Create() {
   CHECK(!read_only_);
 
-  deque<ScopedFileDeleter*> delete_on_failure;
-  ElementDeleter d(&delete_on_failure);
+  vector<string> dirs_to_delete;
+  vector<string> files_to_delete;
+  auto deleter = MakeScopedCleanup([&]() {
+    // Delete files first so that the directories will be empty when deleted.
+    for (const auto& f : files_to_delete) {
+      WARN_NOT_OK(env_->DeleteFile(f), "Could not delete file " + f);
+    }
+    // Delete directories in reverse order since parent directories will have
+    // been added before child directories.
+    for (auto it = dirs_to_delete.rbegin(); it != dirs_to_delete.rend(); it++) {
+      WARN_NOT_OK(env_->DeleteDir(*it), "Could not delete dir " + *it);
+    }
+  });
 
   // The UUIDs and indices will be included in every instance file.
   ObjectIdGenerator gen;
@@ -390,7 +401,7 @@ Status DataDirManager::Create() {
     RETURN_NOT_OK_PREPEND(env_util::CreateDirIfMissing(env_, data_dir, &created),
         Substitute("Could not create directory $0", data_dir));
     if (created) {
-      delete_on_failure.push_front(new ScopedFileDeleter(env_, data_dir));
+      dirs_to_delete.emplace_back(data_dir);
       to_sync.insert(root.path);
     }
 
@@ -402,7 +413,7 @@ Status DataDirManager::Create() {
     PathInstanceMetadataFile metadata(env_, block_manager_type_,
                                       instance_filename);
     RETURN_NOT_OK_PREPEND(metadata.Create(all_uuids[idx], all_uuids), instance_filename);
-    delete_on_failure.push_front(new ScopedFileDeleter(env_, instance_filename));
+    files_to_delete.emplace_back(instance_filename);
 
     idx++;
   }
@@ -414,9 +425,7 @@ Status DataDirManager::Create() {
   }
 
   // Success: don't delete any files.
-  for (ScopedFileDeleter* deleter : delete_on_failure) {
-    deleter->Cancel();
-  }
+  deleter.cancel();
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/27a62b0c/src/kudu/fs/fs_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager.cc b/src/kudu/fs/fs_manager.cc
index a68769c..670c397 100644
--- a/src/kudu/fs/fs_manager.cc
+++ b/src/kudu/fs/fs_manager.cc
@@ -19,7 +19,6 @@
 
 #include <cinttypes>
 #include <ctime>
-#include <deque>
 #include <iostream>
 #include <map>
 #include <set>
@@ -42,7 +41,6 @@
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
-#include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/split.h"
@@ -58,6 +56,7 @@
 #include "kudu/util/oid_generator.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/pb_util.h"
+#include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/website_util.h"
 
@@ -92,7 +91,6 @@ DEFINE_string(fs_data_dirs, "",
               "block directory.");
 TAG_FLAG(fs_data_dirs, stable);
 
-using kudu::env_util::ScopedFileDeleter;
 using kudu::fs::BlockManagerOptions;
 using kudu::fs::CreateBlockOptions;
 using kudu::fs::DataDirManager;
@@ -386,8 +384,19 @@ Status FsManager::CreateInitialFileSystemLayout(boost::optional<string> uuid) {
   // subdirectories.
   //
   // In the event of failure, delete everything we created.
-  std::deque<ScopedFileDeleter*> delete_on_failure;
-  ElementDeleter d(&delete_on_failure);
+  vector<string> dirs_to_delete;
+  vector<string> files_to_delete;
+  auto deleter = MakeScopedCleanup([&]() {
+    // Delete files first so that the directories will be empty when deleted.
+    for (const auto& f : files_to_delete) {
+      WARN_NOT_OK(env_->DeleteFile(f), "Could not delete file " + f);
+    }
+    // Delete directories in reverse order since parent directories will have
+    // been added before child directories.
+    for (auto it = dirs_to_delete.rbegin(); it != dirs_to_delete.rend(); it++) {
+      WARN_NOT_OK(env_->DeleteDir(*it), "Could not delete dir " + *it);
+    }
+  });
 
   InstanceMetadataPB metadata;
   RETURN_NOT_OK(CreateInstanceMetadata(std::move(uuid), &metadata));
@@ -401,13 +410,12 @@ Status FsManager::CreateInitialFileSystemLayout(boost::optional<string> uuid) {
     RETURN_NOT_OK_PREPEND(CreateDirIfMissing(root_name, &created),
                           "Unable to create FSManager root");
     if (created) {
-      delete_on_failure.push_front(new ScopedFileDeleter(env_, root_name));
+      dirs_to_delete.emplace_back(root_name);
       to_sync.insert(DirName(root_name));
     }
     RETURN_NOT_OK_PREPEND(WriteInstanceMetadata(metadata, root_name),
                           "Unable to write instance metadata");
-    delete_on_failure.push_front(new ScopedFileDeleter(
-        env_, GetInstanceMetadataPath(root_name)));
+    files_to_delete.emplace_back(GetInstanceMetadataPath(root_name));
   }
 
   // Initialize ancillary directories.
@@ -419,7 +427,7 @@ Status FsManager::CreateInitialFileSystemLayout(boost::optional<string> uuid) {
     RETURN_NOT_OK_PREPEND(CreateDirIfMissing(dir, &created),
                           Substitute("Unable to create directory $0", dir));
     if (created) {
-      delete_on_failure.push_front(new ScopedFileDeleter(env_, dir));
+      dirs_to_delete.emplace_back(dir);
       to_sync.insert(DirName(dir));
     }
   }
@@ -442,9 +450,7 @@ Status FsManager::CreateInitialFileSystemLayout(boost::optional<string> uuid) {
   }
 
   // Success: don't delete any files.
-  for (ScopedFileDeleter* deleter : delete_on_failure) {
-    deleter->Cancel();
-  }
+  deleter.cancel();
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/27a62b0c/src/kudu/fs/log_block_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager.cc b/src/kudu/fs/log_block_manager.cc
index b9a9322..1b489b8 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -60,7 +60,6 @@
 #include "kudu/util/array_view.h"
 #include "kudu/util/atomic.h"
 #include "kudu/util/env.h"
-#include "kudu/util/env_util.h"
 #include "kudu/util/file_cache.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/locks.h"
@@ -2600,7 +2599,11 @@ Status LogBlockManager::RewriteMetadataFile(const LogBlockContainer& container,
   RETURN_NOT_OK_LBM_DISK_FAILURE_PREPEND(env_->NewTempRWFile(RWFileOptions(), tmpl,
                                                              &tmp_file_name, &tmp_file),
                                          "could not create temporary metadata file");
-  env_util::ScopedFileDeleter tmp_deleter(env_, tmp_file_name);
+  auto tmp_deleter = MakeScopedCleanup([&]() {
+    WARN_NOT_OK(env_->DeleteFile(tmp_file_name),
+                "Could not delete file " + tmp_file_name);
+
+  });
   WritablePBContainerFile pb_file(std::move(tmp_file));
   RETURN_NOT_OK_LBM_DISK_FAILURE_PREPEND(pb_file.CreateNew(BlockRecordPB()),
                                          "could not initialize temporary metadata file");
@@ -2622,7 +2625,7 @@ Status LogBlockManager::RewriteMetadataFile(const LogBlockContainer& container,
   // old file descriptor pointing to the now-deleted old version.
   file_cache_.Invalidate(metadata_file_name);
 
-  tmp_deleter.Cancel();
+  tmp_deleter.cancel();
   *file_bytes_delta = (static_cast<int64_t>(old_metadata_size) - new_metadata_size);
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/27a62b0c/src/kudu/tools/tool_action_pbc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_pbc.cc b/src/kudu/tools/tool_action_pbc.cc
index 5a6acc0..c26a89d 100644
--- a/src/kudu/tools/tool_action_pbc.cc
+++ b/src/kudu/tools/tool_action_pbc.cc
@@ -39,10 +39,10 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/env.h"
-#include "kudu/util/env_util.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/pb_util.h"
+#include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 #include "kudu/util/subprocess.h"
@@ -142,7 +142,10 @@ Status EditFile(const RunnerContext& context) {
   const string tmp_out_path = path + ".new";
   unique_ptr<RWFile> out_rwfile;
   RETURN_NOT_OK_PREPEND(env->NewRWFile(tmp_out_path, &out_rwfile), "couldn't open output PBC file");
-  env_util::ScopedFileDeleter delete_tmp_output(env, tmp_out_path);
+  auto delete_tmp_output = MakeScopedCleanup([&]() {
+    WARN_NOT_OK(env->DeleteFile(tmp_out_path),
+                "Could not delete file " + tmp_out_path);
+  });
 
   // Also make a tmp file where we'll write the PBC in JSON format for
   // easy editing.
@@ -153,7 +156,10 @@ Status EditFile(const RunnerContext& context) {
                                                  JoinPathSegments(dir, tmp_template),
                                                  &tmp_json_path, &tmp_json_file),
                         "couldn't create temporary file");
-  env_util::ScopedFileDeleter delete_tmp_json(env, tmp_json_path);
+  auto delete_tmp_json = MakeScopedCleanup([&]() {
+    WARN_NOT_OK(env->DeleteFile(tmp_json_path),
+                "Could not delete file " + tmp_json_path);
+  });
 
   // Dump the contents in JSON to the temporary file.
   {
@@ -202,7 +208,7 @@ Status EditFile(const RunnerContext& context) {
   LOG(INFO) << "Moved original file to " << backup_path;
   RETURN_NOT_OK_PREPEND(env->RenameFile(tmp_out_path, path),
                         "couldn't move new file into place");
-  delete_tmp_output.Cancel();
+  delete_tmp_output.cancel();
   WARN_NOT_OK(env->SyncDir(dir), "couldn't sync directory");
 
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/27a62b0c/src/kudu/util/env_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.cc b/src/kudu/util/env_util.cc
index 04a9844..ac46b01 100644
--- a/src/kudu/util/env_util.cc
+++ b/src/kudu/util/env_util.cc
@@ -30,6 +30,7 @@
 #include <glog/logging.h>
 
 #include "kudu/gutil/bind.h"
+#include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
@@ -266,27 +267,5 @@ Status DeleteTmpFilesRecursively(Env* env, const string& path) {
   return env->Walk(path, Env::PRE_ORDER, Bind(&DeleteTmpFilesRecursivelyCb, env));
 }
 
-ScopedFileDeleter::ScopedFileDeleter(Env* env, std::string path)
-    : env_(DCHECK_NOTNULL(env)), path_(std::move(path)), should_delete_(true) {}
-
-ScopedFileDeleter::~ScopedFileDeleter() {
-  if (should_delete_) {
-    bool is_dir;
-    Status s = env_->IsDirectory(path_, &is_dir);
-    WARN_NOT_OK(s, Substitute(
-        "Failed to determine if path is a directory: $0", path_));
-    if (!s.ok()) {
-      return;
-    }
-    if (is_dir) {
-      WARN_NOT_OK(env_->DeleteDir(path_),
-                  Substitute("Failed to remove directory: $0", path_));
-    } else {
-      WARN_NOT_OK(env_->DeleteFile(path_),
-          Substitute("Failed to remove file: $0", path_));
-    }
-  }
-}
-
 } // namespace env_util
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/27a62b0c/src/kudu/util/env_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.h b/src/kudu/util/env_util.h
index 21bf0ff..442596f 100644
--- a/src/kudu/util/env_util.h
+++ b/src/kudu/util/env_util.h
@@ -22,7 +22,6 @@
 #include <memory>
 #include <string>
 
-#include "kudu/gutil/macros.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
@@ -89,29 +88,6 @@ Status DeleteExcessFilesByPattern(Env* env, const std::string& pattern, int max_
 // Deletion errors generate warnings but do not halt the traversal.
 Status DeleteTmpFilesRecursively(Env* env, const std::string& path);
 
-// Deletes a file or directory when this object goes out of scope.
-//
-// The deletion may be cancelled by calling .Cancel().
-// This is typically useful for cleaning up temporary files if the
-// creation of the tmp file may fail.
-class ScopedFileDeleter {
- public:
-  ScopedFileDeleter(Env* env, std::string path);
-  ~ScopedFileDeleter();
-
-  // Do not delete the file when this object goes out of scope.
-  void Cancel() {
-    should_delete_ = false;
-  }
-
- private:
-  Env* const env_;
-  const std::string path_;
-  bool should_delete_;
-
-  DISALLOW_COPY_AND_ASSIGN(ScopedFileDeleter);
-};
-
 } // namespace env_util
 } // namespace kudu
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/27a62b0c/src/kudu/util/pb_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/pb_util.cc b/src/kudu/util/pb_util.cc
index 874896c..5c5764a 100644
--- a/src/kudu/util/pb_util.cc
+++ b/src/kudu/util/pb_util.cc
@@ -68,6 +68,7 @@
 #include "kudu/util/path_util.h"
 #include "kudu/util/pb_util-internal.h"
 #include "kudu/util/pb_util.pb.h"
+#include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 
@@ -476,7 +477,9 @@ Status WritePBToPath(Env* env, const std::string& path,
 
   unique_ptr<WritableFile> file;
   RETURN_NOT_OK(env->NewTempWritableFile(WritableFileOptions(), tmp_template, &tmp_path, &file));
-  env_util::ScopedFileDeleter tmp_deleter(env, tmp_path);
+  auto tmp_deleter = MakeScopedCleanup([&]() {
+    WARN_NOT_OK(env->DeleteFile(tmp_path), "Could not delete file " + tmp_path);
+  });
 
   WritableFileOutputStream output(file.get());
   bool res = msg.SerializeToZeroCopyStream(&output);
@@ -489,7 +492,7 @@ Status WritePBToPath(Env* env, const std::string& path,
   }
   RETURN_NOT_OK_PREPEND(file->Close(), "Failed to Close() " + tmp_path);
   RETURN_NOT_OK_PREPEND(env->RenameFile(tmp_path, path), "Failed to rename tmp file to " + path);
-  tmp_deleter.Cancel();
+  tmp_deleter.cancel();
   if (sync == pb_util::SYNC) {
     RETURN_NOT_OK_PREPEND(env->SyncDir(DirName(path)), "Failed to SyncDir() parent of " + path);
   }
@@ -978,7 +981,9 @@ Status WritePBContainerToPath(Env* env, const std::string& path,
 
   unique_ptr<RWFile> file;
   RETURN_NOT_OK(env->NewTempRWFile(RWFileOptions(), tmp_template, &tmp_path, &file));
-  env_util::ScopedFileDeleter tmp_deleter(env, tmp_path);
+  auto tmp_deleter = MakeScopedCleanup([&]() {
+    WARN_NOT_OK(env->DeleteFile(tmp_path), "Could not delete file " + tmp_path);
+  });
 
   WritablePBContainerFile pb_file(std::move(file));
   RETURN_NOT_OK(pb_file.CreateNew(msg));
@@ -989,7 +994,7 @@ Status WritePBContainerToPath(Env* env, const std::string& path,
   RETURN_NOT_OK(pb_file.Close());
   RETURN_NOT_OK_PREPEND(env->RenameFile(tmp_path, path),
                         "Failed to rename tmp file to " + path);
-  tmp_deleter.Cancel();
+  tmp_deleter.cancel();
   if (sync == pb_util::SYNC) {
     RETURN_NOT_OK_PREPEND(env->SyncDir(DirName(path)),
                           "Failed to SyncDir() parent of " + path);


[2/3] kudu git commit: fs: move IsDirectoryEmpty to env_util

Posted by jd...@apache.org.
fs: move IsDirectoryEmpty to env_util

I thought I'd need this in a follow-on patch. I didn't, but I still had the
unit tests written, and I think env_util is a better home for this.

Change-Id: I6f6324ac7507801e5ba8c6ed87c9bb3a3a9ffc1a
Reviewed-on: http://gerrit.cloudera.org:8080/8289
Tested-by: Adar Dembo <ad...@cloudera.com>
Reviewed-by: Andrew Wong <aw...@cloudera.com>
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 7a460f0455e64ca001994f66e0ce0142fc6f1980
Parents: 27a62b0
Author: Adar Dembo <ad...@cloudera.com>
Authored: Fri Oct 13 16:17:26 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Oct 18 19:33:54 2017 +0000

----------------------------------------------------------------------
 src/kudu/fs/fs_manager.cc      | 17 +----------------
 src/kudu/fs/fs_manager.h       |  6 ------
 src/kudu/util/env_util-test.cc | 16 ++++++++++++++++
 src/kudu/util/env_util.cc      | 14 ++++++++++++++
 src/kudu/util/env_util.h       |  6 ++++++
 5 files changed, 37 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/7a460f04/src/kudu/fs/fs_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager.cc b/src/kudu/fs/fs_manager.cc
index 670c397..bf88c63 100644
--- a/src/kudu/fs/fs_manager.cc
+++ b/src/kudu/fs/fs_manager.cc
@@ -371,7 +371,7 @@ Status FsManager::CreateInitialFileSystemLayout(boost::optional<string> uuid) {
       continue;
     }
     bool is_empty;
-    RETURN_NOT_OK_PREPEND(IsDirectoryEmpty(root.path, &is_empty),
+    RETURN_NOT_OK_PREPEND(env_util::IsDirectoryEmpty(env_, root.path, &is_empty),
                           "Unable to check if FSManager root is empty");
     if (!is_empty) {
       return Status::AlreadyPresent(
@@ -490,21 +490,6 @@ Status FsManager::WriteInstanceMetadata(const InstanceMetadataPB& metadata,
   return Status::OK();
 }
 
-Status FsManager::IsDirectoryEmpty(const string& path, bool* is_empty) {
-  vector<string> children;
-  RETURN_NOT_OK(env_->GetChildren(path, &children));
-  for (const string& child : children) {
-    if (child == "." || child == "..") {
-      continue;
-    } else {
-      *is_empty = false;
-      return Status::OK();
-    }
-  }
-  *is_empty = true;
-  return Status::OK();
-}
-
 Status FsManager::CreateDirIfMissing(const string& path, bool* created) {
   return env_util::CreateDirIfMissing(env_, path, created);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/7a460f04/src/kudu/fs/fs_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager.h b/src/kudu/fs/fs_manager.h
index f5ec089..b0488a8 100644
--- a/src/kudu/fs/fs_manager.h
+++ b/src/kudu/fs/fs_manager.h
@@ -255,12 +255,6 @@ class FsManager {
   Status WriteInstanceMetadata(const InstanceMetadataPB& metadata,
                                const std::string& root);
 
-  // Checks if 'path' is an empty directory.
-  //
-  // Returns an error if it's not a directory. Otherwise, sets 'is_empty'
-  // accordingly.
-  Status IsDirectoryEmpty(const std::string& path, bool* is_empty);
-
   // ==========================================================================
   //  file-system helpers
   // ==========================================================================

http://git-wip-us.apache.org/repos/asf/kudu/blob/7a460f04/src/kudu/util/env_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util-test.cc b/src/kudu/util/env_util-test.cc
index b919b4a..9c8266f 100644
--- a/src/kudu/util/env_util-test.cc
+++ b/src/kudu/util/env_util-test.cc
@@ -172,5 +172,21 @@ TEST_F(EnvUtilTest, TestDeleteExcessFilesByPattern) {
   ASSERT_EQ(expected_set, children_set) << children;
 }
 
+TEST_F(EnvUtilTest, TestIsDirectoryEmpty) {
+  const string kDir = JoinPathSegments(test_dir_, "foo");
+  const string kFile = JoinPathSegments(kDir, "bar");
+
+  bool is_empty;
+  ASSERT_TRUE(env_util::IsDirectoryEmpty(env_, kDir, &is_empty).IsNotFound());
+  ASSERT_OK(env_->CreateDir(kDir));
+  ASSERT_OK(env_util::IsDirectoryEmpty(env_, kDir, &is_empty));
+  ASSERT_TRUE(is_empty);
+
+  unique_ptr<WritableFile> file;
+  ASSERT_OK(env_->NewWritableFile(WritableFileOptions(), kFile, &file));
+  ASSERT_OK(env_util::IsDirectoryEmpty(env_, kDir, &is_empty));
+  ASSERT_FALSE(is_empty);
+}
+
 } // namespace env_util
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/7a460f04/src/kudu/util/env_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.cc b/src/kudu/util/env_util.cc
index ac46b01..1a7a5bc 100644
--- a/src/kudu/util/env_util.cc
+++ b/src/kudu/util/env_util.cc
@@ -267,5 +267,19 @@ Status DeleteTmpFilesRecursively(Env* env, const string& path) {
   return env->Walk(path, Env::PRE_ORDER, Bind(&DeleteTmpFilesRecursivelyCb, env));
 }
 
+Status IsDirectoryEmpty(Env* env, const string& path, bool* is_empty) {
+  vector<string> children;
+  RETURN_NOT_OK(env->GetChildren(path, &children));
+  for (const auto& c : children) {
+    if (c == "." || c == "..") {
+      continue;
+    }
+    *is_empty = false;
+    return Status::OK();
+  }
+  *is_empty = true;
+  return Status::OK();
+}
+
 } // namespace env_util
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/7a460f04/src/kudu/util/env_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.h b/src/kudu/util/env_util.h
index 442596f..faba197 100644
--- a/src/kudu/util/env_util.h
+++ b/src/kudu/util/env_util.h
@@ -88,6 +88,12 @@ Status DeleteExcessFilesByPattern(Env* env, const std::string& pattern, int max_
 // Deletion errors generate warnings but do not halt the traversal.
 Status DeleteTmpFilesRecursively(Env* env, const std::string& path);
 
+// Checks if 'path' is an empty directory.
+//
+// Returns an error if it's not a directory. Otherwise, sets 'is_empty'
+// accordingly.
+Status IsDirectoryEmpty(Env* env, const std::string& path, bool* is_empty);
+
 } // namespace env_util
 } // namespace kudu
 


[3/3] kudu git commit: kudu-hive: add provided scope dependency on hadoop-mapreduce-client-common

Posted by jd...@apache.org.
kudu-hive: add provided scope dependency on hadoop-mapreduce-client-common

This appears to be necessary to get the kudu-hive tests to run against
some versions of Hadoop, otherwise NoClassDefFound exceptions occur for
the following classes while attempting to create the Hive configuration
object:

- org.apache.hadoop.mapreduce.TaskAttemptContext
- org.apache.hadoop.mapred.MRVersion

I haven't untangled the chain of Hadoop modules and Maven dependencies
to figure out exactly why this occurs, but since the fix is harmless
enough I'm inclined to keep my sanity and let it lie.

Note: this isn't a problem with Gradle.

Change-Id: I62d62ae31ebd82a2fd39030bfa6201aef6ff82ff
Reviewed-on: http://gerrit.cloudera.org:8080/8316
Tested-by: Kudu Jenkins
Reviewed-by: Jean-Daniel Cryans <jd...@apache.org>


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

Branch: refs/heads/master
Commit: 67382e9a408086a9ad25598d4f0d68650c176ed1
Parents: 7a460f0
Author: Dan Burkert <da...@apache.org>
Authored: Wed Oct 18 10:20:34 2017 -0700
Committer: Jean-Daniel Cryans <jd...@apache.org>
Committed: Wed Oct 18 20:54:19 2017 +0000

----------------------------------------------------------------------
 java/kudu-hive/pom.xml | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/67382e9a/java/kudu-hive/pom.xml
----------------------------------------------------------------------
diff --git a/java/kudu-hive/pom.xml b/java/kudu-hive/pom.xml
index fa67977..d6e3aa7 100644
--- a/java/kudu-hive/pom.xml
+++ b/java/kudu-hive/pom.xml
@@ -64,6 +64,12 @@
             <version>${hadoop.version}</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-common</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>provided</scope>
+        </dependency>
 
         <dependency>
             <groupId>log4j</groupId>