From 1fb1ea527c9b5321e6da6c2543859db2ec3eaf7c Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 22 Oct 2015 07:58:41 -0400 Subject: [PATCH 01/50] Refactored NameNodeConnection --- .../native/libhdfspp/lib/fs/filesystem.cc | 45 +++++++++++-------- .../main/native/libhdfspp/lib/fs/filesystem.h | 34 ++++++++++---- .../native/libhdfspp/lib/fs/inputstream.cc | 8 ++-- .../libhdfspp/lib/fs/inputstream_impl.h | 7 ++- .../libhdfspp/tests/inputstream_test.cc | 14 +++--- 5 files changed, 67 insertions(+), 41 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 0b958a828e9b6..b7b81de35a9ed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -32,6 +32,26 @@ static const int kNamenodeProtocolVersion = 1; using ::asio::ip::tcp; +void NameNodeConnection::Connect(const std::string &server, + const std::string &service, + std::function &handler) { + using namespace continuation; + typedef std::vector State; + auto m = Pipeline::Create(); + m->Push(Resolve(io_service_, server, service, + std::back_inserter(m->state()))) + .Push(Bind([this, m](const Continuation::Next &next) { + engine_.Connect(m->state().front(), next); + })); + m->Run([this, handler](const Status &status, const State &) { + if (status.ok()) { + engine_.Start(); + } + handler(status); + }); +} + + FileSystem::~FileSystem() {} void FileSystem::New( @@ -51,28 +71,14 @@ void FileSystem::New( FileSystemImpl::FileSystemImpl(IoService *io_service, const Options &options) : io_service_(static_cast(io_service)), - engine_(&io_service_->io_service(), options, + nn_(&io_service_->io_service(), options, RpcEngine::GetRandomClientName(), kNamenodeProtocol, - kNamenodeProtocolVersion), - namenode_(&engine_) {} + kNamenodeProtocolVersion) {} void FileSystemImpl::Connect(const std::string &server, const std::string &service, std::function &&handler) { - using namespace continuation; - typedef std::vector State; - auto m = Pipeline::Create(); - m->Push(Resolve(&io_service_->io_service(), server, service, - std::back_inserter(m->state()))) - .Push(Bind([this, m](const Continuation::Next &next) { - engine_.Connect(m->state().front(), next); - })); - m->Run([this, handler](const Status &status, const State &) { - if (status.ok()) { - engine_.Start(); - } - handler(status); - }); + nn_.Connect(server, service, handler); } void FileSystemImpl::Open( @@ -96,10 +102,11 @@ void FileSystemImpl::Open( State *s = &m->state(); m->Push(continuation::Bind( [this, s](const continuation::Continuation::Next &next) { - namenode_.GetBlockLocations(&s->req, s->resp, next); + nn_.namenode_.GetBlockLocations(&s->req, s->resp, next); })); + //TODO: Put client name et. al. into "ClusterInfo" object m->Run([this, handler](const Status &stat, const State &s) { - handler(stat, stat.ok() ? new InputStreamImpl(this, &s.resp->locations()) + handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), nn_.engine_.client_name(), this, &s.resp->locations()) : nullptr); }); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 72f80b7b3456d..c65c4647796b4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -26,6 +26,24 @@ namespace hdfs { +class NameNodeConnection { +public: + NameNodeConnection(::asio::io_service *io_service, const Options &options, + const std::string &client_name, const char *protocol_name, + int protocol_version) : + io_service_(io_service), + engine_(io_service, options, client_name, protocol_name, protocol_version), + namenode_(& engine_) {} + + void Connect(const std::string &server, + const std::string &service, + std::function &handler); + + ::asio::io_service * io_service_; + RpcEngine engine_; + ClientNamenodeProtocol namenode_; +}; + class FileSystemImpl : public FileSystem { public: FileSystemImpl(IoService *io_service, const Options &options); @@ -34,18 +52,17 @@ class FileSystemImpl : public FileSystem { virtual void Open(const std::string &path, const std::function &handler) override; - RpcEngine &rpc_engine() { return engine_; } +// RpcEngine &rpc_engine() { return engine_; } private: IoServiceImpl *io_service_; - RpcEngine engine_; - ClientNamenodeProtocol namenode_; + NameNodeConnection nn_; }; class InputStreamImpl : public InputStream { public: - InputStreamImpl(FileSystemImpl *fs, - const ::hadoop::hdfs::LocatedBlocksProto *blocks); + InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, + FileSystemImpl *fs, const ::hadoop::hdfs::LocatedBlocksProto *blocks); virtual void PositionRead(void *buf, size_t nbyte, uint64_t offset, const std::set &excluded_datanodes, @@ -56,14 +73,15 @@ class InputStreamImpl : public InputStream { const std::set &excluded_datanodes, const Handler &handler); template - void AsyncReadBlock(const std::string &client_name, - const hadoop::hdfs::LocatedBlockProto &block, + void AsyncReadBlock(const hadoop::hdfs::LocatedBlockProto &block, const hadoop::hdfs::DatanodeInfoProto &dn, size_t offset, const MutableBufferSequence &buffers, const Handler &handler); private: - FileSystemImpl *fs_; + ::asio::io_service *io_service_; + const std::string client_name_; + FileSystemImpl *fs_; //TODO: get rid of this? unsigned long long file_length_; std::vector<::hadoop::hdfs::LocatedBlockProto> blocks_; template struct HandshakeContinuation; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index b47dcb1a8fad2..f5f191347ebfe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -24,9 +24,11 @@ using ::hadoop::hdfs::LocatedBlocksProto; InputStream::~InputStream() {} -InputStreamImpl::InputStreamImpl(FileSystemImpl *fs, - const LocatedBlocksProto *blocks) - : fs_(fs), file_length_(blocks->filelength()) { +InputStreamImpl::InputStreamImpl(::asio::io_service *io_service, + const std::string &client_name, + FileSystemImpl *fs, + const ::hadoop::hdfs::LocatedBlocksProto *blocks) + : io_service_(io_service), client_name_(client_name), fs_(fs), file_length_(blocks->filelength()) { for (const auto &block : blocks->blocks()) { blocks_.push_back(block); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 0d3b3027bb862..1b28bb0d914b6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -162,23 +162,22 @@ void InputStreamImpl::AsyncPreadSome( it->b().numbytes() - offset_within_block, asio::buffer_size(buffers)); AsyncReadBlock( - fs_->rpc_engine().client_name(), *it, *chosen_dn, offset_within_block, + *it, *chosen_dn, offset_within_block, asio::buffer(buffers, size_within_block), handler); } template void InputStreamImpl::AsyncReadBlock( - const std::string &client_name, const hadoop::hdfs::LocatedBlockProto &block, const hadoop::hdfs::DatanodeInfoProto &dn, size_t offset, const MutableBufferSequence &buffers, const Handler &handler) { typedef typename BlockReaderTrait::Reader Reader; auto m = - BlockReaderTrait::CreatePipeline(&fs_->rpc_engine().io_service(), dn); + BlockReaderTrait::CreatePipeline(io_service_, dn); auto &s = m->state(); size_t size = asio::buffer_size(buffers); - m->Push(new HandshakeContinuation(s.reader(), client_name, nullptr, + m->Push(new HandshakeContinuation(s.reader(), client_name_, nullptr, &block.b(), size, offset)) .Push(new ReadBlockContinuation( s.reader(), buffers, s.transferred())); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index aa95256b7e1fe..5d624043dbe8b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -78,7 +78,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&fs, &blocks); + InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &fs, &blocks); Status stat; size_t read = 0; struct Trait { @@ -92,7 +92,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { }; is.AsyncReadBlock>( - "client", block, dn, 0, asio::buffer(buf, sizeof(buf)), + block, dn, 0, asio::buffer(buf, sizeof(buf)), [&stat, &read](const Status &status, const std::string &, size_t transferred) { stat = status; read = transferred; @@ -112,7 +112,7 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&fs, &blocks); + InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &fs, &blocks); Status stat; size_t read = 0; struct Trait { @@ -127,7 +127,7 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { }; is.AsyncReadBlock>( - "client", block, dn, 0, asio::buffer(buf, sizeof(buf)), + block, dn, 0, asio::buffer(buf, sizeof(buf)), [&stat, &read](const Status &status, const std::string &, size_t transferred) { stat = status; @@ -148,7 +148,7 @@ TEST(InputStreamTest, TestReadError) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&fs, &blocks); + InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &fs, &blocks); Status stat; size_t read = 0; struct Trait { @@ -165,7 +165,7 @@ TEST(InputStreamTest, TestReadError) { }; is.AsyncReadBlock>( - "client", block, dn, 0, asio::buffer(buf, sizeof(buf)), + block, dn, 0, asio::buffer(buf, sizeof(buf)), [&stat, &read](const Status &status, const std::string &, size_t transferred) { stat = status; @@ -195,7 +195,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&fs, &blocks); + InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &fs, &blocks); Status stat; size_t read = 0; struct Trait { From c6cf5175b9c21561bdcbd22be27f50e22a1d3ebd Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 22 Oct 2015 08:01:36 -0400 Subject: [PATCH 02/50] Removed fs_ from InputStream --- .../src/main/native/libhdfspp/lib/fs/filesystem.cc | 2 +- .../src/main/native/libhdfspp/lib/fs/filesystem.h | 3 +-- .../src/main/native/libhdfspp/lib/fs/inputstream.cc | 3 +-- .../src/main/native/libhdfspp/tests/inputstream_test.cc | 8 ++++---- 4 files changed, 7 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index b7b81de35a9ed..980f97362c8ae 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -106,7 +106,7 @@ void FileSystemImpl::Open( })); //TODO: Put client name et. al. into "ClusterInfo" object m->Run([this, handler](const Status &stat, const State &s) { - handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), nn_.engine_.client_name(), this, &s.resp->locations()) + handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), nn_.engine_.client_name(), &s.resp->locations()) : nullptr); }); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index c65c4647796b4..4ebbd7d3ee107 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -62,7 +62,7 @@ class FileSystemImpl : public FileSystem { class InputStreamImpl : public InputStream { public: InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, - FileSystemImpl *fs, const ::hadoop::hdfs::LocatedBlocksProto *blocks); + const ::hadoop::hdfs::LocatedBlocksProto *blocks); virtual void PositionRead(void *buf, size_t nbyte, uint64_t offset, const std::set &excluded_datanodes, @@ -81,7 +81,6 @@ class InputStreamImpl : public InputStream { private: ::asio::io_service *io_service_; const std::string client_name_; - FileSystemImpl *fs_; //TODO: get rid of this? unsigned long long file_length_; std::vector<::hadoop::hdfs::LocatedBlockProto> blocks_; template struct HandshakeContinuation; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index f5f191347ebfe..28b31f51972ca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -26,9 +26,8 @@ InputStream::~InputStream() {} InputStreamImpl::InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, - FileSystemImpl *fs, const ::hadoop::hdfs::LocatedBlocksProto *blocks) - : io_service_(io_service), client_name_(client_name), fs_(fs), file_length_(blocks->filelength()) { + : io_service_(io_service), client_name_(client_name), file_length_(blocks->filelength()) { for (const auto &block : blocks->blocks()) { blocks_.push_back(block); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 5d624043dbe8b..87339bf9b406c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -78,7 +78,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &fs, &blocks); + InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); Status stat; size_t read = 0; struct Trait { @@ -112,7 +112,7 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &fs, &blocks); + InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); Status stat; size_t read = 0; struct Trait { @@ -148,7 +148,7 @@ TEST(InputStreamTest, TestReadError) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &fs, &blocks); + InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); Status stat; size_t read = 0; struct Trait { @@ -195,7 +195,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &fs, &blocks); + InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); Status stat; size_t read = 0; struct Trait { From 8b8190d334224d8acec9a4bef97d5e0226c1045a Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 22 Oct 2015 09:05:53 -0400 Subject: [PATCH 03/50] Moved GetBlockInfo to NN connection --- .../native/libhdfspp/lib/fs/filesystem.cc | 34 +++++-------------- .../main/native/libhdfspp/lib/fs/filesystem.h | 11 ++++-- 2 files changed, 17 insertions(+), 28 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 980f97362c8ae..95039daeb03a1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -51,7 +51,6 @@ void NameNodeConnection::Connect(const std::string &server, }); } - FileSystem::~FileSystem() {} void FileSystem::New( @@ -71,6 +70,7 @@ void FileSystem::New( FileSystemImpl::FileSystemImpl(IoService *io_service, const Options &options) : io_service_(static_cast(io_service)), + client_name_(RpcEngine::GetRandomClientName()), nn_(&io_service_->io_service(), options, RpcEngine::GetRandomClientName(), kNamenodeProtocol, kNamenodeProtocolVersion) {} @@ -84,30 +84,14 @@ void FileSystemImpl::Connect(const std::string &server, void FileSystemImpl::Open( const std::string &path, const std::function &handler) { - using ::hadoop::hdfs::GetBlockLocationsRequestProto; - using ::hadoop::hdfs::GetBlockLocationsResponseProto; - - struct State { - GetBlockLocationsRequestProto req; - std::shared_ptr resp; - }; - - auto m = continuation::Pipeline::Create(); - auto &req = m->state().req; - req.set_src(path); - req.set_offset(0); - req.set_length(std::numeric_limits::max()); - m->state().resp.reset(new GetBlockLocationsResponseProto()); - - State *s = &m->state(); - m->Push(continuation::Bind( - [this, s](const continuation::Continuation::Next &next) { - nn_.namenode_.GetBlockLocations(&s->req, s->resp, next); - })); - //TODO: Put client name et. al. into "ClusterInfo" object - m->Run([this, handler](const Status &stat, const State &s) { - handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), nn_.engine_.client_name(), &s.resp->locations()) - : nullptr); + + auto foo = [this, handler](const Status &stat, const ::hadoop::hdfs::LocatedBlocksProto* locations){}; + nn_.GetBlockLocations(path, foo); + + + nn_.GetBlockLocations(path, [this, handler](const Status &stat, const ::hadoop::hdfs::LocatedBlocksProto* locations) { +// handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), client_name_, locations) +// : nullptr); }); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 4ebbd7d3ee107..d15b7c8458e6c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -38,7 +38,12 @@ class NameNodeConnection { void Connect(const std::string &server, const std::string &service, std::function &handler); - + + // GetBlockLocations: + // handler: void (const Status &stat, const ::hadoop::hdfs::LocatedBlocksProto* locations) + template + void GetBlockLocations(const std::string & path, Handler handler); +private: ::asio::io_service * io_service_; RpcEngine engine_; ClientNamenodeProtocol namenode_; @@ -52,10 +57,9 @@ class FileSystemImpl : public FileSystem { virtual void Open(const std::string &path, const std::function &handler) override; -// RpcEngine &rpc_engine() { return engine_; } - private: IoServiceImpl *io_service_; + const std::string client_name_; NameNodeConnection nn_; }; @@ -91,5 +95,6 @@ class InputStreamImpl : public InputStream { } #include "inputstream_impl.h" +#include "namenodeconnection_impl.h" #endif From 108b54f3079ed21149a59b9222d6d9832ee05d79 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 22 Oct 2015 09:20:56 -0400 Subject: [PATCH 04/50] Moved GetBlockLocations to std::function --- .../native/libhdfspp/lib/fs/filesystem.cc | 40 ++++++++++++++++--- .../main/native/libhdfspp/lib/fs/filesystem.h | 8 +--- 2 files changed, 36 insertions(+), 12 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 95039daeb03a1..f6dace1ada3e1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -51,6 +51,38 @@ void NameNodeConnection::Connect(const std::string &server, }); } +void NameNodeConnection::GetBlockLocations(const std::string & path, + std::function handler) +{ + using ::hadoop::hdfs::GetBlockLocationsRequestProto; + using ::hadoop::hdfs::GetBlockLocationsResponseProto; + + struct State { + GetBlockLocationsRequestProto req; + std::shared_ptr resp; + }; + + auto m = continuation::Pipeline::Create(); + auto &req = m->state().req; + req.set_src(path); + req.set_offset(0); + req.set_length(std::numeric_limits::max()); + m->state().resp.reset(new GetBlockLocationsResponseProto()); + + State *s = &m->state(); + m->Push(continuation::Bind( + [this, s](const continuation::Continuation::Next &next) { + namenode_.GetBlockLocations(&s->req, s->resp, next); + })); + //TODO-BTH: Put client name et. al. into "ClusterInfo" object + m->Run([this, handler](const Status &stat, const State &s) { + handler(stat, stat.ok() ? &s.resp->locations() + : nullptr); + }); +} + + + FileSystem::~FileSystem() {} void FileSystem::New( @@ -85,13 +117,9 @@ void FileSystemImpl::Open( const std::string &path, const std::function &handler) { - auto foo = [this, handler](const Status &stat, const ::hadoop::hdfs::LocatedBlocksProto* locations){}; - nn_.GetBlockLocations(path, foo); - - nn_.GetBlockLocations(path, [this, handler](const Status &stat, const ::hadoop::hdfs::LocatedBlocksProto* locations) { -// handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), client_name_, locations) -// : nullptr); + handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), client_name_, locations) + : nullptr); }); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index d15b7c8458e6c..aac78e2bc0cab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -38,11 +38,8 @@ class NameNodeConnection { void Connect(const std::string &server, const std::string &service, std::function &handler); - - // GetBlockLocations: - // handler: void (const Status &stat, const ::hadoop::hdfs::LocatedBlocksProto* locations) - template - void GetBlockLocations(const std::string & path, Handler handler); + + void GetBlockLocations(const std::string & path, std::function handler); private: ::asio::io_service * io_service_; RpcEngine engine_; @@ -95,6 +92,5 @@ class InputStreamImpl : public InputStream { } #include "inputstream_impl.h" -#include "namenodeconnection_impl.h" #endif From 6d112a17048bcec437701b422209641e56f6196e Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 22 Oct 2015 09:48:02 -0400 Subject: [PATCH 05/50] Added comments --- .../main/native/libhdfspp/lib/fs/filesystem.h | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index aac78e2bc0cab..ced4c2a240cf1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -26,6 +26,14 @@ namespace hdfs { +/** + * NameNodeConnection: abstracts the details of communicating with a NameNode + * and the implementation of the communications protocol. + * + * Will eventually handle retry and failover. + * + * Threading model: thread-safe; all operations can be called concurrently + */ class NameNodeConnection { public: NameNodeConnection(::asio::io_service *io_service, const Options &options, @@ -45,7 +53,18 @@ class NameNodeConnection { RpcEngine engine_; ClientNamenodeProtocol namenode_; }; - + +/* + * FileSystem: The consumer's main point of interaction with the cluster as + * a whole. + * + * Initially constructed in a disconnected state; call Connect before operating + * on the FileSystem. + * + * All open files must be closed before the FileSystem is destroyed. + * + * Threading model: thread-safe for all operations + */ class FileSystemImpl : public FileSystem { public: FileSystemImpl(IoService *io_service, const Options &options); @@ -60,6 +79,14 @@ class FileSystemImpl : public FileSystem { NameNodeConnection nn_; }; +/* + * InputStreamImpl: connects to DNs, does one-shot reads. + * + * This will eventually be split into DNConnection and ReadOperation classes + * + * Threading model: not thread-safe; consumers and io_service should not call + * concurrently + */ class InputStreamImpl : public InputStream { public: InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, From e57b0ed02e29781f347499f0f3546659870aabab Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 22 Oct 2015 09:52:39 -0400 Subject: [PATCH 06/50] Stripped whitespace --- .../native/libhdfspp/lib/fs/filesystem.cc | 10 +++++----- .../main/native/libhdfspp/lib/fs/filesystem.h | 20 +++++++++---------- .../native/libhdfspp/lib/fs/inputstream.cc | 4 ++-- .../tests/remote_block_reader_test.cc | 2 +- 4 files changed, 18 insertions(+), 18 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index f6dace1ada3e1..32b947855bab5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -51,7 +51,7 @@ void NameNodeConnection::Connect(const std::string &server, }); } -void NameNodeConnection::GetBlockLocations(const std::string & path, +void NameNodeConnection::GetBlockLocations(const std::string & path, std::function handler) { using ::hadoop::hdfs::GetBlockLocationsRequestProto; @@ -60,8 +60,8 @@ void NameNodeConnection::GetBlockLocations(const std::string & path, struct State { GetBlockLocationsRequestProto req; std::shared_ptr resp; - }; - + }; + auto m = continuation::Pipeline::Create(); auto &req = m->state().req; req.set_src(path); @@ -82,7 +82,7 @@ void NameNodeConnection::GetBlockLocations(const std::string & path, } - + FileSystem::~FileSystem() {} void FileSystem::New( @@ -116,7 +116,7 @@ void FileSystemImpl::Connect(const std::string &server, void FileSystemImpl::Open( const std::string &path, const std::function &handler) { - + nn_.GetBlockLocations(path, [this, handler](const Status &stat, const ::hadoop::hdfs::LocatedBlocksProto* locations) { handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), client_name_, locations) : nullptr); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index ced4c2a240cf1..4f2d52edf08d9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -29,9 +29,9 @@ namespace hdfs { /** * NameNodeConnection: abstracts the details of communicating with a NameNode * and the implementation of the communications protocol. - * + * * Will eventually handle retry and failover. - * + * * Threading model: thread-safe; all operations can be called concurrently */ class NameNodeConnection { @@ -53,16 +53,16 @@ class NameNodeConnection { RpcEngine engine_; ClientNamenodeProtocol namenode_; }; - + /* * FileSystem: The consumer's main point of interaction with the cluster as - * a whole. - * + * a whole. + * * Initially constructed in a disconnected state; call Connect before operating * on the FileSystem. - * + * * All open files must be closed before the FileSystem is destroyed. - * + * * Threading model: thread-safe for all operations */ class FileSystemImpl : public FileSystem { @@ -81,15 +81,15 @@ class FileSystemImpl : public FileSystem { /* * InputStreamImpl: connects to DNs, does one-shot reads. - * + * * This will eventually be split into DNConnection and ReadOperation classes - * + * * Threading model: not thread-safe; consumers and io_service should not call * concurrently */ class InputStreamImpl : public InputStream { public: - InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, + InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, const ::hadoop::hdfs::LocatedBlocksProto *blocks); virtual void PositionRead(void *buf, size_t nbyte, uint64_t offset, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index 28b31f51972ca..90e17ba361fb7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -24,8 +24,8 @@ using ::hadoop::hdfs::LocatedBlocksProto; InputStream::~InputStream() {} -InputStreamImpl::InputStreamImpl(::asio::io_service *io_service, - const std::string &client_name, +InputStreamImpl::InputStreamImpl(::asio::io_service *io_service, + const std::string &client_name, const ::hadoop::hdfs::LocatedBlocksProto *blocks) : io_service_(io_service), client_name_(client_name), file_length_(blocks->filelength()) { for (const auto &block : blocks->blocks()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 6ae657ce91d97..df4f506c48af7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -128,7 +128,7 @@ TEST(RemoteBlockReaderTest, TestReadWholeBlock) { block.set_poolid("foo"); block.set_blockid(0); block.set_generationstamp(0); - + std::string data(kChunkSize, 0); ReadContent(&conn, nullptr, block, kChunkSize, 0, buffer(const_cast(data.c_str()), data.size()), From c9c82125e8c0b742ee3a70d6fdbdedca180cdd4f Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 27 Oct 2015 12:07:33 -0400 Subject: [PATCH 07/50] Renamed NameNodeConnection to NameNodeOperations --- .../src/main/native/libhdfspp/lib/fs/filesystem.cc | 4 ++-- .../src/main/native/libhdfspp/lib/fs/filesystem.h | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 32b947855bab5..a8572bbf73f41 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -32,7 +32,7 @@ static const int kNamenodeProtocolVersion = 1; using ::asio::ip::tcp; -void NameNodeConnection::Connect(const std::string &server, +void NameNodeOperations::Connect(const std::string &server, const std::string &service, std::function &handler) { using namespace continuation; @@ -51,7 +51,7 @@ void NameNodeConnection::Connect(const std::string &server, }); } -void NameNodeConnection::GetBlockLocations(const std::string & path, +void NameNodeOperations::GetBlockLocations(const std::string & path, std::function handler) { using ::hadoop::hdfs::GetBlockLocationsRequestProto; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 4f2d52edf08d9..89bd512c0142f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -34,9 +34,9 @@ namespace hdfs { * * Threading model: thread-safe; all operations can be called concurrently */ -class NameNodeConnection { +class NameNodeOperations { public: - NameNodeConnection(::asio::io_service *io_service, const Options &options, + NameNodeOperations(::asio::io_service *io_service, const Options &options, const std::string &client_name, const char *protocol_name, int protocol_version) : io_service_(io_service), @@ -76,7 +76,7 @@ class FileSystemImpl : public FileSystem { private: IoServiceImpl *io_service_; const std::string client_name_; - NameNodeConnection nn_; + NameNodeOperations nn_; }; /* From 01499b6027ec771ebf04d4723899ee976b2a6044 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 27 Oct 2015 19:26:26 -0400 Subject: [PATCH 08/50] Renamed input_stream and asio_continuation --- .../libhdfspp/lib/common/continuation/asio.h | 4 +++- .../libhdfspp/lib/common/continuation/protobuf.h | 2 +- .../main/native/libhdfspp/lib/fs/filesystem.cc | 4 ++-- .../src/main/native/libhdfspp/lib/fs/filesystem.h | 10 +++++----- .../main/native/libhdfspp/lib/fs/inputstream.cc | 4 ++-- .../native/libhdfspp/lib/fs/inputstream_impl.h | 15 ++++++++------- .../libhdfspp/lib/reader/datatransfer_impl.h | 4 ++-- .../lib/reader/remote_block_reader_impl.h | 4 ++-- .../native/libhdfspp/tests/inputstream_test.cc | 8 ++++---- 9 files changed, 29 insertions(+), 26 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h index 5630934ed0105..cf1bdcbd52a99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h @@ -29,7 +29,9 @@ #include namespace hdfs { -namespace continuation { +namespace asio_continuation { + +using namespace continuation; template class ReadContinuation : public Continuation { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h index 08caf0ddbf78f..49a8330b58e54 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h @@ -102,7 +102,7 @@ struct WriteDelimitedPBMessageContinuation : Continuation { os.WriteVarint32(size); msg_->SerializeToCodedStream(&os); write_coroutine_ = - std::shared_ptr(Write(stream_, asio::buffer(buf_))); + std::shared_ptr(asio_continuation::Write(stream_, asio::buffer(buf_))); write_coroutine_->Run([next](const Status &stat) { next(stat); }); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index a8572bbf73f41..059e15e472a51 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -35,7 +35,7 @@ using ::asio::ip::tcp; void NameNodeOperations::Connect(const std::string &server, const std::string &service, std::function &handler) { - using namespace continuation; + using namespace asio_continuation; typedef std::vector State; auto m = Pipeline::Create(); m->Push(Resolve(io_service_, server, service, @@ -118,7 +118,7 @@ void FileSystemImpl::Open( const std::function &handler) { nn_.GetBlockLocations(path, [this, handler](const Status &stat, const ::hadoop::hdfs::LocatedBlocksProto* locations) { - handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), client_name_, locations) + handler(stat, stat.ok() ? new ReadOperation(&io_service_->io_service(), client_name_, locations) : nullptr); }); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 89bd512c0142f..c6cd9031ce35d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -79,17 +79,17 @@ class FileSystemImpl : public FileSystem { NameNodeOperations nn_; }; + + /* - * InputStreamImpl: connects to DNs, does one-shot reads. - * - * This will eventually be split into DNConnection and ReadOperation classes + * ReadOperation: given DN connection, does one-shot reads. * * Threading model: not thread-safe; consumers and io_service should not call * concurrently */ -class InputStreamImpl : public InputStream { +class ReadOperation : public InputStream { public: - InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, + ReadOperation(::asio::io_service *io_service, const std::string &client_name, const ::hadoop::hdfs::LocatedBlocksProto *blocks); virtual void PositionRead(void *buf, size_t nbyte, uint64_t offset, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index 90e17ba361fb7..996da7400acfd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -24,7 +24,7 @@ using ::hadoop::hdfs::LocatedBlocksProto; InputStream::~InputStream() {} -InputStreamImpl::InputStreamImpl(::asio::io_service *io_service, +ReadOperation::ReadOperation(::asio::io_service *io_service, const std::string &client_name, const ::hadoop::hdfs::LocatedBlocksProto *blocks) : io_service_(io_service), client_name_(client_name), file_length_(blocks->filelength()) { @@ -37,7 +37,7 @@ InputStreamImpl::InputStreamImpl(::asio::io_service *io_service, } } -void InputStreamImpl::PositionRead( +void ReadOperation::PositionRead( void *buf, size_t nbyte, uint64_t offset, const std::set &excluded_datanodes, const std::function diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 1b28bb0d914b6..e9fe0a6185fd1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -29,12 +29,13 @@ namespace hdfs { -struct InputStreamImpl::RemoteBlockReaderTrait { +struct ReadOperation::RemoteBlockReaderTrait { typedef RemoteBlockReader Reader; struct State { std::unique_ptr conn_; - std::shared_ptr reader_; std::array endpoints_; + + std::shared_ptr reader_; size_t transferred_; Reader *reader() { return reader_.get(); } size_t *transferred() { return &transferred_; } @@ -52,14 +53,14 @@ struct InputStreamImpl::RemoteBlockReaderTrait { s.endpoints_[0] = tcp::endpoint(address::from_string(datanode.ipaddr()), datanode.xferport()); - m->Push(continuation::Connect(s.conn_.get(), s.endpoints_.begin(), + m->Push(asio_continuation::Connect(s.conn_.get(), s.endpoints_.begin(), s.endpoints_.end())); return m; } }; template -struct InputStreamImpl::HandshakeContinuation : continuation::Continuation { +struct ReadOperation::HandshakeContinuation : continuation::Continuation { HandshakeContinuation(Reader *reader, const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, @@ -88,7 +89,7 @@ struct InputStreamImpl::HandshakeContinuation : continuation::Continuation { }; template -struct InputStreamImpl::ReadBlockContinuation : continuation::Continuation { +struct ReadOperation::ReadBlockContinuation : continuation::Continuation { ReadBlockContinuation(Reader *reader, MutableBufferSequence buffer, size_t *transferred) : reader_(reader), buffer_(buffer), @@ -127,7 +128,7 @@ struct InputStreamImpl::ReadBlockContinuation : continuation::Continuation { }; template -void InputStreamImpl::AsyncPreadSome( +void ReadOperation::AsyncPreadSome( size_t offset, const MutableBufferSequence &buffers, const std::set &excluded_datanodes, const Handler &handler) { using ::hadoop::hdfs::DatanodeInfoProto; @@ -167,7 +168,7 @@ void InputStreamImpl::AsyncPreadSome( } template -void InputStreamImpl::AsyncReadBlock( +void ReadOperation::AsyncReadBlock( const hadoop::hdfs::LocatedBlockProto &block, const hadoop::hdfs::DatanodeInfoProto &dn, size_t offset, const MutableBufferSequence &buffers, const Handler &handler) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h index 088b86e5dc6ff..5666f2ee31543 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h @@ -97,8 +97,8 @@ template template void DataTransferSaslStream::Handshake(const Handler &next) { using ::hadoop::hdfs::DataTransferEncryptorMessageProto; - using ::hdfs::continuation::Write; - using ::hdfs::continuation::WriteDelimitedPBMessage; + using ::hdfs::asio_continuation::Write; + using ::hdfs::asio_continuation::WriteDelimitedPBMessage; static const int kMagicNumber = htonl(kDataTransferSasl); static const asio::const_buffers_1 kMagicNumberBuffer = asio::buffer( diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index 35c2ce46ea9e5..8b0ca5318740c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -68,8 +68,8 @@ void RemoteBlockReader::async_connect( new continuation::ReadDelimitedPBMessageContinuation( stream_, &s->response); - m->Push(continuation::Write(stream_, asio::buffer(s->header))) - .Push(continuation::WriteDelimitedPBMessage(stream_, &s->request)) + m->Push(asio_continuation::Write(stream_, asio::buffer(s->header))) + .Push(asio_continuation::WriteDelimitedPBMessage(stream_, &s->request)) .Push(read_pb_message); m->Run([this, handler, offset](const Status &status, const State &s) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 87339bf9b406c..3a212af06c0b5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -78,7 +78,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); + ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); Status stat; size_t read = 0; struct Trait { @@ -112,7 +112,7 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); + ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); Status stat; size_t read = 0; struct Trait { @@ -148,7 +148,7 @@ TEST(InputStreamTest, TestReadError) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); + ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); Status stat; size_t read = 0; struct Trait { @@ -195,7 +195,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); + ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); Status stat; size_t read = 0; struct Trait { From 02c67837fe832e45286a675f1a27fa29e1b80a9a Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 27 Oct 2015 19:30:44 -0400 Subject: [PATCH 09/50] Renamed CreatePipeline to Connect --- .../src/main/native/libhdfspp/lib/fs/inputstream_impl.h | 4 ++-- .../src/main/native/libhdfspp/tests/inputstream_test.cc | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index e9fe0a6185fd1..a5a19778ade9c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -42,7 +42,7 @@ struct ReadOperation::RemoteBlockReaderTrait { const size_t *transferred() const { return &transferred_; } }; static continuation::Pipeline * - CreatePipeline(::asio::io_service *io_service, + Connect(::asio::io_service *io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn) { using namespace ::asio::ip; auto m = continuation::Pipeline::Create(); @@ -175,7 +175,7 @@ void ReadOperation::AsyncReadBlock( typedef typename BlockReaderTrait::Reader Reader; auto m = - BlockReaderTrait::CreatePipeline(io_service_, dn); + BlockReaderTrait::Connect(io_service_, dn); auto &s = m->state(); size_t size = asio::buffer_size(buffers); m->Push(new HandshakeContinuation(s.reader(), client_name_, nullptr, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 3a212af06c0b5..73547e2897a01 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -59,7 +59,7 @@ template struct MockBlockReaderTrait { }; static continuation::Pipeline * - CreatePipeline(::asio::io_service *, const DatanodeInfoProto &) { + Connect(::asio::io_service *, const DatanodeInfoProto &) { auto m = continuation::Pipeline::Create(); *m->state().transferred() = 0; Trait::InitializeMockReader(m->state().reader()); From 5d28d02e1752be74975647f8dc656776ab9e2cbf Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 27 Oct 2015 19:58:18 -0400 Subject: [PATCH 10/50] Rename async_connect to async_request --- .../main/native/libhdfspp/lib/fs/inputstream_impl.h | 2 +- .../main/native/libhdfspp/lib/reader/block_reader.h | 4 ++-- .../libhdfspp/lib/reader/remote_block_reader_impl.h | 6 +++--- .../main/native/libhdfspp/tests/inputstream_test.cc | 10 +++++----- .../native/libhdfspp/tests/remote_block_reader_test.cc | 4 ++-- hadoop-hdfs-project/pom.xml | 1 - 6 files changed, 13 insertions(+), 14 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index a5a19778ade9c..8b8e9f65a8110 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -75,7 +75,7 @@ struct ReadOperation::HandshakeContinuation : continuation::Continuation { } virtual void Run(const Next &next) override { - reader_->async_connect(client_name_, token_.get(), &block_, length_, + reader_->async_request(client_name_, token_.get(), &block_, length_, offset_, next); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index 81636b9bfff4f..10c3a7a139e23 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -70,13 +70,13 @@ class RemoteBlockReader template size_t read_some(const MutableBufferSequence &buffers, Status *status); - Status connect(const std::string &client_name, + Status request(const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset); template - void async_connect(const std::string &client_name, + void async_request(const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index 8b0ca5318740c..09a478351faf4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -40,7 +40,7 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, template template -void RemoteBlockReader::async_connect( +void RemoteBlockReader::async_request( const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const ConnectHandler &handler) { @@ -327,13 +327,13 @@ RemoteBlockReader::read_some(const MutableBufferSequence &buffers, } template -Status RemoteBlockReader::connect( +Status RemoteBlockReader::request( const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset) { auto stat = std::make_shared>(); std::future future(stat->get_future()); - async_connect(client_name, token, block, length, offset, + async_request(client_name, token, block, length, offset, [stat](const Status &status) { stat->set_value(status); }); return future.get(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 73547e2897a01..36ac260d7805e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -42,7 +42,7 @@ class MockReader { void(const asio::mutable_buffers_1 &, const std::function &)); - MOCK_METHOD6(async_connect, + MOCK_METHOD6(async_request, void(const std::string &, TokenProto *, ExtendedBlockProto *, uint64_t, uint64_t, const std::function &)); @@ -83,7 +83,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { size_t read = 0; struct Trait { static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_connect(_, _, _, _, _, _)) + EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); EXPECT_CALL(*reader, async_read_some(_, _)) @@ -117,7 +117,7 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { size_t read = 0; struct Trait { static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_connect(_, _, _, _, _, _)) + EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); EXPECT_CALL(*reader, async_read_some(_, _)) @@ -153,7 +153,7 @@ TEST(InputStreamTest, TestReadError) { size_t read = 0; struct Trait { static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_connect(_, _, _, _, _, _)) + EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); EXPECT_CALL(*reader, async_read_some(_, _)) @@ -200,7 +200,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { size_t read = 0; struct Trait { static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_connect(_, _, _, _, _, _)) + EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); EXPECT_CALL(*reader, async_read_some(_, _)) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index df4f506c48af7..47357f773f3af 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -101,7 +101,7 @@ ReadContent(Stream *conn, TokenProto *token, const ExtendedBlockProto &block, BlockReaderOptions options; auto reader = std::make_shared>(options, conn); Status result; - reader->async_connect("libhdfs++", token, &block, length, offset, + reader->async_request("libhdfs++", token, &block, length, offset, [buf, reader, handler](const Status &stat) { if (!stat.ok()) { handler(stat, 0); @@ -203,7 +203,7 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { BlockReaderOptions options; auto reader = std::make_shared >(options, &conn); Status result; - reader->async_connect( + reader->async_request( "libhdfs++", nullptr, &block, data.size(), 0, [buf, reader, &data, &io_service](const Status &stat) { ASSERT_TRUE(stat.ok()); diff --git a/hadoop-hdfs-project/pom.xml b/hadoop-hdfs-project/pom.xml index 4d53eddccf9ba..0a61c4674d8ac 100644 --- a/hadoop-hdfs-project/pom.xml +++ b/hadoop-hdfs-project/pom.xml @@ -33,7 +33,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> hadoop-hdfs hadoop-hdfs-client - hadoop-hdfs-native-client hadoop-hdfs-httpfs hadoop-hdfs/src/contrib/bkjournal hadoop-hdfs-nfs From 9d98bf41091c923103cbeeadb5459c3119b50584 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 28 Oct 2015 09:01:38 -0400 Subject: [PATCH 11/50] Renamed read_some to read_packet --- hadoop-common-project/pom.xml | 1 - .../main/native/libhdfspp/lib/fs/inputstream_impl.h | 2 +- .../main/native/libhdfspp/lib/reader/block_reader.h | 4 ++-- .../libhdfspp/lib/reader/remote_block_reader_impl.h | 6 +++--- .../main/native/libhdfspp/tests/inputstream_test.cc | 10 +++++----- .../native/libhdfspp/tests/remote_block_reader_test.cc | 6 +++--- hadoop-hdfs-project/pom.xml | 1 - pom.xml | 1 - 8 files changed, 14 insertions(+), 17 deletions(-) diff --git a/hadoop-common-project/pom.xml b/hadoop-common-project/pom.xml index ef49f9ce08bc0..cc641da138d84 100644 --- a/hadoop-common-project/pom.xml +++ b/hadoop-common-project/pom.xml @@ -33,7 +33,6 @@ hadoop-auth hadoop-auth-examples - hadoop-common hadoop-annotations hadoop-nfs hadoop-minikdc diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 8b8e9f65a8110..0fcee54a76d93 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -120,7 +120,7 @@ struct ReadOperation::ReadBlockContinuation : continuation::Continuation { } else if (*transferred_ >= buffer_size_) { next_(status); } else { - reader_->async_read_some( + reader_->async_read_packet( asio::buffer(buffer_ + *transferred_, buffer_size_ - *transferred_), std::bind(&ReadBlockContinuation::OnReadData, this, _1, _2)); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index 10c3a7a139e23..e384cb8932175 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -64,11 +64,11 @@ class RemoteBlockReader chunk_padding_bytes_(0) {} template - void async_read_some(const MutableBufferSequence &buffers, + void async_read_packet(const MutableBufferSequence &buffers, const ReadHandler &handler); template - size_t read_some(const MutableBufferSequence &buffers, Status *status); + size_t read_packet(const MutableBufferSequence &buffers, Status *status); Status request(const std::string &client_name, const hadoop::common::TokenProto *token, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index 09a478351faf4..a3960161d619b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -285,7 +285,7 @@ struct RemoteBlockReader::AckRead : continuation::Continuation { template template -void RemoteBlockReader::async_read_some( +void RemoteBlockReader::async_read_packet( const MutableBufferSequence &buffers, const ReadHandler &handler) { assert(state_ != kOpen && "Not connected"); @@ -311,12 +311,12 @@ void RemoteBlockReader::async_read_some( template template size_t -RemoteBlockReader::read_some(const MutableBufferSequence &buffers, +RemoteBlockReader::read_packet(const MutableBufferSequence &buffers, Status *status) { size_t transferred = 0; auto done = std::make_shared>(); auto future = done->get_future(); - async_read_some(buffers, + async_read_packet(buffers, [status, &transferred, done](const Status &stat, size_t t) { *status = stat; transferred = t; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 36ac260d7805e..82fc1dd096ba0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -38,7 +38,7 @@ class MockReader { public: virtual ~MockReader() {} MOCK_METHOD2( - async_read_some, + async_read_packet, void(const asio::mutable_buffers_1 &, const std::function &)); @@ -86,7 +86,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); - EXPECT_CALL(*reader, async_read_some(_, _)) + EXPECT_CALL(*reader, async_read_packet(_, _)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); } }; @@ -120,7 +120,7 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); - EXPECT_CALL(*reader, async_read_some(_, _)) + EXPECT_CALL(*reader, async_read_packet(_, _)) .Times(4) .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); } @@ -156,7 +156,7 @@ TEST(InputStreamTest, TestReadError) { EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); - EXPECT_CALL(*reader, async_read_some(_, _)) + EXPECT_CALL(*reader, async_read_packet(_, _)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) @@ -203,7 +203,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); - EXPECT_CALL(*reader, async_read_some(_, _)) + EXPECT_CALL(*reader, async_read_packet(_, _)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); } }; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 47357f773f3af..6056b0de6bf49 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -106,7 +106,7 @@ ReadContent(Stream *conn, TokenProto *token, const ExtendedBlockProto &block, if (!stat.ok()) { handler(stat, 0); } else { - reader->async_read_some(buf, handler); + reader->async_read_packet(buf, handler); } }); return reader; @@ -207,7 +207,7 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { "libhdfs++", nullptr, &block, data.size(), 0, [buf, reader, &data, &io_service](const Status &stat) { ASSERT_TRUE(stat.ok()); - reader->async_read_some( + reader->async_read_packet( buf, [buf, reader, &data, &io_service](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); ASSERT_EQ(kChunkSize, transferred); @@ -215,7 +215,7 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { data.clear(); data.resize(kChunkSize); transferred = 0; - reader->async_read_some( + reader->async_read_packet( buf, [&data,&io_service](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); ASSERT_EQ(kChunkSize, transferred); diff --git a/hadoop-hdfs-project/pom.xml b/hadoop-hdfs-project/pom.xml index 0a61c4674d8ac..0d65422dec7fe 100644 --- a/hadoop-hdfs-project/pom.xml +++ b/hadoop-hdfs-project/pom.xml @@ -31,7 +31,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> pom - hadoop-hdfs hadoop-hdfs-client hadoop-hdfs-httpfs hadoop-hdfs/src/contrib/bkjournal diff --git a/pom.xml b/pom.xml index 4e6f7a1ed96ec..d1508699ae54e 100644 --- a/pom.xml +++ b/pom.xml @@ -115,7 +115,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs hadoop-assemblies hadoop-maven-plugins hadoop-common-project - hadoop-hdfs-project hadoop-yarn-project hadoop-mapreduce-project hadoop-tools From 6ced4a97e297ce0e833db8dbd4b38c91c966d71c Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 28 Oct 2015 09:15:50 -0400 Subject: [PATCH 12/50] Renamed async_request to async_request_block --- .../main/native/libhdfspp/lib/fs/inputstream_impl.h | 2 +- .../main/native/libhdfspp/lib/reader/block_reader.h | 2 +- .../libhdfspp/lib/reader/remote_block_reader_impl.h | 4 ++-- .../main/native/libhdfspp/tests/inputstream_test.cc | 10 +++++----- .../native/libhdfspp/tests/remote_block_reader_test.cc | 4 ++-- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 0fcee54a76d93..42735320d31ac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -75,7 +75,7 @@ struct ReadOperation::HandshakeContinuation : continuation::Continuation { } virtual void Run(const Next &next) override { - reader_->async_request(client_name_, token_.get(), &block_, length_, + reader_->async_request_block(client_name_, token_.get(), &block_, length_, offset_, next); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index e384cb8932175..4c07cfbb5e9cd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -76,7 +76,7 @@ class RemoteBlockReader uint64_t offset); template - void async_request(const std::string &client_name, + void async_request_block(const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index a3960161d619b..32e1c69f7fd57 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -40,7 +40,7 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, template template -void RemoteBlockReader::async_request( +void RemoteBlockReader::async_request_block( const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const ConnectHandler &handler) { @@ -333,7 +333,7 @@ Status RemoteBlockReader::request( uint64_t offset) { auto stat = std::make_shared>(); std::future future(stat->get_future()); - async_request(client_name, token, block, length, offset, + async_request_block(client_name, token, block, length, offset, [stat](const Status &status) { stat->set_value(status); }); return future.get(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 82fc1dd096ba0..e0131cffb3c33 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -42,7 +42,7 @@ class MockReader { void(const asio::mutable_buffers_1 &, const std::function &)); - MOCK_METHOD6(async_request, + MOCK_METHOD6(async_request_block, void(const std::string &, TokenProto *, ExtendedBlockProto *, uint64_t, uint64_t, const std::function &)); @@ -83,7 +83,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { size_t read = 0; struct Trait { static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) + EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); EXPECT_CALL(*reader, async_read_packet(_, _)) @@ -117,7 +117,7 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { size_t read = 0; struct Trait { static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) + EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); EXPECT_CALL(*reader, async_read_packet(_, _)) @@ -153,7 +153,7 @@ TEST(InputStreamTest, TestReadError) { size_t read = 0; struct Trait { static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) + EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); EXPECT_CALL(*reader, async_read_packet(_, _)) @@ -200,7 +200,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { size_t read = 0; struct Trait { static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request(_, _, _, _, _, _)) + EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) .WillOnce(InvokeArgument<5>(Status::OK())); EXPECT_CALL(*reader, async_read_packet(_, _)) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 6056b0de6bf49..719777c87708d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -101,7 +101,7 @@ ReadContent(Stream *conn, TokenProto *token, const ExtendedBlockProto &block, BlockReaderOptions options; auto reader = std::make_shared>(options, conn); Status result; - reader->async_request("libhdfs++", token, &block, length, offset, + reader->async_request_block("libhdfs++", token, &block, length, offset, [buf, reader, handler](const Status &stat) { if (!stat.ok()) { handler(stat, 0); @@ -203,7 +203,7 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { BlockReaderOptions options; auto reader = std::make_shared >(options, &conn); Status result; - reader->async_request( + reader->async_request_block( "libhdfs++", nullptr, &block, data.size(), 0, [buf, reader, &data, &io_service](const Status &stat) { ASSERT_TRUE(stat.ok()); From f05a771e578969b9b281de4e0c97887f98b0f2cf Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 28 Oct 2015 09:19:09 -0400 Subject: [PATCH 13/50] Renamed BlockReader::request to request_block --- .../src/main/native/libhdfspp/lib/reader/block_reader.h | 2 +- .../main/native/libhdfspp/lib/reader/remote_block_reader_impl.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index 4c07cfbb5e9cd..7c68bbd726df3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -70,7 +70,7 @@ class RemoteBlockReader template size_t read_packet(const MutableBufferSequence &buffers, Status *status); - Status request(const std::string &client_name, + Status request_block(const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index 32e1c69f7fd57..f2e278528ec81 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -327,7 +327,7 @@ RemoteBlockReader::read_packet(const MutableBufferSequence &buffers, } template -Status RemoteBlockReader::request( +Status RemoteBlockReader::request_block( const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset) { From fcf1585bf67f84ef8c0acc72660d2ad250005e3b Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 28 Oct 2015 15:12:39 -0400 Subject: [PATCH 14/50] Moved to file_info --- .../native/libhdfspp/lib/fs/filesystem.cc | 28 +++++++++++++++---- .../main/native/libhdfspp/lib/fs/filesystem.h | 12 +++++--- .../native/libhdfspp/lib/fs/inputstream.cc | 11 ++------ .../libhdfspp/lib/fs/inputstream_impl.h | 4 +-- .../libhdfspp/tests/inputstream_test.cc | 22 ++++++++------- 5 files changed, 46 insertions(+), 31 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 059e15e472a51..1e4d55d50ca3e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -52,7 +52,7 @@ void NameNodeOperations::Connect(const std::string &server, } void NameNodeOperations::GetBlockLocations(const std::string & path, - std::function handler) + std::function)> handler) { using ::hadoop::hdfs::GetBlockLocationsRequestProto; using ::hadoop::hdfs::GetBlockLocationsResponseProto; @@ -74,10 +74,26 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, [this, s](const continuation::Continuation::Next &next) { namenode_.GetBlockLocations(&s->req, s->resp, next); })); - //TODO-BTH: Put client name et. al. into "ClusterInfo" object + m->Run([this, handler](const Status &stat, const State &s) { - handler(stat, stat.ok() ? &s.resp->locations() - : nullptr); + if (stat.ok()) { + auto file_info = std::make_shared(); + auto locations = s.resp->locations(); + + file_info->file_length_ = locations.filelength(); + + for (const auto &block : locations.blocks()) { + file_info->blocks_.push_back(block); + } + + if (locations.has_lastblock() && locations.lastblock().b().numbytes()) { + file_info->blocks_.push_back(locations.lastblock()); + } + + handler(stat, file_info); + } else { + handler(stat, nullptr); + } }); } @@ -117,8 +133,8 @@ void FileSystemImpl::Open( const std::string &path, const std::function &handler) { - nn_.GetBlockLocations(path, [this, handler](const Status &stat, const ::hadoop::hdfs::LocatedBlocksProto* locations) { - handler(stat, stat.ok() ? new ReadOperation(&io_service_->io_service(), client_name_, locations) + nn_.GetBlockLocations(path, [this, handler](const Status &stat, std::shared_ptr file_info) { + handler(stat, stat.ok() ? new ReadOperation(&io_service_->io_service(), client_name_, file_info) : nullptr); }); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index c6cd9031ce35d..43480e4eba2f9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -26,6 +26,11 @@ namespace hdfs { + struct FileInfo { + unsigned long long file_length_; + std::vector<::hadoop::hdfs::LocatedBlockProto> blocks_; + }; + /** * NameNodeConnection: abstracts the details of communicating with a NameNode * and the implementation of the communications protocol. @@ -47,7 +52,7 @@ class NameNodeOperations { const std::string &service, std::function &handler); - void GetBlockLocations(const std::string & path, std::function handler); + void GetBlockLocations(const std::string & path, std::function)> handler); private: ::asio::io_service * io_service_; RpcEngine engine_; @@ -90,7 +95,7 @@ class FileSystemImpl : public FileSystem { class ReadOperation : public InputStream { public: ReadOperation(::asio::io_service *io_service, const std::string &client_name, - const ::hadoop::hdfs::LocatedBlocksProto *blocks); + const std::shared_ptr file_info); virtual void PositionRead(void *buf, size_t nbyte, uint64_t offset, const std::set &excluded_datanodes, @@ -109,8 +114,7 @@ class ReadOperation : public InputStream { private: ::asio::io_service *io_service_; const std::string client_name_; - unsigned long long file_length_; - std::vector<::hadoop::hdfs::LocatedBlockProto> blocks_; + const std::shared_ptr file_info_; template struct HandshakeContinuation; template struct ReadBlockContinuation; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index 996da7400acfd..13fc932347009 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -26,15 +26,8 @@ InputStream::~InputStream() {} ReadOperation::ReadOperation(::asio::io_service *io_service, const std::string &client_name, - const ::hadoop::hdfs::LocatedBlocksProto *blocks) - : io_service_(io_service), client_name_(client_name), file_length_(blocks->filelength()) { - for (const auto &block : blocks->blocks()) { - blocks_.push_back(block); - } - - if (blocks->has_lastblock() && blocks->lastblock().b().numbytes()) { - blocks_.push_back(blocks->lastblock()); - } + const std::shared_ptr file_info) + : io_service_(io_service), client_name_(client_name), file_info_(file_info) { } void ReadOperation::PositionRead( diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 42735320d31ac..113e49cb6f28b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -135,11 +135,11 @@ void ReadOperation::AsyncPreadSome( using ::hadoop::hdfs::LocatedBlockProto; auto it = std::find_if( - blocks_.begin(), blocks_.end(), [offset](const LocatedBlockProto &p) { + file_info_->blocks_.begin(), file_info_->blocks_.end(), [offset](const LocatedBlockProto &p) { return p.offset() <= offset && offset < p.offset() + p.b().numbytes(); }); - if (it == blocks_.end()) { + if (it == file_info_->blocks_.end()) { handler(Status::InvalidArgument("Cannot find corresponding blocks"), "", 0); return; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index e0131cffb3c33..fc364c6d0ae18 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -69,6 +69,7 @@ template struct MockBlockReaderTrait { } TEST(InputStreamTest, TestReadSingleTrunk) { + auto file_info = std::make_shared(); LocatedBlocksProto blocks; LocatedBlockProto block; DatanodeInfoProto dn; @@ -78,7 +79,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); + ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); Status stat; size_t read = 0; struct Trait { @@ -103,7 +104,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { } TEST(InputStreamTest, TestReadMultipleTrunk) { - LocatedBlocksProto blocks; + auto file_info = std::make_shared(); LocatedBlockProto block; DatanodeInfoProto dn; char buf[4096] = { @@ -112,7 +113,7 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); + ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); Status stat; size_t read = 0; struct Trait { @@ -139,7 +140,7 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { } TEST(InputStreamTest, TestReadError) { - LocatedBlocksProto blocks; + auto file_info = std::make_shared(); LocatedBlockProto block; DatanodeInfoProto dn; char buf[4096] = { @@ -148,7 +149,7 @@ TEST(InputStreamTest, TestReadError) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); + ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); Status stat; size_t read = 0; struct Trait { @@ -177,15 +178,16 @@ TEST(InputStreamTest, TestReadError) { } TEST(InputStreamTest, TestExcludeDataNode) { - LocatedBlocksProto blocks; - LocatedBlockProto *block = blocks.add_blocks(); - ExtendedBlockProto *b = block->mutable_b(); + auto file_info = std::make_shared(); + file_info->blocks_.push_back(LocatedBlockProto()); + LocatedBlockProto & block = file_info->blocks_[0]; + ExtendedBlockProto *b = block.mutable_b(); b->set_poolid(""); b->set_blockid(1); b->set_generationstamp(1); b->set_numbytes(4096); - DatanodeInfoProto *di = block->add_locs(); + DatanodeInfoProto *di = block.add_locs(); DatanodeIDProto *dnid = di->mutable_id(); dnid->set_datanodeuuid("foo"); @@ -195,7 +197,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { IoServiceImpl io_service; Options options; FileSystemImpl fs(&io_service, options); - ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), &blocks); + ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); Status stat; size_t read = 0; struct Trait { From a3fd975285b25a3eae448e5ac46d0118a14d6610 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 28 Oct 2015 15:16:20 -0400 Subject: [PATCH 15/50] Made file_info pointers const --- .../main/native/libhdfspp/lib/fs/filesystem.cc | 4 ++-- .../main/native/libhdfspp/lib/fs/filesystem.h | 18 +++++++++++------- .../native/libhdfspp/lib/fs/inputstream.cc | 2 +- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 1e4d55d50ca3e..d087c6941abb9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -52,7 +52,7 @@ void NameNodeOperations::Connect(const std::string &server, } void NameNodeOperations::GetBlockLocations(const std::string & path, - std::function)> handler) + std::function)> handler) { using ::hadoop::hdfs::GetBlockLocationsRequestProto; using ::hadoop::hdfs::GetBlockLocationsResponseProto; @@ -133,7 +133,7 @@ void FileSystemImpl::Open( const std::string &path, const std::function &handler) { - nn_.GetBlockLocations(path, [this, handler](const Status &stat, std::shared_ptr file_info) { + nn_.GetBlockLocations(path, [this, handler](const Status &stat, std::shared_ptr file_info) { handler(stat, stat.ok() ? new ReadOperation(&io_service_->io_service(), client_name_, file_info) : nullptr); }); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 43480e4eba2f9..b555c6c05642d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -26,10 +26,14 @@ namespace hdfs { - struct FileInfo { - unsigned long long file_length_; - std::vector<::hadoop::hdfs::LocatedBlockProto> blocks_; - }; +/** + * Information that is assumed to be unchaning about a file for the duration of + * the operations. + */ +struct FileInfo { + unsigned long long file_length_; + std::vector<::hadoop::hdfs::LocatedBlockProto> blocks_; +}; /** * NameNodeConnection: abstracts the details of communicating with a NameNode @@ -52,7 +56,7 @@ class NameNodeOperations { const std::string &service, std::function &handler); - void GetBlockLocations(const std::string & path, std::function)> handler); +void GetBlockLocations(const std::string & path, std::function)> handler); private: ::asio::io_service * io_service_; RpcEngine engine_; @@ -95,7 +99,7 @@ class FileSystemImpl : public FileSystem { class ReadOperation : public InputStream { public: ReadOperation(::asio::io_service *io_service, const std::string &client_name, - const std::shared_ptr file_info); + const std::shared_ptr file_info); virtual void PositionRead(void *buf, size_t nbyte, uint64_t offset, const std::set &excluded_datanodes, @@ -114,7 +118,7 @@ class ReadOperation : public InputStream { private: ::asio::io_service *io_service_; const std::string client_name_; - const std::shared_ptr file_info_; + const std::shared_ptr file_info_; template struct HandshakeContinuation; template struct ReadBlockContinuation; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index 13fc932347009..d71dfdb809bf3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -26,7 +26,7 @@ InputStream::~InputStream() {} ReadOperation::ReadOperation(::asio::io_service *io_service, const std::string &client_name, - const std::shared_ptr file_info) + const std::shared_ptr file_info) : io_service_(io_service), client_name_(client_name), file_info_(file_info) { } From 366f488b8e8364eba3f1966b931216d2bf404ae1 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 28 Oct 2015 17:37:46 -0400 Subject: [PATCH 16/50] Refactored DataNodeConnection, etc. --- .../native/libhdfspp/lib/fs/filesystem.cc | 12 +- .../main/native/libhdfspp/lib/fs/filesystem.h | 48 +++- .../native/libhdfspp/lib/fs/inputstream.cc | 5 +- .../libhdfspp/lib/fs/inputstream_impl.h | 65 ++--- .../libhdfspp/tests/inputstream_test.cc | 250 +++++++++--------- 5 files changed, 212 insertions(+), 168 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index d087c6941abb9..489f3e344b0ee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -22,6 +22,7 @@ #include +#include #include namespace hdfs { @@ -98,6 +99,15 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, } +void DataNodeConnection::Connect( + std::function dn)> handler) { + asio::async_connect(*conn_, endpoints_.begin(), endpoints_.end(), + [this, handler](const asio::error_code &ec, std::array::iterator it) { + (void)it; + handler(ToStatus(ec), shared_from_this()); }); +} + + FileSystem::~FileSystem() {} @@ -134,7 +144,7 @@ void FileSystemImpl::Open( const std::function &handler) { nn_.GetBlockLocations(path, [this, handler](const Status &stat, std::shared_ptr file_info) { - handler(stat, stat.ok() ? new ReadOperation(&io_service_->io_service(), client_name_, file_info) + handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), client_name_, file_info) : nullptr); }); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index b555c6c05642d..09364c74dc204 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -89,6 +89,29 @@ class FileSystemImpl : public FileSystem { }; +class DataNodeConnection : public std::enable_shared_from_this { +public: + std::unique_ptr conn_; + std::array endpoints_; + std::string uuid_; + + + DataNodeConnection(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto) { + using namespace ::asio::ip; + + conn_.reset(new tcp::socket(*io_service)); + auto datanode_addr = dn_proto.id(); + endpoints_[0] = tcp::endpoint(address::from_string(datanode_addr.ipaddr()), + datanode_addr.xferport()); + uuid_ = dn_proto.id().datanodeuuid(); + } + + // Just for test, for now + DataNodeConnection() { + } + + void Connect(std::function dn)> handler); +}; /* * ReadOperation: given DN connection, does one-shot reads. @@ -96,9 +119,9 @@ class FileSystemImpl : public FileSystem { * Threading model: not thread-safe; consumers and io_service should not call * concurrently */ -class ReadOperation : public InputStream { +class InputStreamImpl : public InputStream { public: - ReadOperation(::asio::io_service *io_service, const std::string &client_name, + InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, const std::shared_ptr file_info); virtual void PositionRead(void *buf, size_t nbyte, uint64_t offset, @@ -109,20 +132,27 @@ class ReadOperation : public InputStream { void AsyncPreadSome(size_t offset, const MutableBufferSequence &buffers, const std::set &excluded_datanodes, const Handler &handler); - template - void AsyncReadBlock(const hadoop::hdfs::LocatedBlockProto &block, - const hadoop::hdfs::DatanodeInfoProto &dn, size_t offset, - const MutableBufferSequence &buffers, - const Handler &handler); - private: ::asio::io_service *io_service_; + std::shared_ptr dn_; // The last DN connected to const std::string client_name_; const std::shared_ptr file_info_; + struct RemoteBlockReaderTrait; +}; + +class ReadOperation { +public: + template + static void AsyncReadBlock( + std::shared_ptr dn, + const std::string & client_name, + const hadoop::hdfs::LocatedBlockProto &block, size_t offset, + const MutableBufferSequence &buffers, + const Handler &handler); +private: template struct HandshakeContinuation; template struct ReadBlockContinuation; - struct RemoteBlockReaderTrait; }; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index d71dfdb809bf3..ee0879fb69f88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -24,13 +24,12 @@ using ::hadoop::hdfs::LocatedBlocksProto; InputStream::~InputStream() {} -ReadOperation::ReadOperation(::asio::io_service *io_service, - const std::string &client_name, +InputStreamImpl::InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, const std::shared_ptr file_info) : io_service_(io_service), client_name_(client_name), file_info_(file_info) { } -void ReadOperation::PositionRead( +void InputStreamImpl::PositionRead( void *buf, size_t nbyte, uint64_t offset, const std::set &excluded_datanodes, const std::function diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 113e49cb6f28b..6af76c07e9f56 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -22,6 +22,7 @@ #include "common/continuation/asio.h" #include "common/continuation/protobuf.h" +#include "filesystem.h" #include #include @@ -29,12 +30,10 @@ namespace hdfs { -struct ReadOperation::RemoteBlockReaderTrait { +struct InputStreamImpl::RemoteBlockReaderTrait { typedef RemoteBlockReader Reader; struct State { - std::unique_ptr conn_; - std::array endpoints_; - + std::shared_ptr dn_; std::shared_ptr reader_; size_t transferred_; Reader *reader() { return reader_.get(); } @@ -42,19 +41,10 @@ struct ReadOperation::RemoteBlockReaderTrait { const size_t *transferred() const { return &transferred_; } }; static continuation::Pipeline * - Connect(::asio::io_service *io_service, - const ::hadoop::hdfs::DatanodeInfoProto &dn) { - using namespace ::asio::ip; + CreatePipeline(std::shared_ptr dn) { auto m = continuation::Pipeline::Create(); auto &s = m->state(); - s.conn_.reset(new tcp::socket(*io_service)); - s.reader_ = std::make_shared(BlockReaderOptions(), s.conn_.get()); - auto datanode = dn.id(); - s.endpoints_[0] = tcp::endpoint(address::from_string(datanode.ipaddr()), - datanode.xferport()); - - m->Push(asio_continuation::Connect(s.conn_.get(), s.endpoints_.begin(), - s.endpoints_.end())); + s.reader_ = std::make_shared(BlockReaderOptions(), dn->conn_.get()); return m; } }; @@ -128,7 +118,7 @@ struct ReadOperation::ReadBlockContinuation : continuation::Continuation { }; template -void ReadOperation::AsyncPreadSome( +void InputStreamImpl::AsyncPreadSome( size_t offset, const MutableBufferSequence &buffers, const std::set &excluded_datanodes, const Handler &handler) { using ::hadoop::hdfs::DatanodeInfoProto; @@ -144,9 +134,11 @@ void ReadOperation::AsyncPreadSome( return; } + ::hadoop::hdfs::LocatedBlockProto targetBlock = *it; + const DatanodeInfoProto *chosen_dn = nullptr; - for (int i = 0; i < it->locs_size(); ++i) { - const auto &di = it->locs(i); + for (int i = 0; i < targetBlock.locs_size(); ++i) { + const auto &di = targetBlock.locs(i); if (!excluded_datanodes.count(di.id().datanodeuuid())) { chosen_dn = &di; break; @@ -158,31 +150,42 @@ void ReadOperation::AsyncPreadSome( return; } - uint64_t offset_within_block = offset - it->offset(); + uint64_t offset_within_block = offset - targetBlock.offset(); uint64_t size_within_block = std::min( - it->b().numbytes() - offset_within_block, asio::buffer_size(buffers)); - - AsyncReadBlock( - *it, *chosen_dn, offset_within_block, - asio::buffer(buffers, size_within_block), handler); + targetBlock.b().numbytes() - offset_within_block, asio::buffer_size(buffers)); + + //TODO: re-use DN connection + dn_ = std::make_shared(io_service_, *chosen_dn); + dn_->Connect([this,handler,targetBlock,offset_within_block,size_within_block, buffers] + (Status status, std::shared_ptr dn) { + if (status.ok()) { + ReadOperation::AsyncReadBlock( + dn, client_name_, targetBlock, offset_within_block, + asio::buffer(buffers, size_within_block), handler); + } else { + handler(status, "", 0); + } + }); } template void ReadOperation::AsyncReadBlock( + std::shared_ptr dn, + const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, - const hadoop::hdfs::DatanodeInfoProto &dn, size_t offset, + size_t offset, const MutableBufferSequence &buffers, const Handler &handler) { typedef typename BlockReaderTrait::Reader Reader; - auto m = - BlockReaderTrait::Connect(io_service_, dn); + auto m = BlockReaderTrait::CreatePipeline(dn); auto &s = m->state(); + size_t size = asio::buffer_size(buffers); - m->Push(new HandshakeContinuation(s.reader(), client_name_, nullptr, + m->Push(new HandshakeContinuation(s.reader(), client_name, nullptr, &block.b(), size, offset)) - .Push(new ReadBlockContinuation( - s.reader(), buffers, s.transferred())); - const std::string &dnid = dn.id().datanodeuuid(); + .Push(new ReadBlockContinuation( + s.reader(), buffers, s.transferred())); + const std::string &dnid = dn->uuid_; m->Run([handler, dnid](const Status &status, const typename BlockReaderTrait::State &state) { handler(status, dnid, *state.transferred()); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index fc364c6d0ae18..0e4bffc31f0c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -59,7 +59,8 @@ template struct MockBlockReaderTrait { }; static continuation::Pipeline * - Connect(::asio::io_service *, const DatanodeInfoProto &) { + CreatePipeline(std::shared_ptr dn) { + (void) dn; auto m = continuation::Pipeline::Create(); *m->state().transferred() = 0; Trait::InitializeMockReader(m->state().reader()); @@ -76,10 +77,10 @@ TEST(InputStreamTest, TestReadSingleTrunk) { char buf[4096] = { 0, }; - IoServiceImpl io_service; - Options options; - FileSystemImpl fs(&io_service, options); - ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); +// IoServiceImpl io_service; +// Options options; +// FileSystemImpl fs(&io_service, options); +// InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); Status stat; size_t read = 0; struct Trait { @@ -92,8 +93,9 @@ TEST(InputStreamTest, TestReadSingleTrunk) { } }; - is.AsyncReadBlock>( - block, dn, 0, asio::buffer(buf, sizeof(buf)), + auto conn = std::make_shared(); + ReadOperation::AsyncReadBlock>( + conn, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), [&stat, &read](const Status &status, const std::string &, size_t transferred) { stat = status; read = transferred; @@ -103,123 +105,123 @@ TEST(InputStreamTest, TestReadSingleTrunk) { read = 0; } -TEST(InputStreamTest, TestReadMultipleTrunk) { - auto file_info = std::make_shared(); - LocatedBlockProto block; - DatanodeInfoProto dn; - char buf[4096] = { - 0, - }; - IoServiceImpl io_service; - Options options; - FileSystemImpl fs(&io_service, options); - ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); - Status stat; - size_t read = 0; - struct Trait { - static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); - - EXPECT_CALL(*reader, async_read_packet(_, _)) - .Times(4) - .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); - } - }; - - is.AsyncReadBlock>( - block, dn, 0, asio::buffer(buf, sizeof(buf)), - [&stat, &read](const Status &status, const std::string &, - size_t transferred) { - stat = status; - read = transferred; - }); - ASSERT_TRUE(stat.ok()); - ASSERT_EQ(sizeof(buf), read); - read = 0; -} - -TEST(InputStreamTest, TestReadError) { - auto file_info = std::make_shared(); - LocatedBlockProto block; - DatanodeInfoProto dn; - char buf[4096] = { - 0, - }; - IoServiceImpl io_service; - Options options; - FileSystemImpl fs(&io_service, options); - ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); - Status stat; - size_t read = 0; - struct Trait { - static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); - - EXPECT_CALL(*reader, async_read_packet(_, _)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) - .WillOnce(InvokeArgument<1>(Status::Error("error"), 0)); - } - }; - - is.AsyncReadBlock>( - block, dn, 0, asio::buffer(buf, sizeof(buf)), - [&stat, &read](const Status &status, const std::string &, - size_t transferred) { - stat = status; - read = transferred; - }); - ASSERT_FALSE(stat.ok()); - ASSERT_EQ(sizeof(buf) / 4 * 3, read); - read = 0; -} - -TEST(InputStreamTest, TestExcludeDataNode) { - auto file_info = std::make_shared(); - file_info->blocks_.push_back(LocatedBlockProto()); - LocatedBlockProto & block = file_info->blocks_[0]; - ExtendedBlockProto *b = block.mutable_b(); - b->set_poolid(""); - b->set_blockid(1); - b->set_generationstamp(1); - b->set_numbytes(4096); - - DatanodeInfoProto *di = block.add_locs(); - DatanodeIDProto *dnid = di->mutable_id(); - dnid->set_datanodeuuid("foo"); - - char buf[4096] = { - 0, - }; - IoServiceImpl io_service; - Options options; - FileSystemImpl fs(&io_service, options); - ReadOperation is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); - Status stat; - size_t read = 0; - struct Trait { - static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); - - EXPECT_CALL(*reader, async_read_packet(_, _)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); - } - }; - - - std::set excluded_dn({"foo"}); - is.AsyncPreadSome(0, asio::buffer(buf, sizeof(buf)), excluded_dn, - [&stat, &read](const Status &status, const std::string &, size_t transferred) { - stat = status; - read = transferred; - }); - ASSERT_EQ(static_cast(std::errc::resource_unavailable_try_again), stat.code()); - ASSERT_EQ(0UL, read); -} +//TEST(InputStreamTest, TestReadMultipleTrunk) { +// auto file_info = std::make_shared(); +// LocatedBlockProto block; +// DatanodeInfoProto dn; +// char buf[4096] = { +// 0, +// }; +// IoServiceImpl io_service; +// Options options; +// FileSystemImpl fs(&io_service, options); +// InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); +// Status stat; +// size_t read = 0; +// struct Trait { +// static void InitializeMockReader(MockReader *reader) { +// EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) +// .WillOnce(InvokeArgument<5>(Status::OK())); +// +// EXPECT_CALL(*reader, async_read_packet(_, _)) +// .Times(4) +// .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); +// } +// }; +// +// is.AsyncReadBlock>( +// block, dn, 0, asio::buffer(buf, sizeof(buf)), +// [&stat, &read](const Status &status, const std::string &, +// size_t transferred) { +// stat = status; +// read = transferred; +// }); +// ASSERT_TRUE(stat.ok()); +// ASSERT_EQ(sizeof(buf), read); +// read = 0; +//} +// +//TEST(InputStreamTest, TestReadError) { +// auto file_info = std::make_shared(); +// LocatedBlockProto block; +// DatanodeInfoProto dn; +// char buf[4096] = { +// 0, +// }; +// IoServiceImpl io_service; +// Options options; +// FileSystemImpl fs(&io_service, options); +// InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); +// Status stat; +// size_t read = 0; +// struct Trait { +// static void InitializeMockReader(MockReader *reader) { +// EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) +// .WillOnce(InvokeArgument<5>(Status::OK())); +// +// EXPECT_CALL(*reader, async_read_packet(_, _)) +// .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) +// .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) +// .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) +// .WillOnce(InvokeArgument<1>(Status::Error("error"), 0)); +// } +// }; +// +// is.AsyncReadBlock>( +// block, dn, 0, asio::buffer(buf, sizeof(buf)), +// [&stat, &read](const Status &status, const std::string &, +// size_t transferred) { +// stat = status; +// read = transferred; +// }); +// ASSERT_FALSE(stat.ok()); +// ASSERT_EQ(sizeof(buf) / 4 * 3, read); +// read = 0; +//} +// +//TEST(InputStreamTest, TestExcludeDataNode) { +// auto file_info = std::make_shared(); +// file_info->blocks_.push_back(LocatedBlockProto()); +// LocatedBlockProto & block = file_info->blocks_[0]; +// ExtendedBlockProto *b = block.mutable_b(); +// b->set_poolid(""); +// b->set_blockid(1); +// b->set_generationstamp(1); +// b->set_numbytes(4096); +// +// DatanodeInfoProto *di = block.add_locs(); +// DatanodeIDProto *dnid = di->mutable_id(); +// dnid->set_datanodeuuid("foo"); +// +// char buf[4096] = { +// 0, +// }; +// IoServiceImpl io_service; +// Options options; +// FileSystemImpl fs(&io_service, options); +// InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); +// Status stat; +// size_t read = 0; +// struct Trait { +// static void InitializeMockReader(MockReader *reader) { +// EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) +// .WillOnce(InvokeArgument<5>(Status::OK())); +// +// EXPECT_CALL(*reader, async_read_packet(_, _)) +// .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); +// } +// }; +// +// +// std::set excluded_dn({"foo"}); +// is.AsyncPreadSome(0, asio::buffer(buf, sizeof(buf)), excluded_dn, +// [&stat, &read](const Status &status, const std::string &, size_t transferred) { +// stat = status; +// read = transferred; +// }); +// ASSERT_EQ(static_cast(std::errc::resource_unavailable_try_again), stat.code()); +// ASSERT_EQ(0UL, read); +//} int main(int argc, char *argv[]) { // The following line must be executed to initialize Google Mock From 418799feb8d12181d9e5bd6b6aa94333bb21e126 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 29 Oct 2015 09:53:46 -0400 Subject: [PATCH 17/50] Added shared_ptr to DN_Connection --- .../src/main/native/libhdfspp/lib/fs/filesystem.cc | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 489f3e344b0ee..6e2926bd3a90b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -101,10 +101,12 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, void DataNodeConnection::Connect( std::function dn)> handler) { + // Keep the DN from being freed until we're done + auto shared_this = shared_from_this(); asio::async_connect(*conn_, endpoints_.begin(), endpoints_.end(), - [this, handler](const asio::error_code &ec, std::array::iterator it) { + [shared_this, handler](const asio::error_code &ec, std::array::iterator it) { (void)it; - handler(ToStatus(ec), shared_from_this()); }); + handler(ToStatus(ec), shared_this); }); } From f043e154a261e9ff64f1ead450e3a256ecd023a2 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 29 Oct 2015 11:31:28 -0400 Subject: [PATCH 18/50] Moved DNConnection into trait --- .../native/libhdfspp/lib/fs/filesystem.cc | 4 +-- .../main/native/libhdfspp/lib/fs/filesystem.h | 15 +++++------ .../libhdfspp/lib/fs/inputstream_impl.h | 12 ++++----- .../libhdfspp/tests/inputstream_test.cc | 26 ++++++++++++++----- 4 files changed, 33 insertions(+), 24 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 6e2926bd3a90b..0d8d32cbc9f10 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -99,8 +99,8 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, } -void DataNodeConnection::Connect( - std::function dn)> handler) { +void DataNodeConnectionImpl::Connect( + std::function dn)> handler) { // Keep the DN from being freed until we're done auto shared_this = shared_from_this(); asio::async_connect(*conn_, endpoints_.begin(), endpoints_.end(), diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 09364c74dc204..7f2fda9ff2590 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -89,14 +89,14 @@ class FileSystemImpl : public FileSystem { }; -class DataNodeConnection : public std::enable_shared_from_this { +class DataNodeConnectionImpl : public std::enable_shared_from_this { public: std::unique_ptr conn_; std::array endpoints_; std::string uuid_; - DataNodeConnection(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto) { + DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto) { using namespace ::asio::ip; conn_.reset(new tcp::socket(*io_service)); @@ -106,11 +106,7 @@ class DataNodeConnection : public std::enable_shared_from_this dn)> handler); + void Connect(std::function dn)> handler); }; /* @@ -134,15 +130,16 @@ class InputStreamImpl : public InputStream { const Handler &handler); private: ::asio::io_service *io_service_; - std::shared_ptr dn_; // The last DN connected to const std::string client_name_; const std::shared_ptr file_info_; struct RemoteBlockReaderTrait; + + std::shared_ptr dn_; // The last DN connected to }; class ReadOperation { public: - template + template static void AsyncReadBlock( std::shared_ptr dn, const std::string & client_name, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 6af76c07e9f56..425184f2c1d72 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -33,7 +33,7 @@ namespace hdfs { struct InputStreamImpl::RemoteBlockReaderTrait { typedef RemoteBlockReader Reader; struct State { - std::shared_ptr dn_; + std::shared_ptr dn_; std::shared_ptr reader_; size_t transferred_; Reader *reader() { return reader_.get(); } @@ -41,7 +41,7 @@ struct InputStreamImpl::RemoteBlockReaderTrait { const size_t *transferred() const { return &transferred_; } }; static continuation::Pipeline * - CreatePipeline(std::shared_ptr dn) { + CreatePipeline(std::shared_ptr dn) { auto m = continuation::Pipeline::Create(); auto &s = m->state(); s.reader_ = std::make_shared(BlockReaderOptions(), dn->conn_.get()); @@ -155,11 +155,11 @@ void InputStreamImpl::AsyncPreadSome( targetBlock.b().numbytes() - offset_within_block, asio::buffer_size(buffers)); //TODO: re-use DN connection - dn_ = std::make_shared(io_service_, *chosen_dn); + dn_ = std::make_shared(io_service_, *chosen_dn); dn_->Connect([this,handler,targetBlock,offset_within_block,size_within_block, buffers] - (Status status, std::shared_ptr dn) { + (Status status, std::shared_ptr dn) { if (status.ok()) { - ReadOperation::AsyncReadBlock( + ReadOperation::AsyncReadBlock( dn, client_name_, targetBlock, offset_within_block, asio::buffer(buffers, size_within_block), handler); } else { @@ -168,7 +168,7 @@ void InputStreamImpl::AsyncPreadSome( }); } -template +template void ReadOperation::AsyncReadBlock( std::shared_ptr dn, const std::string & client_name, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 0e4bffc31f0c9..c80cfe9f4aaed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -18,6 +18,7 @@ #include "fs/filesystem.h" #include +#include using hadoop::common::TokenProto; using hadoop::hdfs::DatanodeInfoProto; @@ -48,6 +49,20 @@ class MockReader { const std::function &)); }; +class MockDataNodeConnection { +public: + MockDataNodeConnection() { + int id; + RAND_pseudo_bytes((unsigned char *)&id, sizeof(id)); + + std::stringstream ss; + ss << "dn_" << id; + uuid_ = ss.str(); + } + + std::string uuid_; +}; + template struct MockBlockReaderTrait { typedef MockReader Reader; struct State { @@ -57,9 +72,9 @@ template struct MockBlockReaderTrait { size_t *transferred() { return &transferred_; } const size_t *transferred() const { return &transferred_; } }; - + static continuation::Pipeline * - CreatePipeline(std::shared_ptr dn) { + CreatePipeline(std::shared_ptr dn) { (void) dn; auto m = continuation::Pipeline::Create(); *m->state().transferred() = 0; @@ -77,10 +92,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { char buf[4096] = { 0, }; -// IoServiceImpl io_service; -// Options options; -// FileSystemImpl fs(&io_service, options); -// InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); + Status stat; size_t read = 0; struct Trait { @@ -93,7 +105,7 @@ TEST(InputStreamTest, TestReadSingleTrunk) { } }; - auto conn = std::make_shared(); + auto conn = std::make_shared(); ReadOperation::AsyncReadBlock>( conn, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), [&stat, &read](const Status &status, const std::string &, size_t transferred) { From aea859ff34a6768c7df29ec25f1abd2b92835b9e Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 29 Oct 2015 11:32:12 -0400 Subject: [PATCH 19/50] Trimmed whitespace --- .../native/libhdfspp/lib/common/continuation/asio.h | 2 +- .../src/main/native/libhdfspp/lib/fs/filesystem.cc | 8 ++++---- .../src/main/native/libhdfspp/lib/fs/filesystem.h | 12 ++++++------ .../main/native/libhdfspp/lib/fs/inputstream_impl.h | 4 ++-- .../main/native/libhdfspp/tests/inputstream_test.cc | 2 +- 5 files changed, 14 insertions(+), 14 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h index cf1bdcbd52a99..a5a0446bca8dd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h @@ -30,7 +30,7 @@ namespace hdfs { namespace asio_continuation { - + using namespace continuation; template diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 0d8d32cbc9f10..40faf3af8c0d5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -80,9 +80,9 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, if (stat.ok()) { auto file_info = std::make_shared(); auto locations = s.resp->locations(); - + file_info->file_length_ = locations.filelength(); - + for (const auto &block : locations.blocks()) { file_info->blocks_.push_back(block); } @@ -90,7 +90,7 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, if (locations.has_lastblock() && locations.lastblock().b().numbytes()) { file_info->blocks_.push_back(locations.lastblock()); } - + handler(stat, file_info); } else { handler(stat, nullptr); @@ -103,7 +103,7 @@ void DataNodeConnectionImpl::Connect( std::function dn)> handler) { // Keep the DN from being freed until we're done auto shared_this = shared_from_this(); - asio::async_connect(*conn_, endpoints_.begin(), endpoints_.end(), + asio::async_connect(*conn_, endpoints_.begin(), endpoints_.end(), [shared_this, handler](const asio::error_code &ec, std::array::iterator it) { (void)it; handler(ToStatus(ec), shared_this); }); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 7f2fda9ff2590..a3d2e547f0c4b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -34,7 +34,7 @@ struct FileInfo { unsigned long long file_length_; std::vector<::hadoop::hdfs::LocatedBlockProto> blocks_; }; - + /** * NameNodeConnection: abstracts the details of communicating with a NameNode * and the implementation of the communications protocol. @@ -94,18 +94,18 @@ class DataNodeConnectionImpl : public std::enable_shared_from_this conn_; std::array endpoints_; std::string uuid_; - - + + DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto) { using namespace ::asio::ip; conn_.reset(new tcp::socket(*io_service)); auto datanode_addr = dn_proto.id(); endpoints_[0] = tcp::endpoint(address::from_string(datanode_addr.ipaddr()), - datanode_addr.xferport()); + datanode_addr.xferport()); uuid_ = dn_proto.id().datanodeuuid(); } - + void Connect(std::function dn)> handler); }; @@ -141,7 +141,7 @@ class ReadOperation { public: template static void AsyncReadBlock( - std::shared_ptr dn, + std::shared_ptr dn, const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, size_t offset, const MutableBufferSequence &buffers, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 425184f2c1d72..0709d56b39b62 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -170,7 +170,7 @@ void InputStreamImpl::AsyncPreadSome( template void ReadOperation::AsyncReadBlock( - std::shared_ptr dn, + std::shared_ptr dn, const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, size_t offset, @@ -179,7 +179,7 @@ void ReadOperation::AsyncReadBlock( typedef typename BlockReaderTrait::Reader Reader; auto m = BlockReaderTrait::CreatePipeline(dn); auto &s = m->state(); - + size_t size = asio::buffer_size(buffers); m->Push(new HandshakeContinuation(s.reader(), client_name, nullptr, &block.b(), size, offset)) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index c80cfe9f4aaed..8d2716a18045e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -72,7 +72,7 @@ template struct MockBlockReaderTrait { size_t *transferred() { return &transferred_; } const size_t *transferred() const { return &transferred_; } }; - + static continuation::Pipeline * CreatePipeline(std::shared_ptr dn) { (void) dn; From 55d7b5dcd92b0fd9d0011e97d8f47e78c3316205 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 29 Oct 2015 13:23:30 -0400 Subject: [PATCH 20/50] Re-enabled IS tests --- .../libhdfspp/tests/inputstream_test.cc | 225 +++++++++--------- 1 file changed, 107 insertions(+), 118 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 8d2716a18045e..4e52d0a2b661b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -88,7 +88,6 @@ TEST(InputStreamTest, TestReadSingleTrunk) { auto file_info = std::make_shared(); LocatedBlocksProto blocks; LocatedBlockProto block; - DatanodeInfoProto dn; char buf[4096] = { 0, }; @@ -117,123 +116,113 @@ TEST(InputStreamTest, TestReadSingleTrunk) { read = 0; } -//TEST(InputStreamTest, TestReadMultipleTrunk) { -// auto file_info = std::make_shared(); -// LocatedBlockProto block; -// DatanodeInfoProto dn; -// char buf[4096] = { -// 0, -// }; -// IoServiceImpl io_service; -// Options options; -// FileSystemImpl fs(&io_service, options); -// InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); -// Status stat; -// size_t read = 0; -// struct Trait { -// static void InitializeMockReader(MockReader *reader) { -// EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) -// .WillOnce(InvokeArgument<5>(Status::OK())); -// -// EXPECT_CALL(*reader, async_read_packet(_, _)) -// .Times(4) -// .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); -// } -// }; -// -// is.AsyncReadBlock>( -// block, dn, 0, asio::buffer(buf, sizeof(buf)), -// [&stat, &read](const Status &status, const std::string &, -// size_t transferred) { -// stat = status; -// read = transferred; -// }); -// ASSERT_TRUE(stat.ok()); -// ASSERT_EQ(sizeof(buf), read); -// read = 0; -//} -// -//TEST(InputStreamTest, TestReadError) { -// auto file_info = std::make_shared(); -// LocatedBlockProto block; -// DatanodeInfoProto dn; -// char buf[4096] = { -// 0, -// }; -// IoServiceImpl io_service; -// Options options; -// FileSystemImpl fs(&io_service, options); -// InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); -// Status stat; -// size_t read = 0; -// struct Trait { -// static void InitializeMockReader(MockReader *reader) { -// EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) -// .WillOnce(InvokeArgument<5>(Status::OK())); -// -// EXPECT_CALL(*reader, async_read_packet(_, _)) -// .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) -// .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) -// .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) -// .WillOnce(InvokeArgument<1>(Status::Error("error"), 0)); -// } -// }; -// -// is.AsyncReadBlock>( -// block, dn, 0, asio::buffer(buf, sizeof(buf)), -// [&stat, &read](const Status &status, const std::string &, -// size_t transferred) { -// stat = status; -// read = transferred; -// }); -// ASSERT_FALSE(stat.ok()); -// ASSERT_EQ(sizeof(buf) / 4 * 3, read); -// read = 0; -//} -// -//TEST(InputStreamTest, TestExcludeDataNode) { -// auto file_info = std::make_shared(); -// file_info->blocks_.push_back(LocatedBlockProto()); -// LocatedBlockProto & block = file_info->blocks_[0]; -// ExtendedBlockProto *b = block.mutable_b(); -// b->set_poolid(""); -// b->set_blockid(1); -// b->set_generationstamp(1); -// b->set_numbytes(4096); -// -// DatanodeInfoProto *di = block.add_locs(); -// DatanodeIDProto *dnid = di->mutable_id(); -// dnid->set_datanodeuuid("foo"); -// -// char buf[4096] = { -// 0, -// }; -// IoServiceImpl io_service; -// Options options; -// FileSystemImpl fs(&io_service, options); -// InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); -// Status stat; -// size_t read = 0; -// struct Trait { -// static void InitializeMockReader(MockReader *reader) { -// EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) -// .WillOnce(InvokeArgument<5>(Status::OK())); -// -// EXPECT_CALL(*reader, async_read_packet(_, _)) -// .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); -// } -// }; -// -// -// std::set excluded_dn({"foo"}); -// is.AsyncPreadSome(0, asio::buffer(buf, sizeof(buf)), excluded_dn, -// [&stat, &read](const Status &status, const std::string &, size_t transferred) { -// stat = status; -// read = transferred; -// }); -// ASSERT_EQ(static_cast(std::errc::resource_unavailable_try_again), stat.code()); -// ASSERT_EQ(0UL, read); -//} +TEST(InputStreamTest, TestReadMultipleTrunk) { + LocatedBlockProto block; + char buf[4096] = { + 0, + }; + Status stat; + size_t read = 0; + struct Trait { + static void InitializeMockReader(MockReader *reader) { + EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) + .WillOnce(InvokeArgument<5>(Status::OK())); + + EXPECT_CALL(*reader, async_read_packet(_, _)) + .Times(4) + .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); + } + }; + + auto conn = std::make_shared(); + ReadOperation::AsyncReadBlock>( + conn, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + [&stat, &read](const Status &status, const std::string &, + size_t transferred) { + stat = status; + read = transferred; + }); + ASSERT_TRUE(stat.ok()); + ASSERT_EQ(sizeof(buf), read); + read = 0; +} + +TEST(InputStreamTest, TestReadError) { + LocatedBlockProto block; + char buf[4096] = { + 0, + }; + Status stat; + size_t read = 0; + struct Trait { + static void InitializeMockReader(MockReader *reader) { + EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) + .WillOnce(InvokeArgument<5>(Status::OK())); + + EXPECT_CALL(*reader, async_read_packet(_, _)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) + .WillOnce(InvokeArgument<1>(Status::Error("error"), 0)); + } + }; + + auto conn = std::make_shared(); + ReadOperation::AsyncReadBlock>( + conn, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + [&stat, &read](const Status &status, const std::string &, + size_t transferred) { + stat = status; + read = transferred; + }); + ASSERT_FALSE(stat.ok()); + ASSERT_EQ(sizeof(buf) / 4 * 3, read); + read = 0; +} + +TEST(InputStreamTest, TestExcludeDataNode) { + auto file_info = std::make_shared(); + file_info->blocks_.push_back(LocatedBlockProto()); + LocatedBlockProto & block = file_info->blocks_[0]; + ExtendedBlockProto *b = block.mutable_b(); + b->set_poolid(""); + b->set_blockid(1); + b->set_generationstamp(1); + b->set_numbytes(4096); + + DatanodeInfoProto *di = block.add_locs(); + DatanodeIDProto *dnid = di->mutable_id(); + dnid->set_datanodeuuid("foo"); + + char buf[4096] = { + 0, + }; + IoServiceImpl io_service; + Options options; + FileSystemImpl fs(&io_service, options); + InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); + Status stat; + size_t read = 0; + struct Trait { + static void InitializeMockReader(MockReader *reader) { + EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) + .WillOnce(InvokeArgument<5>(Status::OK())); + + EXPECT_CALL(*reader, async_read_packet(_, _)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); + } + }; + + + std::set excluded_dn({"foo"}); + is.AsyncPreadSome(0, asio::buffer(buf, sizeof(buf)), excluded_dn, + [&stat, &read](const Status &status, const std::string &, size_t transferred) { + stat = status; + read = transferred; + }); + ASSERT_EQ(static_cast(std::errc::resource_unavailable_try_again), stat.code()); + ASSERT_EQ(0UL, read); +} int main(int argc, char *argv[]) { // The following line must be executed to initialize Google Mock From 142efabbda38852b431d94096d6cef69f5c96393 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 29 Oct 2015 13:31:05 -0400 Subject: [PATCH 21/50] Cleaned up some tests --- .../native/libhdfspp/tests/inputstream_test.cc | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 4e52d0a2b661b..22546bf610528 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -190,6 +190,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { b->set_generationstamp(1); b->set_numbytes(4096); + // Set up the one block to have one datanode holding it DatanodeInfoProto *di = block.add_locs(); DatanodeIDProto *dnid = di->mutable_id(); dnid->set_datanodeuuid("foo"); @@ -198,28 +199,19 @@ TEST(InputStreamTest, TestExcludeDataNode) { 0, }; IoServiceImpl io_service; - Options options; - FileSystemImpl fs(&io_service, options); InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); Status stat; size_t read = 0; - struct Trait { - static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); - - EXPECT_CALL(*reader, async_read_packet(_, _)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); - } - }; - + // Exclude the one datanode with the data std::set excluded_dn({"foo"}); is.AsyncPreadSome(0, asio::buffer(buf, sizeof(buf)), excluded_dn, [&stat, &read](const Status &status, const std::string &, size_t transferred) { stat = status; read = transferred; }); + + // Should fail with no resource available ASSERT_EQ(static_cast(std::errc::resource_unavailable_try_again), stat.code()); ASSERT_EQ(0UL, read); } From 4bc0f448fe52a762a242428a1331272c9fee3247 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 29 Oct 2015 17:53:57 -0400 Subject: [PATCH 22/50] Working on less templates --- .../libhdfspp/lib/common/async_stream.h | 36 +++++++++ .../lib/common/continuation/async_stream.h | 81 +++++++++++++++++++ .../lib/common/continuation/protobuf.h | 21 +++-- .../native/libhdfspp/lib/fs/filesystem.cc | 2 +- .../main/native/libhdfspp/lib/fs/filesystem.h | 25 +++++- .../libhdfspp/lib/fs/inputstream_impl.h | 17 ++-- .../libhdfspp/lib/reader/block_reader.h | 9 +-- .../lib/reader/remote_block_reader_impl.h | 73 ++++++++--------- 8 files changed, 195 insertions(+), 69 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h new file mode 100644 index 0000000000000..373f5276e047a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#ifndef LIB_COMMON_ASYNC_STREAM_H_ +#define LIB_COMMON_ASYNC_STREAM_H_ + +#include + +namespace hdfs { + +class AsyncStream { + virtual void async_read(const asio::mutable_buffer & buffers, + std::function completion_handler, + std::function completed_handler); +}; + +} + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h new file mode 100644 index 0000000000000..48720ecb4a2af --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIB_COMMON_CONTINUATION_ASYNC_STREAM_H_ +#define LIB_COMMON_CONTINUATION_ASYNC_STREAM_H_ + +#include "continuation.h" +#include "common/util.h" + +#include "libhdfspp/status.h" + +#include "common/async_stream.h" + +namespace hdfs { +namespace async_stream_continuation { + +using namespace continuation; + +template +class ReadContinuation : public Continuation { +public: + ReadContinuation(std::shared_ptr stream, const MutableBufferSequence &buffer) + : stream_(stream), buffer_(buffer) {} + virtual void Run(const Next &next) override { + auto handler = + [next](const asio::error_code &ec, size_t) { next(ToStatus(ec)); }; + stream_->async_read(buffer_, handler); + } + +private: + std::shared_ptr stream_; + MutableBufferSequence buffer_; +}; + +template +class WriteContinuation : public Continuation { +public: + WriteContinuation(std::shared_ptr stream, const ConstBufferSequence &buffer) + : stream_(stream), buffer_(buffer) {} + + virtual void Run(const Next &next) override { + auto handler = + [next](const asio::error_code &ec, size_t) { next(ToStatus(ec)); }; + stream_->async_write(buffer_, handler); + } + +private: + std::shared_ptr stream_; + ConstBufferSequence buffer_; +}; + +template +static inline Continuation *Write(std::shared_ptr stream, + const ConstBufferSequence &buffer) { + return new WriteContinuation(stream, buffer); +} + +template +static inline Continuation *Read(std::shared_ptr stream, + const MutableBufferSequence &buffer) { + return new ReadContinuation(stream, buffer); +} + +} +} + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h index 49a8330b58e54..174129291b4fa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h @@ -33,7 +33,7 @@ namespace continuation { template struct ReadDelimitedPBMessageContinuation : public Continuation { - ReadDelimitedPBMessageContinuation(Stream *stream, + ReadDelimitedPBMessageContinuation(std::shared_ptr stream, ::google::protobuf::MessageLite *msg) : stream_(stream), msg_(msg) {} @@ -56,8 +56,8 @@ struct ReadDelimitedPBMessageContinuation : public Continuation { } next(status); }; - asio::async_read( - *stream_, asio::buffer(buf_), + stream_->async_read( + asio::buffer(buf_), std::bind(&ReadDelimitedPBMessageContinuation::CompletionHandler, this, std::placeholders::_1, std::placeholders::_2), handler); @@ -82,14 +82,14 @@ struct ReadDelimitedPBMessageContinuation : public Continuation { return offset ? len + offset - transferred : 1; } - Stream *stream_; + std::shared_ptr stream_; ::google::protobuf::MessageLite *msg_; std::array buf_; }; template struct WriteDelimitedPBMessageContinuation : Continuation { - WriteDelimitedPBMessageContinuation(Stream *stream, + WriteDelimitedPBMessageContinuation(std::shared_ptr stream, const google::protobuf::MessageLite *msg) : stream_(stream), msg_(msg) {} @@ -101,28 +101,25 @@ struct WriteDelimitedPBMessageContinuation : Continuation { pbio::CodedOutputStream os(&ss); os.WriteVarint32(size); msg_->SerializeToCodedStream(&os); - write_coroutine_ = - std::shared_ptr(asio_continuation::Write(stream_, asio::buffer(buf_))); - write_coroutine_->Run([next](const Status &stat) { next(stat); }); + stream_->async_write(asio::buffer(buf_), [next](const asio::error_code &ec, size_t) { next(ToStatus(ec)); } ); } private: - Stream *stream_; + std::shared_ptr stream_; const google::protobuf::MessageLite *msg_; std::string buf_; - std::shared_ptr write_coroutine_; }; template static inline Continuation * -ReadDelimitedPBMessage(Stream *stream, ::google::protobuf::MessageLite *msg) { +ReadDelimitedPBMessage(std::shared_ptr stream, ::google::protobuf::MessageLite *msg) { return new ReadDelimitedPBMessageContinuation(stream, msg); } template static inline Continuation * -WriteDelimitedPBMessage(Stream *stream, ::google::protobuf::MessageLite *msg) { +WriteDelimitedPBMessage(std::shared_ptr stream, ::google::protobuf::MessageLite *msg) { return new WriteDelimitedPBMessageContinuation(stream, msg); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 40faf3af8c0d5..fd4bbae68aa23 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -100,7 +100,7 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, void DataNodeConnectionImpl::Connect( - std::function dn)> handler) { + std::function dn)> handler) { // Keep the DN from being freed until we're done auto shared_this = shared_from_this(); asio::async_connect(*conn_, endpoints_.begin(), endpoints_.end(), diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index a3d2e547f0c4b..b20a0ec1b21af 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -89,7 +89,26 @@ class FileSystemImpl : public FileSystem { }; -class DataNodeConnectionImpl : public std::enable_shared_from_this { +class DataNodeConnection { +public: + std::string uuid_; + + virtual void Connect(std::function dn)> handler) = 0; + virtual void async_read(const asio::mutable_buffer & buffers, + std::function completed_handler) = 0; + virtual void async_read(const asio::mutable_buffer & buffers, + std::function completion_handler, + std::function completed_handler) = 0; + virtual void async_write(const asio::const_buffer & buffers, + std::function handler) = 0; + +}; + + +class DataNodeConnectionImpl : public DataNodeConnection, std::enable_shared_from_this { public: std::unique_ptr conn_; std::array endpoints_; @@ -106,7 +125,7 @@ class DataNodeConnectionImpl : public std::enable_shared_from_this dn)> handler); + void Connect(std::function dn)> handler) override; }; /* @@ -139,7 +158,7 @@ class InputStreamImpl : public InputStream { class ReadOperation { public: - template + template static void AsyncReadBlock( std::shared_ptr dn, const std::string & client_name, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 0709d56b39b62..5a33b349366b2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -31,20 +31,19 @@ namespace hdfs { struct InputStreamImpl::RemoteBlockReaderTrait { - typedef RemoteBlockReader Reader; struct State { - std::shared_ptr dn_; - std::shared_ptr reader_; + std::shared_ptr dn_; + std::shared_ptr reader_; size_t transferred_; - Reader *reader() { return reader_.get(); } + RemoteBlockReader *reader() { return reader_.get(); } size_t *transferred() { return &transferred_; } const size_t *transferred() const { return &transferred_; } }; static continuation::Pipeline * - CreatePipeline(std::shared_ptr dn) { + CreatePipeline(std::shared_ptr dn) { auto m = continuation::Pipeline::Create(); auto &s = m->state(); - s.reader_ = std::make_shared(BlockReaderOptions(), dn->conn_.get()); + s.reader_ = std::make_shared(BlockReaderOptions(), dn); return m; } }; @@ -157,9 +156,9 @@ void InputStreamImpl::AsyncPreadSome( //TODO: re-use DN connection dn_ = std::make_shared(io_service_, *chosen_dn); dn_->Connect([this,handler,targetBlock,offset_within_block,size_within_block, buffers] - (Status status, std::shared_ptr dn) { + (Status status, std::shared_ptr dn) { if (status.ok()) { - ReadOperation::AsyncReadBlock( + ReadOperation::AsyncReadBlock( dn, client_name_, targetBlock, offset_within_block, asio::buffer(buffers, size_within_block), handler); } else { @@ -168,7 +167,7 @@ void InputStreamImpl::AsyncPreadSome( }); } -template +template void ReadOperation::AsyncReadBlock( std::shared_ptr dn, const std::string & client_name, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index 7c68bbd726df3..f19ae4754a1b7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -55,11 +55,10 @@ struct BlockReaderOptions { : verify_checksum(true), encryption_scheme(EncryptionScheme::kNone) {} }; -template class RemoteBlockReader - : public std::enable_shared_from_this> { + : public std::enable_shared_from_this { public: - explicit RemoteBlockReader(const BlockReaderOptions &options, Stream *stream) + explicit RemoteBlockReader(const BlockReaderOptions &options, std::shared_ptr stream) : stream_(stream), state_(kOpen), options_(options), chunk_padding_bytes_(0) {} @@ -86,7 +85,7 @@ class RemoteBlockReader struct ReadPacketHeader; struct ReadChecksum; struct ReadPadding; - template struct ReadData; + struct ReadData; struct AckRead; enum State { kOpen, @@ -97,7 +96,7 @@ class RemoteBlockReader kFinished, }; - Stream *stream_; + std::shared_ptr stream_; hadoop::hdfs::PacketHeaderProto header_; State state_; BlockReaderOptions options_; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index f2e278528ec81..530b85dfafdcd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -20,6 +20,7 @@ #include "datatransfer.h" #include "common/continuation/asio.h" +#include "common/continuation/async_stream.h" #include "common/continuation/protobuf.h" #include @@ -38,9 +39,8 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset); -template template -void RemoteBlockReader::async_request_block( +void RemoteBlockReader::async_request_block( const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const ConnectHandler &handler) { @@ -65,15 +65,14 @@ void RemoteBlockReader::async_request_block( token, block, length, offset)); auto read_pb_message = - new continuation::ReadDelimitedPBMessageContinuation( + new continuation::ReadDelimitedPBMessageContinuation( stream_, &s->response); - m->Push(asio_continuation::Write(stream_, asio::buffer(s->header))) + m->Push(async_stream_continuation::Write(stream_, asio::buffer(s->header))) .Push(asio_continuation::WriteDelimitedPBMessage(stream_, &s->request)) .Push(read_pb_message); - m->Run([this, handler, offset](const Status &status, const State &s) { - Status stat = status; + m->Run([this, handler, offset](const Status &status, const State &s) { Status stat = status; if (stat.ok()) { const auto &resp = s.response; if (resp.status() == ::hadoop::hdfs::Status::SUCCESS) { @@ -90,10 +89,9 @@ void RemoteBlockReader::async_request_block( }); } -template -struct RemoteBlockReader::ReadPacketHeader +struct RemoteBlockReader::ReadPacketHeader : continuation::Continuation { - ReadPacketHeader(RemoteBlockReader *parent) : parent_(parent) {} + ReadPacketHeader(RemoteBlockReader *parent) : parent_(parent) {} virtual void Run(const Next &next) override { parent_->packet_data_read_bytes_ = 0; @@ -113,7 +111,7 @@ struct RemoteBlockReader::ReadPacketHeader next(status); }; - asio::async_read(*parent_->stream_, asio::buffer(buf_), + parent_->stream_->async_read(asio::buffer(buf_), std::bind(&ReadPacketHeader::CompletionHandler, this, std::placeholders::_1, std::placeholders::_2), handler); @@ -127,7 +125,7 @@ struct RemoteBlockReader::ReadPacketHeader static const size_t kHeaderLenSize = sizeof(int16_t); static const size_t kHeaderStart = kPayloadLenSize + kHeaderLenSize; - RemoteBlockReader *parent_; + RemoteBlockReader *parent_; std::array buf_; size_t packet_length() const { @@ -149,9 +147,8 @@ struct RemoteBlockReader::ReadPacketHeader } }; -template -struct RemoteBlockReader::ReadChecksum : continuation::Continuation { - ReadChecksum(RemoteBlockReader *parent) : parent_(parent) {} +struct RemoteBlockReader::ReadChecksum : continuation::Continuation { + ReadChecksum(RemoteBlockReader *parent) : parent_(parent) {} virtual void Run(const Next &next) override { auto parent = parent_; @@ -172,20 +169,19 @@ struct RemoteBlockReader::ReadChecksum : continuation::Continuation { }; parent->checksum_.resize(parent->packet_len_ - sizeof(int) - parent->header_.datalen()); - asio::async_read(*parent->stream_, asio::buffer(parent->checksum_), + parent->stream_->async_read(asio::buffer(parent->checksum_), handler); } private: - RemoteBlockReader *parent_; + RemoteBlockReader *parent_; }; -template -struct RemoteBlockReader::ReadPadding : continuation::Continuation { - ReadPadding(RemoteBlockReader *parent) +struct RemoteBlockReader::ReadPadding : continuation::Continuation { + ReadPadding(RemoteBlockReader *parent) : parent_(parent), padding_(parent->chunk_padding_bytes_), bytes_transferred_(std::make_shared(0)), - read_data_(new ReadData( + read_data_(new ReadData( parent, bytes_transferred_, asio::buffer(padding_))) {} virtual void Run(const Next &next) override { @@ -207,7 +203,7 @@ struct RemoteBlockReader::ReadPadding : continuation::Continuation { } private: - RemoteBlockReader *parent_; + RemoteBlockReader *parent_; std::vector padding_; std::shared_ptr bytes_transferred_; std::shared_ptr read_data_; @@ -215,12 +211,11 @@ struct RemoteBlockReader::ReadPadding : continuation::Continuation { ReadPadding &operator=(const ReadPadding &) = delete; }; -template -template -struct RemoteBlockReader::ReadData : continuation::Continuation { - ReadData(RemoteBlockReader *parent, + +struct RemoteBlockReader::ReadData : continuation::Continuation { + ReadData(RemoteBlockReader *parent, std::shared_ptr bytes_transferred, - const MutableBufferSequence &buf) + const asio::mutable_buffer &buf) : parent_(parent), bytes_transferred_(bytes_transferred), buf_(buf) {} virtual void Run(const Next &next) override { @@ -241,19 +236,19 @@ struct RemoteBlockReader::ReadData : continuation::Continuation { auto data_len = parent_->header_.datalen() - parent_->packet_data_read_bytes_; - async_read(*parent_->stream_, buf_, asio::transfer_exactly(data_len), + parent_->stream_->async_read(buf_, asio::transfer_exactly(data_len), handler); } private: - RemoteBlockReader *parent_; + RemoteBlockReader *parent_; std::shared_ptr bytes_transferred_; MutableBufferSequence buf_; }; -template -struct RemoteBlockReader::AckRead : continuation::Continuation { - AckRead(RemoteBlockReader *parent) : parent_(parent) {} + +struct RemoteBlockReader::AckRead : continuation::Continuation { + AckRead(RemoteBlockReader *parent) : parent_(parent) {} virtual void Run(const Next &next) override { if (parent_->bytes_to_read_ > 0) { @@ -273,19 +268,19 @@ struct RemoteBlockReader::AckRead : continuation::Continuation { m->Run([this, next](const Status &status, const hadoop::hdfs::ClientReadStatusProto &) { if (status.ok()) { - parent_->state_ = RemoteBlockReader::kFinished; + parent_->state_ = RemoteBlockReader::kFinished; } next(status); }); } private: - RemoteBlockReader *parent_; + RemoteBlockReader *parent_; }; -template + template -void RemoteBlockReader::async_read_packet( +void RemoteBlockReader::async_read_packet( const MutableBufferSequence &buffers, const ReadHandler &handler) { assert(state_ != kOpen && "Not connected"); @@ -308,10 +303,10 @@ void RemoteBlockReader::async_read_packet( }); } -template + template size_t -RemoteBlockReader::read_packet(const MutableBufferSequence &buffers, +RemoteBlockReader::read_packet(const MutableBufferSequence &buffers, Status *status) { size_t transferred = 0; auto done = std::make_shared>(); @@ -326,8 +321,8 @@ RemoteBlockReader::read_packet(const MutableBufferSequence &buffers, return transferred; } -template -Status RemoteBlockReader::request_block( + +Status RemoteBlockReader::request_block( const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset) { From dd16d4fa9f08f55f9d4140219471f002eca5a8ed Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Thu, 29 Oct 2015 19:28:01 -0400 Subject: [PATCH 23/50] Compiles! --- .../libhdfspp/lib/common/async_stream.h | 16 ++-- .../main/native/libhdfspp/lib/fs/filesystem.h | 33 ++++--- .../libhdfspp/lib/fs/inputstream_impl.h | 3 +- .../libhdfspp/lib/reader/block_reader.h | 5 +- .../libhdfspp/lib/reader/datatransfer.h | 30 ++++--- .../libhdfspp/lib/reader/datatransfer_impl.h | 25 ++---- .../lib/reader/remote_block_reader.cc | 12 +++ .../lib/reader/remote_block_reader_impl.h | 85 ++++++++----------- .../libhdfspp/tests/inputstream_test.cc | 20 ++++- .../tests/remote_block_reader_test.cc | 52 ++++++++---- 10 files changed, 162 insertions(+), 119 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h index 373f5276e047a..fa38b53cb6a0a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h @@ -24,11 +24,17 @@ namespace hdfs { class AsyncStream { - virtual void async_read(const asio::mutable_buffer & buffers, - std::function completion_handler, - std::function completed_handler); +public: + virtual void async_read(const asio::mutable_buffers_1 & buffers, + std::function completed_handler) = 0; + virtual void async_read(const asio::mutable_buffers_1 & buffers, + std::function completion_handler, + std::function completed_handler) = 0; + virtual void async_write(const asio::const_buffers_1 & buffers, + std::function handler) = 0; }; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index b20a0ec1b21af..94bc34c91527d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -19,11 +19,14 @@ #define LIBHDFSPP_LIB_FS_FILESYSTEM_H_ #include "common/hdfs_public_api.h" +#include "common/async_stream.h" #include "libhdfspp/hdfs.h" #include "rpc/rpc_engine.h" #include "ClientNamenodeProtocol.pb.h" #include "ClientNamenodeProtocol.hrpc.inl" +#include "asio.hpp" + namespace hdfs { /** @@ -89,26 +92,15 @@ class FileSystemImpl : public FileSystem { }; -class DataNodeConnection { +class DataNodeConnection : public AsyncStream { public: std::string uuid_; virtual void Connect(std::function dn)> handler) = 0; - virtual void async_read(const asio::mutable_buffer & buffers, - std::function completed_handler) = 0; - virtual void async_read(const asio::mutable_buffer & buffers, - std::function completion_handler, - std::function completed_handler) = 0; - virtual void async_write(const asio::const_buffer & buffers, - std::function handler) = 0; - }; -class DataNodeConnectionImpl : public DataNodeConnection, std::enable_shared_from_this { +class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_shared_from_this { public: std::unique_ptr conn_; std::array endpoints_; @@ -126,6 +118,21 @@ class DataNodeConnectionImpl : public DataNodeConnection, std::enable_shared_fro } void Connect(std::function dn)> handler) override; + + virtual void async_read(const asio::mutable_buffers_1 & buffers, + std::function completed_handler) + { asio::async_read(*conn_, buffers, completed_handler); } + + virtual void async_read(const asio::mutable_buffers_1 & buffers, + std::function completion_handler, + std::function completed_handler) + { asio::async_read(*conn_, buffers, completion_handler, completed_handler); } + virtual void async_write(const asio::const_buffers_1 & buffers, + std::function handler) + { asio::async_write(*conn_, buffers, handler); } }; /* diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 5a33b349366b2..5e8867f41d1ac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -31,6 +31,7 @@ namespace hdfs { struct InputStreamImpl::RemoteBlockReaderTrait { + typedef RemoteBlockReader Reader; struct State { std::shared_ptr dn_; std::shared_ptr reader_; @@ -158,7 +159,7 @@ void InputStreamImpl::AsyncPreadSome( dn_->Connect([this,handler,targetBlock,offset_within_block,size_within_block, buffers] (Status status, std::shared_ptr dn) { if (status.ok()) { - ReadOperation::AsyncReadBlock( + ReadOperation::AsyncReadBlock( dn, client_name_, targetBlock, offset_within_block, asio::buffer(buffers, size_within_block), handler); } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index f19ae4754a1b7..bf0e4010a8de5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -19,6 +19,7 @@ #define BLOCK_READER_H_ #include "libhdfspp/status.h" +#include "common/async_stream.h" #include "datatransfer.pb.h" #include @@ -58,7 +59,7 @@ struct BlockReaderOptions { class RemoteBlockReader : public std::enable_shared_from_this { public: - explicit RemoteBlockReader(const BlockReaderOptions &options, std::shared_ptr stream) + explicit RemoteBlockReader(const BlockReaderOptions &options, std::shared_ptr stream) : stream_(stream), state_(kOpen), options_(options), chunk_padding_bytes_(0) {} @@ -96,7 +97,7 @@ class RemoteBlockReader kFinished, }; - std::shared_ptr stream_; + std::shared_ptr stream_; hadoop::hdfs::PacketHeaderProto header_; State state_; BlockReaderOptions options_; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h index 511c2eb9e208a..212edbe039477 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h @@ -19,6 +19,9 @@ #define LIB_READER_DATA_TRANSFER_H_ #include "common/sasl_authenticator.h" +#include "common/async_stream.h" +#include + namespace hdfs { @@ -32,26 +35,33 @@ enum Operation { kReadBlock = 81, }; -template class DataTransferSaslStream { +template class DataTransferSaslStream : public AsyncStream { public: - DataTransferSaslStream(Stream *stream, const std::string &username, + DataTransferSaslStream(std::shared_ptr stream, const std::string &username, const std::string &password) : stream_(stream), authenticator_(username, password) {} template void Handshake(const Handler &next); - template - void async_read_some(const MutableBufferSequence &buffers, - ReadHandler &&handler); - - template - void async_write_some(const ConstBufferSequence &buffers, - WriteHandler &&handler); + virtual void async_read(const asio::mutable_buffers_1 & buffers, + std::function completed_handler) + { stream_->async_read(buffers, completed_handler); } + + virtual void async_read(const asio::mutable_buffers_1 & buffers, + std::function completion_handler, + std::function completed_handler) + { stream_->async_read(buffers, completion_handler, completed_handler); } + virtual void async_write(const asio::const_buffers_1 & buffers, + std::function handler) + { stream_->async_write(buffers, handler); } private: DataTransferSaslStream(const DataTransferSaslStream &) = delete; DataTransferSaslStream &operator=(const DataTransferSaslStream &) = delete; - Stream *stream_; + std::shared_ptr stream_; DigestMD5Authenticator authenticator_; struct ReadSaslMessage; struct Authenticator; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h index 5666f2ee31543..762b46aaff590 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h @@ -20,7 +20,7 @@ #include "datatransfer.pb.h" #include "common/continuation/continuation.h" -#include "common/continuation/asio.h" +#include "common/continuation/async_stream.h" #include "common/continuation/protobuf.h" #include @@ -70,7 +70,7 @@ struct DataTransferSaslStream::Authenticator template struct DataTransferSaslStream::ReadSaslMessage : continuation::Continuation { - ReadSaslMessage(Stream *stream, std::string *data) + ReadSaslMessage(std::shared_ptr stream, std::string *data) : stream_(stream), data_(data), read_pb_(stream, &resp_) {} virtual void Run(const Next &next) override { @@ -87,7 +87,7 @@ struct DataTransferSaslStream::ReadSaslMessage } private: - Stream *stream_; + std::shared_ptr stream_; std::string *data_; hadoop::hdfs::DataTransferEncryptorMessageProto resp_; continuation::ReadDelimitedPBMessageContinuation read_pb_; @@ -97,8 +97,8 @@ template template void DataTransferSaslStream::Handshake(const Handler &next) { using ::hadoop::hdfs::DataTransferEncryptorMessageProto; - using ::hdfs::asio_continuation::Write; - using ::hdfs::asio_continuation::WriteDelimitedPBMessage; + using ::hdfs::async_stream_continuation::Write; + using ::hdfs::continuation::WriteDelimitedPBMessage; static const int kMagicNumber = htonl(kDataTransferSasl); static const asio::const_buffers_1 kMagicNumberBuffer = asio::buffer( @@ -109,7 +109,7 @@ void DataTransferSaslStream::Handshake(const Handler &next) { std::string resp0; DataTransferEncryptorMessageProto req1; std::string resp1; - Stream *stream; + std::shared_ptr stream; }; auto m = continuation::Pipeline::Create(); State *s = &m->state(); @@ -126,19 +126,6 @@ void DataTransferSaslStream::Handshake(const Handler &next) { m->Run([next](const Status &status, const State &) { next(status); }); } -template -template -void DataTransferSaslStream::async_read_some( - const MutableBufferSequence &buffers, ReadHandler &&handler) { - stream_->async_read_some(buffers, handler); -} - -template -template -void DataTransferSaslStream::async_write_some( - const ConstBufferSequence &buffers, WriteHandler &&handler) { - stream_->async_write_some(buffers, handler); -} } #endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc index 68bc4ee3376ee..7a16758ae39cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc @@ -43,4 +43,16 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, // TODO: p.set_allocated_cachingstrategy(); return p; } + +Status RemoteBlockReader::request_block( + const std::string &client_name, const hadoop::common::TokenProto *token, + const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, + uint64_t offset) { + auto stat = std::make_shared>(); + std::future future(stat->get_future()); + async_request_block(client_name, token, block, length, offset, + [stat](const Status &status) { stat->set_value(status); }); + return future.get(); +} + } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index 530b85dfafdcd..4564453074a88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -65,7 +65,7 @@ void RemoteBlockReader::async_request_block( token, block, length, offset)); auto read_pb_message = - new continuation::ReadDelimitedPBMessageContinuation( + new continuation::ReadDelimitedPBMessageContinuation( stream_, &s->response); m->Push(async_stream_continuation::Write(stream_, asio::buffer(s->header))) @@ -177,6 +177,40 @@ struct RemoteBlockReader::ReadChecksum : continuation::Continuation { RemoteBlockReader *parent_; }; +struct RemoteBlockReader::ReadData : continuation::Continuation { + ReadData(RemoteBlockReader *parent, + std::shared_ptr bytes_transferred, + const asio::mutable_buffers_1 &buf) + : parent_(parent), bytes_transferred_(bytes_transferred), buf_(buf) {} + + virtual void Run(const Next &next) override { + auto handler = + [next, this](const asio::error_code &ec, size_t transferred) { + Status status; + if (ec) { + status = Status(ec.value(), ec.message().c_str()); + } + *bytes_transferred_ += transferred; + parent_->bytes_to_read_ -= transferred; + parent_->packet_data_read_bytes_ += transferred; + if (parent_->packet_data_read_bytes_ >= parent_->header_.datalen()) { + parent_->state_ = kReadPacketHeader; + } + next(status); + }; + + auto data_len = + parent_->header_.datalen() - parent_->packet_data_read_bytes_; + parent_->stream_->async_read(buf_, asio::transfer_exactly(data_len), + handler); + } + +private: + RemoteBlockReader *parent_; + std::shared_ptr bytes_transferred_; + const asio::mutable_buffers_1 & buf_; +}; + struct RemoteBlockReader::ReadPadding : continuation::Continuation { ReadPadding(RemoteBlockReader *parent) : parent_(parent), padding_(parent->chunk_padding_bytes_), @@ -212,41 +246,6 @@ struct RemoteBlockReader::ReadPadding : continuation::Continuation { }; -struct RemoteBlockReader::ReadData : continuation::Continuation { - ReadData(RemoteBlockReader *parent, - std::shared_ptr bytes_transferred, - const asio::mutable_buffer &buf) - : parent_(parent), bytes_transferred_(bytes_transferred), buf_(buf) {} - - virtual void Run(const Next &next) override { - auto handler = - [next, this](const asio::error_code &ec, size_t transferred) { - Status status; - if (ec) { - status = Status(ec.value(), ec.message().c_str()); - } - *bytes_transferred_ += transferred; - parent_->bytes_to_read_ -= transferred; - parent_->packet_data_read_bytes_ += transferred; - if (parent_->packet_data_read_bytes_ >= parent_->header_.datalen()) { - parent_->state_ = kReadPacketHeader; - } - next(status); - }; - - auto data_len = - parent_->header_.datalen() - parent_->packet_data_read_bytes_; - parent_->stream_->async_read(buf_, asio::transfer_exactly(data_len), - handler); - } - -private: - RemoteBlockReader *parent_; - std::shared_ptr bytes_transferred_; - MutableBufferSequence buf_; -}; - - struct RemoteBlockReader::AckRead : continuation::Continuation { AckRead(RemoteBlockReader *parent) : parent_(parent) {} @@ -293,7 +292,7 @@ void RemoteBlockReader::async_read_packet( m->Push(new ReadPacketHeader(this)) .Push(new ReadChecksum(this)) .Push(new ReadPadding(this)) - .Push(new ReadData( + .Push(new ReadData( this, m->state().bytes_transferred, buffers)) .Push(new AckRead(this)); @@ -320,18 +319,6 @@ RemoteBlockReader::read_packet(const MutableBufferSequence &buffers, future.wait(); return transferred; } - - -Status RemoteBlockReader::request_block( - const std::string &client_name, const hadoop::common::TokenProto *token, - const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, - uint64_t offset) { - auto stat = std::make_shared>(); - std::future future(stat->get_future()); - async_request_block(client_name, token, block, length, offset, - [stat](const Status &status) { stat->set_value(status); }); - return future.get(); -} } #endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 22546bf610528..846f2cfb0aea9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -49,7 +49,7 @@ class MockReader { const std::function &)); }; -class MockDataNodeConnection { +class MockDataNodeConnection : public DataNodeConnection, public std::enable_shared_from_this { public: MockDataNodeConnection() { int id; @@ -60,6 +60,22 @@ class MockDataNodeConnection { uuid_ = ss.str(); } + MOCK_METHOD1(Connect, void(std::function)>)); + MOCK_METHOD2(async_read, + void(const asio::mutable_buffers_1 & buffers, + std::function)); + MOCK_METHOD3(async_read, + void(const asio::mutable_buffers_1 & buffers, + std::function, + std::function)); + MOCK_METHOD2(async_write, + void(const asio::const_buffers_1 & buffers, + std::function)); + std::string uuid_; }; @@ -74,7 +90,7 @@ template struct MockBlockReaderTrait { }; static continuation::Pipeline * - CreatePipeline(std::shared_ptr dn) { + CreatePipeline(std::shared_ptr dn) { (void) dn; auto m = continuation::Pipeline::Create(); *m->state().transferred() = 0; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 719777c87708d..48316bd3dc8c3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -49,11 +49,27 @@ namespace pbio = pb::io; namespace hdfs { -class MockDNConnection : public MockConnectionBase { +class MockDNConnection : public MockConnectionBase, public AsyncStream { public: MockDNConnection(::asio::io_service &io_service) : MockConnectionBase(&io_service) {} MOCK_METHOD0(Produce, ProducerResult()); + + // MOCK_METHOD1(Connect, void(std::function)>)); + MOCK_METHOD2(async_read, + void(const asio::mutable_buffers_1 & buffers, + std::function)); + MOCK_METHOD3(async_read, + void(const asio::mutable_buffers_1 & buffers, + std::function, + std::function)); + MOCK_METHOD2(async_write, + void(const asio::const_buffers_1 & buffers, + std::function)); }; } @@ -94,12 +110,12 @@ ProducePacket(const std::string &data, const std::string &checksum, } template -static std::shared_ptr> -ReadContent(Stream *conn, TokenProto *token, const ExtendedBlockProto &block, +static std::shared_ptr +ReadContent(std::shared_ptr conn, TokenProto *token, const ExtendedBlockProto &block, uint64_t length, uint64_t offset, const mutable_buffers_1 &buf, const Handler &handler) { BlockReaderOptions options; - auto reader = std::make_shared>(options, conn); + auto reader = std::make_shared(options, conn); Status result; reader->async_request_block("libhdfs++", token, &block, length, offset, [buf, reader, handler](const Status &stat) { @@ -116,11 +132,11 @@ TEST(RemoteBlockReaderTest, TestReadWholeBlock) { static const size_t kChunkSize = 512; static const string kChunkData(kChunkSize, 'a'); ::asio::io_service io_service; - MockDNConnection conn(io_service); + auto conn = std::make_shared(io_service); BlockOpResponseProto block_op_resp; block_op_resp.set_status(::hadoop::hdfs::Status::SUCCESS); - EXPECT_CALL(conn, Produce()) + EXPECT_CALL(*conn, Produce()) .WillOnce(Return(Produce(ToDelimitedString(&block_op_resp)))) .WillOnce(Return(ProducePacket(kChunkData, "", 0, 1, true))); @@ -130,7 +146,7 @@ TEST(RemoteBlockReaderTest, TestReadWholeBlock) { block.set_generationstamp(0); std::string data(kChunkSize, 0); - ReadContent(&conn, nullptr, block, kChunkSize, 0, + ReadContent(conn, nullptr, block, kChunkSize, 0, buffer(const_cast(data.c_str()), data.size()), [&data, &io_service](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); @@ -148,7 +164,7 @@ TEST(RemoteBlockReaderTest, TestReadWithinChunk) { static const string kChunkData = string(kOffset, 'a') + string(kLength, 'b'); ::asio::io_service io_service; - MockDNConnection conn(io_service); + auto conn = std::make_shared(io_service); BlockOpResponseProto block_op_resp; ReadOpChecksumInfoProto *checksum_info = block_op_resp.mutable_readopchecksuminfo(); @@ -158,7 +174,7 @@ TEST(RemoteBlockReaderTest, TestReadWithinChunk) { checksum->set_bytesperchecksum(512); block_op_resp.set_status(::hadoop::hdfs::Status::SUCCESS); - EXPECT_CALL(conn, Produce()) + EXPECT_CALL(*conn, Produce()) .WillOnce(Return(Produce(ToDelimitedString(&block_op_resp)))) .WillOnce(Return(ProducePacket(kChunkData, "", kOffset, 1, true))); @@ -168,7 +184,7 @@ TEST(RemoteBlockReaderTest, TestReadWithinChunk) { block.set_generationstamp(0); string data(kLength, 0); - ReadContent(&conn, nullptr, block, data.size(), kOffset, + ReadContent(conn, nullptr, block, data.size(), kOffset, buffer(const_cast(data.c_str()), data.size()), [&data, &io_service](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); @@ -184,11 +200,11 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { static const string kChunkData(kChunkSize, 'a'); ::asio::io_service io_service; - MockDNConnection conn(io_service); + auto conn = std::make_shared(io_service); BlockOpResponseProto block_op_resp; block_op_resp.set_status(::hadoop::hdfs::Status::SUCCESS); - EXPECT_CALL(conn, Produce()) + EXPECT_CALL(*conn, Produce()) .WillOnce(Return(Produce(ToDelimitedString(&block_op_resp)))) .WillOnce(Return(ProducePacket(kChunkData, "", 0, 1, false))) .WillOnce(Return(ProducePacket(kChunkData, "", kChunkSize, 2, true))); @@ -201,7 +217,7 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { string data(kChunkSize, 0); mutable_buffers_1 buf = buffer(const_cast(data.c_str()), data.size()); BlockReaderOptions options; - auto reader = std::make_shared >(options, &conn); + auto reader = std::make_shared(options, conn); Status result; reader->async_request_block( "libhdfs++", nullptr, &block, data.size(), 0, @@ -234,7 +250,7 @@ TEST(RemoteBlockReaderTest, TestSaslConnection) { "qKah8qh4QZLoOLCDcTtEKhlS\",qop=\"auth\"," "charset=utf-8,algorithm=md5-sess"; ::asio::io_service io_service; - MockDNConnection conn(io_service); + auto conn = std::make_shared(io_service); BlockOpResponseProto block_op_resp; block_op_resp.set_status(::hadoop::hdfs::Status::SUCCESS); @@ -247,23 +263,23 @@ TEST(RemoteBlockReaderTest, TestSaslConnection) { ::hadoop::hdfs:: DataTransferEncryptorMessageProto_DataTransferEncryptorStatus_SUCCESS); - EXPECT_CALL(conn, Produce()) + EXPECT_CALL(*conn, Produce()) .WillOnce(Return(Produce(ToDelimitedString(&sasl_resp0)))) .WillOnce(Return(Produce(ToDelimitedString(&sasl_resp1)))) .WillOnce(Return(Produce(ToDelimitedString(&block_op_resp)))) .WillOnce(Return(ProducePacket(kChunkData, "", 0, 1, true))); - DataTransferSaslStream sasl_conn(&conn, "foo", "bar"); + auto sasl_conn = std::make_shared >(conn, "foo", "bar"); ExtendedBlockProto block; block.set_poolid("foo"); block.set_blockid(0); block.set_generationstamp(0); std::string data(kChunkSize, 0); - sasl_conn.Handshake([&sasl_conn, &block, &data, &io_service]( + sasl_conn->Handshake([sasl_conn, &block, &data, &io_service]( const Status &s) { ASSERT_TRUE(s.ok()); - ReadContent(&sasl_conn, nullptr, block, kChunkSize, 0, + ReadContent(sasl_conn, nullptr, block, kChunkSize, 0, buffer(const_cast(data.c_str()), data.size()), [&data, &io_service](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); From 2b14efa8277c66a3e9e0fb67af925501757d39f8 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Fri, 30 Oct 2015 16:46:52 -0400 Subject: [PATCH 24/50] Fixed DNconnection signature --- .../libhdfspp/lib/common/async_stream.h | 2 +- .../main/native/libhdfspp/lib/fs/filesystem.h | 2 +- .../libhdfspp/lib/reader/datatransfer.h | 2 +- .../libhdfspp/tests/inputstream_test.cc | 2 +- .../native/libhdfspp/tests/mock_connection.h | 2 + .../tests/remote_block_reader_test.cc | 48 ++++++++++++------- 6 files changed, 36 insertions(+), 22 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h index fa38b53cb6a0a..6d3873ffbb4c1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h @@ -29,7 +29,7 @@ class AsyncStream { std::function completed_handler) = 0; virtual void async_read(const asio::mutable_buffers_1 & buffers, - std::function completion_handler, std::function completed_handler) = 0; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 94bc34c91527d..a6ee2260bbc73 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -125,7 +125,7 @@ class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_sha { asio::async_read(*conn_, buffers, completed_handler); } virtual void async_read(const asio::mutable_buffers_1 & buffers, - std::function completion_handler, std::function completed_handler) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h index 212edbe039477..bcacbc3b259b0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h @@ -49,7 +49,7 @@ template class DataTransferSaslStream : public AsyncStream { { stream_->async_read(buffers, completed_handler); } virtual void async_read(const asio::mutable_buffers_1 & buffers, - std::function completion_handler, std::function completed_handler) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 846f2cfb0aea9..a039a5b566870 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -67,7 +67,7 @@ class MockDataNodeConnection : public DataNodeConnection, public std::enable_sha std::size_t) >)); MOCK_METHOD3(async_read, void(const asio::mutable_buffers_1 & buffers, - std::function, std::function)); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h index 8c0ef8cf3a019..6e7e2984e241c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h @@ -18,6 +18,8 @@ #ifndef LIBHDFSPP_TEST_MOCK_CONNECTION_H_ #define LIBHDFSPP_TEST_MOCK_CONNECTION_H_ +#include "common/async_stream.h" + #include #include #include diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 48316bd3dc8c3..93d71bfa6292b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -40,6 +40,8 @@ using ::hadoop::hdfs::ReadOpChecksumInfoProto; using ::asio::buffer; using ::asio::error_code; using ::asio::mutable_buffers_1; +using ::testing::_; +using ::testing::InvokeArgument; using ::testing::Return; using std::make_pair; using std::string; @@ -49,27 +51,34 @@ namespace pbio = pb::io; namespace hdfs { -class MockDNConnection : public MockConnectionBase, public AsyncStream { +class MockDNConnection : public MockConnectionBase, public AsyncStream{ public: MockDNConnection(::asio::io_service &io_service) : MockConnectionBase(&io_service) {} MOCK_METHOD0(Produce, ProducerResult()); - - // MOCK_METHOD1(Connect, void(std::function)>)); - MOCK_METHOD2(async_read, - void(const asio::mutable_buffers_1 & buffers, - std::function)); - MOCK_METHOD3(async_read, - void(const asio::mutable_buffers_1 & buffers, - std::function, - std::function)); - MOCK_METHOD2(async_write, - void(const asio::const_buffers_1 & buffers, - std::function)); + + + // Satisfy the AsyncStream contract, delegating to MocKConnectionBase + void async_read(const asio::mutable_buffers_1 & buf, + std::function handler) { + asio::async_read(*this, buf, handler); + } + + void async_read(const asio::mutable_buffers_1 & buf, + std::function completion_handler, + std::function completed_handler) { + asio::async_read(*this, buf, completion_handler, completed_handler); + } + + void async_write(const asio::const_buffers_1 & buf, + std::function handler) { + asio::async_write(*this, buf, handler); + } + + }; } @@ -145,16 +154,19 @@ TEST(RemoteBlockReaderTest, TestReadWholeBlock) { block.set_blockid(0); block.set_generationstamp(0); + bool done = false; std::string data(kChunkSize, 0); ReadContent(conn, nullptr, block, kChunkSize, 0, buffer(const_cast(data.c_str()), data.size()), - [&data, &io_service](const Status &stat, size_t transferred) { + [&data, &io_service, &done](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); ASSERT_EQ(kChunkSize, transferred); ASSERT_EQ(kChunkData, data); io_service.stop(); + done = true; }); io_service.run(); + ASSERT_TRUE(done); } TEST(RemoteBlockReaderTest, TestReadWithinChunk) { From 8d143e789a98431f8cd2cb08db37a0a05f4d9c77 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Mon, 2 Nov 2015 11:35:54 -0500 Subject: [PATCH 25/50] Fixed segfault in ReadData --- .../libhdfspp/lib/reader/remote_block_reader_impl.h | 10 ++++++++-- .../native/libhdfspp/tests/remote_block_reader_test.cc | 8 ++++++-- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index 4564453074a88..564f524244f06 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -181,7 +181,13 @@ struct RemoteBlockReader::ReadData : continuation::Continuation { ReadData(RemoteBlockReader *parent, std::shared_ptr bytes_transferred, const asio::mutable_buffers_1 &buf) - : parent_(parent), bytes_transferred_(bytes_transferred), buf_(buf) {} + : parent_(parent), bytes_transferred_(bytes_transferred), buf_(buf) { + buf_.begin(); + } + + ~ReadData() { + buf_.end(); + } virtual void Run(const Next &next) override { auto handler = @@ -208,7 +214,7 @@ struct RemoteBlockReader::ReadData : continuation::Continuation { private: RemoteBlockReader *parent_; std::shared_ptr bytes_transferred_; - const asio::mutable_buffers_1 & buf_; + const asio::mutable_buffers_1 buf_; }; struct RemoteBlockReader::ReadPadding : continuation::Continuation { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 93d71bfa6292b..d4de57e2dd8fd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -162,8 +162,8 @@ TEST(RemoteBlockReaderTest, TestReadWholeBlock) { ASSERT_TRUE(stat.ok()); ASSERT_EQ(kChunkSize, transferred); ASSERT_EQ(kChunkData, data); - io_service.stop(); done = true; + io_service.stop(); }); io_service.run(); ASSERT_TRUE(done); @@ -195,16 +195,20 @@ TEST(RemoteBlockReaderTest, TestReadWithinChunk) { block.set_blockid(0); block.set_generationstamp(0); + bool done = false; + string data(kLength, 0); ReadContent(conn, nullptr, block, data.size(), kOffset, buffer(const_cast(data.c_str()), data.size()), - [&data, &io_service](const Status &stat, size_t transferred) { + [&data, &io_service,&done](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); ASSERT_EQ(kLength, transferred); ASSERT_EQ(kChunkData.substr(kOffset, kLength), data); + done = true; io_service.stop(); }); io_service.run(); + ASSERT_TRUE(done); } TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { From b6f5454e626c1caa1b76398c9edf220fc1252be9 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Mon, 2 Nov 2015 13:36:15 -0500 Subject: [PATCH 26/50] Removed BlockReader callback templates --- .../libhdfspp/lib/reader/block_reader.h | 7 +-- .../lib/reader/remote_block_reader.cc | 49 ++++++++++++++++ .../lib/reader/remote_block_reader_impl.h | 56 +------------------ 3 files changed, 55 insertions(+), 57 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index bf0e4010a8de5..ca9accf2b5ee1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -63,9 +63,9 @@ class RemoteBlockReader : stream_(stream), state_(kOpen), options_(options), chunk_padding_bytes_(0) {} - template + template void async_read_packet(const MutableBufferSequence &buffers, - const ReadHandler &handler); + const std::function &handler); template size_t read_packet(const MutableBufferSequence &buffers, Status *status); @@ -75,12 +75,11 @@ class RemoteBlockReader const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset); - template void async_request_block(const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, - const ConnectHandler &handler); + const std::function &handler); private: struct ReadPacketHeader; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc index 7a16758ae39cb..258209bbb311c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc @@ -44,6 +44,55 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, return p; } +void RemoteBlockReader::async_request_block( + const std::string &client_name, const hadoop::common::TokenProto *token, + const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, + uint64_t offset, const std::function &handler) { + // The total number of bytes that we need to transfer from the DN is + // the amount that the user wants (bytesToRead), plus the padding at + // the beginning in order to chunk-align. Note that the DN may elect + // to send more than this amount if the read starts/ends mid-chunk. + bytes_to_read_ = length; + + struct State { + std::string header; + hadoop::hdfs::OpReadBlockProto request; + hadoop::hdfs::BlockOpResponseProto response; + }; + + auto m = continuation::Pipeline::Create(); + State *s = &m->state(); + + s->header.insert(s->header.begin(), + {0, kDataTransferVersion, Operation::kReadBlock}); + s->request = std::move(ReadBlockProto(client_name, options_.verify_checksum, + token, block, length, offset)); + + auto read_pb_message = + new continuation::ReadDelimitedPBMessageContinuation( + stream_, &s->response); + + m->Push(async_stream_continuation::Write(stream_, asio::buffer(s->header))) + .Push(asio_continuation::WriteDelimitedPBMessage(stream_, &s->request)) + .Push(read_pb_message); + + m->Run([this, handler, offset](const Status &status, const State &s) { Status stat = status; + if (stat.ok()) { + const auto &resp = s.response; + if (resp.status() == ::hadoop::hdfs::Status::SUCCESS) { + if (resp.has_readopchecksuminfo()) { + const auto &checksum_info = resp.readopchecksuminfo(); + chunk_padding_bytes_ = offset - checksum_info.chunkoffset(); + } + state_ = kReadPacketHeader; + } else { + stat = Status::Error(s.response.message().c_str()); + } + } + handler(stat); + }); +} + Status RemoteBlockReader::request_block( const std::string &client_name, const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index 564f524244f06..7702de4e2d9b3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -39,56 +39,6 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset); -template -void RemoteBlockReader::async_request_block( - const std::string &client_name, const hadoop::common::TokenProto *token, - const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, - uint64_t offset, const ConnectHandler &handler) { - // The total number of bytes that we need to transfer from the DN is - // the amount that the user wants (bytesToRead), plus the padding at - // the beginning in order to chunk-align. Note that the DN may elect - // to send more than this amount if the read starts/ends mid-chunk. - bytes_to_read_ = length; - - struct State { - std::string header; - hadoop::hdfs::OpReadBlockProto request; - hadoop::hdfs::BlockOpResponseProto response; - }; - - auto m = continuation::Pipeline::Create(); - State *s = &m->state(); - - s->header.insert(s->header.begin(), - {0, kDataTransferVersion, Operation::kReadBlock}); - s->request = std::move(ReadBlockProto(client_name, options_.verify_checksum, - token, block, length, offset)); - - auto read_pb_message = - new continuation::ReadDelimitedPBMessageContinuation( - stream_, &s->response); - - m->Push(async_stream_continuation::Write(stream_, asio::buffer(s->header))) - .Push(asio_continuation::WriteDelimitedPBMessage(stream_, &s->request)) - .Push(read_pb_message); - - m->Run([this, handler, offset](const Status &status, const State &s) { Status stat = status; - if (stat.ok()) { - const auto &resp = s.response; - if (resp.status() == ::hadoop::hdfs::Status::SUCCESS) { - if (resp.has_readopchecksuminfo()) { - const auto &checksum_info = resp.readopchecksuminfo(); - chunk_padding_bytes_ = offset - checksum_info.chunkoffset(); - } - state_ = kReadPacketHeader; - } else { - stat = Status::Error(s.response.message().c_str()); - } - } - handler(stat); - }); -} - struct RemoteBlockReader::ReadPacketHeader : continuation::Continuation { ReadPacketHeader(RemoteBlockReader *parent) : parent_(parent) {} @@ -283,10 +233,10 @@ struct RemoteBlockReader::AckRead : continuation::Continuation { RemoteBlockReader *parent_; }; - -template +template void RemoteBlockReader::async_read_packet( - const MutableBufferSequence &buffers, const ReadHandler &handler) { + const MutableBufferSequence &buffers, + const std::function &handler) { assert(state_ != kOpen && "Not connected"); struct State { From 3b5d712b454f5b817c22909bac2f3477a64624fe Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Mon, 2 Nov 2015 13:52:16 -0500 Subject: [PATCH 27/50] Removed last templates from BlockReader --- .../libhdfspp/lib/common/async_stream.h | 9 +- .../libhdfspp/lib/reader/block_reader.h | 6 +- .../lib/reader/remote_block_reader.cc | 241 +++++++++++++++++ .../lib/reader/remote_block_reader_impl.h | 242 ------------------ 4 files changed, 249 insertions(+), 249 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h index 6d3873ffbb4c1..ba50fa594e557 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h @@ -23,17 +23,20 @@ namespace hdfs { +typedef asio::mutable_buffers_1 MutableBuffers; +typedef asio::const_buffers_1 ConstBuffers; + class AsyncStream { public: - virtual void async_read(const asio::mutable_buffers_1 & buffers, + virtual void async_read(const MutableBuffers & buffers, std::function completed_handler) = 0; - virtual void async_read(const asio::mutable_buffers_1 & buffers, + virtual void async_read(const MutableBuffers & buffers, std::function completion_handler, std::function completed_handler) = 0; - virtual void async_write(const asio::const_buffers_1 & buffers, + virtual void async_write(const ConstBuffers & buffers, std::function handler) = 0; }; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index ca9accf2b5ee1..9030b1a9b612c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -63,12 +63,10 @@ class RemoteBlockReader : stream_(stream), state_(kOpen), options_(options), chunk_padding_bytes_(0) {} - template - void async_read_packet(const MutableBufferSequence &buffers, + void async_read_packet(const MutableBuffers &buffers, const std::function &handler); - template - size_t read_packet(const MutableBufferSequence &buffers, Status *status); + size_t read_packet(const MutableBuffers &buffers, Status *status); Status request_block(const std::string &client_name, const hadoop::common::TokenProto *token, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc index 258209bbb311c..098e0bb189633 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc @@ -104,4 +104,245 @@ Status RemoteBlockReader::request_block( return future.get(); } +hadoop::hdfs::OpReadBlockProto +ReadBlockProto(const std::string &client_name, bool verify_checksum, + const hadoop::common::TokenProto *token, + const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, + uint64_t offset); + +struct RemoteBlockReader::ReadPacketHeader + : continuation::Continuation { + ReadPacketHeader(RemoteBlockReader *parent) : parent_(parent) {} + + virtual void Run(const Next &next) override { + parent_->packet_data_read_bytes_ = 0; + parent_->packet_len_ = 0; + auto handler = [next, this](const asio::error_code &ec, size_t) { + Status status; + if (ec) { + status = Status(ec.value(), ec.message().c_str()); + } else { + parent_->packet_len_ = packet_length(); + parent_->header_.Clear(); + bool v = parent_->header_.ParseFromArray(&buf_[kHeaderStart], + header_length()); + assert(v && "Failed to parse the header"); + parent_->state_ = kReadChecksum; + } + next(status); + }; + + parent_->stream_->async_read(asio::buffer(buf_), + std::bind(&ReadPacketHeader::CompletionHandler, this, + std::placeholders::_1, std::placeholders::_2), + handler); + } + +private: + static const size_t kMaxHeaderSize = 512; + static const size_t kPayloadLenOffset = 0; + static const size_t kPayloadLenSize = sizeof(int); + static const size_t kHeaderLenOffset = 4; + static const size_t kHeaderLenSize = sizeof(short); + static const size_t kHeaderStart = kPayloadLenSize + kHeaderLenSize; + + RemoteBlockReader *parent_; + std::array buf_; + + size_t packet_length() const { + return ntohl(*reinterpret_cast(&buf_[kPayloadLenOffset])); + } + + size_t header_length() const { + return ntohs(*reinterpret_cast(&buf_[kHeaderLenOffset])); + } + + size_t CompletionHandler(const asio::error_code &ec, size_t transferred) { + if (ec) { + return 0; + } else if (transferred < kHeaderStart) { + return kHeaderStart - transferred; + } else { + return kHeaderStart + header_length() - transferred; + } + } +}; + +struct RemoteBlockReader::ReadChecksum : continuation::Continuation { + ReadChecksum(RemoteBlockReader *parent) : parent_(parent) {} + + virtual void Run(const Next &next) override { + auto parent = parent_; + if (parent->state_ != kReadChecksum) { + next(Status::OK()); + return; + } + + auto handler = [parent, next](const asio::error_code &ec, size_t) { + Status status; + if (ec) { + status = Status(ec.value(), ec.message().c_str()); + } else { + parent->state_ = + parent->chunk_padding_bytes_ ? kReadPadding : kReadData; + } + next(status); + }; + parent->checksum_.resize(parent->packet_len_ - sizeof(int) - + parent->header_.datalen()); + parent->stream_->async_read(asio::buffer(parent->checksum_), + handler); + } + +private: + RemoteBlockReader *parent_; +}; + +struct RemoteBlockReader::ReadData : continuation::Continuation { + ReadData(RemoteBlockReader *parent, + std::shared_ptr bytes_transferred, + const asio::mutable_buffers_1 &buf) + : parent_(parent), bytes_transferred_(bytes_transferred), buf_(buf) { + buf_.begin(); + } + + ~ReadData() { + buf_.end(); + } + + virtual void Run(const Next &next) override { + auto handler = + [next, this](const asio::error_code &ec, size_t transferred) { + Status status; + if (ec) { + status = Status(ec.value(), ec.message().c_str()); + } + *bytes_transferred_ += transferred; + parent_->bytes_to_read_ -= transferred; + parent_->packet_data_read_bytes_ += transferred; + if (parent_->packet_data_read_bytes_ >= parent_->header_.datalen()) { + parent_->state_ = kReadPacketHeader; + } + next(status); + }; + + auto data_len = + parent_->header_.datalen() - parent_->packet_data_read_bytes_; + parent_->stream_->async_read(buf_, asio::transfer_exactly(data_len), + handler); + } + +private: + RemoteBlockReader *parent_; + std::shared_ptr bytes_transferred_; + const asio::mutable_buffers_1 buf_; +}; + +struct RemoteBlockReader::ReadPadding : continuation::Continuation { + ReadPadding(RemoteBlockReader *parent) + : parent_(parent), padding_(parent->chunk_padding_bytes_), + bytes_transferred_(std::make_shared(0)), + read_data_(new ReadData( + parent, bytes_transferred_, asio::buffer(padding_))) {} + + virtual void Run(const Next &next) override { + if (parent_->state_ != kReadPadding || !parent_->chunk_padding_bytes_) { + next(Status::OK()); + return; + } + + auto h = [next, this](const Status &status) { + if (status.ok()) { + assert(reinterpret_cast(*bytes_transferred_) == + parent_->chunk_padding_bytes_); + parent_->chunk_padding_bytes_ = 0; + parent_->state_ = kReadData; + } + next(status); + }; + read_data_->Run(h); + } + +private: + RemoteBlockReader *parent_; + std::vector padding_; + std::shared_ptr bytes_transferred_; + std::shared_ptr read_data_; + ReadPadding(const ReadPadding &) = delete; + ReadPadding &operator=(const ReadPadding &) = delete; +}; + + +struct RemoteBlockReader::AckRead : continuation::Continuation { + AckRead(RemoteBlockReader *parent) : parent_(parent) {} + + virtual void Run(const Next &next) override { + if (parent_->bytes_to_read_ > 0) { + next(Status::OK()); + return; + } + + auto m = + continuation::Pipeline::Create(); + m->state().set_status(parent_->options_.verify_checksum + ? hadoop::hdfs::Status::CHECKSUM_OK + : hadoop::hdfs::Status::SUCCESS); + + m->Push( + continuation::WriteDelimitedPBMessage(parent_->stream_, &m->state())); + + m->Run([this, next](const Status &status, + const hadoop::hdfs::ClientReadStatusProto &) { + if (status.ok()) { + parent_->state_ = RemoteBlockReader::kFinished; + } + next(status); + }); + } + +private: + RemoteBlockReader *parent_; +}; + +void RemoteBlockReader::async_read_packet( + const MutableBuffers &buffers, + const std::function &handler) { + assert(state_ != kOpen && "Not connected"); + + struct State { + std::shared_ptr bytes_transferred; + }; + auto m = continuation::Pipeline::Create(); + m->state().bytes_transferred = std::make_shared(0); + + m->Push(new ReadPacketHeader(this)) + .Push(new ReadChecksum(this)) + .Push(new ReadPadding(this)) + .Push(new ReadData( + this, m->state().bytes_transferred, buffers)) + .Push(new AckRead(this)); + + auto self = this->shared_from_this(); + m->Run([self, handler](const Status &status, const State &state) { + handler(status, *state.bytes_transferred); + }); +} + + +size_t +RemoteBlockReader::read_packet(const MutableBuffers &buffers, + Status *status) { + size_t transferred = 0; + auto done = std::make_shared>(); + auto future = done->get_future(); + async_read_packet(buffers, + [status, &transferred, done](const Status &stat, size_t t) { + *status = stat; + transferred = t; + done->set_value(); + }); + future.wait(); + return transferred; +} + } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h index 7702de4e2d9b3..003d924dd5614 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h @@ -33,248 +33,6 @@ namespace hdfs { -hadoop::hdfs::OpReadBlockProto -ReadBlockProto(const std::string &client_name, bool verify_checksum, - const hadoop::common::TokenProto *token, - const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, - uint64_t offset); - -struct RemoteBlockReader::ReadPacketHeader - : continuation::Continuation { - ReadPacketHeader(RemoteBlockReader *parent) : parent_(parent) {} - - virtual void Run(const Next &next) override { - parent_->packet_data_read_bytes_ = 0; - parent_->packet_len_ = 0; - auto handler = [next, this](const asio::error_code &ec, size_t) { - Status status; - if (ec) { - status = Status(ec.value(), ec.message().c_str()); - } else { - parent_->packet_len_ = packet_length(); - parent_->header_.Clear(); - bool v = parent_->header_.ParseFromArray(&buf_[kHeaderStart], - header_length()); - assert(v && "Failed to parse the header"); - parent_->state_ = kReadChecksum; - } - next(status); - }; - - parent_->stream_->async_read(asio::buffer(buf_), - std::bind(&ReadPacketHeader::CompletionHandler, this, - std::placeholders::_1, std::placeholders::_2), - handler); - } - -private: - static const size_t kMaxHeaderSize = 512; - static const size_t kPayloadLenOffset = 0; - static const size_t kPayloadLenSize = sizeof(int32_t); - static const size_t kHeaderLenOffset = 4; - static const size_t kHeaderLenSize = sizeof(int16_t); - static const size_t kHeaderStart = kPayloadLenSize + kHeaderLenSize; - - RemoteBlockReader *parent_; - std::array buf_; - - size_t packet_length() const { - return ntohl(*reinterpret_cast(&buf_[kPayloadLenOffset])); - } - - size_t header_length() const { - return ntohs(*reinterpret_cast(&buf_[kHeaderLenOffset])); - } - - size_t CompletionHandler(const asio::error_code &ec, size_t transferred) { - if (ec) { - return 0; - } else if (transferred < kHeaderStart) { - return kHeaderStart - transferred; - } else { - return kHeaderStart + header_length() - transferred; - } - } -}; - -struct RemoteBlockReader::ReadChecksum : continuation::Continuation { - ReadChecksum(RemoteBlockReader *parent) : parent_(parent) {} - - virtual void Run(const Next &next) override { - auto parent = parent_; - if (parent->state_ != kReadChecksum) { - next(Status::OK()); - return; - } - - auto handler = [parent, next](const asio::error_code &ec, size_t) { - Status status; - if (ec) { - status = Status(ec.value(), ec.message().c_str()); - } else { - parent->state_ = - parent->chunk_padding_bytes_ ? kReadPadding : kReadData; - } - next(status); - }; - parent->checksum_.resize(parent->packet_len_ - sizeof(int) - - parent->header_.datalen()); - parent->stream_->async_read(asio::buffer(parent->checksum_), - handler); - } - -private: - RemoteBlockReader *parent_; -}; - -struct RemoteBlockReader::ReadData : continuation::Continuation { - ReadData(RemoteBlockReader *parent, - std::shared_ptr bytes_transferred, - const asio::mutable_buffers_1 &buf) - : parent_(parent), bytes_transferred_(bytes_transferred), buf_(buf) { - buf_.begin(); - } - - ~ReadData() { - buf_.end(); - } - - virtual void Run(const Next &next) override { - auto handler = - [next, this](const asio::error_code &ec, size_t transferred) { - Status status; - if (ec) { - status = Status(ec.value(), ec.message().c_str()); - } - *bytes_transferred_ += transferred; - parent_->bytes_to_read_ -= transferred; - parent_->packet_data_read_bytes_ += transferred; - if (parent_->packet_data_read_bytes_ >= parent_->header_.datalen()) { - parent_->state_ = kReadPacketHeader; - } - next(status); - }; - - auto data_len = - parent_->header_.datalen() - parent_->packet_data_read_bytes_; - parent_->stream_->async_read(buf_, asio::transfer_exactly(data_len), - handler); - } - -private: - RemoteBlockReader *parent_; - std::shared_ptr bytes_transferred_; - const asio::mutable_buffers_1 buf_; -}; - -struct RemoteBlockReader::ReadPadding : continuation::Continuation { - ReadPadding(RemoteBlockReader *parent) - : parent_(parent), padding_(parent->chunk_padding_bytes_), - bytes_transferred_(std::make_shared(0)), - read_data_(new ReadData( - parent, bytes_transferred_, asio::buffer(padding_))) {} - - virtual void Run(const Next &next) override { - if (parent_->state_ != kReadPadding || !parent_->chunk_padding_bytes_) { - next(Status::OK()); - return; - } - - auto h = [next, this](const Status &status) { - if (status.ok()) { - assert(reinterpret_cast(*bytes_transferred_) == - parent_->chunk_padding_bytes_); - parent_->chunk_padding_bytes_ = 0; - parent_->state_ = kReadData; - } - next(status); - }; - read_data_->Run(h); - } - -private: - RemoteBlockReader *parent_; - std::vector padding_; - std::shared_ptr bytes_transferred_; - std::shared_ptr read_data_; - ReadPadding(const ReadPadding &) = delete; - ReadPadding &operator=(const ReadPadding &) = delete; -}; - - -struct RemoteBlockReader::AckRead : continuation::Continuation { - AckRead(RemoteBlockReader *parent) : parent_(parent) {} - - virtual void Run(const Next &next) override { - if (parent_->bytes_to_read_ > 0) { - next(Status::OK()); - return; - } - - auto m = - continuation::Pipeline::Create(); - m->state().set_status(parent_->options_.verify_checksum - ? hadoop::hdfs::Status::CHECKSUM_OK - : hadoop::hdfs::Status::SUCCESS); - - m->Push( - continuation::WriteDelimitedPBMessage(parent_->stream_, &m->state())); - - m->Run([this, next](const Status &status, - const hadoop::hdfs::ClientReadStatusProto &) { - if (status.ok()) { - parent_->state_ = RemoteBlockReader::kFinished; - } - next(status); - }); - } - -private: - RemoteBlockReader *parent_; -}; - -template -void RemoteBlockReader::async_read_packet( - const MutableBufferSequence &buffers, - const std::function &handler) { - assert(state_ != kOpen && "Not connected"); - - struct State { - std::shared_ptr bytes_transferred; - }; - auto m = continuation::Pipeline::Create(); - m->state().bytes_transferred = std::make_shared(0); - - m->Push(new ReadPacketHeader(this)) - .Push(new ReadChecksum(this)) - .Push(new ReadPadding(this)) - .Push(new ReadData( - this, m->state().bytes_transferred, buffers)) - .Push(new AckRead(this)); - - auto self = this->shared_from_this(); - m->Run([self, handler](const Status &status, const State &state) { - handler(status, *state.bytes_transferred); - }); -} - - -template -size_t -RemoteBlockReader::read_packet(const MutableBufferSequence &buffers, - Status *status) { - size_t transferred = 0; - auto done = std::make_shared>(); - auto future = done->get_future(); - async_read_packet(buffers, - [status, &transferred, done](const Status &stat, size_t t) { - *status = stat; - transferred = t; - done->set_value(); - }); - future.wait(); - return transferred; -} } #endif From d9b9241f12a957226df7ccacad07d8e1a0d98cca Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Mon, 2 Nov 2015 15:56:43 -0500 Subject: [PATCH 28/50] Moved entirely over to BlockReader w/out templates --- .../main/native/libhdfspp/lib/fs/filesystem.h | 18 +-- .../native/libhdfspp/lib/fs/inputstream.cc | 149 ++++++++++++++++++ .../libhdfspp/lib/fs/inputstream_impl.h | 141 ----------------- .../libhdfspp/lib/reader/block_reader.h | 19 ++- .../libhdfspp/tests/inputstream_test.cc | 139 +++++----------- 5 files changed, 212 insertions(+), 254 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index a6ee2260bbc73..47fd991e9c6c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -22,6 +22,7 @@ #include "common/async_stream.h" #include "libhdfspp/hdfs.h" #include "rpc/rpc_engine.h" +#include "reader/block_reader.h" #include "ClientNamenodeProtocol.pb.h" #include "ClientNamenodeProtocol.hrpc.inl" @@ -150,10 +151,10 @@ class InputStreamImpl : public InputStream { const std::set &excluded_datanodes, const std::function &handler) override; - template - void AsyncPreadSome(size_t offset, const MutableBufferSequence &buffers, + + void AsyncPreadSome(size_t offset, const MutableBuffers &buffers, const std::set &excluded_datanodes, - const Handler &handler); + const std::function handler); private: ::asio::io_service *io_service_; const std::string client_name_; @@ -165,18 +166,17 @@ class InputStreamImpl : public InputStream { class ReadOperation { public: - template static void AsyncReadBlock( - std::shared_ptr dn, + BlockReader * reader, const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, size_t offset, - const MutableBufferSequence &buffers, - const Handler &handler); + const MutableBuffers &buffers, + const std::function handler); private: - template struct HandshakeContinuation; - template + struct HandshakeContinuation; struct ReadBlockContinuation; }; + } #include "inputstream_impl.h" diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index ee0879fb69f88..b3ba1e4b2bf5e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -36,4 +36,153 @@ void InputStreamImpl::PositionRead( &handler) { AsyncPreadSome(offset, asio::buffer(buf, nbyte), excluded_datanodes, handler); } + +struct ReadOperation::HandshakeContinuation : continuation::Continuation { + HandshakeContinuation(BlockReader *reader, const std::string &client_name, + const hadoop::common::TokenProto *token, + const hadoop::hdfs::ExtendedBlockProto *block, + uint64_t length, uint64_t offset) + : reader_(reader), client_name_(client_name), length_(length), + offset_(offset) { + if (token) { + token_.reset(new hadoop::common::TokenProto()); + token_->CheckTypeAndMergeFrom(*token); + } + block_.CheckTypeAndMergeFrom(*block); + } + + virtual void Run(const Next &next) override { + reader_->async_request_block(client_name_, token_.get(), &block_, length_, + offset_, next); + } + +private: + BlockReader *reader_; + const std::string client_name_; + std::unique_ptr token_; + hadoop::hdfs::ExtendedBlockProto block_; + uint64_t length_; + uint64_t offset_; +}; + +struct ReadOperation::ReadBlockContinuation : continuation::Continuation { + ReadBlockContinuation(BlockReader *reader, MutableBuffers buffer, + size_t *transferred) + : reader_(reader), buffer_(buffer), + buffer_size_(asio::buffer_size(buffer)), transferred_(transferred) { + } + + virtual void Run(const Next &next) override { + *transferred_ = 0; + next_ = next; + OnReadData(Status::OK(), 0); + } + +private: + BlockReader *reader_; + const MutableBuffers buffer_; + const size_t buffer_size_; + size_t *transferred_; + std::function next_; + + void OnReadData(const Status &status, size_t transferred) { + using std::placeholders::_1; + using std::placeholders::_2; + *transferred_ += transferred; + if (!status.ok()) { + next_(status); + } else if (*transferred_ >= buffer_size_) { + next_(status); + } else { + reader_->async_read_packet( + asio::buffer(buffer_ + *transferred_, buffer_size_ - *transferred_), + std::bind(&ReadBlockContinuation::OnReadData, this, _1, _2)); + } + } +}; + +void InputStreamImpl::AsyncPreadSome( + size_t offset, const MutableBuffers &buffers, + const std::set &excluded_datanodes, + const std::function handler) { + using ::hadoop::hdfs::DatanodeInfoProto; + using ::hadoop::hdfs::LocatedBlockProto; + + auto it = std::find_if( + file_info_->blocks_.begin(), file_info_->blocks_.end(), [offset](const LocatedBlockProto &p) { + return p.offset() <= offset && offset < p.offset() + p.b().numbytes(); + }); + + if (it == file_info_->blocks_.end()) { + handler(Status::InvalidArgument("Cannot find corresponding blocks"), "", 0); + return; + } + + ::hadoop::hdfs::LocatedBlockProto targetBlock = *it; + + const DatanodeInfoProto *chosen_dn = nullptr; + for (int i = 0; i < targetBlock.locs_size(); ++i) { + const auto &di = targetBlock.locs(i); + if (!excluded_datanodes.count(di.id().datanodeuuid())) { + chosen_dn = &di; + break; + } + } + + if (!chosen_dn) { + handler(Status::ResourceUnavailable("No datanodes available"), "", 0); + return; + } + + uint64_t offset_within_block = offset - targetBlock.offset(); + uint64_t size_within_block = std::min( + targetBlock.b().numbytes() - offset_within_block, asio::buffer_size(buffers)); + + // This is where we will put the logic for re-using a DN connection + dn_ = std::make_shared(io_service_, *chosen_dn); + std::string dn_id = dn_->uuid_; + + std::shared_ptr reader; + reader.reset(new RemoteBlockReader(BlockReaderOptions(), dn_)); + + auto read_handler = [dn_id, handler](const Status & status, size_t transferred) { + handler(status, dn_id, transferred); + }; + + dn_->Connect([this,handler,read_handler,targetBlock,offset_within_block,size_within_block, buffers, reader] + (Status status, std::shared_ptr dn) { + (void)dn; + if (status.ok()) { + ReadOperation::AsyncReadBlock( + reader.get(), client_name_, targetBlock, offset_within_block, + asio::buffer(buffers, size_within_block), read_handler); + } else { + handler(status, "", 0); + } + }); +} + +void ReadOperation::AsyncReadBlock( + BlockReader * reader, + const std::string & client_name, + const hadoop::hdfs::LocatedBlockProto &block, + size_t offset, + const MutableBuffers &buffers, + const std::function handler) { + + auto m = continuation::Pipeline::Create(); + size_t * bytesTransferred = &m->state(); + + size_t size = asio::buffer_size(buffers); + + m->Push(new HandshakeContinuation(reader, client_name, nullptr, + &block.b(), size, offset)) + .Push(new ReadBlockContinuation(reader, buffers, bytesTransferred)); + + m->Run([handler] (const Status &status, + const size_t totalBytesTransferred) { + handler(status, totalBytesTransferred); + }); +} + } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h index 5e8867f41d1ac..0c050f312550e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h @@ -49,148 +49,7 @@ struct InputStreamImpl::RemoteBlockReaderTrait { } }; -template -struct ReadOperation::HandshakeContinuation : continuation::Continuation { - HandshakeContinuation(Reader *reader, const std::string &client_name, - const hadoop::common::TokenProto *token, - const hadoop::hdfs::ExtendedBlockProto *block, - uint64_t length, uint64_t offset) - : reader_(reader), client_name_(client_name), length_(length), - offset_(offset) { - if (token) { - token_.reset(new hadoop::common::TokenProto()); - token_->CheckTypeAndMergeFrom(*token); - } - block_.CheckTypeAndMergeFrom(*block); - } - - virtual void Run(const Next &next) override { - reader_->async_request_block(client_name_, token_.get(), &block_, length_, - offset_, next); - } - -private: - Reader *reader_; - const std::string client_name_; - std::unique_ptr token_; - hadoop::hdfs::ExtendedBlockProto block_; - uint64_t length_; - uint64_t offset_; -}; - -template -struct ReadOperation::ReadBlockContinuation : continuation::Continuation { - ReadBlockContinuation(Reader *reader, MutableBufferSequence buffer, - size_t *transferred) - : reader_(reader), buffer_(buffer), - buffer_size_(asio::buffer_size(buffer)), transferred_(transferred) { - static_assert(!std::is_reference::value, - "Buffer must not be a reference type"); - } - - virtual void Run(const Next &next) override { - *transferred_ = 0; - next_ = next; - OnReadData(Status::OK(), 0); - } - -private: - Reader *reader_; - const MutableBufferSequence buffer_; - const size_t buffer_size_; - size_t *transferred_; - std::function next_; - - void OnReadData(const Status &status, size_t transferred) { - using std::placeholders::_1; - using std::placeholders::_2; - *transferred_ += transferred; - if (!status.ok()) { - next_(status); - } else if (*transferred_ >= buffer_size_) { - next_(status); - } else { - reader_->async_read_packet( - asio::buffer(buffer_ + *transferred_, buffer_size_ - *transferred_), - std::bind(&ReadBlockContinuation::OnReadData, this, _1, _2)); - } - } -}; - -template -void InputStreamImpl::AsyncPreadSome( - size_t offset, const MutableBufferSequence &buffers, - const std::set &excluded_datanodes, const Handler &handler) { - using ::hadoop::hdfs::DatanodeInfoProto; - using ::hadoop::hdfs::LocatedBlockProto; - auto it = std::find_if( - file_info_->blocks_.begin(), file_info_->blocks_.end(), [offset](const LocatedBlockProto &p) { - return p.offset() <= offset && offset < p.offset() + p.b().numbytes(); - }); - - if (it == file_info_->blocks_.end()) { - handler(Status::InvalidArgument("Cannot find corresponding blocks"), "", 0); - return; - } - - ::hadoop::hdfs::LocatedBlockProto targetBlock = *it; - - const DatanodeInfoProto *chosen_dn = nullptr; - for (int i = 0; i < targetBlock.locs_size(); ++i) { - const auto &di = targetBlock.locs(i); - if (!excluded_datanodes.count(di.id().datanodeuuid())) { - chosen_dn = &di; - break; - } - } - - if (!chosen_dn) { - handler(Status::ResourceUnavailable("No datanodes available"), "", 0); - return; - } - - uint64_t offset_within_block = offset - targetBlock.offset(); - uint64_t size_within_block = std::min( - targetBlock.b().numbytes() - offset_within_block, asio::buffer_size(buffers)); - - //TODO: re-use DN connection - dn_ = std::make_shared(io_service_, *chosen_dn); - dn_->Connect([this,handler,targetBlock,offset_within_block,size_within_block, buffers] - (Status status, std::shared_ptr dn) { - if (status.ok()) { - ReadOperation::AsyncReadBlock( - dn, client_name_, targetBlock, offset_within_block, - asio::buffer(buffers, size_within_block), handler); - } else { - handler(status, "", 0); - } - }); -} - -template -void ReadOperation::AsyncReadBlock( - std::shared_ptr dn, - const std::string & client_name, - const hadoop::hdfs::LocatedBlockProto &block, - size_t offset, - const MutableBufferSequence &buffers, const Handler &handler) { - - typedef typename BlockReaderTrait::Reader Reader; - auto m = BlockReaderTrait::CreatePipeline(dn); - auto &s = m->state(); - - size_t size = asio::buffer_size(buffers); - m->Push(new HandshakeContinuation(s.reader(), client_name, nullptr, - &block.b(), size, offset)) - .Push(new ReadBlockContinuation( - s.reader(), buffers, s.transferred())); - const std::string &dnid = dn->uuid_; - m->Run([handler, dnid](const Status &status, - const typename BlockReaderTrait::State &state) { - handler(status, dnid, *state.transferred()); - }); -} } #endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index 9030b1a9b612c..8c2781b035908 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -56,15 +56,27 @@ struct BlockReaderOptions { : verify_checksum(true), encryption_scheme(EncryptionScheme::kNone) {} }; +class BlockReader { +public: + virtual void async_read_packet(const MutableBuffers &buffers, + const std::function &handler) = 0; + + virtual void async_request_block(const std::string &client_name, + const hadoop::common::TokenProto *token, + const hadoop::hdfs::ExtendedBlockProto *block, + uint64_t length, uint64_t offset, + const std::function &handler) = 0; +}; + class RemoteBlockReader - : public std::enable_shared_from_this { + : public BlockReader, public std::enable_shared_from_this { public: explicit RemoteBlockReader(const BlockReaderOptions &options, std::shared_ptr stream) : stream_(stream), state_(kOpen), options_(options), chunk_padding_bytes_(0) {} void async_read_packet(const MutableBuffers &buffers, - const std::function &handler); + const std::function &handler) override; size_t read_packet(const MutableBuffers &buffers, Status *status); @@ -77,8 +89,7 @@ class RemoteBlockReader const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, - const std::function &handler); - + const std::function &handler) override; private: struct ReadPacketHeader; struct ReadChecksum; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index a039a5b566870..c4963b353c058 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -35,7 +35,7 @@ using namespace hdfs; namespace hdfs { -class MockReader { +class MockReader : public BlockReader { public: virtual ~MockReader() {} MOCK_METHOD2( @@ -44,60 +44,13 @@ class MockReader { const std::function &)); MOCK_METHOD6(async_request_block, - void(const std::string &, TokenProto *, ExtendedBlockProto *, - uint64_t, uint64_t, - const std::function &)); + void(const std::string &client_name, + const hadoop::common::TokenProto *token, + const hadoop::hdfs::ExtendedBlockProto *block, + uint64_t length, uint64_t offset, + const std::function &handler)); }; -class MockDataNodeConnection : public DataNodeConnection, public std::enable_shared_from_this { -public: - MockDataNodeConnection() { - int id; - RAND_pseudo_bytes((unsigned char *)&id, sizeof(id)); - - std::stringstream ss; - ss << "dn_" << id; - uuid_ = ss.str(); - } - - MOCK_METHOD1(Connect, void(std::function)>)); - MOCK_METHOD2(async_read, - void(const asio::mutable_buffers_1 & buffers, - std::function)); - MOCK_METHOD3(async_read, - void(const asio::mutable_buffers_1 & buffers, - std::function, - std::function)); - MOCK_METHOD2(async_write, - void(const asio::const_buffers_1 & buffers, - std::function)); - - std::string uuid_; -}; - -template struct MockBlockReaderTrait { - typedef MockReader Reader; - struct State { - MockReader reader_; - size_t transferred_; - Reader *reader() { return &reader_; } - size_t *transferred() { return &transferred_; } - const size_t *transferred() const { return &transferred_; } - }; - - static continuation::Pipeline * - CreatePipeline(std::shared_ptr dn) { - (void) dn; - auto m = continuation::Pipeline::Create(); - *m->state().transferred() = 0; - Trait::InitializeMockReader(m->state().reader()); - return m; - } -}; } TEST(InputStreamTest, TestReadSingleTrunk) { @@ -110,20 +63,15 @@ TEST(InputStreamTest, TestReadSingleTrunk) { Status stat; size_t read = 0; - struct Trait { - static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); - - EXPECT_CALL(*reader, async_read_packet(_, _)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); - } - }; - - auto conn = std::make_shared(); - ReadOperation::AsyncReadBlock>( - conn, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), - [&stat, &read](const Status &status, const std::string &, size_t transferred) { + MockReader reader; + EXPECT_CALL(reader, async_request_block(_, _, _, _, _, _)) + .WillOnce(InvokeArgument<5>(Status::OK())); + EXPECT_CALL(reader, async_read_packet(_, _)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); + + ReadOperation::AsyncReadBlock( + &reader, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + [&stat, &read](const Status &status, size_t transferred) { stat = status; read = transferred; }); @@ -139,22 +87,18 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { }; Status stat; size_t read = 0; - struct Trait { - static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); - - EXPECT_CALL(*reader, async_read_packet(_, _)) - .Times(4) - .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); - } - }; + + MockReader reader; + EXPECT_CALL(reader, async_request_block(_, _, _, _, _, _)) + .WillOnce(InvokeArgument<5>(Status::OK())); - auto conn = std::make_shared(); - ReadOperation::AsyncReadBlock>( - conn, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), - [&stat, &read](const Status &status, const std::string &, - size_t transferred) { + EXPECT_CALL(reader, async_read_packet(_, _)) + .Times(4) + .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); + + ReadOperation::AsyncReadBlock( + &reader, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + [&stat, &read](const Status &status, size_t transferred) { stat = status; read = transferred; }); @@ -170,24 +114,19 @@ TEST(InputStreamTest, TestReadError) { }; Status stat; size_t read = 0; - struct Trait { - static void InitializeMockReader(MockReader *reader) { - EXPECT_CALL(*reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); - - EXPECT_CALL(*reader, async_read_packet(_, _)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) - .WillOnce(InvokeArgument<1>(Status::Error("error"), 0)); - } - }; - - auto conn = std::make_shared(); - ReadOperation::AsyncReadBlock>( - conn, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), - [&stat, &read](const Status &status, const std::string &, - size_t transferred) { + MockReader reader; + EXPECT_CALL(reader, async_request_block(_, _, _, _, _, _)) + .WillOnce(InvokeArgument<5>(Status::OK())); + + EXPECT_CALL(reader, async_read_packet(_, _)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) + .WillOnce(InvokeArgument<1>(Status::Error("error"), 0)); + + ReadOperation::AsyncReadBlock( + &reader, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + [&stat, &read](const Status &status, size_t transferred) { stat = status; read = transferred; }); From 5de0bce35fb52b7a688d3fc4ad02748106fca38e Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Mon, 2 Nov 2015 16:06:25 -0500 Subject: [PATCH 29/50] Removed unnecessary impls --- .../main/native/libhdfspp/lib/fs/filesystem.h | 2 - .../native/libhdfspp/lib/fs/inputstream.cc | 1 + .../libhdfspp/lib/fs/inputstream_impl.h | 55 ------------------- .../libhdfspp/lib/reader/block_reader.h | 2 - .../lib/reader/remote_block_reader.cc | 7 ++- .../lib/reader/remote_block_reader_impl.h | 38 ------------- .../tests/remote_block_reader_test.cc | 1 + 7 files changed, 8 insertions(+), 98 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 47fd991e9c6c4..41830f12501ed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -179,6 +179,4 @@ class ReadOperation { } -#include "inputstream_impl.h" - #endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index b3ba1e4b2bf5e..81f78f1f25e03 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -17,6 +17,7 @@ */ #include "filesystem.h" +#include "common/continuation/continuation.h" namespace hdfs { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h deleted file mode 100644 index 0c050f312550e..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream_impl.h +++ /dev/null @@ -1,55 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#ifndef FS_INPUTSTREAM_IMPL_H_ -#define FS_INPUTSTREAM_IMPL_H_ - -#include "reader/block_reader.h" - -#include "common/continuation/asio.h" -#include "common/continuation/protobuf.h" -#include "filesystem.h" - -#include -#include -#include - -namespace hdfs { - -struct InputStreamImpl::RemoteBlockReaderTrait { - typedef RemoteBlockReader Reader; - struct State { - std::shared_ptr dn_; - std::shared_ptr reader_; - size_t transferred_; - RemoteBlockReader *reader() { return reader_.get(); } - size_t *transferred() { return &transferred_; } - const size_t *transferred() const { return &transferred_; } - }; - static continuation::Pipeline * - CreatePipeline(std::shared_ptr dn) { - auto m = continuation::Pipeline::Create(); - auto &s = m->state(); - s.reader_ = std::make_shared(BlockReaderOptions(), dn); - return m; - } -}; - - -} - -#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index 8c2781b035908..33d245aabc89e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -117,6 +117,4 @@ class RemoteBlockReader }; } -#include "remote_block_reader_impl.h" - #endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc index 098e0bb189633..80f2af5d05459 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc @@ -15,7 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "block_reader.h" +#include "reader/block_reader.h" +#include "reader/datatransfer.h" +#include "common/continuation/continuation.h" +#include "common/continuation/asio.h" + +#include namespace hdfs { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h deleted file mode 100644 index 003d924dd5614..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader_impl.h +++ /dev/null @@ -1,38 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#ifndef LIBHDFSPP_READER_REMOTE_BLOCK_READER_IMPL_H_ -#define LIBHDFSPP_READER_REMOTE_BLOCK_READER_IMPL_H_ - -#include "datatransfer.h" -#include "common/continuation/asio.h" -#include "common/continuation/async_stream.h" -#include "common/continuation/protobuf.h" - -#include -#include -#include - -#include - -#include - -namespace hdfs { - -} - -#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index d4de57e2dd8fd..949130effd968 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -21,6 +21,7 @@ #include "datatransfer.pb.h" #include "common/util.h" #include "reader/block_reader.h" +#include "reader/datatransfer.h" #include #include From d5baa8784643bdfed454c8a4ba0edb102d73f40a Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 10:00:50 -0500 Subject: [PATCH 30/50] Moved DN to its own file --- .../main/native/libhdfspp/lib/CMakeLists.txt | 1 + .../libhdfspp/lib/connection/CMakeLists.txt | 2 + .../lib/connection/datanodeconnection.cc | 35 ++++++++ .../lib/connection/datanodeconnection.h | 81 +++++++++++++++++++ .../native/libhdfspp/lib/fs/filesystem.cc | 12 --- .../main/native/libhdfspp/lib/fs/filesystem.h | 43 ---------- .../native/libhdfspp/lib/fs/inputstream.cc | 12 +-- .../libhdfspp/lib/reader/block_reader.h | 10 +-- .../libhdfspp/lib/reader/datatransfer.h | 5 +- .../lib/reader/remote_block_reader.cc | 22 ++--- .../native/libhdfspp/tests/CMakeLists.txt | 6 +- .../libhdfspp/tests/inputstream_test.cc | 15 ++-- .../tests/remote_block_reader_test.cc | 15 ++-- 13 files changed, 159 insertions(+), 100 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/CMakeLists.txt create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/CMakeLists.txt index 434dc4e8eff8d..c8515979d75ce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/CMakeLists.txt @@ -21,4 +21,5 @@ add_subdirectory(fs) add_subdirectory(reader) add_subdirectory(rpc) add_subdirectory(proto) +add_subdirectory(connection) add_subdirectory(bindings) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/CMakeLists.txt new file mode 100644 index 0000000000000..54ba96c6b825f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/CMakeLists.txt @@ -0,0 +1,2 @@ +add_library(connection datanodeconnection.cc) +add_dependencies(connection proto) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.cc new file mode 100644 index 0000000000000..1b2cc3fd0288a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.cc @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "datanodeconnection.h" +#include "common/util.h" + +namespace hdfs { + +void DataNodeConnectionImpl::Connect( + std::function dn)> handler) { + // Keep the DN from being freed until we're done + auto shared_this = shared_from_this(); + asio::async_connect(*conn_, endpoints_.begin(), endpoints_.end(), + [shared_this, handler](const asio::error_code &ec, std::array::iterator it) { + (void)it; + handler(ToStatus(ec), shared_this); }); +} + + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h new file mode 100644 index 0000000000000..ac20ef5cafb74 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBHDFSPP_LIB_CONNECTION_DATANODECONNECTION_H_ +#define LIBHDFSPP_LIB_CONNECTION_DATANODECONNECTION_H_ + +#include "common/hdfs_public_api.h" +#include "common/async_stream.h" +#include "ClientNamenodeProtocol.pb.h" + +#include "asio.hpp" + +namespace hdfs { + +class DataNodeConnection : public AsyncStream { +public: + std::string uuid_; + std::unique_ptr token_; + + virtual void Connect(std::function dn)> handler) = 0; +}; + + +class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_shared_from_this { +public: + std::unique_ptr conn_; + std::array endpoints_; + std::string uuid_; + + + DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto, + const hadoop::common::TokenProto *token) { + using namespace ::asio::ip; + + conn_.reset(new tcp::socket(*io_service)); + auto datanode_addr = dn_proto.id(); + endpoints_[0] = tcp::endpoint(address::from_string(datanode_addr.ipaddr()), + datanode_addr.xferport()); + uuid_ = dn_proto.id().datanodeuuid(); + + if (token) { + token_.reset(new hadoop::common::TokenProto()); + token_->CheckTypeAndMergeFrom(*token); + } + } + + void Connect(std::function dn)> handler) override; + + virtual void async_read(const asio::mutable_buffers_1 & buffers, + std::function completed_handler) + { asio::async_read(*conn_, buffers, completed_handler); } + + virtual void async_read(const asio::mutable_buffers_1 & buffers, + std::function completion_handler, + std::function completed_handler) + { asio::async_read(*conn_, buffers, completion_handler, completed_handler); } + virtual void async_write(const asio::const_buffers_1 & buffers, + std::function handler) + { asio::async_write(*conn_, buffers, handler); } +}; + +} + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index fd4bbae68aa23..6dec8e63dad56 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -99,18 +99,6 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, } -void DataNodeConnectionImpl::Connect( - std::function dn)> handler) { - // Keep the DN from being freed until we're done - auto shared_this = shared_from_this(); - asio::async_connect(*conn_, endpoints_.begin(), endpoints_.end(), - [shared_this, handler](const asio::error_code &ec, std::array::iterator it) { - (void)it; - handler(ToStatus(ec), shared_this); }); -} - - - FileSystem::~FileSystem() {} void FileSystem::New( diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 41830f12501ed..875c5964264f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -93,49 +93,6 @@ class FileSystemImpl : public FileSystem { }; -class DataNodeConnection : public AsyncStream { -public: - std::string uuid_; - - virtual void Connect(std::function dn)> handler) = 0; -}; - - -class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_shared_from_this { -public: - std::unique_ptr conn_; - std::array endpoints_; - std::string uuid_; - - - DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto) { - using namespace ::asio::ip; - - conn_.reset(new tcp::socket(*io_service)); - auto datanode_addr = dn_proto.id(); - endpoints_[0] = tcp::endpoint(address::from_string(datanode_addr.ipaddr()), - datanode_addr.xferport()); - uuid_ = dn_proto.id().datanodeuuid(); - } - - void Connect(std::function dn)> handler) override; - - virtual void async_read(const asio::mutable_buffers_1 & buffers, - std::function completed_handler) - { asio::async_read(*conn_, buffers, completed_handler); } - - virtual void async_read(const asio::mutable_buffers_1 & buffers, - std::function completion_handler, - std::function completed_handler) - { asio::async_read(*conn_, buffers, completion_handler, completed_handler); } - virtual void async_write(const asio::const_buffers_1 & buffers, - std::function handler) - { asio::async_write(*conn_, buffers, handler); } -}; - /* * ReadOperation: given DN connection, does one-shot reads. * diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index 81f78f1f25e03..3324e5f31be14 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -40,27 +40,21 @@ void InputStreamImpl::PositionRead( struct ReadOperation::HandshakeContinuation : continuation::Continuation { HandshakeContinuation(BlockReader *reader, const std::string &client_name, - const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset) : reader_(reader), client_name_(client_name), length_(length), offset_(offset) { - if (token) { - token_.reset(new hadoop::common::TokenProto()); - token_->CheckTypeAndMergeFrom(*token); - } block_.CheckTypeAndMergeFrom(*block); } virtual void Run(const Next &next) override { - reader_->async_request_block(client_name_, token_.get(), &block_, length_, + reader_->async_request_block(client_name_, &block_, length_, offset_, next); } private: BlockReader *reader_; const std::string client_name_; - std::unique_ptr token_; hadoop::hdfs::ExtendedBlockProto block_; uint64_t length_; uint64_t offset_; @@ -140,7 +134,7 @@ void InputStreamImpl::AsyncPreadSome( targetBlock.b().numbytes() - offset_within_block, asio::buffer_size(buffers)); // This is where we will put the logic for re-using a DN connection - dn_ = std::make_shared(io_service_, *chosen_dn); + dn_ = std::make_shared(io_service_, *chosen_dn, nullptr); std::string dn_id = dn_->uuid_; std::shared_ptr reader; @@ -176,7 +170,7 @@ void ReadOperation::AsyncReadBlock( size_t size = asio::buffer_size(buffers); - m->Push(new HandshakeContinuation(reader, client_name, nullptr, + m->Push(new HandshakeContinuation(reader, client_name, &block.b(), size, offset)) .Push(new ReadBlockContinuation(reader, buffers, bytesTransferred)); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index 33d245aabc89e..9c08823a3f515 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -21,6 +21,7 @@ #include "libhdfspp/status.h" #include "common/async_stream.h" #include "datatransfer.pb.h" +#include "connection/datanodeconnection.h" #include @@ -62,7 +63,6 @@ class BlockReader { const std::function &handler) = 0; virtual void async_request_block(const std::string &client_name, - const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler) = 0; @@ -71,8 +71,8 @@ class BlockReader { class RemoteBlockReader : public BlockReader, public std::enable_shared_from_this { public: - explicit RemoteBlockReader(const BlockReaderOptions &options, std::shared_ptr stream) - : stream_(stream), state_(kOpen), options_(options), + explicit RemoteBlockReader(const BlockReaderOptions &options, std::shared_ptr dn) + : dn_(dn), state_(kOpen), options_(options), chunk_padding_bytes_(0) {} void async_read_packet(const MutableBuffers &buffers, @@ -81,12 +81,10 @@ class RemoteBlockReader size_t read_packet(const MutableBuffers &buffers, Status *status); Status request_block(const std::string &client_name, - const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset); void async_request_block(const std::string &client_name, - const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler) override; @@ -105,7 +103,7 @@ class RemoteBlockReader kFinished, }; - std::shared_ptr stream_; + std::shared_ptr dn_; hadoop::hdfs::PacketHeaderProto header_; State state_; BlockReaderOptions options_; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h index bcacbc3b259b0..4dffd1f866251 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h @@ -20,6 +20,7 @@ #include "common/sasl_authenticator.h" #include "common/async_stream.h" +#include "connection/datanodeconnection.h" #include @@ -35,7 +36,7 @@ enum Operation { kReadBlock = 81, }; -template class DataTransferSaslStream : public AsyncStream { +template class DataTransferSaslStream : public DataNodeConnection { public: DataTransferSaslStream(std::shared_ptr stream, const std::string &username, const std::string &password) @@ -58,6 +59,8 @@ template class DataTransferSaslStream : public AsyncStream { std::function handler) { stream_->async_write(buffers, handler); } + void Connect(std::function dn)> handler) override + {(void)handler; /*TODO: Handshaking goes here*/}; private: DataTransferSaslStream(const DataTransferSaslStream &) = delete; DataTransferSaslStream &operator=(const DataTransferSaslStream &) = delete; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc index 80f2af5d05459..48c4f4f61c0cd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc @@ -50,7 +50,7 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, } void RemoteBlockReader::async_request_block( - const std::string &client_name, const hadoop::common::TokenProto *token, + const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler) { // The total number of bytes that we need to transfer from the DN is @@ -71,14 +71,14 @@ void RemoteBlockReader::async_request_block( s->header.insert(s->header.begin(), {0, kDataTransferVersion, Operation::kReadBlock}); s->request = std::move(ReadBlockProto(client_name, options_.verify_checksum, - token, block, length, offset)); + dn_->token_.get(), block, length, offset)); auto read_pb_message = new continuation::ReadDelimitedPBMessageContinuation( - stream_, &s->response); + dn_, &s->response); - m->Push(async_stream_continuation::Write(stream_, asio::buffer(s->header))) - .Push(asio_continuation::WriteDelimitedPBMessage(stream_, &s->request)) + m->Push(async_stream_continuation::Write(dn_, asio::buffer(s->header))) + .Push(asio_continuation::WriteDelimitedPBMessage(dn_, &s->request)) .Push(read_pb_message); m->Run([this, handler, offset](const Status &status, const State &s) { Status stat = status; @@ -99,12 +99,12 @@ void RemoteBlockReader::async_request_block( } Status RemoteBlockReader::request_block( - const std::string &client_name, const hadoop::common::TokenProto *token, + const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset) { auto stat = std::make_shared>(); std::future future(stat->get_future()); - async_request_block(client_name, token, block, length, offset, + async_request_block(client_name, block, length, offset, [stat](const Status &status) { stat->set_value(status); }); return future.get(); } @@ -137,7 +137,7 @@ struct RemoteBlockReader::ReadPacketHeader next(status); }; - parent_->stream_->async_read(asio::buffer(buf_), + parent_->dn_->async_read(asio::buffer(buf_), std::bind(&ReadPacketHeader::CompletionHandler, this, std::placeholders::_1, std::placeholders::_2), handler); @@ -195,7 +195,7 @@ struct RemoteBlockReader::ReadChecksum : continuation::Continuation { }; parent->checksum_.resize(parent->packet_len_ - sizeof(int) - parent->header_.datalen()); - parent->stream_->async_read(asio::buffer(parent->checksum_), + parent->dn_->async_read(asio::buffer(parent->checksum_), handler); } @@ -233,7 +233,7 @@ struct RemoteBlockReader::ReadData : continuation::Continuation { auto data_len = parent_->header_.datalen() - parent_->packet_data_read_bytes_; - parent_->stream_->async_read(buf_, asio::transfer_exactly(data_len), + parent_->dn_->async_read(buf_, asio::transfer_exactly(data_len), handler); } @@ -294,7 +294,7 @@ struct RemoteBlockReader::AckRead : continuation::Continuation { : hadoop::hdfs::Status::SUCCESS); m->Push( - continuation::WriteDelimitedPBMessage(parent_->stream_, &m->state())); + continuation::WriteDelimitedPBMessage(parent_->dn_, &m->state())); m->Run([this, next](const Status &status, const hadoop::hdfs::ClientReadStatusProto &) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt index eca878e1d249a..7615ba0394f27 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt @@ -26,7 +26,7 @@ protobuf_generate_cpp(PROTO_TEST_SRCS PROTO_TEST_HDRS ) add_executable(remote_block_reader_test remote_block_reader_test.cc $) -target_link_libraries(remote_block_reader_test reader proto common ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT}) +target_link_libraries(remote_block_reader_test reader proto common connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT}) add_test(remote_block_reader remote_block_reader_test) add_executable(sasl_digest_md5_test sasl_digest_md5_test.cc) @@ -34,10 +34,10 @@ target_link_libraries(sasl_digest_md5_test common ${OPENSSL_LIBRARIES} gmock_mai add_test(sasl_digest_md5 sasl_digest_md5_test) add_executable(inputstream_test inputstream_test.cc) -target_link_libraries(inputstream_test fs rpc reader proto common ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT}) +target_link_libraries(inputstream_test fs rpc reader proto common connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT}) add_test(inputstream inputstream_test) include_directories(${CMAKE_CURRENT_BINARY_DIR}) add_executable(rpc_engine_test rpc_engine_test.cc ${PROTO_TEST_SRCS} ${PROTO_TEST_HDRS} $) -target_link_libraries(rpc_engine_test rpc proto common ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT}) +target_link_libraries(rpc_engine_test rpc proto common connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT}) add_test(rpc_engine rpc_engine_test) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index c4963b353c058..4bdd8c6d87a1c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -43,9 +43,8 @@ class MockReader : public BlockReader { void(const asio::mutable_buffers_1 &, const std::function &)); - MOCK_METHOD6(async_request_block, + MOCK_METHOD5(async_request_block, void(const std::string &client_name, - const hadoop::common::TokenProto *token, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler)); @@ -64,8 +63,8 @@ TEST(InputStreamTest, TestReadSingleTrunk) { Status stat; size_t read = 0; MockReader reader; - EXPECT_CALL(reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); + EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) + .WillOnce(InvokeArgument<4>(Status::OK())); EXPECT_CALL(reader, async_read_packet(_, _)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); @@ -89,8 +88,8 @@ TEST(InputStreamTest, TestReadMultipleTrunk) { size_t read = 0; MockReader reader; - EXPECT_CALL(reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); + EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) + .WillOnce(InvokeArgument<4>(Status::OK())); EXPECT_CALL(reader, async_read_packet(_, _)) .Times(4) @@ -115,8 +114,8 @@ TEST(InputStreamTest, TestReadError) { Status stat; size_t read = 0; MockReader reader; - EXPECT_CALL(reader, async_request_block(_, _, _, _, _, _)) - .WillOnce(InvokeArgument<5>(Status::OK())); + EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) + .WillOnce(InvokeArgument<4>(Status::OK())); EXPECT_CALL(reader, async_read_packet(_, _)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 949130effd968..eff2d48170794 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -52,12 +52,13 @@ namespace pbio = pb::io; namespace hdfs { -class MockDNConnection : public MockConnectionBase, public AsyncStream{ +class MockDNConnection : public MockConnectionBase, public DataNodeConnection{ public: MockDNConnection(::asio::io_service &io_service) : MockConnectionBase(&io_service) {} MOCK_METHOD0(Produce, ProducerResult()); + MOCK_METHOD1(Connect, void(std::function dn)>)); // Satisfy the AsyncStream contract, delegating to MocKConnectionBase void async_read(const asio::mutable_buffers_1 & buf, @@ -121,13 +122,13 @@ ProducePacket(const std::string &data, const std::string &checksum, template static std::shared_ptr -ReadContent(std::shared_ptr conn, TokenProto *token, const ExtendedBlockProto &block, +ReadContent(std::shared_ptr conn, const ExtendedBlockProto &block, uint64_t length, uint64_t offset, const mutable_buffers_1 &buf, const Handler &handler) { BlockReaderOptions options; auto reader = std::make_shared(options, conn); Status result; - reader->async_request_block("libhdfs++", token, &block, length, offset, + reader->async_request_block("libhdfs++", &block, length, offset, [buf, reader, handler](const Status &stat) { if (!stat.ok()) { handler(stat, 0); @@ -157,7 +158,7 @@ TEST(RemoteBlockReaderTest, TestReadWholeBlock) { bool done = false; std::string data(kChunkSize, 0); - ReadContent(conn, nullptr, block, kChunkSize, 0, + ReadContent(conn, block, kChunkSize, 0, buffer(const_cast(data.c_str()), data.size()), [&data, &io_service, &done](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); @@ -199,7 +200,7 @@ TEST(RemoteBlockReaderTest, TestReadWithinChunk) { bool done = false; string data(kLength, 0); - ReadContent(conn, nullptr, block, data.size(), kOffset, + ReadContent(conn, block, data.size(), kOffset, buffer(const_cast(data.c_str()), data.size()), [&data, &io_service,&done](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); @@ -237,7 +238,7 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { auto reader = std::make_shared(options, conn); Status result; reader->async_request_block( - "libhdfs++", nullptr, &block, data.size(), 0, + "libhdfs++", &block, data.size(), 0, [buf, reader, &data, &io_service](const Status &stat) { ASSERT_TRUE(stat.ok()); reader->async_read_packet( @@ -296,7 +297,7 @@ TEST(RemoteBlockReaderTest, TestSaslConnection) { sasl_conn->Handshake([sasl_conn, &block, &data, &io_service]( const Status &s) { ASSERT_TRUE(s.ok()); - ReadContent(sasl_conn, nullptr, block, kChunkSize, 0, + ReadContent(sasl_conn, block, kChunkSize, 0, buffer(const_cast(data.c_str()), data.size()), [&data, &io_service](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); From 9ae3112f0ac693d8cf833fcdfcbe432cd7e23c13 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 11:16:01 -0500 Subject: [PATCH 31/50] Moved AsyncReadBlock to blockreader --- .../main/native/libhdfspp/lib/common/util.h | 17 +++ .../native/libhdfspp/lib/fs/filesystem.cc | 4 +- .../main/native/libhdfspp/lib/fs/filesystem.h | 24 +--- .../native/libhdfspp/lib/fs/inputstream.cc | 83 +------------- .../libhdfspp/lib/reader/block_reader.h | 17 +++ .../native/libhdfspp/lib/reader/fileinfo.h | 36 ++++++ .../lib/reader/remote_block_reader.cc | 84 ++++++++++++++ .../native/libhdfspp/lib/rpc/rpc_engine.cc | 14 --- .../libhdfspp/tests/inputstream_test.cc | 94 ++-------------- .../tests/remote_block_reader_test.cc | 106 +++++++++++++++++- 10 files changed, 273 insertions(+), 206 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/fileinfo.h diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h index ff9f36c8891cf..b93ce9d69e780 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h @@ -20,7 +20,11 @@ #include "libhdfspp/status.h" + +#include + #include +#include #include #include @@ -53,6 +57,19 @@ static inline void ReadDelimitedPBMessage( std::string Base64Encode(const std::string &src); +static inline std::string GetRandomClientName() { + unsigned char buf[6] = { + 0, + }; + RAND_pseudo_bytes(buf, sizeof(buf)); + + std::stringstream ss; + ss << "libhdfs++_" + << Base64Encode(std::string(reinterpret_cast(buf), sizeof(buf))); + return ss.str(); +} + + } #endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 6dec8e63dad56..5041097b5a43a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -118,9 +118,9 @@ void FileSystem::New( FileSystemImpl::FileSystemImpl(IoService *io_service, const Options &options) : io_service_(static_cast(io_service)), - client_name_(RpcEngine::GetRandomClientName()), + client_name_(GetRandomClientName()), nn_(&io_service_->io_service(), options, - RpcEngine::GetRandomClientName(), kNamenodeProtocol, + GetRandomClientName(), kNamenodeProtocol, kNamenodeProtocolVersion) {} void FileSystemImpl::Connect(const std::string &server, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 875c5964264f6..f1081f1fc96ce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -23,6 +23,7 @@ #include "libhdfspp/hdfs.h" #include "rpc/rpc_engine.h" #include "reader/block_reader.h" +#include "reader/fileinfo.h" #include "ClientNamenodeProtocol.pb.h" #include "ClientNamenodeProtocol.hrpc.inl" @@ -30,15 +31,6 @@ namespace hdfs { -/** - * Information that is assumed to be unchaning about a file for the duration of - * the operations. - */ -struct FileInfo { - unsigned long long file_length_; - std::vector<::hadoop::hdfs::LocatedBlockProto> blocks_; -}; - /** * NameNodeConnection: abstracts the details of communicating with a NameNode * and the implementation of the communications protocol. @@ -116,24 +108,10 @@ class InputStreamImpl : public InputStream { ::asio::io_service *io_service_; const std::string client_name_; const std::shared_ptr file_info_; - struct RemoteBlockReaderTrait; std::shared_ptr dn_; // The last DN connected to }; -class ReadOperation { -public: - static void AsyncReadBlock( - BlockReader * reader, - const std::string & client_name, - const hadoop::hdfs::LocatedBlockProto &block, size_t offset, - const MutableBuffers &buffers, - const std::function handler); -private: - struct HandshakeContinuation; - struct ReadBlockContinuation; -}; - } #endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index 3324e5f31be14..fed78e1773cd2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -38,64 +38,6 @@ void InputStreamImpl::PositionRead( AsyncPreadSome(offset, asio::buffer(buf, nbyte), excluded_datanodes, handler); } -struct ReadOperation::HandshakeContinuation : continuation::Continuation { - HandshakeContinuation(BlockReader *reader, const std::string &client_name, - const hadoop::hdfs::ExtendedBlockProto *block, - uint64_t length, uint64_t offset) - : reader_(reader), client_name_(client_name), length_(length), - offset_(offset) { - block_.CheckTypeAndMergeFrom(*block); - } - - virtual void Run(const Next &next) override { - reader_->async_request_block(client_name_, &block_, length_, - offset_, next); - } - -private: - BlockReader *reader_; - const std::string client_name_; - hadoop::hdfs::ExtendedBlockProto block_; - uint64_t length_; - uint64_t offset_; -}; - -struct ReadOperation::ReadBlockContinuation : continuation::Continuation { - ReadBlockContinuation(BlockReader *reader, MutableBuffers buffer, - size_t *transferred) - : reader_(reader), buffer_(buffer), - buffer_size_(asio::buffer_size(buffer)), transferred_(transferred) { - } - - virtual void Run(const Next &next) override { - *transferred_ = 0; - next_ = next; - OnReadData(Status::OK(), 0); - } - -private: - BlockReader *reader_; - const MutableBuffers buffer_; - const size_t buffer_size_; - size_t *transferred_; - std::function next_; - - void OnReadData(const Status &status, size_t transferred) { - using std::placeholders::_1; - using std::placeholders::_2; - *transferred_ += transferred; - if (!status.ok()) { - next_(status); - } else if (*transferred_ >= buffer_size_) { - next_(status); - } else { - reader_->async_read_packet( - asio::buffer(buffer_ + *transferred_, buffer_size_ - *transferred_), - std::bind(&ReadBlockContinuation::OnReadData, this, _1, _2)); - } - } -}; - void InputStreamImpl::AsyncPreadSome( size_t offset, const MutableBuffers &buffers, const std::set &excluded_datanodes, @@ -148,7 +90,7 @@ void InputStreamImpl::AsyncPreadSome( (Status status, std::shared_ptr dn) { (void)dn; if (status.ok()) { - ReadOperation::AsyncReadBlock( + reader->AsyncReadBlock( reader.get(), client_name_, targetBlock, offset_within_block, asio::buffer(buffers, size_within_block), read_handler); } else { @@ -157,27 +99,4 @@ void InputStreamImpl::AsyncPreadSome( }); } -void ReadOperation::AsyncReadBlock( - BlockReader * reader, - const std::string & client_name, - const hadoop::hdfs::LocatedBlockProto &block, - size_t offset, - const MutableBuffers &buffers, - const std::function handler) { - - auto m = continuation::Pipeline::Create(); - size_t * bytesTransferred = &m->state(); - - size_t size = asio::buffer_size(buffers); - - m->Push(new HandshakeContinuation(reader, client_name, - &block.b(), size, offset)) - .Push(new ReadBlockContinuation(reader, buffers, bytesTransferred)); - - m->Run([handler] (const Status &status, - const size_t totalBytesTransferred) { - handler(status, totalBytesTransferred); - }); -} - } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index 9c08823a3f515..c603c1e6dc69a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -66,6 +66,13 @@ class BlockReader { const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler) = 0; + + virtual void AsyncReadBlock( + BlockReader * reader, + const std::string & client_name, + const hadoop::hdfs::LocatedBlockProto &block, size_t offset, + const MutableBuffers &buffers, + const std::function handler) = 0; }; class RemoteBlockReader @@ -88,7 +95,17 @@ class RemoteBlockReader const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler) override; + + void AsyncReadBlock( + BlockReader * reader, + const std::string & client_name, + const hadoop::hdfs::LocatedBlockProto &block, size_t offset, + const MutableBuffers &buffers, + const std::function handler); private: + struct RequestBlockContinuation; + struct ReadBlockContinuation; + struct ReadPacketHeader; struct ReadChecksum; struct ReadPadding; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/fileinfo.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/fileinfo.h new file mode 100644 index 0000000000000..ad10165d5e546 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/fileinfo.h @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIB_READER_FILEINFO_H_ +#define LIB_READER_FILEINFO_H_ + +#include "ClientNamenodeProtocol.pb.h" + +namespace hdfs { + +/** + * Information that is assumed to be unchanging about a file for the duration of + * the operations. + */ +struct FileInfo { + unsigned long long file_length_; + std::vector<::hadoop::hdfs::LocatedBlockProto> blocks_; +}; + +} + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc index 48c4f4f61c0cd..bf8b6ad157056 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc @@ -350,4 +350,88 @@ RemoteBlockReader::read_packet(const MutableBuffers &buffers, return transferred; } + +struct RemoteBlockReader::RequestBlockContinuation : continuation::Continuation { + RequestBlockContinuation(BlockReader *reader, const std::string &client_name, + const hadoop::hdfs::ExtendedBlockProto *block, + uint64_t length, uint64_t offset) + : reader_(reader), client_name_(client_name), length_(length), + offset_(offset) { + block_.CheckTypeAndMergeFrom(*block); + } + + virtual void Run(const Next &next) override { + reader_->async_request_block(client_name_, &block_, length_, + offset_, next); + } + +private: + BlockReader *reader_; + const std::string client_name_; + hadoop::hdfs::ExtendedBlockProto block_; + uint64_t length_; + uint64_t offset_; +}; + +struct RemoteBlockReader::ReadBlockContinuation : continuation::Continuation { + ReadBlockContinuation(BlockReader *reader, MutableBuffers buffer, + size_t *transferred) + : reader_(reader), buffer_(buffer), + buffer_size_(asio::buffer_size(buffer)), transferred_(transferred) { + } + + virtual void Run(const Next &next) override { + *transferred_ = 0; + next_ = next; + OnReadData(Status::OK(), 0); + } + +private: + BlockReader *reader_; + const MutableBuffers buffer_; + const size_t buffer_size_; + size_t *transferred_; + std::function next_; + + void OnReadData(const Status &status, size_t transferred) { + using std::placeholders::_1; + using std::placeholders::_2; + *transferred_ += transferred; + if (!status.ok()) { + next_(status); + } else if (*transferred_ >= buffer_size_) { + next_(status); + } else { + reader_->async_read_packet( + asio::buffer(buffer_ + *transferred_, buffer_size_ - *transferred_), + std::bind(&ReadBlockContinuation::OnReadData, this, _1, _2)); + } + } +}; + +void RemoteBlockReader::AsyncReadBlock( + BlockReader * reader, + const std::string & client_name, + const hadoop::hdfs::LocatedBlockProto &block, + size_t offset, + const MutableBuffers &buffers, + const std::function handler) { + + auto m = continuation::Pipeline::Create(); + size_t * bytesTransferred = &m->state(); + + size_t size = asio::buffer_size(buffers); + + m->Push(new RequestBlockContinuation(reader, client_name, + &block.b(), size, offset)) + .Push(new ReadBlockContinuation(reader, buffers, bytesTransferred)); + + m->Run([handler] (const Status &status, + const size_t totalBytesTransferred) { + handler(status, totalBytesTransferred); + }); +} + + + } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc index 83721a76c5dbb..29d455fa65293 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc @@ -19,9 +19,6 @@ #include "rpc_connection.h" #include "common/util.h" -#include - -#include #include namespace hdfs { @@ -83,15 +80,4 @@ Status RpcEngine::RawRpc(const std::string &method_name, const std::string &req, return future.get(); } -std::string RpcEngine::GetRandomClientName() { - unsigned char buf[6] = { - 0, - }; - RAND_pseudo_bytes(buf, sizeof(buf)); - - std::stringstream ss; - ss << "libhdfs++_" - << Base64Encode(std::string(reinterpret_cast(buf), sizeof(buf))); - return ss.str(); -} } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 4bdd8c6d87a1c..b6a3eb34da493 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -17,6 +17,8 @@ */ #include "fs/filesystem.h" +#include "common/util.h" + #include #include @@ -37,7 +39,6 @@ namespace hdfs { class MockReader : public BlockReader { public: - virtual ~MockReader() {} MOCK_METHOD2( async_read_packet, void(const asio::mutable_buffers_1 &, @@ -48,90 +49,15 @@ class MockReader : public BlockReader { const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler)); -}; - -} - -TEST(InputStreamTest, TestReadSingleTrunk) { - auto file_info = std::make_shared(); - LocatedBlocksProto blocks; - LocatedBlockProto block; - char buf[4096] = { - 0, - }; - - Status stat; - size_t read = 0; - MockReader reader; - EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) - .WillOnce(InvokeArgument<4>(Status::OK())); - EXPECT_CALL(reader, async_read_packet(_, _)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); - - ReadOperation::AsyncReadBlock( - &reader, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), - [&stat, &read](const Status &status, size_t transferred) { - stat = status; - read = transferred; - }); - ASSERT_TRUE(stat.ok()); - ASSERT_EQ(sizeof(buf), read); - read = 0; -} - -TEST(InputStreamTest, TestReadMultipleTrunk) { - LocatedBlockProto block; - char buf[4096] = { - 0, - }; - Status stat; - size_t read = 0; - MockReader reader; - EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) - .WillOnce(InvokeArgument<4>(Status::OK())); - - EXPECT_CALL(reader, async_read_packet(_, _)) - .Times(4) - .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); - - ReadOperation::AsyncReadBlock( - &reader, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), - [&stat, &read](const Status &status, size_t transferred) { - stat = status; - read = transferred; - }); - ASSERT_TRUE(stat.ok()); - ASSERT_EQ(sizeof(buf), read); - read = 0; -} - -TEST(InputStreamTest, TestReadError) { - LocatedBlockProto block; - char buf[4096] = { - 0, - }; - Status stat; - size_t read = 0; - MockReader reader; - EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) - .WillOnce(InvokeArgument<4>(Status::OK())); - - EXPECT_CALL(reader, async_read_packet(_, _)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) - .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) - .WillOnce(InvokeArgument<1>(Status::Error("error"), 0)); + MOCK_METHOD6(AsyncReadBlock, void( + BlockReader * reader, + const std::string & client_name, + const hadoop::hdfs::LocatedBlockProto &block, size_t offset, + const MutableBuffers &buffers, + const std::function handler)); +}; - ReadOperation::AsyncReadBlock( - &reader, RpcEngine::GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), - [&stat, &read](const Status &status, size_t transferred) { - stat = status; - read = transferred; - }); - ASSERT_FALSE(stat.ok()); - ASSERT_EQ(sizeof(buf) / 4 * 3, read); - read = 0; } TEST(InputStreamTest, TestExcludeDataNode) { @@ -153,7 +79,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { 0, }; IoServiceImpl io_service; - InputStreamImpl is(&io_service.io_service(), RpcEngine::GetRandomClientName(), file_info); + InputStreamImpl is(&io_service.io_service(), GetRandomClientName(), file_info); Status stat; size_t read = 0; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index eff2d48170794..0cc006b4515f7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -22,6 +22,7 @@ #include "common/util.h" #include "reader/block_reader.h" #include "reader/datatransfer.h" +#include "reader/fileinfo.h" #include #include @@ -37,6 +38,8 @@ using ::hadoop::hdfs::DataTransferEncryptorMessageProto; using ::hadoop::hdfs::ExtendedBlockProto; using ::hadoop::hdfs::PacketHeaderProto; using ::hadoop::hdfs::ReadOpChecksumInfoProto; +using ::hadoop::hdfs::LocatedBlockProto; +using ::hadoop::hdfs::LocatedBlocksProto; using ::asio::buffer; using ::asio::error_code; @@ -79,9 +82,28 @@ class MockDNConnection : public MockConnectionBase, public DataNodeConnection{ std::function handler) { asio::async_write(*this, buf, handler); } +}; + +// Mocks async_read_packet and async_request_block but not AsyncReadBlock, so we +// can test the logic of AsyncReadBlock +class PartialMockReader : public RemoteBlockReader { +public: + PartialMockReader() : + RemoteBlockReader(BlockReaderOptions(), std::shared_ptr()) {}; - + MOCK_METHOD2( + async_read_packet, + void(const asio::mutable_buffers_1 &, + const std::function &)); + + MOCK_METHOD5(async_request_block, + void(const std::string &client_name, + const hadoop::hdfs::ExtendedBlockProto *block, + uint64_t length, uint64_t offset, + const std::function &handler)); }; + + } static inline string ToDelimitedString(const pb::MessageLite *msg) { @@ -120,6 +142,88 @@ ProducePacket(const std::string &data, const std::string &checksum, return std::make_pair(error_code(), std::move(payload)); } +TEST(RemoteBlockReaderTest, TestReadSingleTrunk) { + auto file_info = std::make_shared(); + LocatedBlocksProto blocks; + LocatedBlockProto block; + char buf[4096] = { + 0, + }; + + Status stat; + size_t read = 0; + PartialMockReader reader; + EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) + .WillOnce(InvokeArgument<4>(Status::OK())); + EXPECT_CALL(reader, async_read_packet(_, _)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); + + reader.AsyncReadBlock( + &reader, GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + [&stat, &read](const Status &status, size_t transferred) { + stat = status; + read = transferred; + }); + ASSERT_TRUE(stat.ok()); + ASSERT_EQ(sizeof(buf), read); + read = 0; +} + +TEST(RemoteBlockReaderTest, TestReadMultipleTrunk) { + LocatedBlockProto block; + char buf[4096] = { + 0, + }; + Status stat; + size_t read = 0; + + PartialMockReader reader; + EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) + .WillOnce(InvokeArgument<4>(Status::OK())); + + EXPECT_CALL(reader, async_read_packet(_, _)) + .Times(4) + .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); + + reader.AsyncReadBlock( + &reader, GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + [&stat, &read](const Status &status, size_t transferred) { + stat = status; + read = transferred; + }); + ASSERT_TRUE(stat.ok()); + ASSERT_EQ(sizeof(buf), read); + read = 0; +} + +TEST(RemoteBlockReaderTest, TestReadError) { + LocatedBlockProto block; + char buf[4096] = { + 0, + }; + Status stat; + size_t read = 0; + PartialMockReader reader; + EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) + .WillOnce(InvokeArgument<4>(Status::OK())); + + EXPECT_CALL(reader, async_read_packet(_, _)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) + .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) + .WillOnce(InvokeArgument<1>(Status::Error("error"), 0)); + + reader.AsyncReadBlock( + &reader, GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + [&stat, &read](const Status &status, size_t transferred) { + stat = status; + read = transferred; + }); + ASSERT_FALSE(stat.ok()); + ASSERT_EQ(sizeof(buf) / 4 * 3, read); + read = 0; +} + template static std::shared_ptr ReadContent(std::shared_ptr conn, const ExtendedBlockProto &block, From f180c994f46761a1c0069e555f3894ad06d2cb1b Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 11:30:29 -0500 Subject: [PATCH 32/50] Renamed RemoteBlockReader to BlockReaderImpl --- .../native/libhdfspp/lib/fs/inputstream.cc | 4 +- .../libhdfspp/lib/reader/CMakeLists.txt | 2 +- ...remote_block_reader.cc => block_reader.cc} | 59 +++++++++---------- .../libhdfspp/lib/reader/block_reader.h | 59 +++++++++++-------- .../libhdfspp/tests/inputstream_test.cc | 7 +-- .../tests/remote_block_reader_test.cc | 44 +++++++------- 6 files changed, 90 insertions(+), 85 deletions(-) rename hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/{remote_block_reader.cc => block_reader.cc} (89%) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index fed78e1773cd2..637e0eb23fda4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -80,7 +80,7 @@ void InputStreamImpl::AsyncPreadSome( std::string dn_id = dn_->uuid_; std::shared_ptr reader; - reader.reset(new RemoteBlockReader(BlockReaderOptions(), dn_)); + reader.reset(new BlockReaderImpl(BlockReaderOptions(), dn_)); auto read_handler = [dn_id, handler](const Status & status, size_t transferred) { handler(status, dn_id, transferred); @@ -91,7 +91,7 @@ void InputStreamImpl::AsyncPreadSome( (void)dn; if (status.ok()) { reader->AsyncReadBlock( - reader.get(), client_name_, targetBlock, offset_within_block, + client_name_, targetBlock, offset_within_block, asio::buffer(buffers, size_within_block), read_handler); } else { handler(status, "", 0); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/CMakeLists.txt index 71e28ac619901..0dcae29e88d18 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/CMakeLists.txt @@ -16,5 +16,5 @@ # limitations under the License. # -add_library(reader remote_block_reader.cc datatransfer.cc) +add_library(reader block_reader.cc datatransfer.cc) add_dependencies(reader proto) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc similarity index 89% rename from hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc rename to hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc index bf8b6ad157056..91598044e703a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/remote_block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc @@ -49,7 +49,7 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, return p; } -void RemoteBlockReader::async_request_block( +void BlockReaderImpl::AsyncRequestBlock( const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler) { @@ -98,13 +98,13 @@ void RemoteBlockReader::async_request_block( }); } -Status RemoteBlockReader::request_block( +Status BlockReaderImpl::RequestBlock( const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset) { auto stat = std::make_shared>(); std::future future(stat->get_future()); - async_request_block(client_name, block, length, offset, + AsyncRequestBlock(client_name, block, length, offset, [stat](const Status &status) { stat->set_value(status); }); return future.get(); } @@ -115,9 +115,9 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset); -struct RemoteBlockReader::ReadPacketHeader +struct BlockReaderImpl::ReadPacketHeader : continuation::Continuation { - ReadPacketHeader(RemoteBlockReader *parent) : parent_(parent) {} + ReadPacketHeader(BlockReaderImpl *parent) : parent_(parent) {} virtual void Run(const Next &next) override { parent_->packet_data_read_bytes_ = 0; @@ -151,7 +151,7 @@ struct RemoteBlockReader::ReadPacketHeader static const size_t kHeaderLenSize = sizeof(short); static const size_t kHeaderStart = kPayloadLenSize + kHeaderLenSize; - RemoteBlockReader *parent_; + BlockReaderImpl *parent_; std::array buf_; size_t packet_length() const { @@ -173,8 +173,8 @@ struct RemoteBlockReader::ReadPacketHeader } }; -struct RemoteBlockReader::ReadChecksum : continuation::Continuation { - ReadChecksum(RemoteBlockReader *parent) : parent_(parent) {} +struct BlockReaderImpl::ReadChecksum : continuation::Continuation { + ReadChecksum(BlockReaderImpl *parent) : parent_(parent) {} virtual void Run(const Next &next) override { auto parent = parent_; @@ -200,11 +200,11 @@ struct RemoteBlockReader::ReadChecksum : continuation::Continuation { } private: - RemoteBlockReader *parent_; + BlockReaderImpl *parent_; }; -struct RemoteBlockReader::ReadData : continuation::Continuation { - ReadData(RemoteBlockReader *parent, +struct BlockReaderImpl::ReadData : continuation::Continuation { + ReadData(BlockReaderImpl *parent, std::shared_ptr bytes_transferred, const asio::mutable_buffers_1 &buf) : parent_(parent), bytes_transferred_(bytes_transferred), buf_(buf) { @@ -238,13 +238,13 @@ struct RemoteBlockReader::ReadData : continuation::Continuation { } private: - RemoteBlockReader *parent_; + BlockReaderImpl *parent_; std::shared_ptr bytes_transferred_; const asio::mutable_buffers_1 buf_; }; -struct RemoteBlockReader::ReadPadding : continuation::Continuation { - ReadPadding(RemoteBlockReader *parent) +struct BlockReaderImpl::ReadPadding : continuation::Continuation { + ReadPadding(BlockReaderImpl *parent) : parent_(parent), padding_(parent->chunk_padding_bytes_), bytes_transferred_(std::make_shared(0)), read_data_(new ReadData( @@ -269,7 +269,7 @@ struct RemoteBlockReader::ReadPadding : continuation::Continuation { } private: - RemoteBlockReader *parent_; + BlockReaderImpl *parent_; std::vector padding_; std::shared_ptr bytes_transferred_; std::shared_ptr read_data_; @@ -278,8 +278,8 @@ struct RemoteBlockReader::ReadPadding : continuation::Continuation { }; -struct RemoteBlockReader::AckRead : continuation::Continuation { - AckRead(RemoteBlockReader *parent) : parent_(parent) {} +struct BlockReaderImpl::AckRead : continuation::Continuation { + AckRead(BlockReaderImpl *parent) : parent_(parent) {} virtual void Run(const Next &next) override { if (parent_->bytes_to_read_ > 0) { @@ -299,17 +299,17 @@ struct RemoteBlockReader::AckRead : continuation::Continuation { m->Run([this, next](const Status &status, const hadoop::hdfs::ClientReadStatusProto &) { if (status.ok()) { - parent_->state_ = RemoteBlockReader::kFinished; + parent_->state_ = BlockReaderImpl::kFinished; } next(status); }); } private: - RemoteBlockReader *parent_; + BlockReaderImpl *parent_; }; -void RemoteBlockReader::async_read_packet( +void BlockReaderImpl::AsyncReadPacket( const MutableBuffers &buffers, const std::function &handler) { assert(state_ != kOpen && "Not connected"); @@ -335,12 +335,12 @@ void RemoteBlockReader::async_read_packet( size_t -RemoteBlockReader::read_packet(const MutableBuffers &buffers, +BlockReaderImpl::ReadPacket(const MutableBuffers &buffers, Status *status) { size_t transferred = 0; auto done = std::make_shared>(); auto future = done->get_future(); - async_read_packet(buffers, + AsyncReadPacket(buffers, [status, &transferred, done](const Status &stat, size_t t) { *status = stat; transferred = t; @@ -351,7 +351,7 @@ RemoteBlockReader::read_packet(const MutableBuffers &buffers, } -struct RemoteBlockReader::RequestBlockContinuation : continuation::Continuation { +struct BlockReaderImpl::RequestBlockContinuation : continuation::Continuation { RequestBlockContinuation(BlockReader *reader, const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset) @@ -361,7 +361,7 @@ struct RemoteBlockReader::RequestBlockContinuation : continuation::Continuation } virtual void Run(const Next &next) override { - reader_->async_request_block(client_name_, &block_, length_, + reader_->AsyncRequestBlock(client_name_, &block_, length_, offset_, next); } @@ -373,7 +373,7 @@ struct RemoteBlockReader::RequestBlockContinuation : continuation::Continuation uint64_t offset_; }; -struct RemoteBlockReader::ReadBlockContinuation : continuation::Continuation { +struct BlockReaderImpl::ReadBlockContinuation : continuation::Continuation { ReadBlockContinuation(BlockReader *reader, MutableBuffers buffer, size_t *transferred) : reader_(reader), buffer_(buffer), @@ -402,15 +402,14 @@ struct RemoteBlockReader::ReadBlockContinuation : continuation::Continuation { } else if (*transferred_ >= buffer_size_) { next_(status); } else { - reader_->async_read_packet( + reader_->AsyncReadPacket( asio::buffer(buffer_ + *transferred_, buffer_size_ - *transferred_), std::bind(&ReadBlockContinuation::OnReadData, this, _1, _2)); } } }; -void RemoteBlockReader::AsyncReadBlock( - BlockReader * reader, +void BlockReaderImpl::AsyncReadBlock( const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, size_t offset, @@ -422,9 +421,9 @@ void RemoteBlockReader::AsyncReadBlock( size_t size = asio::buffer_size(buffers); - m->Push(new RequestBlockContinuation(reader, client_name, + m->Push(new RequestBlockContinuation(this, client_name, &block.b(), size, offset)) - .Push(new ReadBlockContinuation(reader, buffers, bytesTransferred)); + .Push(new ReadBlockContinuation(this, buffers, bytesTransferred)); m->Run([handler] (const Status &status, const size_t totalBytesTransferred) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index c603c1e6dc69a..ee68fc8f9342c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -59,49 +59,56 @@ struct BlockReaderOptions { class BlockReader { public: - virtual void async_read_packet(const MutableBuffers &buffers, - const std::function &handler) = 0; + virtual void AsyncReadPacket( + const MutableBuffers &buffers, + const std::function &handler) = 0; - virtual void async_request_block(const std::string &client_name, - const hadoop::hdfs::ExtendedBlockProto *block, - uint64_t length, uint64_t offset, - const std::function &handler) = 0; + virtual void AsyncRequestBlock( + const std::string &client_name, + const hadoop::hdfs::ExtendedBlockProto *block, + uint64_t length, + uint64_t offset, + const std::function &handler) = 0; virtual void AsyncReadBlock( - BlockReader * reader, const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, size_t offset, const MutableBuffers &buffers, const std::function handler) = 0; }; -class RemoteBlockReader - : public BlockReader, public std::enable_shared_from_this { +class BlockReaderImpl + : public BlockReader, public std::enable_shared_from_this { public: - explicit RemoteBlockReader(const BlockReaderOptions &options, std::shared_ptr dn) + explicit BlockReaderImpl(const BlockReaderOptions &options, std::shared_ptr dn) : dn_(dn), state_(kOpen), options_(options), chunk_padding_bytes_(0) {} - void async_read_packet(const MutableBuffers &buffers, - const std::function &handler) override; - - size_t read_packet(const MutableBuffers &buffers, Status *status); - - Status request_block(const std::string &client_name, - const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, - uint64_t offset); - - void async_request_block(const std::string &client_name, - const hadoop::hdfs::ExtendedBlockProto *block, - uint64_t length, uint64_t offset, - const std::function &handler) override; + virtual void AsyncReadPacket( + const MutableBuffers &buffers, + const std::function &handler) override; + + virtual void AsyncRequestBlock( + const std::string &client_name, + const hadoop::hdfs::ExtendedBlockProto *block, + uint64_t length, + uint64_t offset, + const std::function &handler) override; - void AsyncReadBlock( - BlockReader * reader, + virtual void AsyncReadBlock( const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, size_t offset, const MutableBuffers &buffers, - const std::function handler); + const std::function handler) override; + + size_t ReadPacket(const MutableBuffers &buffers, Status *status); + + Status RequestBlock( + const std::string &client_name, + const hadoop::hdfs::ExtendedBlockProto *block, + uint64_t length, + uint64_t offset); + private: struct RequestBlockContinuation; struct ReadBlockContinuation; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index b6a3eb34da493..9ac928740937a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -40,18 +40,17 @@ namespace hdfs { class MockReader : public BlockReader { public: MOCK_METHOD2( - async_read_packet, + AsyncReadPacket, void(const asio::mutable_buffers_1 &, const std::function &)); - MOCK_METHOD5(async_request_block, + MOCK_METHOD5(AsyncRequestBlock, void(const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler)); - MOCK_METHOD6(AsyncReadBlock, void( - BlockReader * reader, + MOCK_METHOD5(AsyncReadBlock, void( const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, size_t offset, const MutableBuffers &buffers, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 0cc006b4515f7..312ac46e4d20f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -84,19 +84,19 @@ class MockDNConnection : public MockConnectionBase, public DataNodeConnection{ } }; -// Mocks async_read_packet and async_request_block but not AsyncReadBlock, so we +// Mocks AsyncReadPacket and AsyncRequestBlock but not AsyncReadBlock, so we // can test the logic of AsyncReadBlock -class PartialMockReader : public RemoteBlockReader { +class PartialMockReader : public BlockReaderImpl { public: PartialMockReader() : - RemoteBlockReader(BlockReaderOptions(), std::shared_ptr()) {}; + BlockReaderImpl(BlockReaderOptions(), std::shared_ptr()) {}; MOCK_METHOD2( - async_read_packet, + AsyncReadPacket, void(const asio::mutable_buffers_1 &, const std::function &)); - MOCK_METHOD5(async_request_block, + MOCK_METHOD5(AsyncRequestBlock, void(const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, @@ -153,13 +153,13 @@ TEST(RemoteBlockReaderTest, TestReadSingleTrunk) { Status stat; size_t read = 0; PartialMockReader reader; - EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) + EXPECT_CALL(reader, AsyncRequestBlock(_, _, _, _, _)) .WillOnce(InvokeArgument<4>(Status::OK())); - EXPECT_CALL(reader, async_read_packet(_, _)) + EXPECT_CALL(reader, AsyncReadPacket(_, _)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf))); reader.AsyncReadBlock( - &reader, GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), [&stat, &read](const Status &status, size_t transferred) { stat = status; read = transferred; @@ -178,15 +178,15 @@ TEST(RemoteBlockReaderTest, TestReadMultipleTrunk) { size_t read = 0; PartialMockReader reader; - EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) + EXPECT_CALL(reader, AsyncRequestBlock(_, _, _, _, _)) .WillOnce(InvokeArgument<4>(Status::OK())); - EXPECT_CALL(reader, async_read_packet(_, _)) + EXPECT_CALL(reader, AsyncReadPacket(_, _)) .Times(4) .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)); reader.AsyncReadBlock( - &reader, GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), [&stat, &read](const Status &status, size_t transferred) { stat = status; read = transferred; @@ -204,17 +204,17 @@ TEST(RemoteBlockReaderTest, TestReadError) { Status stat; size_t read = 0; PartialMockReader reader; - EXPECT_CALL(reader, async_request_block(_, _, _, _, _)) + EXPECT_CALL(reader, AsyncRequestBlock(_, _, _, _, _)) .WillOnce(InvokeArgument<4>(Status::OK())); - EXPECT_CALL(reader, async_read_packet(_, _)) + EXPECT_CALL(reader, AsyncReadPacket(_, _)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4)) .WillOnce(InvokeArgument<1>(Status::Error("error"), 0)); reader.AsyncReadBlock( - &reader, GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), + GetRandomClientName(), block, 0, asio::buffer(buf, sizeof(buf)), [&stat, &read](const Status &status, size_t transferred) { stat = status; read = transferred; @@ -225,19 +225,19 @@ TEST(RemoteBlockReaderTest, TestReadError) { } template -static std::shared_ptr +static std::shared_ptr ReadContent(std::shared_ptr conn, const ExtendedBlockProto &block, uint64_t length, uint64_t offset, const mutable_buffers_1 &buf, const Handler &handler) { BlockReaderOptions options; - auto reader = std::make_shared(options, conn); + auto reader = std::make_shared(options, conn); Status result; - reader->async_request_block("libhdfs++", &block, length, offset, + reader->AsyncRequestBlock("libhdfs++", &block, length, offset, [buf, reader, handler](const Status &stat) { if (!stat.ok()) { handler(stat, 0); } else { - reader->async_read_packet(buf, handler); + reader->AsyncReadPacket(buf, handler); } }); return reader; @@ -339,13 +339,13 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { string data(kChunkSize, 0); mutable_buffers_1 buf = buffer(const_cast(data.c_str()), data.size()); BlockReaderOptions options; - auto reader = std::make_shared(options, conn); + auto reader = std::make_shared(options, conn); Status result; - reader->async_request_block( + reader->AsyncRequestBlock( "libhdfs++", &block, data.size(), 0, [buf, reader, &data, &io_service](const Status &stat) { ASSERT_TRUE(stat.ok()); - reader->async_read_packet( + reader->AsyncReadPacket( buf, [buf, reader, &data, &io_service](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); ASSERT_EQ(kChunkSize, transferred); @@ -353,7 +353,7 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) { data.clear(); data.resize(kChunkSize); transferred = 0; - reader->async_read_packet( + reader->AsyncReadPacket( buf, [&data,&io_service](const Status &stat, size_t transferred) { ASSERT_TRUE(stat.ok()); ASSERT_EQ(kChunkSize, transferred); From ba823b492b95b43011ce1263c97d008ff1f2efb1 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 11:52:21 -0500 Subject: [PATCH 33/50] Make AsyncStream match asio stream reqs --- .../libhdfspp/lib/common/async_stream.h | 17 +++--- .../lib/common/continuation/async_stream.h | 2 +- .../lib/common/continuation/protobuf.h | 4 +- .../lib/connection/datanodeconnection.h | 57 +++++++++---------- .../libhdfspp/lib/reader/block_reader.cc | 9 +-- .../libhdfspp/lib/reader/datatransfer.h | 25 ++++---- .../native/libhdfspp/tests/mock_connection.h | 13 +++-- .../tests/remote_block_reader_test.cc | 26 ++++----- 8 files changed, 69 insertions(+), 84 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h index ba50fa594e557..002b29290d915 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h @@ -28,16 +28,13 @@ typedef asio::const_buffers_1 ConstBuffers; class AsyncStream { public: - virtual void async_read(const MutableBuffers & buffers, - std::function completed_handler) = 0; - virtual void async_read(const MutableBuffers & buffers, - std::function completion_handler, - std::function completed_handler) = 0; - virtual void async_write(const ConstBuffers & buffers, - std::function handler) = 0; + virtual void async_read_some(const MutableBuffers &buf, + std::function handler) = 0; + + virtual void async_write_some(const ConstBuffers &buf, + std::function handler) = 0; }; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h index 48720ecb4a2af..535701e3acb7a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h @@ -55,7 +55,7 @@ class WriteContinuation : public Continuation { virtual void Run(const Next &next) override { auto handler = [next](const asio::error_code &ec, size_t) { next(ToStatus(ec)); }; - stream_->async_write(buffer_, handler); + asio::async_write(*stream_, buffer_, handler); } private: diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h index 174129291b4fa..54caeed2eb2ba 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/protobuf.h @@ -56,7 +56,7 @@ struct ReadDelimitedPBMessageContinuation : public Continuation { } next(status); }; - stream_->async_read( + asio::async_read(*stream_, asio::buffer(buf_), std::bind(&ReadDelimitedPBMessageContinuation::CompletionHandler, this, std::placeholders::_1, std::placeholders::_2), @@ -101,7 +101,7 @@ struct WriteDelimitedPBMessageContinuation : Continuation { pbio::CodedOutputStream os(&ss); os.WriteVarint32(size); msg_->SerializeToCodedStream(&os); - stream_->async_write(asio::buffer(buf_), [next](const asio::error_code &ec, size_t) { next(ToStatus(ec)); } ); + asio::async_write(*stream_, asio::buffer(buf_), [next](const asio::error_code &ec, size_t) { next(ToStatus(ec)); } ); } private: diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h index ac20ef5cafb74..3e15528d08626 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h @@ -37,43 +37,40 @@ class DataNodeConnection : public AsyncStream { class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_shared_from_this { public: - std::unique_ptr conn_; - std::array endpoints_; - std::string uuid_; + std::unique_ptr conn_; + std::array endpoints_; + std::string uuid_; - DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto, - const hadoop::common::TokenProto *token) { - using namespace ::asio::ip; + DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto, + const hadoop::common::TokenProto *token) { + using namespace ::asio::ip; - conn_.reset(new tcp::socket(*io_service)); - auto datanode_addr = dn_proto.id(); - endpoints_[0] = tcp::endpoint(address::from_string(datanode_addr.ipaddr()), - datanode_addr.xferport()); - uuid_ = dn_proto.id().datanodeuuid(); + conn_.reset(new tcp::socket(*io_service)); + auto datanode_addr = dn_proto.id(); + endpoints_[0] = tcp::endpoint(address::from_string(datanode_addr.ipaddr()), + datanode_addr.xferport()); + uuid_ = dn_proto.id().datanodeuuid(); - if (token) { - token_.reset(new hadoop::common::TokenProto()); - token_->CheckTypeAndMergeFrom(*token); - } + if (token) { + token_.reset(new hadoop::common::TokenProto()); + token_->CheckTypeAndMergeFrom(*token); } + } + + void Connect(std::function dn)> handler) override; - void Connect(std::function dn)> handler) override; + virtual void async_read_some(const MutableBuffers &buf, + std::function handler) { + conn_->async_read_some(buf, handler); + }; - virtual void async_read(const asio::mutable_buffers_1 & buffers, - std::function completed_handler) - { asio::async_read(*conn_, buffers, completed_handler); } - - virtual void async_read(const asio::mutable_buffers_1 & buffers, - std::function completion_handler, - std::function completed_handler) - { asio::async_read(*conn_, buffers, completion_handler, completed_handler); } - virtual void async_write(const asio::const_buffers_1 & buffers, - std::function handler) - { asio::async_write(*conn_, buffers, handler); } + virtual void async_write_some(const ConstBuffers &buf, + std::function handler) { + conn_->async_write_some(buf, handler); + } }; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc index 91598044e703a..40719c50b24d1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc @@ -137,7 +137,7 @@ struct BlockReaderImpl::ReadPacketHeader next(status); }; - parent_->dn_->async_read(asio::buffer(buf_), + asio::async_read(*parent_->dn_, asio::buffer(buf_), std::bind(&ReadPacketHeader::CompletionHandler, this, std::placeholders::_1, std::placeholders::_2), handler); @@ -195,8 +195,7 @@ struct BlockReaderImpl::ReadChecksum : continuation::Continuation { }; parent->checksum_.resize(parent->packet_len_ - sizeof(int) - parent->header_.datalen()); - parent->dn_->async_read(asio::buffer(parent->checksum_), - handler); + asio::async_read(*parent->dn_, asio::buffer(parent->checksum_), handler); } private: @@ -233,7 +232,7 @@ struct BlockReaderImpl::ReadData : continuation::Continuation { auto data_len = parent_->header_.datalen() - parent_->packet_data_read_bytes_; - parent_->dn_->async_read(buf_, asio::transfer_exactly(data_len), + asio::async_read(*parent_->dn_, buf_, asio::transfer_exactly(data_len), handler); } @@ -431,6 +430,4 @@ void BlockReaderImpl::AsyncReadBlock( }); } - - } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h index 4dffd1f866251..1387106c893bb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h @@ -44,20 +44,17 @@ template class DataTransferSaslStream : public DataNodeConnection template void Handshake(const Handler &next); - virtual void async_read(const asio::mutable_buffers_1 & buffers, - std::function completed_handler) - { stream_->async_read(buffers, completed_handler); } - - virtual void async_read(const asio::mutable_buffers_1 & buffers, - std::function completion_handler, - std::function completed_handler) - { stream_->async_read(buffers, completion_handler, completed_handler); } - virtual void async_write(const asio::const_buffers_1 & buffers, - std::function handler) - { stream_->async_write(buffers, handler); } + virtual void async_read_some(const MutableBuffers &buf, + std::function handler) { + stream_->async_read_some(buf, handler); + } + + virtual void async_write_some(const ConstBuffers &buf, + std::function handler) { + stream_->async_write_some(buf, handler); + } void Connect(std::function dn)> handler) override {(void)handler; /*TODO: Handshaking goes here*/}; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h index 6e7e2984e241c..98939befa567e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h @@ -29,13 +29,15 @@ namespace hdfs { -class MockConnectionBase { +class MockConnectionBase : public AsyncStream{ public: MockConnectionBase(::asio::io_service *io_service); virtual ~MockConnectionBase(); typedef std::pair ProducerResult; - template - void async_read_some(const MutableBufferSequence &buf, Handler &&handler) { + + void async_read_some(const MutableBuffers &buf, + std::function handler) override { if (produced_.size() == 0) { ProducerResult r = Produce(); if (r.first) { @@ -53,8 +55,9 @@ class MockConnectionBase { io_service_->post(std::bind(handler, asio::error_code(), len)); } - template - void async_write_some(const ConstBufferSequence &buf, Handler &&handler) { + void async_write_some(const ConstBuffers &buf, + std::function handler) override { // CompletionResult res = OnWrite(buf); io_service_->post(std::bind(handler, asio::error_code(), asio::buffer_size(buf))); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 312ac46e4d20f..0d3186bc61800 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -63,25 +63,19 @@ class MockDNConnection : public MockConnectionBase, public DataNodeConnection{ MOCK_METHOD1(Connect, void(std::function dn)>)); - // Satisfy the AsyncStream contract, delegating to MocKConnectionBase - void async_read(const asio::mutable_buffers_1 & buf, - std::function handler) { - asio::async_read(*this, buf, handler); - } - - void async_read(const asio::mutable_buffers_1 & buf, - std::function completion_handler, - std::function completed_handler) { - asio::async_read(*this, buf, completion_handler, completed_handler); + void async_read_some(const MutableBuffers &buf, + std::function handler) override { + this->MockConnectionBase::async_read_some(buf, handler); } - void async_write(const asio::const_buffers_1 & buf, - std::function handler) { - asio::async_write(*this, buf, handler); + void async_write_some(const ConstBuffers &buf, + std::function handler) override { + // CompletionResult res = OnWrite(buf); + this->MockConnectionBase::async_write_some(buf, handler); } + }; // Mocks AsyncReadPacket and AsyncRequestBlock but not AsyncReadBlock, so we From 43eec6ae523db830777610ffa088a76af9287bc7 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 11:57:36 -0500 Subject: [PATCH 34/50] Removed stream continuations --- .../lib/common/continuation/async_stream.h | 81 ------------------- .../libhdfspp/lib/reader/block_reader.cc | 2 +- .../libhdfspp/lib/reader/datatransfer_impl.h | 6 +- 3 files changed, 4 insertions(+), 85 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h deleted file mode 100644 index 535701e3acb7a..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/async_stream.h +++ /dev/null @@ -1,81 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#ifndef LIB_COMMON_CONTINUATION_ASYNC_STREAM_H_ -#define LIB_COMMON_CONTINUATION_ASYNC_STREAM_H_ - -#include "continuation.h" -#include "common/util.h" - -#include "libhdfspp/status.h" - -#include "common/async_stream.h" - -namespace hdfs { -namespace async_stream_continuation { - -using namespace continuation; - -template -class ReadContinuation : public Continuation { -public: - ReadContinuation(std::shared_ptr stream, const MutableBufferSequence &buffer) - : stream_(stream), buffer_(buffer) {} - virtual void Run(const Next &next) override { - auto handler = - [next](const asio::error_code &ec, size_t) { next(ToStatus(ec)); }; - stream_->async_read(buffer_, handler); - } - -private: - std::shared_ptr stream_; - MutableBufferSequence buffer_; -}; - -template -class WriteContinuation : public Continuation { -public: - WriteContinuation(std::shared_ptr stream, const ConstBufferSequence &buffer) - : stream_(stream), buffer_(buffer) {} - - virtual void Run(const Next &next) override { - auto handler = - [next](const asio::error_code &ec, size_t) { next(ToStatus(ec)); }; - asio::async_write(*stream_, buffer_, handler); - } - -private: - std::shared_ptr stream_; - ConstBufferSequence buffer_; -}; - -template -static inline Continuation *Write(std::shared_ptr stream, - const ConstBufferSequence &buffer) { - return new WriteContinuation(stream, buffer); -} - -template -static inline Continuation *Read(std::shared_ptr stream, - const MutableBufferSequence &buffer) { - return new ReadContinuation(stream, buffer); -} - -} -} - -#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc index 40719c50b24d1..97fc2b28d80d1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc @@ -77,7 +77,7 @@ void BlockReaderImpl::AsyncRequestBlock( new continuation::ReadDelimitedPBMessageContinuation( dn_, &s->response); - m->Push(async_stream_continuation::Write(dn_, asio::buffer(s->header))) + m->Push(asio_continuation::Write(dn_.get(), asio::buffer(s->header))) .Push(asio_continuation::WriteDelimitedPBMessage(dn_, &s->request)) .Push(read_pb_message); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h index 762b46aaff590..3ca16e958165a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h @@ -20,7 +20,7 @@ #include "datatransfer.pb.h" #include "common/continuation/continuation.h" -#include "common/continuation/async_stream.h" +#include "common/continuation/asio.h" #include "common/continuation/protobuf.h" #include @@ -97,7 +97,7 @@ template template void DataTransferSaslStream::Handshake(const Handler &next) { using ::hadoop::hdfs::DataTransferEncryptorMessageProto; - using ::hdfs::async_stream_continuation::Write; + using ::hdfs::asio_continuation::Write; using ::hdfs::continuation::WriteDelimitedPBMessage; static const int kMagicNumber = htonl(kDataTransferSasl); @@ -117,7 +117,7 @@ void DataTransferSaslStream::Handshake(const Handler &next) { DataTransferSaslStreamUtil::PrepareInitialHandshake(&s->req0); - m->Push(Write(stream_, kMagicNumberBuffer)) + m->Push(Write(stream_.get(), kMagicNumberBuffer)) .Push(WriteDelimitedPBMessage(stream_, &s->req0)) .Push(new ReadSaslMessage(stream_, &s->resp0)) .Push(new Authenticator(&authenticator_, &s->resp0, &s->req1)) From c3320a2b1619f691ecd192540122ac775b0919f7 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 12:43:36 -0500 Subject: [PATCH 35/50] Re-played int change --- .../src/main/native/libhdfspp/lib/reader/block_reader.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc index 97fc2b28d80d1..4bf5fca50f9cd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc @@ -146,9 +146,9 @@ struct BlockReaderImpl::ReadPacketHeader private: static const size_t kMaxHeaderSize = 512; static const size_t kPayloadLenOffset = 0; - static const size_t kPayloadLenSize = sizeof(int); + static const size_t kPayloadLenSize = sizeof(int32_t); static const size_t kHeaderLenOffset = 4; - static const size_t kHeaderLenSize = sizeof(short); + static const size_t kHeaderLenSize = sizeof(int16_t); static const size_t kHeaderStart = kPayloadLenSize + kHeaderLenSize; BlockReaderImpl *parent_; From 5d02748f2561d63fd37181b71d10324ee00129df Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 13:25:15 -0500 Subject: [PATCH 36/50] Made IS.PRead concurrent --- .../libhdfspp/lib/bindings/c/hdfs_cpp.h | 1 + .../native/libhdfspp/lib/fs/inputstream.cc | 21 ++++++++++++------- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.h index 5822ff4c7b1f9..e10d261f75923 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.h @@ -50,6 +50,7 @@ class FileHandle { class HadoopFileSystem { public: HadoopFileSystem() : service_(IoService::New()) {} + HadoopFileSystem(IoService *&ioservice) {service_.reset(ioservice); ioservice = nullptr; } virtual ~HadoopFileSystem(); /* attempt to connect to namenode, return false on failure */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index 637e0eb23fda4..b5180557ff561 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -75,26 +75,31 @@ void InputStreamImpl::AsyncPreadSome( uint64_t size_within_block = std::min( targetBlock.b().numbytes() - offset_within_block, asio::buffer_size(buffers)); - // This is where we will put the logic for re-using a DN connection - dn_ = std::make_shared(io_service_, *chosen_dn, nullptr); - std::string dn_id = dn_->uuid_; - + // This is where we will put the logic for re-using a DN connection; we can + // steal the InputStream's dn and put it back when we're done + std::shared_ptr dn = std::make_shared(io_service_, *chosen_dn, nullptr /*token*/); + std::string dn_id = dn->uuid_; + std::string client_name = client_name_; + + // Wrap the DN in a block reader to handle the state and logic of the + // block request protocol std::shared_ptr reader; - reader.reset(new BlockReaderImpl(BlockReaderOptions(), dn_)); + reader.reset(new BlockReaderImpl(BlockReaderOptions(), dn)); + auto read_handler = [dn_id, handler](const Status & status, size_t transferred) { handler(status, dn_id, transferred); }; - dn_->Connect([this,handler,read_handler,targetBlock,offset_within_block,size_within_block, buffers, reader] + dn->Connect([handler,read_handler,targetBlock,offset_within_block,size_within_block, buffers, reader, dn_id, client_name] (Status status, std::shared_ptr dn) { (void)dn; if (status.ok()) { reader->AsyncReadBlock( - client_name_, targetBlock, offset_within_block, + client_name, targetBlock, offset_within_block, asio::buffer(buffers, size_within_block), read_handler); } else { - handler(status, "", 0); + handler(status, dn_id, 0); } }); } From 979500acd801de46dde1b9697396dd926da57170 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 14:59:36 -0500 Subject: [PATCH 37/50] Moved C lib to talk to FS --- .../native/libhdfspp/include/libhdfspp/hdfs.h | 12 ++ .../libhdfspp/lib/bindings/c/CMakeLists.txt | 2 +- .../native/libhdfspp/lib/bindings/c/hdfs.cc | 40 ++--- .../libhdfspp/lib/bindings/c/hdfs_cpp.cc | 157 ------------------ .../libhdfspp/lib/bindings/c/hdfs_cpp.h | 83 --------- .../native/libhdfspp/lib/fs/filesystem.cc | 115 ++++++++++++- .../main/native/libhdfspp/lib/fs/filesystem.h | 38 ++++- .../native/libhdfspp/lib/fs/inputstream.cc | 28 ++++ 8 files changed, 204 insertions(+), 271 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.cc delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.h diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h index a3b185392c76c..4c3ed4a437c48 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h @@ -76,6 +76,9 @@ class InputStream { const std::set &excluded_datanodes, const std::function &handler) = 0; + + virtual size_t PositionRead(void *buf, size_t nbyte, off_t offset) = 0; + virtual ~InputStream(); }; @@ -93,6 +96,12 @@ class FileSystem { New(IoService *io_service, const Options &options, const std::string &server, const std::string &service, const std::function &handler); + + /* Synchronous call of New*/ + static FileSystem * + New(IoService *io_service, const Options &options, const std::string &server, + const std::string &service); + /** * Open a file on HDFS. The call issues an RPC to the NameNode to * gather the locations of all blocks in the file and to return a @@ -101,7 +110,10 @@ class FileSystem { virtual void Open(const std::string &path, const std::function &handler) = 0; + Status Open(const std::string &path, InputStream **handle); + virtual ~FileSystem(); + }; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/CMakeLists.txt index e170370c92040..664518ac684ad 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/CMakeLists.txt @@ -16,5 +16,5 @@ # under the License. -add_library(bindings_c hdfs.cc hdfs_cpp.cc) +add_library(bindings_c hdfs.cc) add_dependencies(bindings_c fs rpc reader proto common fs rpc reader proto common) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc index 9b985a932ad02..cb2cd75bc7622 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc @@ -16,8 +16,10 @@ * limitations under the License. */ -#include "hdfs_cpp.h" +#include "fs/filesystem.h" +#include +#include #include #include @@ -25,26 +27,26 @@ using namespace hdfs; /* Seperate the handles used by the C api from the C++ API*/ struct hdfs_internal { - hdfs_internal(HadoopFileSystem *p) : filesystem_(p) {} - hdfs_internal(std::unique_ptr p) + hdfs_internal(FileSystem *p) : filesystem_(p) {} + hdfs_internal(std::unique_ptr p) : filesystem_(std::move(p)) {} virtual ~hdfs_internal(){}; - HadoopFileSystem *get_impl() { return filesystem_.get(); } - const HadoopFileSystem *get_impl() const { return filesystem_.get(); } + FileSystem *get_impl() { return filesystem_.get(); } + const FileSystem *get_impl() const { return filesystem_.get(); } private: - std::unique_ptr filesystem_; + std::unique_ptr filesystem_; }; struct hdfsFile_internal { - hdfsFile_internal(FileHandle *p) : file_(p) {} - hdfsFile_internal(std::unique_ptr p) : file_(std::move(p)) {} + hdfsFile_internal(InputStream *p) : file_(p) {} + hdfsFile_internal(std::unique_ptr p) : file_(std::move(p)) {} virtual ~hdfsFile_internal(){}; - FileHandle *get_impl() { return file_.get(); } - const FileHandle *get_impl() const { return file_.get(); } + InputStream *get_impl() { return file_.get(); } + const InputStream *get_impl() const { return file_.get(); } private: - std::unique_ptr file_; + std::unique_ptr file_; }; /* Error handling with optional debug to stderr */ @@ -65,17 +67,17 @@ static void ReportError(int errnum, std::string msg) { int hdfsFileIsOpenForRead(hdfsFile file) { /* files can only be open for reads at the moment, do a quick check */ if (file) { - return file->get_impl()->IsOpenForRead(); + return true; // Update implementation when we get file writing } return false; } hdfsFS hdfsConnect(const char *nn, tPort port) { - HadoopFileSystem *fs = new HadoopFileSystem(); - Status stat = fs->Connect(nn, port); - if (!stat.ok()) { + std::string port_as_string = std::to_string(port); + IoService * io_service = IoService::New(); + FileSystem *fs = FileSystem::New(io_service, Options(), nn, port_as_string); + if (!fs) { ReportError(ENODEV, "Unable to connect to NameNode."); - delete fs; return nullptr; } return new hdfs_internal(fs); @@ -101,8 +103,8 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const char *path, int flags, int bufferSize, ReportError(ENODEV, "Cannot perform FS operations with null FS handle."); return nullptr; } - FileHandle *f = nullptr; - Status stat = fs->get_impl()->OpenFileForRead(path, &f); + InputStream *f = nullptr; + Status stat = fs->get_impl()->Open(path, &f); if (!stat.ok()) { return nullptr; } @@ -133,5 +135,5 @@ tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position, void *buffer, return -1; } - return file->get_impl()->Pread(buffer, length, position); + return file->get_impl()->PositionRead(buffer, length, position); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.cc deleted file mode 100644 index 3f3fb8dc3d190..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.cc +++ /dev/null @@ -1,157 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include "hdfs_cpp.h" - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include "libhdfspp/hdfs.h" -#include "libhdfspp/status.h" -#include "fs/filesystem.h" -#include "common/hdfs_public_api.h" - -namespace hdfs { - -ssize_t FileHandle::Pread(void *buf, size_t nbyte, off_t offset) { - auto stat = std::make_shared>(); - std::future future(stat->get_future()); - - /* wrap async call with promise/future to make it blocking */ - size_t read_count = 0; - auto callback = [stat, &read_count](const Status &s, const std::string &dn, - size_t bytes) { - (void)dn; - stat->set_value(s); - read_count = bytes; - }; - - input_stream_->PositionRead(buf, nbyte, offset, std::set(), - callback); - - /* wait for async to finish */ - auto s = future.get(); - - if (!s.ok()) { - return -1; - } - return (ssize_t)read_count; -} - -bool FileHandle::IsOpenForRead() { - /* for now just check if InputStream exists */ - if (!input_stream_) { - return false; - } - return true; -} - -HadoopFileSystem::~HadoopFileSystem() { - /** - * Note: IoService must be stopped before getting rid of worker threads. - * Once worker threads are joined and deleted the service can be deleted. - **/ - - file_system_.reset(nullptr); - service_->Stop(); - worker_threads_.clear(); - service_.reset(nullptr); -} - -Status HadoopFileSystem::Connect(const char *nn, tPort port, - unsigned int threads) { - /* IoService::New can return nullptr */ - if (!service_) { - return Status::Error("Null IoService"); - } - /* spawn background threads for asio delegation */ - for (unsigned int i = 0; i < threads; i++) { - AddWorkerThread(); - } - /* synchronized */ - FileSystem *fs = nullptr; - auto stat = std::make_shared>(); - std::future future = stat->get_future(); - - auto callback = [stat, &fs](const Status &s, FileSystem *f) { - fs = f; - stat->set_value(s); - }; - - /* dummy options object until this is hooked up to HDFS-9117 */ - Options options_object; - FileSystem::New(service_.get(), options_object, nn, std::to_string(port), - callback); - - /* block until promise is set */ - auto s = future.get(); - - /* check and see if it worked */ - if (!fs) { - service_->Stop(); - worker_threads_.clear(); - return s; - } - - file_system_ = std::unique_ptr(fs); - return s; -} - -int HadoopFileSystem::AddWorkerThread() { - auto service_task = [](IoService *service) { service->Run(); }; - worker_threads_.push_back( - WorkerPtr(new std::thread(service_task, service_.get()))); - return worker_threads_.size(); -} - -Status HadoopFileSystem::OpenFileForRead(const std::string &path, - FileHandle **handle) { - auto stat = std::make_shared>(); - std::future future = stat->get_future(); - - /* wrap async FileSystem::Open with promise to make it a blocking call */ - InputStream *input_stream = nullptr; - auto h = [stat, &input_stream](const Status &s, InputStream *is) { - stat->set_value(s); - input_stream = is; - }; - - file_system_->Open(path, h); - - /* block until promise is set */ - auto s = future.get(); - - if (!s.ok()) { - delete input_stream; - return s; - } - if (!input_stream) { - return s; - } - - *handle = new FileHandle(input_stream); - return s; -} -} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.h deleted file mode 100644 index e10d261f75923..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs_cpp.h +++ /dev/null @@ -1,83 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#ifndef LIBHDFSPP_BINDINGS_HDFSCPP_H -#define LIBHDFSPP_BINDINGS_HDFSCPP_H - -#include -#include -#include - -#include "libhdfspp/hdfs.h" -#include - -namespace hdfs { - -/** - * Implement a very simple 'it just works' interface in C++ - * that provides posix-like file operations + extra stuff for hadoop. - * Then provide very thin C wrappers over each method. - */ - -class FileHandle { - public: - virtual ~FileHandle(){}; - ssize_t Pread(void *buf, size_t nbyte, off_t offset); - bool IsOpenForRead(); - - private: - /* handle should only be created by fs */ - friend class HadoopFileSystem; - FileHandle(InputStream *is) : input_stream_(is){}; - std::unique_ptr input_stream_; -}; - -class HadoopFileSystem { - public: - HadoopFileSystem() : service_(IoService::New()) {} - HadoopFileSystem(IoService *&ioservice) {service_.reset(ioservice); ioservice = nullptr; } - virtual ~HadoopFileSystem(); - - /* attempt to connect to namenode, return false on failure */ - Status Connect(const char *nn, tPort port, unsigned int threads = 1); - - /* how many worker threads are servicing asio requests */ - int WorkerThreadCount() { return worker_threads_.size(); } - - /* add a new thread to handle asio requests, return number of threads in pool - */ - int AddWorkerThread(); - - Status OpenFileForRead(const std::string &path, FileHandle **handle); - - private: - std::unique_ptr service_; - /* std::thread needs to join before deletion */ - struct WorkerDeleter { - void operator()(std::thread *t) { - t->join(); - delete t; - } - }; - typedef std::unique_ptr WorkerPtr; - std::vector worker_threads_; - std::unique_ptr file_system_; -}; -} - -#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 5041097b5a43a..454923e119e23 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -24,6 +24,7 @@ #include #include +#include namespace hdfs { @@ -99,8 +100,6 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, } -FileSystem::~FileSystem() {} - void FileSystem::New( IoService *io_service, const Options &options, const std::string &server, const std::string &service, @@ -116,19 +115,82 @@ void FileSystem::New( }); } -FileSystemImpl::FileSystemImpl(IoService *io_service, const Options &options) - : io_service_(static_cast(io_service)), - client_name_(GetRandomClientName()), - nn_(&io_service_->io_service(), options, - GetRandomClientName(), kNamenodeProtocol, - kNamenodeProtocolVersion) {} +FileSystem * FileSystem::New( + IoService *io_service, const Options &options, const std::string &server, + const std::string &service) { + auto stat = std::make_shared>>(); + std::future> future = stat->get_future(); + + auto callback = [stat](const Status &s, FileSystem * fs) { + std::pair value(s, fs); + stat->set_value(value); + }; + + New(io_service, options, server, service, callback); + + /* block until promise is set */ + auto s = future.get(); + + if (s.first.ok()) + return s.second; + else + return nullptr; +} + + FileSystemImpl::FileSystemImpl(IoService *&io_service, const Options &options) + : io_service_(static_cast(io_service)), + nn_(&io_service_->io_service(), options, + GetRandomClientName(), kNamenodeProtocol, + kNamenodeProtocolVersion), + client_name_(GetRandomClientName()) +{ + // 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(); + } +} + +FileSystemImpl::~FileSystemImpl() { + /** + * Note: IoService must be stopped before getting rid of worker threads. + * Once worker threads are joined and deleted the service can be deleted. + **/ + io_service_->Stop(); + worker_threads_.clear(); + io_service_.reset(nullptr); +} void FileSystemImpl::Connect(const std::string &server, const std::string &service, std::function &&handler) { + /* IoService::New can return nullptr */ + if (!io_service_) { + handler (Status::Error("Null IoService")); + } nn_.Connect(server, service, handler); } +Status FileSystemImpl::Connect(const std::string &server, const std::string &service) { + /* synchronized */ + auto stat = std::make_shared>(); + std::future future = stat->get_future(); + + auto callback = [stat](const Status &s) { + stat->set_value(s); + }; + + Connect(server, service, callback); + + /* block until promise is set */ + auto s = future.get(); + + return s; +} + void FileSystemImpl::Open( const std::string &path, const std::function &handler) { @@ -138,4 +200,41 @@ void FileSystemImpl::Open( : nullptr); }); } + +int FileSystemImpl::AddWorkerThread() { + auto service_task = [](IoService *service) { service->Run(); }; + worker_threads_.push_back( + WorkerPtr(new std::thread(service_task, io_service_.get()))); + return worker_threads_.size(); +} + +Status FileSystemImpl::Open(const std::string &path, + InputStream **handle) { + auto stat = std::make_shared>(); + std::future future = stat->get_future(); + + /* wrap async FileSystem::Open with promise to make it a blocking call */ + InputStream *input_stream = nullptr; + auto h = [stat, &input_stream](const Status &s, InputStream *is) { + stat->set_value(s); + input_stream = is; + }; + + Open(path, h); + + /* block until promise is set */ + auto s = future.get(); + + if (!s.ok()) { + delete input_stream; + return s; + } + if (!input_stream) { + return s; + } + + *handle = input_stream; + return s; +} + } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index f1081f1fc96ce..8cc76cba0bc75 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -29,6 +29,8 @@ #include "asio.hpp" +#include + namespace hdfs { /** @@ -72,16 +74,44 @@ void GetBlockLocations(const std::string & path, std::function &&handler); + /* attempt to connect to namenode, return bad status on failure */ + Status Connect(const std::string &server, const std::string &service); + + virtual void Open(const std::string &path, const std::function &handler) override; + Status Open(const std::string &path, InputStream **handle); + + + /* add a new thread to handle asio requests, return number of threads in pool + */ + int AddWorkerThread(); + + /* how many worker threads are servicing asio requests */ + int WorkerThreadCount() { return worker_threads_.size(); } + + private: - IoServiceImpl *io_service_; - const std::string client_name_; + std::unique_ptr io_service_; NameNodeOperations nn_; + const std::string client_name_; + + struct WorkerDeleter { + void operator()(std::thread *t) { + t->join(); + delete t; + } + }; + typedef std::unique_ptr WorkerPtr; + std::vector worker_threads_; + }; @@ -101,6 +131,8 @@ class InputStreamImpl : public InputStream { const std::function &handler) override; + size_t PositionRead(void *buf, size_t nbyte, off_t offset) override; + void AsyncPreadSome(size_t offset, const MutableBuffers &buffers, const std::set &excluded_datanodes, const std::function handler); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index b5180557ff561..a9ea1b2ab3db0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -19,6 +19,8 @@ #include "filesystem.h" #include "common/continuation/continuation.h" +#include + namespace hdfs { using ::hadoop::hdfs::LocatedBlocksProto; @@ -38,6 +40,31 @@ void InputStreamImpl::PositionRead( AsyncPreadSome(offset, asio::buffer(buf, nbyte), excluded_datanodes, handler); } +size_t InputStreamImpl::PositionRead(void *buf, size_t nbyte, off_t offset) { + auto stat = std::make_shared>(); + std::future future(stat->get_future()); + + /* wrap async call with promise/future to make it blocking */ + size_t read_count = 0; + auto callback = [stat, &read_count](const Status &s, const std::string &dn, + size_t bytes) { + (void)dn; + stat->set_value(s); + read_count = bytes; + }; + + PositionRead(buf, nbyte, offset, std::set(), + callback); + + /* wait for async to finish */ + auto s = future.get(); + + if (!s.ok()) { + return -1; + } + return (ssize_t)read_count; +} + void InputStreamImpl::AsyncPreadSome( size_t offset, const MutableBuffers &buffers, const std::set &excluded_datanodes, @@ -104,4 +131,5 @@ void InputStreamImpl::AsyncPreadSome( }); } + } From c7aeebefc3c3cdd708d912de2adf54698c88799b Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 15:03:20 -0500 Subject: [PATCH 38/50] Renamed InputStream to FileHandle --- .../main/native/libhdfspp/include/libhdfspp/hdfs.h | 10 +++++----- .../src/main/native/libhdfspp/lib/bindings/c/hdfs.cc | 12 ++++++------ .../src/main/native/libhdfspp/lib/fs/filesystem.cc | 10 +++++----- .../src/main/native/libhdfspp/lib/fs/filesystem.h | 8 ++++---- .../src/main/native/libhdfspp/lib/fs/inputstream.cc | 12 ++++++------ .../main/native/libhdfspp/tests/inputstream_test.cc | 4 ++-- 6 files changed, 28 insertions(+), 28 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h index 4c3ed4a437c48..fb0d5e4e013af 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h @@ -54,9 +54,9 @@ class IoService { }; /** - * Applications opens an InputStream to read files in HDFS. + * Applications opens a FileHandle to read files in HDFS. **/ -class InputStream { +class FileHandle { public: /** * Read data from a specific position. The current implementation @@ -79,7 +79,7 @@ class InputStream { virtual size_t PositionRead(void *buf, size_t nbyte, off_t offset) = 0; - virtual ~InputStream(); + virtual ~FileHandle(); }; /** @@ -109,8 +109,8 @@ class FileSystem { **/ virtual void Open(const std::string &path, - const std::function &handler) = 0; - Status Open(const std::string &path, InputStream **handle); + const std::function &handler) = 0; + Status Open(const std::string &path, FileHandle **handle); virtual ~FileSystem(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc index cb2cd75bc7622..cd3af77c2cd4c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc @@ -39,14 +39,14 @@ struct hdfs_internal { }; struct hdfsFile_internal { - hdfsFile_internal(InputStream *p) : file_(p) {} - hdfsFile_internal(std::unique_ptr p) : file_(std::move(p)) {} + hdfsFile_internal(FileHandle *p) : file_(p) {} + hdfsFile_internal(std::unique_ptr p) : file_(std::move(p)) {} virtual ~hdfsFile_internal(){}; - InputStream *get_impl() { return file_.get(); } - const InputStream *get_impl() const { return file_.get(); } + FileHandle *get_impl() { return file_.get(); } + const FileHandle *get_impl() const { return file_.get(); } private: - std::unique_ptr file_; + std::unique_ptr file_; }; /* Error handling with optional debug to stderr */ @@ -103,7 +103,7 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const char *path, int flags, int bufferSize, ReportError(ENODEV, "Cannot perform FS operations with null FS handle."); return nullptr; } - InputStream *f = nullptr; + FileHandle *f = nullptr; Status stat = fs->get_impl()->Open(path, &f); if (!stat.ok()) { return nullptr; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 454923e119e23..8c60ef0d0bdfb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -193,10 +193,10 @@ Status FileSystemImpl::Connect(const std::string &server, const std::string &ser void FileSystemImpl::Open( const std::string &path, - const std::function &handler) { + const std::function &handler) { nn_.GetBlockLocations(path, [this, handler](const Status &stat, std::shared_ptr file_info) { - handler(stat, stat.ok() ? new InputStreamImpl(&io_service_->io_service(), client_name_, file_info) + handler(stat, stat.ok() ? new FileHandleImpl(&io_service_->io_service(), client_name_, file_info) : nullptr); }); } @@ -209,13 +209,13 @@ int FileSystemImpl::AddWorkerThread() { } Status FileSystemImpl::Open(const std::string &path, - InputStream **handle) { + FileHandle **handle) { auto stat = std::make_shared>(); std::future future = stat->get_future(); /* wrap async FileSystem::Open with promise to make it a blocking call */ - InputStream *input_stream = nullptr; - auto h = [stat, &input_stream](const Status &s, InputStream *is) { + FileHandle *input_stream = nullptr; + auto h = [stat, &input_stream](const Status &s, FileHandle *is) { stat->set_value(s); input_stream = is; }; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 8cc76cba0bc75..0d7030a360894 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -85,9 +85,9 @@ class FileSystemImpl : public FileSystem { virtual void Open(const std::string &path, - const std::function + const std::function &handler) override; - Status Open(const std::string &path, InputStream **handle); + Status Open(const std::string &path, FileHandle **handle); /* add a new thread to handle asio requests, return number of threads in pool @@ -121,9 +121,9 @@ class FileSystemImpl : public FileSystem { * Threading model: not thread-safe; consumers and io_service should not call * concurrently */ -class InputStreamImpl : public InputStream { +class FileHandleImpl : public FileHandle { public: - InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, + FileHandleImpl(::asio::io_service *io_service, const std::string &client_name, const std::shared_ptr file_info); virtual void PositionRead(void *buf, size_t nbyte, uint64_t offset, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc index a9ea1b2ab3db0..45d9b0c9efb9d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc @@ -25,14 +25,14 @@ namespace hdfs { using ::hadoop::hdfs::LocatedBlocksProto; -InputStream::~InputStream() {} +FileHandle::~FileHandle() {} -InputStreamImpl::InputStreamImpl(::asio::io_service *io_service, const std::string &client_name, +FileHandleImpl::FileHandleImpl(::asio::io_service *io_service, const std::string &client_name, const std::shared_ptr file_info) : io_service_(io_service), client_name_(client_name), file_info_(file_info) { } -void InputStreamImpl::PositionRead( +void FileHandleImpl::PositionRead( void *buf, size_t nbyte, uint64_t offset, const std::set &excluded_datanodes, const std::function @@ -40,7 +40,7 @@ void InputStreamImpl::PositionRead( AsyncPreadSome(offset, asio::buffer(buf, nbyte), excluded_datanodes, handler); } -size_t InputStreamImpl::PositionRead(void *buf, size_t nbyte, off_t offset) { +size_t FileHandleImpl::PositionRead(void *buf, size_t nbyte, off_t offset) { auto stat = std::make_shared>(); std::future future(stat->get_future()); @@ -65,7 +65,7 @@ size_t InputStreamImpl::PositionRead(void *buf, size_t nbyte, off_t offset) { return (ssize_t)read_count; } -void InputStreamImpl::AsyncPreadSome( +void FileHandleImpl::AsyncPreadSome( size_t offset, const MutableBuffers &buffers, const std::set &excluded_datanodes, const std::function handler) { @@ -103,7 +103,7 @@ void InputStreamImpl::AsyncPreadSome( targetBlock.b().numbytes() - offset_within_block, asio::buffer_size(buffers)); // This is where we will put the logic for re-using a DN connection; we can - // steal the InputStream's dn and put it back when we're done + // steal the FileHandle's dn and put it back when we're done std::shared_ptr dn = std::make_shared(io_service_, *chosen_dn, nullptr /*token*/); std::string dn_id = dn->uuid_; std::string client_name = client_name_; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index 9ac928740937a..dcb3faec05e74 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -59,7 +59,7 @@ class MockReader : public BlockReader { } -TEST(InputStreamTest, TestExcludeDataNode) { +TEST(FileHandleTest, TestExcludeDataNode) { auto file_info = std::make_shared(); file_info->blocks_.push_back(LocatedBlockProto()); LocatedBlockProto & block = file_info->blocks_[0]; @@ -78,7 +78,7 @@ TEST(InputStreamTest, TestExcludeDataNode) { 0, }; IoServiceImpl io_service; - InputStreamImpl is(&io_service.io_service(), GetRandomClientName(), file_info); + FileHandleImpl is(&io_service.io_service(), GetRandomClientName(), file_info); Status stat; size_t read = 0; From ab8573e6c7358d16d8a760d9d58aca7d0c51f326 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 15:04:19 -0500 Subject: [PATCH 39/50] Moved InputStream.cc to FileHandle.cc --- .../native/libhdfspp/lib/fs/CMakeLists.txt | 2 +- .../native/libhdfspp/lib/fs/inputstream.cc | 135 ------------------ 2 files changed, 1 insertion(+), 136 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/CMakeLists.txt index f386688ab169b..8344022ead376 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/CMakeLists.txt @@ -1,2 +1,2 @@ -add_library(fs filesystem.cc inputstream.cc) +add_library(fs filesystem.cc filehandle.cc) add_dependencies(fs proto) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc deleted file mode 100644 index 45d9b0c9efb9d..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/inputstream.cc +++ /dev/null @@ -1,135 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include "filesystem.h" -#include "common/continuation/continuation.h" - -#include - -namespace hdfs { - -using ::hadoop::hdfs::LocatedBlocksProto; - -FileHandle::~FileHandle() {} - -FileHandleImpl::FileHandleImpl(::asio::io_service *io_service, const std::string &client_name, - const std::shared_ptr file_info) - : io_service_(io_service), client_name_(client_name), file_info_(file_info) { -} - -void FileHandleImpl::PositionRead( - void *buf, size_t nbyte, uint64_t offset, - const std::set &excluded_datanodes, - const std::function - &handler) { - AsyncPreadSome(offset, asio::buffer(buf, nbyte), excluded_datanodes, handler); -} - -size_t FileHandleImpl::PositionRead(void *buf, size_t nbyte, off_t offset) { - auto stat = std::make_shared>(); - std::future future(stat->get_future()); - - /* wrap async call with promise/future to make it blocking */ - size_t read_count = 0; - auto callback = [stat, &read_count](const Status &s, const std::string &dn, - size_t bytes) { - (void)dn; - stat->set_value(s); - read_count = bytes; - }; - - PositionRead(buf, nbyte, offset, std::set(), - callback); - - /* wait for async to finish */ - auto s = future.get(); - - if (!s.ok()) { - return -1; - } - return (ssize_t)read_count; -} - -void FileHandleImpl::AsyncPreadSome( - size_t offset, const MutableBuffers &buffers, - const std::set &excluded_datanodes, - const std::function handler) { - using ::hadoop::hdfs::DatanodeInfoProto; - using ::hadoop::hdfs::LocatedBlockProto; - - auto it = std::find_if( - file_info_->blocks_.begin(), file_info_->blocks_.end(), [offset](const LocatedBlockProto &p) { - return p.offset() <= offset && offset < p.offset() + p.b().numbytes(); - }); - - if (it == file_info_->blocks_.end()) { - handler(Status::InvalidArgument("Cannot find corresponding blocks"), "", 0); - return; - } - - ::hadoop::hdfs::LocatedBlockProto targetBlock = *it; - - const DatanodeInfoProto *chosen_dn = nullptr; - for (int i = 0; i < targetBlock.locs_size(); ++i) { - const auto &di = targetBlock.locs(i); - if (!excluded_datanodes.count(di.id().datanodeuuid())) { - chosen_dn = &di; - break; - } - } - - if (!chosen_dn) { - handler(Status::ResourceUnavailable("No datanodes available"), "", 0); - return; - } - - uint64_t offset_within_block = offset - targetBlock.offset(); - uint64_t size_within_block = std::min( - targetBlock.b().numbytes() - offset_within_block, asio::buffer_size(buffers)); - - // This is where we will put the logic for re-using a DN connection; we can - // steal the FileHandle's dn and put it back when we're done - std::shared_ptr dn = std::make_shared(io_service_, *chosen_dn, nullptr /*token*/); - std::string dn_id = dn->uuid_; - std::string client_name = client_name_; - - // Wrap the DN in a block reader to handle the state and logic of the - // block request protocol - std::shared_ptr reader; - reader.reset(new BlockReaderImpl(BlockReaderOptions(), dn)); - - - auto read_handler = [dn_id, handler](const Status & status, size_t transferred) { - handler(status, dn_id, transferred); - }; - - dn->Connect([handler,read_handler,targetBlock,offset_within_block,size_within_block, buffers, reader, dn_id, client_name] - (Status status, std::shared_ptr dn) { - (void)dn; - if (status.ok()) { - reader->AsyncReadBlock( - client_name, targetBlock, offset_within_block, - asio::buffer(buffers, size_within_block), read_handler); - } else { - handler(status, dn_id, 0); - } - }); -} - - -} From 802c923129610407e057be92a0e68575208b8f0a Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 15:07:44 -0500 Subject: [PATCH 40/50] Removed whitespace --- .../main/native/libhdfspp/include/libhdfspp/hdfs.h | 4 ++-- .../native/libhdfspp/lib/common/async_stream.h | 6 +++--- .../libhdfspp/lib/connection/datanodeconnection.h | 8 ++++---- .../src/main/native/libhdfspp/lib/fs/filesystem.cc | 6 +++--- .../src/main/native/libhdfspp/lib/fs/filesystem.h | 6 +++--- .../native/libhdfspp/lib/reader/block_reader.cc | 14 +++++++------- .../native/libhdfspp/lib/reader/block_reader.h | 10 +++++----- .../native/libhdfspp/lib/reader/datatransfer.h | 6 +++--- .../native/libhdfspp/tests/inputstream_test.cc | 4 ++-- .../main/native/libhdfspp/tests/mock_connection.h | 4 ++-- .../libhdfspp/tests/remote_block_reader_test.cc | 14 +++++++------- 11 files changed, 41 insertions(+), 41 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h index fb0d5e4e013af..c65da93d8f643 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h @@ -76,7 +76,7 @@ class FileHandle { const std::set &excluded_datanodes, const std::function &handler) = 0; - + virtual size_t PositionRead(void *buf, size_t nbyte, off_t offset) = 0; virtual ~FileHandle(); @@ -113,7 +113,7 @@ class FileSystem { Status Open(const std::string &path, FileHandle **handle); virtual ~FileSystem(); - + }; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h index 002b29290d915..51e83e765ea48 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h @@ -25,14 +25,14 @@ namespace hdfs { typedef asio::mutable_buffers_1 MutableBuffers; typedef asio::const_buffers_1 ConstBuffers; - + class AsyncStream { public: - virtual void async_read_some(const MutableBuffers &buf, + virtual void async_read_some(const MutableBuffers &buf, std::function handler) = 0; - virtual void async_write_some(const ConstBuffers &buf, + virtual void async_write_some(const ConstBuffers &buf, std::function handler) = 0; }; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h index 3e15528d08626..8e79a24362279 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h @@ -30,7 +30,7 @@ class DataNodeConnection : public AsyncStream { public: std::string uuid_; std::unique_ptr token_; - + virtual void Connect(std::function dn)> handler) = 0; }; @@ -42,7 +42,7 @@ class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_sha std::string uuid_; - DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto, + DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto, const hadoop::common::TokenProto *token) { using namespace ::asio::ip; @@ -60,13 +60,13 @@ class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_sha void Connect(std::function dn)> handler) override; - virtual void async_read_some(const MutableBuffers &buf, + virtual void async_read_some(const MutableBuffers &buf, std::function handler) { conn_->async_read_some(buf, handler); }; - virtual void async_write_some(const ConstBuffers &buf, + virtual void async_write_some(const ConstBuffers &buf, std::function handler) { conn_->async_write_some(buf, handler); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index 8c60ef0d0bdfb..f0d6433dec95f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -136,7 +136,7 @@ FileSystem * FileSystem::New( else return nullptr; } - + FileSystemImpl::FileSystemImpl(IoService *&io_service, const Options &options) : io_service_(static_cast(io_service)), nn_(&io_service_->io_service(), options, @@ -146,9 +146,9 @@ FileSystem * FileSystem::New( { // Poor man's move io_service = nullptr; - + /* spawn background threads for asio delegation */ - unsigned int threads = 1 /* options.io_threads_, pending HDFS-9117 */; + unsigned int threads = 1 /* options.io_threads_, pending HDFS-9117 */; for (unsigned int i = 0; i < threads; i++) { AddWorkerThread(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 0d7030a360894..7879e5386cac5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -88,8 +88,8 @@ class FileSystemImpl : public FileSystem { const std::function &handler) override; Status Open(const std::string &path, FileHandle **handle); - - + + /* add a new thread to handle asio requests, return number of threads in pool */ int AddWorkerThread(); @@ -97,7 +97,7 @@ class FileSystemImpl : public FileSystem { /* how many worker threads are servicing asio requests */ int WorkerThreadCount() { return worker_threads_.size(); } - + private: std::unique_ptr io_service_; NameNodeOperations nn_; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc index 4bf5fca50f9cd..a4e21de11baeb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc @@ -50,7 +50,7 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum, } void BlockReaderImpl::AsyncRequestBlock( - const std::string &client_name, + const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler) { // The total number of bytes that we need to transfer from the DN is @@ -99,7 +99,7 @@ void BlockReaderImpl::AsyncRequestBlock( } Status BlockReaderImpl::RequestBlock( - const std::string &client_name, + const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset) { auto stat = std::make_shared>(); @@ -209,7 +209,7 @@ struct BlockReaderImpl::ReadData : continuation::Continuation { : parent_(parent), bytes_transferred_(bytes_transferred), buf_(buf) { buf_.begin(); } - + ~ReadData() { buf_.end(); } @@ -309,7 +309,7 @@ struct BlockReaderImpl::AckRead : continuation::Continuation { }; void BlockReaderImpl::AsyncReadPacket( - const MutableBuffers &buffers, + const MutableBuffers &buffers, const std::function &handler) { assert(state_ != kOpen && "Not connected"); @@ -412,7 +412,7 @@ void BlockReaderImpl::AsyncReadBlock( const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, size_t offset, - const MutableBuffers &buffers, + const MutableBuffers &buffers, const std::function handler) { auto m = continuation::Pipeline::Create(); @@ -420,10 +420,10 @@ void BlockReaderImpl::AsyncReadBlock( size_t size = asio::buffer_size(buffers); - m->Push(new RequestBlockContinuation(this, client_name, + m->Push(new RequestBlockContinuation(this, client_name, &block.b(), size, offset)) .Push(new ReadBlockContinuation(this, buffers, bytesTransferred)); - + m->Run([handler] (const Status &status, const size_t totalBytesTransferred) { handler(status, totalBytesTransferred); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index ee68fc8f9342c..43059d99c31c1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -62,11 +62,11 @@ class BlockReader { virtual void AsyncReadPacket( const MutableBuffers &buffers, const std::function &handler) = 0; - + virtual void AsyncRequestBlock( const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, - uint64_t length, + uint64_t length, uint64_t offset, const std::function &handler) = 0; @@ -87,11 +87,11 @@ class BlockReaderImpl virtual void AsyncReadPacket( const MutableBuffers &buffers, const std::function &handler) override; - + virtual void AsyncRequestBlock( const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, - uint64_t length, + uint64_t length, uint64_t offset, const std::function &handler) override; @@ -105,7 +105,7 @@ class BlockReaderImpl Status RequestBlock( const std::string &client_name, - const hadoop::hdfs::ExtendedBlockProto *block, + const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h index 1387106c893bb..a67cb52911a39 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h @@ -44,19 +44,19 @@ template class DataTransferSaslStream : public DataNodeConnection template void Handshake(const Handler &next); - virtual void async_read_some(const MutableBuffers &buf, + virtual void async_read_some(const MutableBuffers &buf, std::function handler) { stream_->async_read_some(buf, handler); } - virtual void async_write_some(const ConstBuffers &buf, + virtual void async_write_some(const ConstBuffers &buf, std::function handler) { stream_->async_write_some(buf, handler); } - void Connect(std::function dn)> handler) override + void Connect(std::function dn)> handler) override {(void)handler; /*TODO: Handshaking goes here*/}; private: DataTransferSaslStream(const DataTransferSaslStream &) = delete; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc index dcb3faec05e74..c6f2c20e0f9a7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/inputstream_test.cc @@ -49,7 +49,7 @@ class MockReader : public BlockReader { const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, const std::function &handler)); - + MOCK_METHOD5(AsyncReadBlock, void( const std::string & client_name, const hadoop::hdfs::LocatedBlockProto &block, size_t offset, @@ -89,7 +89,7 @@ TEST(FileHandleTest, TestExcludeDataNode) { stat = status; read = transferred; }); - + // Should fail with no resource available ASSERT_EQ(static_cast(std::errc::resource_unavailable_try_again), stat.code()); ASSERT_EQ(0UL, read); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h index 98939befa567e..4c15375205d69 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h @@ -35,7 +35,7 @@ class MockConnectionBase : public AsyncStream{ virtual ~MockConnectionBase(); typedef std::pair ProducerResult; - void async_read_some(const MutableBuffers &buf, + void async_read_some(const MutableBuffers &buf, std::function handler) override { if (produced_.size() == 0) { @@ -55,7 +55,7 @@ class MockConnectionBase : public AsyncStream{ io_service_->post(std::bind(handler, asio::error_code(), len)); } - void async_write_some(const ConstBuffers &buf, + void async_write_some(const ConstBuffers &buf, std::function handler) override { // CompletionResult res = OnWrite(buf); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 0d3186bc61800..dd0f5dae90650 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -60,16 +60,16 @@ class MockDNConnection : public MockConnectionBase, public DataNodeConnection{ MockDNConnection(::asio::io_service &io_service) : MockConnectionBase(&io_service) {} MOCK_METHOD0(Produce, ProducerResult()); - + MOCK_METHOD1(Connect, void(std::function dn)>)); - - void async_read_some(const MutableBuffers &buf, + + void async_read_some(const MutableBuffers &buf, std::function handler) override { this->MockConnectionBase::async_read_some(buf, handler); } - void async_write_some(const ConstBuffers &buf, + void async_write_some(const ConstBuffers &buf, std::function handler) override { // CompletionResult res = OnWrite(buf); @@ -84,7 +84,7 @@ class PartialMockReader : public BlockReaderImpl { public: PartialMockReader() : BlockReaderImpl(BlockReaderOptions(), std::shared_ptr()) {}; - + MOCK_METHOD2( AsyncReadPacket, void(const asio::mutable_buffers_1 &, @@ -94,7 +94,7 @@ class PartialMockReader : public BlockReaderImpl { void(const std::string &client_name, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, uint64_t offset, - const std::function &handler)); + const std::function &handler)); }; @@ -170,7 +170,7 @@ TEST(RemoteBlockReaderTest, TestReadMultipleTrunk) { }; Status stat; size_t read = 0; - + PartialMockReader reader; EXPECT_CALL(reader, AsyncRequestBlock(_, _, _, _, _)) .WillOnce(InvokeArgument<4>(Status::OK())); From 967b35caf38c37ef87b133cbeb57071cd757d974 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Tue, 3 Nov 2015 15:14:14 -0500 Subject: [PATCH 41/50] Rolled back pom.xml changes --- hadoop-hdfs-project/pom.xml | 2 ++ pom.xml | 1 + 2 files changed, 3 insertions(+) diff --git a/hadoop-hdfs-project/pom.xml b/hadoop-hdfs-project/pom.xml index 0d65422dec7fe..4d53eddccf9ba 100644 --- a/hadoop-hdfs-project/pom.xml +++ b/hadoop-hdfs-project/pom.xml @@ -31,7 +31,9 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> pom + hadoop-hdfs hadoop-hdfs-client + hadoop-hdfs-native-client hadoop-hdfs-httpfs hadoop-hdfs/src/contrib/bkjournal hadoop-hdfs-nfs diff --git a/pom.xml b/pom.xml index d1508699ae54e..4e6f7a1ed96ec 100644 --- a/pom.xml +++ b/pom.xml @@ -115,6 +115,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs hadoop-assemblies hadoop-maven-plugins hadoop-common-project + hadoop-hdfs-project hadoop-yarn-project hadoop-mapreduce-project hadoop-tools From 951602c6bed9d7413a860006324c78ab48bc1760 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 4 Nov 2015 08:39:33 -0500 Subject: [PATCH 42/50] Added cancel --- .../native/libhdfspp/include/libhdfspp/hdfs.h | 3 ++- .../libhdfspp/lib/common/async_stream.h | 1 + .../lib/connection/datanodeconnection.h | 23 ++++++++++++++----- .../main/native/libhdfspp/lib/fs/filesystem.h | 4 ++-- .../libhdfspp/lib/reader/datatransfer.h | 20 ++++++++-------- .../tests/remote_block_reader_test.cc | 10 ++++---- 6 files changed, 38 insertions(+), 23 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h index c65da93d8f643..0c3e9346c7774 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h @@ -20,6 +20,7 @@ #include "libhdfspp/options.h" #include "libhdfspp/status.h" +#include "common/cancelable.h" #include #include @@ -71,7 +72,7 @@ class FileHandle { * The handler returns the datanode that serves the block and the number of * bytes has read. **/ - virtual void + virtual CancelHandle PositionRead(void *buf, size_t nbyte, uint64_t offset, const std::set &excluded_datanodes, const std::function namespace hdfs { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h index 8e79a24362279..8dbcffc9e64b2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h @@ -26,16 +26,18 @@ namespace hdfs { -class DataNodeConnection : public AsyncStream { +class DataNodeConnection : public AsyncStream, public Cancelable { public: std::string uuid_; std::unique_ptr token_; virtual void Connect(std::function dn)> handler) = 0; + + virtual void cancel() = 0; }; -class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_shared_from_this { +class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_shared_from_this{ public: std::unique_ptr conn_; std::array endpoints_; @@ -60,17 +62,26 @@ class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_sha void Connect(std::function dn)> handler) override; - virtual void async_read_some(const MutableBuffers &buf, + void async_read_some(const MutableBuffers &buf, std::function handler) { + std::size_t bytes_transferred) > handler) override { conn_->async_read_some(buf, handler); }; - virtual void async_write_some(const ConstBuffers &buf, + void async_write_some(const ConstBuffers &buf, std::function handler) { + std::size_t bytes_transferred) > handler) override { conn_->async_write_some(buf, handler); } + + void cancel() override { + // Can't portably use asio::cancel; see + // http://www.boost.org/doc/libs/1_59_0/doc/html/boost_asio/reference/basic_stream_socket/cancel/overload2.html + // + // When we implement connection re-use, we need to be sure to throw out + // connections on error + conn_->close(); + } }; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 7879e5386cac5..3aff55d336bbc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -125,7 +125,7 @@ class FileHandleImpl : public FileHandle { public: FileHandleImpl(::asio::io_service *io_service, const std::string &client_name, const std::shared_ptr file_info); - virtual void + virtual CancelHandle PositionRead(void *buf, size_t nbyte, uint64_t offset, const std::set &excluded_datanodes, const std::function &excluded_datanodes, const std::function handler); private: diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h index a67cb52911a39..c70a075f6e6b7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h @@ -44,17 +44,19 @@ template class DataTransferSaslStream : public DataNodeConnection template void Handshake(const Handler &next); - virtual void async_read_some(const MutableBuffers &buf, + void async_read_some(const MutableBuffers &buf, + std::function handler) override { + stream_->async_read_some(buf, handler); + } + + void async_write_some(const ConstBuffers &buf, std::function handler) { - stream_->async_read_some(buf, handler); - } + std::size_t bytes_transferred) > handler) override { + stream_->async_write_some(buf, handler); + } - virtual void async_write_some(const ConstBuffers &buf, - std::function handler) { - stream_->async_write_some(buf, handler); - } + void cancel() override { stream_->cancel(); } void Connect(std::function dn)> handler) override {(void)handler; /*TODO: Handshaking goes here*/}; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index dd0f5dae90650..9f252bf99c8d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -63,19 +63,19 @@ class MockDNConnection : public MockConnectionBase, public DataNodeConnection{ MOCK_METHOD1(Connect, void(std::function dn)>)); - void async_read_some(const MutableBuffers &buf, - std::function handler) override { + void async_read_some(const MutableBuffers &buf, + std::function handler) override { this->MockConnectionBase::async_read_some(buf, handler); } void async_write_some(const ConstBuffers &buf, std::function handler) override { - // CompletionResult res = OnWrite(buf); this->MockConnectionBase::async_write_some(buf, handler); } - + + void cancel() override {}; }; // Mocks AsyncReadPacket and AsyncRequestBlock but not AsyncReadBlock, so we From 999a36815403a4ef133e5b44470e0559e05dad22 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 4 Nov 2015 09:13:27 -0500 Subject: [PATCH 43/50] Move FileHandle into its own module --- .../native/libhdfspp/include/libhdfspp/hdfs.h | 4 +-- .../main/native/libhdfspp/lib/fs/filesystem.h | 32 ++----------------- 2 files changed, 4 insertions(+), 32 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h index 0c3e9346c7774..1974ea969ccc8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h @@ -74,9 +74,7 @@ class FileHandle { **/ virtual CancelHandle PositionRead(void *buf, size_t nbyte, uint64_t offset, - const std::set &excluded_datanodes, - const std::function &handler) = 0; + const std::function &handler) = 0; virtual size_t PositionRead(void *buf, size_t nbyte, off_t offset) = 0; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 3aff55d336bbc..5dad167052a77 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -18,6 +18,7 @@ #ifndef LIBHDFSPP_LIB_FS_FILESYSTEM_H_ #define LIBHDFSPP_LIB_FS_FILESYSTEM_H_ +#include "filehandle.h" #include "common/hdfs_public_api.h" #include "common/async_stream.h" #include "libhdfspp/hdfs.h" @@ -40,6 +41,7 @@ namespace hdfs { * Will eventually handle retry and failover. * * Threading model: thread-safe; all operations can be called concurrently + * Lifetime: owned by a FileSystemImpl */ class NameNodeOperations { public: @@ -71,6 +73,7 @@ void GetBlockLocations(const std::string & path, std::function file_info); - virtual CancelHandle - PositionRead(void *buf, size_t nbyte, uint64_t offset, - const std::set &excluded_datanodes, - const std::function &handler) override; - - size_t PositionRead(void *buf, size_t nbyte, off_t offset) override; - - CancelHandle AsyncPreadSome(size_t offset, const MutableBuffers &buffers, - const std::set &excluded_datanodes, - const std::function handler); -private: - ::asio::io_service *io_service_; - const std::string client_name_; - const std::shared_ptr file_info_; - - std::shared_ptr dn_; // The last DN connected to -}; - } #endif From 7080f8a7809ba538f13564f06cf884ab9ed9f628 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 4 Nov 2015 10:12:20 -0500 Subject: [PATCH 44/50] Added missing files --- .../native/libhdfspp/lib/common/cancelable.h | 48 ++++++ .../native/libhdfspp/lib/fs/filehandle.cc | 141 ++++++++++++++++++ .../main/native/libhdfspp/lib/fs/filehandle.h | 79 ++++++++++ 3 files changed, 268 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/cancelable.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/cancelable.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/cancelable.h new file mode 100644 index 0000000000000..8d38ae9fedce7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/cancelable.h @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIB_COMMON_CANCELABLE_H_ +#define LIB_COMMON_CANCELABLE_H_ + +#include + +namespace hdfs { + +class Cancelable { +public: + virtual void cancel() = 0; +}; + +class NullCancelable : public Cancelable { +public: + NullCancelable() {}; + virtual void cancel() {}; + +}; + +class CancelHandle : public Cancelable { +public: + CancelHandle(std::shared_ptr target) : target_(target) {} + + void cancel() override { target_->cancel(); } +private: + std::shared_ptr target_; +}; + +} + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc new file mode 100644 index 0000000000000..eba4e180732ba --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc @@ -0,0 +1,141 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "filehandle.h" +#include "common/continuation/continuation.h" +#include "connection/datanodeconnection.h" +#include "reader/block_reader.h" + +#include "ClientNamenodeProtocol.pb.h" + +#include + +namespace hdfs { + +using ::hadoop::hdfs::LocatedBlocksProto; + +FileHandle::~FileHandle() {} + +FileHandleImpl::FileHandleImpl(::asio::io_service *io_service, const std::string &client_name, + const std::shared_ptr file_info) + : io_service_(io_service), client_name_(client_name), file_info_(file_info) { +} + +CancelHandle FileHandleImpl::PositionRead( + void *buf, size_t nbyte, uint64_t offset, + const std::function + &handler) { + return AsyncPreadSome(offset, asio::buffer(buf, nbyte), std::set(), + [handler](const Status &status, const std::string &dn_id, size_t bytes_read){ + (void)dn_id; + handler(status, bytes_read); } + ); +} + +size_t FileHandleImpl::PositionRead(void *buf, size_t nbyte, off_t offset) { + auto stat = std::make_shared>(); + std::future future(stat->get_future()); + + /* wrap async call with promise/future to make it blocking */ + size_t read_count = 0; + auto callback = [stat, &read_count](const Status &s, size_t bytes) { + stat->set_value(s); + read_count = bytes; + }; + + PositionRead(buf, nbyte, offset, callback); + + /* wait for async to finish */ + auto s = future.get(); + + if (!s.ok()) { + return -1; + } + return (ssize_t)read_count; +} + +CancelHandle FileHandleImpl::AsyncPreadSome( + size_t offset, const MutableBuffers &buffers, + const std::set &excluded_datanodes, + const std::function handler) { + using ::hadoop::hdfs::DatanodeInfoProto; + using ::hadoop::hdfs::LocatedBlockProto; + + auto it = std::find_if( + file_info_->blocks_.begin(), file_info_->blocks_.end(), [offset](const LocatedBlockProto &p) { + return p.offset() <= offset && offset < p.offset() + p.b().numbytes(); + }); + + if (it == file_info_->blocks_.end()) { + handler(Status::InvalidArgument("Cannot find corresponding blocks"), "", 0); + return CancelHandle(nullptr); + } + + ::hadoop::hdfs::LocatedBlockProto targetBlock = *it; + + const DatanodeInfoProto *chosen_dn = nullptr; + for (int i = 0; i < targetBlock.locs_size(); ++i) { + const auto &di = targetBlock.locs(i); + if (!excluded_datanodes.count(di.id().datanodeuuid())) { + chosen_dn = &di; + break; + } + } + + if (!chosen_dn) { + handler(Status::ResourceUnavailable("No datanodes available"), "", 0); + return CancelHandle(nullptr); + } + + uint64_t offset_within_block = offset - targetBlock.offset(); + uint64_t size_within_block = std::min( + targetBlock.b().numbytes() - offset_within_block, asio::buffer_size(buffers)); + + // This is where we will put the logic for re-using a DN connection; we can + // steal the FileHandle's dn and put it back when we're done + std::shared_ptr dn = std::make_shared(io_service_, *chosen_dn, nullptr /*token*/); + std::string dn_id = dn->uuid_; + std::string client_name = client_name_; + + // Wrap the DN in a block reader to handle the state and logic of the + // block request protocol + std::shared_ptr reader; + reader.reset(new BlockReaderImpl(BlockReaderOptions(), dn)); + + + auto read_handler = [dn_id, handler](const Status & status, size_t transferred) { + handler(status, dn_id, transferred); + }; + + dn->Connect([handler,read_handler,targetBlock,offset_within_block,size_within_block, buffers, reader, dn_id, client_name] + (Status status, std::shared_ptr dn) { + (void)dn; + if (status.ok()) { + reader->AsyncReadBlock( + client_name, targetBlock, offset_within_block, + asio::buffer(buffers, size_within_block), read_handler); + } else { + handler(status, dn_id, 0); + } + }); + + return CancelHandle(dn); +} + + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h new file mode 100644 index 0000000000000..f7df7486a1d46 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBHDFSPP_LIB_FS_FILEHANDLE_H_ +#define LIBHDFSPP_LIB_FS_FILEHANDLE_H_ + +#include "common/hdfs_public_api.h" +#include "common/async_stream.h" +#include "reader/fileinfo.h" + +#include "asio.hpp" + +#include + +namespace hdfs { + +/* + * FileHandle: coordinates operations on a particular file in HDFS + * + * Threading model: not thread-safe; consumers and io_service should not call + * concurrently. PositionRead and cancel() are the exceptions; they can be + * called concurrently and repeatedly. + * Lifetime: pointer returned to consumer by FileSystem::Open. Consumer is + * resonsible for freeing the object. + */ +class FileHandleImpl : public FileHandle { +public: + FileHandleImpl(::asio::io_service *io_service, const std::string &client_name, + const std::shared_ptr file_info); + + /* + * [Some day reliably] Reads a particular offset into the data file. + * On error, bytes_read returns the number of bytes successfully read; on + * success, bytes_read will equal nbyte + */ + CancelHandle PositionRead( + void *buf, + size_t nbyte, + uint64_t offset, + const std::function &handler + ) override; + size_t PositionRead(void *buf, size_t bytes_read, off_t offset) override; + + /* + * Reads some amount of data into the buffer. Will attempt to find the best + * datanode and read data from it. + * + * If an error occurs during connection or transfer, the callback will be + * called with bytes_read equal to the number of bytes successfully transferred. + * If no data nodes can be found, status will be Status::ResourceUnavailable. + * + */ + CancelHandle AsyncPreadSome(size_t offset, const MutableBuffers &buffers, + const std::set &excluded_datanodes, + const std::function handler); +private: + ::asio::io_service *io_service_; + const std::string client_name_; + const std::shared_ptr file_info_; + // Shared pointer to the FileSystem's dead nodes object goes here +}; + +} + +#endif From 7a0620c1627b74bf755c6a8094b37a8b4e7568d0 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 4 Nov 2015 10:33:54 -0500 Subject: [PATCH 45/50] Roll back pom file --- hadoop-common-project/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-common-project/pom.xml b/hadoop-common-project/pom.xml index cc641da138d84..ef49f9ce08bc0 100644 --- a/hadoop-common-project/pom.xml +++ b/hadoop-common-project/pom.xml @@ -33,6 +33,7 @@ hadoop-auth hadoop-auth-examples + hadoop-common hadoop-annotations hadoop-nfs hadoop-minikdc From 6a4a43a6d51459b210f61daad4470c5bda5240c3 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 4 Nov 2015 10:52:15 -0500 Subject: [PATCH 46/50] Some code cleanup --- .../libhdfspp/lib/common/async_stream.h | 10 ++++- .../native/libhdfspp/lib/common/cancelable.h | 9 +---- .../lib/connection/datanodeconnection.h | 4 +- .../native/libhdfspp/lib/fs/filehandle.cc | 10 +++++ .../main/native/libhdfspp/lib/fs/filehandle.h | 2 +- .../native/libhdfspp/lib/fs/filesystem.cc | 39 ++++++++++++------- .../main/native/libhdfspp/lib/fs/filesystem.h | 4 +- 7 files changed, 51 insertions(+), 27 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h index 1daa3c16f5b55..f25f5f8daf9d0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h @@ -27,7 +27,15 @@ namespace hdfs { typedef asio::mutable_buffers_1 MutableBuffers; typedef asio::const_buffers_1 ConstBuffers; -class AsyncStream { +/* + * asio-compatible stream implementation. + * + * Lifecycle: should be managed using std::shared_ptr so the object can be + * handed from consumer to consumer + * Threading model: async_read_some and async_write_some are not thread-safe. + * Cancel() can be called from any context + */ +class AsyncStream : public Cancelable { public: virtual void async_read_some(const MutableBuffers &buf, std::function target) : target_(target) {} - void cancel() override { target_->cancel(); } + void cancel() override { if (target_) target_->cancel(); } private: std::shared_ptr target_; }; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h index 8dbcffc9e64b2..7f6f81b925f4d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h @@ -26,14 +26,12 @@ namespace hdfs { -class DataNodeConnection : public AsyncStream, public Cancelable { +class DataNodeConnection : public AsyncStream { public: std::string uuid_; std::unique_ptr token_; virtual void Connect(std::function dn)> handler) = 0; - - virtual void cancel() = 0; }; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc index eba4e180732ba..6389adb853ca9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc @@ -36,10 +36,16 @@ FileHandleImpl::FileHandleImpl(::asio::io_service *io_service, const std::string : io_service_(io_service), client_name_(client_name), file_info_(file_info) { } + + + CancelHandle FileHandleImpl::PositionRead( void *buf, size_t nbyte, uint64_t offset, const std::function &handler) { + + // This is where retry and dead DN node elision will occur + return AsyncPreadSome(offset, asio::buffer(buf, nbyte), std::set(), [handler](const Status &status, const std::string &dn_id, size_t bytes_read){ (void)dn_id; @@ -69,6 +75,10 @@ size_t FileHandleImpl::PositionRead(void *buf, size_t nbyte, off_t offset) { return (ssize_t)read_count; } +/* + * Note that this method must be thread-safe w.r.t. the unsafe operations occurring + * on the FileHandle + */ CancelHandle FileHandleImpl::AsyncPreadSome( size_t offset, const MutableBuffers &buffers, const std::set &excluded_datanodes, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h index f7df7486a1d46..0e0d3741be0c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h @@ -68,7 +68,7 @@ class FileHandleImpl : public FileHandle { const std::set &excluded_datanodes, const std::function handler); private: - ::asio::io_service *io_service_; + ::asio::io_service * const io_service_; const std::string client_name_; const std::shared_ptr file_info_; // Shared pointer to the FileSystem's dead nodes object goes here diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc index f0d6433dec95f..5f63aa136bb97 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc @@ -34,6 +34,10 @@ static const int kNamenodeProtocolVersion = 1; using ::asio::ip::tcp; +/***************************************************************************** + * NAMENODE OPERATIONS + ****************************************************************************/ + void NameNodeOperations::Connect(const std::string &server, const std::string &service, std::function &handler) { @@ -100,6 +104,10 @@ void NameNodeOperations::GetBlockLocations(const std::string & path, } +/***************************************************************************** + * FILESYSTEM BASE CLASS + ****************************************************************************/ + void FileSystem::New( IoService *io_service, const Options &options, const std::string &server, const std::string &service, @@ -137,12 +145,16 @@ FileSystem * FileSystem::New( return nullptr; } - FileSystemImpl::FileSystemImpl(IoService *&io_service, const Options &options) - : io_service_(static_cast(io_service)), - nn_(&io_service_->io_service(), options, - GetRandomClientName(), kNamenodeProtocol, - kNamenodeProtocolVersion), - client_name_(GetRandomClientName()) +/***************************************************************************** + * FILESYSTEM IMPLEMENTATION + ****************************************************************************/ + +FileSystemImpl::FileSystemImpl(IoService *&io_service, const Options &options) + : io_service_(static_cast(io_service)), + nn_(&io_service_->io_service(), options, + GetRandomClientName(), kNamenodeProtocol, + kNamenodeProtocolVersion), + client_name_(GetRandomClientName()) { // Poor man's move io_service = nullptr; @@ -191,6 +203,14 @@ Status FileSystemImpl::Connect(const std::string &server, const std::string &ser return s; } + +int FileSystemImpl::AddWorkerThread() { + auto service_task = [](IoService *service) { service->Run(); }; + worker_threads_.push_back( + WorkerPtr(new std::thread(service_task, io_service_.get()))); + return worker_threads_.size(); +} + void FileSystemImpl::Open( const std::string &path, const std::function &handler) { @@ -201,13 +221,6 @@ void FileSystemImpl::Open( }); } -int FileSystemImpl::AddWorkerThread() { - auto service_task = [](IoService *service) { service->Run(); }; - worker_threads_.push_back( - WorkerPtr(new std::thread(service_task, io_service_.get()))); - return worker_threads_.size(); -} - Status FileSystemImpl::Open(const std::string &path, FileHandle **handle) { auto stat = std::make_shared>(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 5dad167052a77..5fc439c76ea42 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -56,7 +56,9 @@ class NameNodeOperations { const std::string &service, std::function &handler); -void GetBlockLocations(const std::string & path, std::function)> handler); + void GetBlockLocations(const std::string & path, + std::function)> handler); + private: ::asio::io_service * io_service_; RpcEngine engine_; From a6c59119b75342cb83a15054666268b882576f64 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 4 Nov 2015 10:53:04 -0500 Subject: [PATCH 47/50] Cleared whitespace --- .../libhdfspp/lib/common/async_stream.h | 4 ++-- .../native/libhdfspp/lib/common/cancelable.h | 2 +- .../lib/connection/datanodeconnection.h | 4 ++-- .../native/libhdfspp/lib/fs/filehandle.cc | 8 ++++---- .../main/native/libhdfspp/lib/fs/filehandle.h | 20 +++++++++---------- .../main/native/libhdfspp/lib/fs/filesystem.h | 4 ++-- .../tests/remote_block_reader_test.cc | 2 +- 7 files changed, 22 insertions(+), 22 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h index f25f5f8daf9d0..80006473a7851 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h @@ -28,8 +28,8 @@ typedef asio::mutable_buffers_1 MutableBuffers; typedef asio::const_buffers_1 ConstBuffers; /* - * asio-compatible stream implementation. - * + * asio-compatible stream implementation. + * * Lifecycle: should be managed using std::shared_ptr so the object can be * handed from consumer to consumer * Threading model: async_read_some and async_write_some are not thread-safe. diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/cancelable.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/cancelable.h index fccc8fbcc7c99..7cd1cfa626032 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/cancelable.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/cancelable.h @@ -30,7 +30,7 @@ class Cancelable { class CancelHandle : public Cancelable { public: CancelHandle(std::shared_ptr target) : target_(target) {} - + void cancel() override { if (target_) target_->cancel(); } private: std::shared_ptr target_; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h index 7f6f81b925f4d..03bc81f200e66 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h @@ -71,12 +71,12 @@ class DataNodeConnectionImpl : public DataNodeConnection, public std::enable_sha std::size_t bytes_transferred) > handler) override { conn_->async_write_some(buf, handler); } - + void cancel() override { // Can't portably use asio::cancel; see // http://www.boost.org/doc/libs/1_59_0/doc/html/boost_asio/reference/basic_stream_socket/cancel/overload2.html // - // When we implement connection re-use, we need to be sure to throw out + // When we implement connection re-use, we need to be sure to throw out // connections on error conn_->close(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc index 6389adb853ca9..b8f4746104866 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc @@ -44,9 +44,9 @@ CancelHandle FileHandleImpl::PositionRead( const std::function &handler) { - // This is where retry and dead DN node elision will occur - - return AsyncPreadSome(offset, asio::buffer(buf, nbyte), std::set(), + // This is where retry and dead DN node elision will occur + + return AsyncPreadSome(offset, asio::buffer(buf, nbyte), std::set(), [handler](const Status &status, const std::string &dn_id, size_t bytes_read){ (void)dn_id; handler(status, bytes_read); } @@ -143,7 +143,7 @@ CancelHandle FileHandleImpl::AsyncPreadSome( handler(status, dn_id, 0); } }); - + return CancelHandle(dn); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h index 0e0d3741be0c8..60f34ba1388be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h @@ -32,24 +32,24 @@ namespace hdfs { * FileHandle: coordinates operations on a particular file in HDFS * * Threading model: not thread-safe; consumers and io_service should not call - * concurrently. PositionRead and cancel() are the exceptions; they can be + * concurrently. PositionRead and cancel() are the exceptions; they can be * called concurrently and repeatedly. - * Lifetime: pointer returned to consumer by FileSystem::Open. Consumer is + * Lifetime: pointer returned to consumer by FileSystem::Open. Consumer is * resonsible for freeing the object. */ class FileHandleImpl : public FileHandle { public: FileHandleImpl(::asio::io_service *io_service, const std::string &client_name, const std::shared_ptr file_info); - + /* - * [Some day reliably] Reads a particular offset into the data file. + * [Some day reliably] Reads a particular offset into the data file. * On error, bytes_read returns the number of bytes successfully read; on * success, bytes_read will equal nbyte */ CancelHandle PositionRead( - void *buf, - size_t nbyte, + void *buf, + size_t nbyte, uint64_t offset, const std::function &handler ) override; @@ -57,12 +57,12 @@ class FileHandleImpl : public FileHandle { /* * Reads some amount of data into the buffer. Will attempt to find the best - * datanode and read data from it. - * - * If an error occurs during connection or transfer, the callback will be + * datanode and read data from it. + * + * If an error occurs during connection or transfer, the callback will be * called with bytes_read equal to the number of bytes successfully transferred. * If no data nodes can be found, status will be Status::ResourceUnavailable. - * + * */ CancelHandle AsyncPreadSome(size_t offset, const MutableBuffers &buffers, const std::set &excluded_datanodes, diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index 5fc439c76ea42..e2aef1dd5cb78 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -56,9 +56,9 @@ class NameNodeOperations { const std::string &service, std::function &handler); - void GetBlockLocations(const std::string & path, + void GetBlockLocations(const std::string & path, std::function)> handler); - + private: ::asio::io_service * io_service_; RpcEngine engine_; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc index 9f252bf99c8d7..80b50af44d4cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc @@ -74,7 +74,7 @@ class MockDNConnection : public MockConnectionBase, public DataNodeConnection{ std::size_t bytes_transferred) > handler) override { this->MockConnectionBase::async_write_some(buf, handler); } - + void cancel() override {}; }; From 4b99eb624704f6dd98e6d67a66bba9e1bd8f1a96 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Wed, 4 Nov 2015 11:01:03 -0500 Subject: [PATCH 48/50] Added docs to blockreader --- .../libhdfspp/lib/reader/block_reader.h | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h index 43059d99c31c1..140286b9fcdbe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h @@ -57,8 +57,21 @@ struct BlockReaderOptions { : verify_checksum(true), encryption_scheme(EncryptionScheme::kNone) {} }; +/** + * Handles the operational state of request and reading a block (or portion of + * a block) from a DataNode. + * + * Threading model: not thread-safe. + * Lifecycle: should be created, used for a single read, then freed. + */ class BlockReader { public: + virtual void AsyncReadBlock( + const std::string & client_name, + const hadoop::hdfs::LocatedBlockProto &block, size_t offset, + const MutableBuffers &buffers, + const std::function handler) = 0; + virtual void AsyncReadPacket( const MutableBuffers &buffers, const std::function &handler) = 0; @@ -69,12 +82,6 @@ class BlockReader { uint64_t length, uint64_t offset, const std::function &handler) = 0; - - virtual void AsyncReadBlock( - const std::string & client_name, - const hadoop::hdfs::LocatedBlockProto &block, size_t offset, - const MutableBuffers &buffers, - const std::function handler) = 0; }; class BlockReaderImpl From 5d30337a7c6befd072767ddbece5574c359a41fd Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Fri, 6 Nov 2015 16:17:58 -0500 Subject: [PATCH 49/50] Fixed virtual method calls in FileSystem --- .../src/main/native/libhdfspp/include/libhdfspp/hdfs.h | 4 ++-- .../src/main/native/libhdfspp/lib/fs/filesystem.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h index 1974ea969ccc8..6961f30017220 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs.h @@ -109,9 +109,9 @@ class FileSystem { virtual void Open(const std::string &path, const std::function &handler) = 0; - Status Open(const std::string &path, FileHandle **handle); + virtual Status Open(const std::string &path, FileHandle **handle); - virtual ~FileSystem(); + virtual ~FileSystem() {}; }; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h index e2aef1dd5cb78..9740e9e26ecf8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h @@ -80,7 +80,7 @@ class NameNodeOperations { class FileSystemImpl : public FileSystem { public: FileSystemImpl(IoService *&io_service, const Options &options); - ~FileSystemImpl(); + ~FileSystemImpl() override; /* attempt to connect to namenode, return bad status on failure */ void Connect(const std::string &server, const std::string &service, @@ -92,7 +92,7 @@ class FileSystemImpl : public FileSystem { virtual void Open(const std::string &path, const std::function &handler) override; - Status Open(const std::string &path, FileHandle **handle); + Status Open(const std::string &path, FileHandle **handle) override; /* add a new thread to handle asio requests, return number of threads in pool From f1382d1621d2954c9369b4dafcfd4274df57dfc8 Mon Sep 17 00:00:00 2001 From: Bob Hansen Date: Fri, 6 Nov 2015 16:18:44 -0500 Subject: [PATCH 50/50] Retain reader through last callback --- .../src/main/native/libhdfspp/lib/fs/filehandle.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc index b8f4746104866..d93e82261385d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc @@ -128,7 +128,7 @@ CancelHandle FileHandleImpl::AsyncPreadSome( reader.reset(new BlockReaderImpl(BlockReaderOptions(), dn)); - auto read_handler = [dn_id, handler](const Status & status, size_t transferred) { + auto read_handler = [reader, dn_id, handler](const Status & status, size_t transferred) { handler(status, dn_id, transferred); };