|
@@ -79,11 +79,20 @@ FileSystem *FileSystem::New(
|
|
|
return new FileSystemImpl(io_service, user_name, options);
|
|
|
}
|
|
|
|
|
|
+FileSystem *FileSystem::New(
|
|
|
+ std::shared_ptr<IoService> io_service, const std::string &user_name, const Options &options) {
|
|
|
+ return new FileSystemImpl(io_service, user_name, options);
|
|
|
+}
|
|
|
+
|
|
|
FileSystem *FileSystem::New() {
|
|
|
// No, this pointer won't be leaked. The FileSystem takes ownership.
|
|
|
- IoService *io_service = IoService::New();
|
|
|
+ std::shared_ptr<IoService> io_service = IoService::MakeShared();
|
|
|
if(!io_service)
|
|
|
return nullptr;
|
|
|
+ int thread_count = io_service->InitDefaultWorkers();
|
|
|
+ if(thread_count < 1)
|
|
|
+ return nullptr;
|
|
|
+
|
|
|
std::string user_name = get_effective_user_name("");
|
|
|
Options options;
|
|
|
return new FileSystemImpl(io_service, user_name, options);
|
|
@@ -123,25 +132,56 @@ const std::string get_effective_user_name(const std::string &user_name) {
|
|
|
}
|
|
|
|
|
|
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("
|
|
|
+ io_service_(static_cast<IoServiceImpl *>(io_service)), options_(options),
|
|
|
+ client_name_(GetRandomClientName()),
|
|
|
+ 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_DEBUG(kFileSystem, << "FileSystemImpl::FileSystemImpl("
|
|
|
<< FMT_THIS_ADDR << ") called");
|
|
|
|
|
|
// Poor man's move
|
|
|
io_service = nullptr;
|
|
|
|
|
|
- /* spawn background threads for asio delegation */
|
|
|
- unsigned int threads = 1 /* options.io_threads_, pending HDFS-9117 */;
|
|
|
- for (unsigned int i = 0; i < threads; i++) {
|
|
|
- AddWorkerThread();
|
|
|
+ unsigned int running_workers = 0;
|
|
|
+ if(options.io_threads_ < 1) {
|
|
|
+ LOG_DEBUG(kFileSystem, << "FileSystemImpl::FileSystemImpl Initializing default number of worker threads");
|
|
|
+ running_workers = io_service_->InitDefaultWorkers();
|
|
|
+ } else {
|
|
|
+ LOG_DEBUG(kFileSystem, << "FileSystemImpl::FileSystenImpl Initializing " << options_.io_threads_ << " worker threads.");
|
|
|
+ running_workers = io_service->InitWorkers(options_.io_threads_);
|
|
|
+ }
|
|
|
+
|
|
|
+ if(running_workers < 1) {
|
|
|
+ LOG_WARN(kFileSystem, << "FileSystemImpl::FileSystemImpl was unable to start worker threads");
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
+FileSystemImpl::FileSystemImpl(std::shared_ptr<IoService> io_service, const std::string& user_name, const Options &options) :
|
|
|
+ io_service_(std::static_pointer_cast<IoServiceImpl>(io_service)), options_(options),
|
|
|
+ client_name_(GetRandomClientName()),
|
|
|
+ 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_DEBUG(kFileSystem, << "FileSystemImpl::FileSystemImpl("
|
|
|
+ << FMT_THIS_ADDR << ", shared IoService@" << io_service_.get() << ") called");
|
|
|
+ int worker_thread_count = io_service_->get_worker_thread_count();
|
|
|
+ if(worker_thread_count < 1) {
|
|
|
+ LOG_WARN(kFileSystem, << "FileSystemImpl::FileSystemImpl IoService provided doesn't have any worker threads. "
|
|
|
+ << "It needs at least 1 worker to connect to an HDFS cluster.")
|
|
|
+ } else {
|
|
|
+ LOG_DEBUG(kFileSystem, << "FileSystemImpl::FileSystemImpl using " << worker_thread_count << " worker threads.");
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -154,7 +194,6 @@ FileSystemImpl::~FileSystemImpl() {
|
|
|
* Once worker threads are joined and deleted the service can be deleted.
|
|
|
**/
|
|
|
io_service_->Stop();
|
|
|
- worker_threads_.clear();
|
|
|
}
|
|
|
|
|
|
void FileSystemImpl::Connect(const std::string &server,
|
|
@@ -230,12 +269,21 @@ void FileSystemImpl::ConnectToDefaultFs(const std::function<void(const Status &,
|
|
|
int FileSystemImpl::AddWorkerThread() {
|
|
|
LOG_DEBUG(kFileSystem, << "FileSystemImpl::AddWorkerThread("
|
|
|
<< FMT_THIS_ADDR << ") called."
|
|
|
- << " Existing thread count = " << worker_threads_.size());
|
|
|
+ << " Existing thread count = " << WorkerThreadCount());
|
|
|
+
|
|
|
+ if(!io_service_)
|
|
|
+ return -1;
|
|
|
|
|
|
- auto service_task = [](IoService *service) { service->Run(); };
|
|
|
- worker_threads_.push_back(
|
|
|
- WorkerPtr(new std::thread(service_task, io_service_.get())));
|
|
|
- return worker_threads_.size();
|
|
|
+ io_service_->AddWorkerThread();
|
|
|
+ return 1;
|
|
|
+}
|
|
|
+
|
|
|
+int FileSystemImpl::WorkerThreadCount() {
|
|
|
+ if(!io_service_) {
|
|
|
+ return -1;
|
|
|
+ } else {
|
|
|
+ return io_service_->get_worker_thread_count();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
void FileSystemImpl::Open(
|
|
@@ -714,21 +762,6 @@ void FileSystemImpl::DisallowSnapshot(const std::string &path,
|
|
|
nn_.DisallowSnapshot(path, handler);
|
|
|
}
|
|
|
|
|
|
-
|
|
|
-void FileSystemImpl::WorkerDeleter::operator()(std::thread *t) {
|
|
|
- // It is far too easy to destroy the filesystem (and thus the threadpool)
|
|
|
- // from within one of the worker threads, leading to a deadlock. Let's
|
|
|
- // provide some explicit protection.
|
|
|
- if(t->get_id() == std::this_thread::get_id()) {
|
|
|
- LOG_ERROR(kFileSystem, << "FileSystemImpl::WorkerDeleter::operator(treadptr="
|
|
|
- << t << ") : FATAL: Attempted to destroy a thread pool"
|
|
|
- "from within a callback of the thread pool!");
|
|
|
- }
|
|
|
- t->join();
|
|
|
- delete t;
|
|
|
-}
|
|
|
-
|
|
|
-
|
|
|
void FileSystemImpl::SetFsEventCallback(fs_event_callback callback) {
|
|
|
if (event_handlers_) {
|
|
|
event_handlers_->set_fs_callback(callback);
|