|
@@ -79,16 +79,16 @@ const std::string get_effective_user_name(const std::string &user_name) {
|
|
|
return "unknown_user";
|
|
|
}
|
|
|
|
|
|
-FileSystemImpl::FileSystemImpl(IoService *&io_service, const std::string &user_name,
|
|
|
- const Options &options)
|
|
|
- : options_(options),
|
|
|
- io_service_(static_cast<IoServiceImpl *>(io_service)),
|
|
|
- nn_(&io_service_->io_service(), options,
|
|
|
- GetRandomClientName(), get_effective_user_name(user_name), kNamenodeProtocol,
|
|
|
- kNamenodeProtocolVersion), client_name_(GetRandomClientName()),
|
|
|
- bad_node_tracker_(std::make_shared<BadDataNodeTracker>()),
|
|
|
- event_handlers_(std::make_shared<LibhdfsEvents>())
|
|
|
-{
|
|
|
+FileSystemImpl::FileSystemImpl(IoService *&io_service, const std::string &user_name, const Options &options) :
|
|
|
+ options_(options), client_name_(GetRandomClientName()), io_service_(
|
|
|
+ static_cast<IoServiceImpl *>(io_service)),
|
|
|
+ nn_(
|
|
|
+ &io_service_->io_service(), options, client_name_,
|
|
|
+ get_effective_user_name(user_name), kNamenodeProtocol,
|
|
|
+ kNamenodeProtocolVersion
|
|
|
+ ), bad_node_tracker_(std::make_shared<BadDataNodeTracker>()),
|
|
|
+ event_handlers_(std::make_shared<LibhdfsEvents>()) {
|
|
|
+
|
|
|
LOG_TRACE(kFileSystem, << "FileSystemImpl::FileSystemImpl("
|
|
|
<< FMT_THIS_ADDR << ") called");
|
|
|
|
|
@@ -396,6 +396,43 @@ Status FileSystemImpl::GetFileInfo(const std::string &path,
|
|
|
return stat;
|
|
|
}
|
|
|
|
|
|
+void FileSystemImpl::GetFsStats(
|
|
|
+ const std::function<void(const Status &, const FsInfo &)> &handler) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::GetFsStats(" << FMT_THIS_ADDR << ") called");
|
|
|
+
|
|
|
+ nn_.GetFsStats([handler](const Status &stat, const FsInfo &fs_info) {
|
|
|
+ handler(stat, fs_info);
|
|
|
+ });
|
|
|
+}
|
|
|
+
|
|
|
+Status FileSystemImpl::GetFsStats(FsInfo & fs_info) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::[sync]GetFsStats(" << FMT_THIS_ADDR << ") called");
|
|
|
+
|
|
|
+ auto callstate = std::make_shared<std::promise<std::tuple<Status, FsInfo>>>();
|
|
|
+ std::future<std::tuple<Status, FsInfo>> future(callstate->get_future());
|
|
|
+
|
|
|
+ /* wrap async FileSystem::GetFsStats with promise to make it a blocking call */
|
|
|
+ auto h = [callstate](const Status &s, const FsInfo &si) {
|
|
|
+ callstate->set_value(std::make_tuple(s, si));
|
|
|
+ };
|
|
|
+
|
|
|
+ GetFsStats(h);
|
|
|
+
|
|
|
+ /* block until promise is set */
|
|
|
+ auto returnstate = future.get();
|
|
|
+ Status stat = std::get<0>(returnstate);
|
|
|
+ FsInfo info = std::get<1>(returnstate);
|
|
|
+
|
|
|
+ if (!stat.ok()) {
|
|
|
+ return stat;
|
|
|
+ }
|
|
|
+
|
|
|
+ fs_info = info;
|
|
|
+ return stat;
|
|
|
+}
|
|
|
+
|
|
|
/**
|
|
|
* Helper function for recursive GetListing calls.
|
|
|
*
|
|
@@ -475,6 +512,157 @@ Status FileSystemImpl::GetListing(const std::string &path, std::shared_ptr<std::
|
|
|
return stat;
|
|
|
}
|
|
|
|
|
|
+void FileSystemImpl::CreateSnapshot(const std::string &path,
|
|
|
+ const std::string &name,
|
|
|
+ const std::function<void(const Status &)> &handler) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::CreateSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
|
|
|
+
|
|
|
+ if (path.empty()) {
|
|
|
+ handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ nn_.CreateSnapshot(path, name, [handler](const Status &stat) {
|
|
|
+ handler(stat);
|
|
|
+ });
|
|
|
+}
|
|
|
+
|
|
|
+Status FileSystemImpl::CreateSnapshot(const std::string &path,
|
|
|
+ const std::string &name) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::[sync]CreateSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
|
|
|
+
|
|
|
+ auto callstate = std::make_shared<std::promise<std::tuple<Status>>>();
|
|
|
+ std::future<std::tuple<Status>> future(callstate->get_future());
|
|
|
+
|
|
|
+ /* wrap async FileSystem::CreateSnapshot with promise to make it a blocking call */
|
|
|
+ auto h = [callstate](const Status &s) {
|
|
|
+ callstate->set_value(std::make_tuple(s));
|
|
|
+ };
|
|
|
+
|
|
|
+ CreateSnapshot(path, name, h);
|
|
|
+
|
|
|
+ /* block until promise is set */
|
|
|
+ auto returnstate = future.get();
|
|
|
+ Status stat = std::get<0>(returnstate);
|
|
|
+
|
|
|
+ return stat;
|
|
|
+}
|
|
|
+
|
|
|
+void FileSystemImpl::DeleteSnapshot(const std::string &path,
|
|
|
+ const std::string &name,
|
|
|
+ const std::function<void(const Status &)> &handler) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::DeleteSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
|
|
|
+
|
|
|
+ if (path.empty()) {
|
|
|
+ handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ if (name.empty()) {
|
|
|
+ handler(Status::InvalidArgument("Argument 'name' cannot be empty"));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ nn_.DeleteSnapshot(path, name, [handler](const Status &stat) {
|
|
|
+ handler(stat);
|
|
|
+ });
|
|
|
+}
|
|
|
+
|
|
|
+Status FileSystemImpl::DeleteSnapshot(const std::string &path,
|
|
|
+ const std::string &name) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::[sync]DeleteSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
|
|
|
+
|
|
|
+ auto callstate = std::make_shared<std::promise<std::tuple<Status>>>();
|
|
|
+ std::future<std::tuple<Status>> future(callstate->get_future());
|
|
|
+
|
|
|
+ /* wrap async FileSystem::DeleteSnapshot with promise to make it a blocking call */
|
|
|
+ auto h = [callstate](const Status &s) {
|
|
|
+ callstate->set_value(std::make_tuple(s));
|
|
|
+ };
|
|
|
+
|
|
|
+ DeleteSnapshot(path, name, h);
|
|
|
+
|
|
|
+ /* block until promise is set */
|
|
|
+ auto returnstate = future.get();
|
|
|
+ Status stat = std::get<0>(returnstate);
|
|
|
+
|
|
|
+ return stat;
|
|
|
+}
|
|
|
+
|
|
|
+void FileSystemImpl::AllowSnapshot(const std::string &path,
|
|
|
+ const std::function<void(const Status &)> &handler) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::AllowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
|
|
|
+
|
|
|
+ if (path.empty()) {
|
|
|
+ handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ nn_.AllowSnapshot(path, [handler](const Status &stat) {
|
|
|
+ handler(stat);
|
|
|
+ });
|
|
|
+}
|
|
|
+
|
|
|
+Status FileSystemImpl::AllowSnapshot(const std::string &path) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::[sync]AllowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
|
|
|
+
|
|
|
+ auto callstate = std::make_shared<std::promise<std::tuple<Status>>>();
|
|
|
+ std::future<std::tuple<Status>> future(callstate->get_future());
|
|
|
+
|
|
|
+ /* wrap async FileSystem::AllowSnapshot with promise to make it a blocking call */
|
|
|
+ auto h = [callstate](const Status &s) {
|
|
|
+ callstate->set_value(std::make_tuple(s));
|
|
|
+ };
|
|
|
+
|
|
|
+ AllowSnapshot(path, h);
|
|
|
+
|
|
|
+ /* block until promise is set */
|
|
|
+ auto returnstate = future.get();
|
|
|
+ Status stat = std::get<0>(returnstate);
|
|
|
+
|
|
|
+ return stat;
|
|
|
+}
|
|
|
+
|
|
|
+void FileSystemImpl::DisallowSnapshot(const std::string &path,
|
|
|
+ const std::function<void(const Status &)> &handler) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::DisallowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
|
|
|
+
|
|
|
+ if (path.empty()) {
|
|
|
+ handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ nn_.DisallowSnapshot(path, [handler](const Status &stat) {
|
|
|
+ handler(stat);
|
|
|
+ });
|
|
|
+}
|
|
|
+
|
|
|
+Status FileSystemImpl::DisallowSnapshot(const std::string &path) {
|
|
|
+ LOG_DEBUG(kFileSystem,
|
|
|
+ << "FileSystemImpl::[sync]DisallowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
|
|
|
+
|
|
|
+ auto callstate = std::make_shared<std::promise<std::tuple<Status>>>();
|
|
|
+ std::future<std::tuple<Status>> future(callstate->get_future());
|
|
|
+
|
|
|
+ /* wrap async FileSystem::DisallowSnapshot with promise to make it a blocking call */
|
|
|
+ auto h = [callstate](const Status &s) {
|
|
|
+ callstate->set_value(std::make_tuple(s));
|
|
|
+ };
|
|
|
+
|
|
|
+ DisallowSnapshot(path, h);
|
|
|
+
|
|
|
+ /* block until promise is set */
|
|
|
+ auto returnstate = future.get();
|
|
|
+ Status stat = std::get<0>(returnstate);
|
|
|
+
|
|
|
+ return stat;
|
|
|
+}
|
|
|
|
|
|
void FileSystemImpl::WorkerDeleter::operator()(std::thread *t) {
|
|
|
// It is far too easy to destroy the filesystem (and thus the threadpool)
|