|
@@ -26,6 +26,7 @@
|
|
|
#include <tuple>
|
|
|
#include <iostream>
|
|
|
#include <pwd.h>
|
|
|
+#include <fnmatch.h>
|
|
|
|
|
|
#define FMT_THIS_ADDR "this=" << (void*)this
|
|
|
|
|
@@ -39,6 +40,34 @@ using ::asio::ip::tcp;
|
|
|
|
|
|
static constexpr uint16_t kDefaultPort = 8020;
|
|
|
|
|
|
+uint32_t FileSystem::GetDefaultFindMaxDepth() {
|
|
|
+ return std::numeric_limits<uint32_t>::max();
|
|
|
+}
|
|
|
+
|
|
|
+uint16_t FileSystem::GetDefaultPermissionMask() {
|
|
|
+ return 0755;
|
|
|
+}
|
|
|
+
|
|
|
+Status FileSystem::CheckValidPermissionMask(uint16_t permissions) {
|
|
|
+ if (permissions > 01777) {
|
|
|
+ std::stringstream errormsg;
|
|
|
+ errormsg << "CheckValidPermissionMask: argument 'permissions' is " << std::oct
|
|
|
+ << std::showbase << permissions << " (should be between 0 and 01777)";
|
|
|
+ return Status::InvalidArgument(errormsg.str().c_str());
|
|
|
+ }
|
|
|
+ return Status::OK();
|
|
|
+}
|
|
|
+
|
|
|
+Status FileSystem::CheckValidReplication(uint16_t replication) {
|
|
|
+ if (replication < 1 || replication > 512) {
|
|
|
+ std::stringstream errormsg;
|
|
|
+ errormsg << "CheckValidReplication: argument 'replication' is "
|
|
|
+ << replication << " (should be between 1 and 512)";
|
|
|
+ return Status::InvalidArgument(errormsg.str().c_str());
|
|
|
+ }
|
|
|
+ return Status::OK();
|
|
|
+}
|
|
|
+
|
|
|
/*****************************************************************************
|
|
|
* FILESYSTEM BASE CLASS
|
|
|
****************************************************************************/
|
|
@@ -446,7 +475,7 @@ void FileSystemImpl::SetReplication(const std::string & path, int16_t replicatio
|
|
|
handler(Status::InvalidArgument("SetReplication: argument 'path' cannot be empty"));
|
|
|
return;
|
|
|
}
|
|
|
- Status replStatus = NameNodeOperations::CheckValidReplication(replication);
|
|
|
+ Status replStatus = FileSystem::CheckValidReplication(replication);
|
|
|
if (!replStatus.ok()) {
|
|
|
handler(replStatus);
|
|
|
return;
|
|
@@ -593,44 +622,43 @@ Status FileSystemImpl::GetFsStats(FsInfo & fs_info) {
|
|
|
* Some compilers don't like recursive lambdas, so we make the lambda call a
|
|
|
* method, which in turn creates a lambda calling itself.
|
|
|
*/
|
|
|
-void FileSystemImpl::GetListingShim(const Status &stat, std::shared_ptr<std::vector<StatInfo>> &stat_infos, bool has_more,
|
|
|
- std::string path,
|
|
|
- const std::function<bool(const Status &, std::shared_ptr<std::vector<StatInfo>>&, bool)> &handler) {
|
|
|
- bool has_next = stat_infos && stat_infos->size() > 0;
|
|
|
+void FileSystemImpl::GetListingShim(const Status &stat, const std::vector<StatInfo> & stat_infos, bool has_more,
|
|
|
+ std::string path, const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> &handler) {
|
|
|
+ bool has_next = !stat_infos.empty();
|
|
|
bool get_more = handler(stat, stat_infos, has_more && has_next);
|
|
|
if (get_more && has_more && has_next ) {
|
|
|
- auto callback = [this, path, handler](const Status &stat, std::shared_ptr<std::vector<StatInfo>> &stat_infos, bool has_more) {
|
|
|
+ auto callback = [this, path, handler](const Status &stat, const std::vector<StatInfo> & stat_infos, bool has_more) {
|
|
|
GetListingShim(stat, stat_infos, has_more, path, handler);
|
|
|
};
|
|
|
|
|
|
- std::string last = stat_infos->back().path;
|
|
|
+ std::string last = stat_infos.back().path;
|
|
|
nn_.GetListing(path, callback, last);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
void FileSystemImpl::GetListing(
|
|
|
const std::string &path,
|
|
|
- const std::function<bool(const Status &, std::shared_ptr<std::vector<StatInfo>>&, bool)> &handler) {
|
|
|
+ const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> &handler) {
|
|
|
LOG_INFO(kFileSystem, << "FileSystemImpl::GetListing("
|
|
|
<< FMT_THIS_ADDR << ", path="
|
|
|
<< path << ") called");
|
|
|
|
|
|
// Caputure the state and push it into the shim
|
|
|
- auto callback = [this, path, handler](const Status &stat, std::shared_ptr<std::vector<StatInfo>> &stat_infos, bool has_more) {
|
|
|
+ auto callback = [this, path, handler](const Status &stat, const std::vector<StatInfo> & stat_infos, bool has_more) {
|
|
|
GetListingShim(stat, stat_infos, has_more, path, handler);
|
|
|
};
|
|
|
|
|
|
nn_.GetListing(path, callback);
|
|
|
}
|
|
|
|
|
|
-Status FileSystemImpl::GetListing(const std::string &path, std::shared_ptr<std::vector<StatInfo>> &stat_infos) {
|
|
|
+Status FileSystemImpl::GetListing(const std::string &path, std::vector<StatInfo> * stat_infos) {
|
|
|
LOG_INFO(kFileSystem, << "FileSystemImpl::[sync]GetListing("
|
|
|
<< FMT_THIS_ADDR << ", path="
|
|
|
<< path << ") called");
|
|
|
|
|
|
- // In this case, we're going to allocate the result on the heap and have the
|
|
|
- // async code populate it.
|
|
|
- auto results = std::make_shared<std::vector<StatInfo>>();
|
|
|
+ if (!stat_infos) {
|
|
|
+ return Status::InvalidArgument("FileSystemImpl::GetListing: argument 'stat_infos' cannot be NULL");
|
|
|
+ }
|
|
|
|
|
|
auto callstate = std::make_shared<std::promise<Status>>();
|
|
|
std::future<Status> future(callstate->get_future());
|
|
@@ -640,9 +668,9 @@ Status FileSystemImpl::GetListing(const std::string &path, std::shared_ptr<std::
|
|
|
Keep requesting more until we get the entire listing, and don't set the promise
|
|
|
* until we have the entire listing.
|
|
|
*/
|
|
|
- auto h = [callstate, results](const Status &s, std::shared_ptr<std::vector<StatInfo>> si, bool has_more) -> bool {
|
|
|
- if (si) {
|
|
|
- results->insert(results->end(), si->begin(), si->end());
|
|
|
+ auto h = [callstate, stat_infos](const Status &s, const std::vector<StatInfo> & si, bool has_more) -> bool {
|
|
|
+ if (!si.empty()) {
|
|
|
+ stat_infos->insert(stat_infos->end(), si.begin(), si.end());
|
|
|
}
|
|
|
|
|
|
bool done = !s.ok() || !has_more;
|
|
@@ -658,11 +686,6 @@ Status FileSystemImpl::GetListing(const std::string &path, std::shared_ptr<std::
|
|
|
/* block until promise is set */
|
|
|
Status stat = future.get();
|
|
|
|
|
|
- if (!stat.ok()) {
|
|
|
- return stat;
|
|
|
- }
|
|
|
-
|
|
|
- stat_infos = results;
|
|
|
return stat;
|
|
|
}
|
|
|
|
|
@@ -677,7 +700,7 @@ void FileSystemImpl::Mkdirs(const std::string & path, uint16_t permissions, bool
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- Status permStatus = NameNodeOperations::CheckValidPermissionMask(permissions);
|
|
|
+ Status permStatus = FileSystem::CheckValidPermissionMask(permissions);
|
|
|
if (!permStatus.ok()) {
|
|
|
handler(permStatus);
|
|
|
return;
|
|
@@ -790,7 +813,7 @@ void FileSystemImpl::SetPermission(const std::string & path,
|
|
|
handler(Status::InvalidArgument("SetPermission: argument 'path' cannot be empty"));
|
|
|
return;
|
|
|
}
|
|
|
- Status permStatus = NameNodeOperations::CheckValidPermissionMask(permissions);
|
|
|
+ Status permStatus = FileSystem::CheckValidPermissionMask(permissions);
|
|
|
if (!permStatus.ok()) {
|
|
|
handler(permStatus);
|
|
|
return;
|
|
@@ -832,8 +855,8 @@ void FileSystemImpl::SetOwner(const std::string & path, const std::string & user
|
|
|
nn_.SetOwner(path, username, groupname, handler);
|
|
|
}
|
|
|
|
|
|
-Status FileSystemImpl::SetOwner(const std::string & path,
|
|
|
- const std::string & username, const std::string & groupname) {
|
|
|
+Status FileSystemImpl::SetOwner(const std::string & path, const std::string & username,
|
|
|
+ const std::string & groupname) {
|
|
|
LOG_DEBUG(kFileSystem,
|
|
|
<< "FileSystemImpl::[sync]SetOwner(" << FMT_THIS_ADDR << ", path=" << path << ", username=" << username << ", groupname=" << groupname << ") called");
|
|
|
|
|
@@ -849,10 +872,179 @@ Status FileSystemImpl::SetOwner(const std::string & path,
|
|
|
|
|
|
/* block until promise is set */
|
|
|
Status stat = future.get();
|
|
|
-
|
|
|
return stat;
|
|
|
}
|
|
|
|
|
|
+/**
|
|
|
+ * Helper function for recursive Find calls.
|
|
|
+ *
|
|
|
+ * Some compilers don't like recursive lambdas, so we make the lambda call a
|
|
|
+ * method, which in turn creates a lambda calling itself.
|
|
|
+ *
|
|
|
+ * ***High-level explanation***
|
|
|
+ *
|
|
|
+ * Since we are allowing to use wild cards in both path and name, we start by expanding the path first.
|
|
|
+ * Boolean search_path is set to true when we search for the path and false when we search for the name.
|
|
|
+ * When we search for the path we break the given path pattern into sub-directories. Starting from the
|
|
|
+ * first sub-directory we list them one-by-one and recursively continue into directories that matched the
|
|
|
+ * path pattern at the current depth. Directories that are large will be requested to continue sending
|
|
|
+ * the results. We keep track of the current depth within the path pattern in the 'depth' variable.
|
|
|
+ * This continues recursively until the depth reaches the end of the path. Next that we start matching
|
|
|
+ * the name pattern. All directories that we find we recurse now, and all names that match the given name
|
|
|
+ * pattern are being stored in outputs and later sent back to the user.
|
|
|
+ */
|
|
|
+void FileSystemImpl::FindShim(const Status &stat, const std::vector<StatInfo> & stat_infos, bool directory_has_more,
|
|
|
+ std::shared_ptr<FindOperationalState> operational_state, std::shared_ptr<FindSharedState> shared_state) {
|
|
|
+ //We buffer the outputs then send them back at the end
|
|
|
+ std::vector<StatInfo> outputs;
|
|
|
+ //Return on error
|
|
|
+ if(!stat.ok()){
|
|
|
+ std::lock_guard<std::mutex> find_lock(shared_state->lock);
|
|
|
+ //We send true becuase we do not want the user code to exit before all our requests finished
|
|
|
+ shared_state->handler(stat, outputs, true);
|
|
|
+ shared_state->aborted = true;
|
|
|
+ }
|
|
|
+ if(!shared_state->aborted){
|
|
|
+ //User did not abort the operation
|
|
|
+ if (directory_has_more) {
|
|
|
+ //Directory is large and has more results
|
|
|
+ //We launch another async call to get more results
|
|
|
+ shared_state->outstanding_requests++;
|
|
|
+ auto callback = [this, operational_state, shared_state](const Status &stat, const std::vector<StatInfo> & stat_infos, bool has_more) {
|
|
|
+ FindShim(stat, stat_infos, has_more, operational_state, shared_state);
|
|
|
+ };
|
|
|
+ std::string last = stat_infos.back().path;
|
|
|
+ nn_.GetListing(operational_state->path, callback, last);
|
|
|
+ }
|
|
|
+ if(operational_state->search_path && operational_state->depth < shared_state->dirs.size() - 1){
|
|
|
+ //We are searching for the path and did not reach the end of the path yet
|
|
|
+ for (StatInfo const& si : stat_infos) {
|
|
|
+ //If we are at the last depth and it matches both path and name, we need to output it.
|
|
|
+ if (operational_state->depth == shared_state->dirs.size() - 2
|
|
|
+ && !fnmatch(shared_state->dirs[operational_state->depth + 1].c_str(), si.path.c_str(), 0)
|
|
|
+ && !fnmatch(shared_state->name.c_str(), si.path.c_str(), 0)) {
|
|
|
+ outputs.push_back(si);
|
|
|
+ }
|
|
|
+ //Skip if not directory
|
|
|
+ if(si.file_type != StatInfo::IS_DIR) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ //Checking for a match with the path at the current depth
|
|
|
+ if(!fnmatch(shared_state->dirs[operational_state->depth + 1].c_str(), si.path.c_str(), 0)){
|
|
|
+ //Launch a new requests for every matched directory
|
|
|
+ shared_state->outstanding_requests++;
|
|
|
+ auto callback = [this, si, operational_state, shared_state](const Status &stat, const std::vector<StatInfo> & stat_infos, bool has_more) {
|
|
|
+ std::shared_ptr<FindOperationalState> new_current_state = std::make_shared<FindOperationalState>(si.full_path, operational_state->depth + 1, true); //true because searching for the path
|
|
|
+ FindShim(stat, stat_infos, has_more, new_current_state, shared_state);
|
|
|
+ };
|
|
|
+ nn_.GetListing(si.full_path, callback);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ else if(shared_state->maxdepth > operational_state->depth - shared_state->dirs.size() + 1){
|
|
|
+ //We are searching for the name now and maxdepth has not been reached
|
|
|
+ for (StatInfo const& si : stat_infos) {
|
|
|
+ //Launch a new request for every directory
|
|
|
+ if(si.file_type == StatInfo::IS_DIR) {
|
|
|
+ shared_state->outstanding_requests++;
|
|
|
+ auto callback = [this, si, operational_state, shared_state](const Status &stat, const std::vector<StatInfo> & stat_infos, bool has_more) {
|
|
|
+ std::shared_ptr<FindOperationalState> new_current_state = std::make_shared<FindOperationalState>(si.full_path, operational_state->depth + 1, false); //false because searching for the name
|
|
|
+ FindShim(stat, stat_infos, has_more, new_current_state, shared_state);
|
|
|
+ };
|
|
|
+ nn_.GetListing(si.full_path, callback);
|
|
|
+ }
|
|
|
+ //All names that match the specified name are saved to outputs
|
|
|
+ if(!fnmatch(shared_state->name.c_str(), si.path.c_str(), 0)){
|
|
|
+ outputs.push_back(si);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ //This section needs a lock to make sure we return the final chunk only once
|
|
|
+ //and no results are sent after aborted is set
|
|
|
+ std::lock_guard<std::mutex> find_lock(shared_state->lock);
|
|
|
+ //Decrement the counter once since we are done with this chunk
|
|
|
+ shared_state->outstanding_requests--;
|
|
|
+ if(shared_state->outstanding_requests == 0){
|
|
|
+ //Send the outputs back to the user and notify that this is the final chunk
|
|
|
+ shared_state->handler(stat, outputs, false);
|
|
|
+ } else {
|
|
|
+ //There will be more results and we are not aborting
|
|
|
+ if (outputs.size() > 0 && !shared_state->aborted){
|
|
|
+ //Send the outputs back to the user and notify that there is more
|
|
|
+ bool user_wants_more = shared_state->handler(stat, outputs, true);
|
|
|
+ if(!user_wants_more) {
|
|
|
+ //Abort if user doesn't want more
|
|
|
+ shared_state->aborted = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
+void FileSystemImpl::Find(
|
|
|
+ const std::string &path, const std::string &name, const uint32_t maxdepth,
|
|
|
+ const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> &handler) {
|
|
|
+ LOG_INFO(kFileSystem, << "FileSystemImpl::Find("
|
|
|
+ << FMT_THIS_ADDR << ", path="
|
|
|
+ << path << ", name="
|
|
|
+ << name << ") called");
|
|
|
+
|
|
|
+ //Populating the operational state, which includes:
|
|
|
+ //current search path, depth within the path, and the indication that we are currently searching for a path (not name yet).
|
|
|
+ std::shared_ptr<FindOperationalState> operational_state = std::make_shared<FindOperationalState>(path, 0, true);
|
|
|
+ //Populating the shared state, which includes:
|
|
|
+ //vector of sub-directories constructed from path, name to search, handler to use for result returning, outstanding_requests counter, and aborted flag.
|
|
|
+ std::shared_ptr<FindSharedState> shared_state = std::make_shared<FindSharedState>(path, name, maxdepth, handler, 1, false);
|
|
|
+ auto callback = [this, operational_state, shared_state](const Status &stat, const std::vector<StatInfo> & stat_infos, bool directory_has_more) {
|
|
|
+ FindShim(stat, stat_infos, directory_has_more, operational_state, shared_state);
|
|
|
+ };
|
|
|
+ nn_.GetListing("/", callback);
|
|
|
+}
|
|
|
+
|
|
|
+Status FileSystemImpl::Find(const std::string &path, const std::string &name, const uint32_t maxdepth, std::vector<StatInfo> * stat_infos) {
|
|
|
+ LOG_INFO(kFileSystem, << "FileSystemImpl::[sync]Find("
|
|
|
+ << FMT_THIS_ADDR << ", path="
|
|
|
+ << path << ", name="
|
|
|
+ << name << ") called");
|
|
|
+
|
|
|
+ if (!stat_infos) {
|
|
|
+ return Status::InvalidArgument("FileSystemImpl::Find: argument 'stat_infos' cannot be NULL");
|
|
|
+ }
|
|
|
+
|
|
|
+ // In this case, we're going to have the async code populate stat_infos.
|
|
|
+
|
|
|
+ std::promise<void> promise = std::promise<void>();
|
|
|
+ std::future<void> future(promise.get_future());
|
|
|
+ Status status = Status::OK();
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Keep requesting more until we get the entire listing. Set the promise
|
|
|
+ * when we have the entire listing to stop.
|
|
|
+ *
|
|
|
+ * Find guarantees that the handler will only be called once at a time,
|
|
|
+ * so we do not need any locking here
|
|
|
+ */
|
|
|
+ auto h = [&status, &promise, stat_infos](const Status &s, const std::vector<StatInfo> & si, bool has_more_results) -> bool {
|
|
|
+ if (!si.empty()) {
|
|
|
+ stat_infos->insert(stat_infos->end(), si.begin(), si.end());
|
|
|
+ }
|
|
|
+ if (!s.ok() && status.ok()){
|
|
|
+ //We make sure we set 'status' only on the first error.
|
|
|
+ status = s;
|
|
|
+ }
|
|
|
+ if (!has_more_results) {
|
|
|
+ promise.set_value();
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ return true;
|
|
|
+ };
|
|
|
+
|
|
|
+ Find(path, name, maxdepth, h);
|
|
|
+
|
|
|
+ /* block until promise is set */
|
|
|
+ future.get();
|
|
|
+ return status;
|
|
|
+}
|
|
|
|
|
|
void FileSystemImpl::CreateSnapshot(const std::string &path,
|
|
|
const std::string &name,
|