|
@@ -21,6 +21,8 @@
|
|
|
#include "datatransfer.pb.h"
|
|
|
#include "common/util.h"
|
|
|
#include "reader/block_reader.h"
|
|
|
+#include "reader/datatransfer.h"
|
|
|
+#include "reader/fileinfo.h"
|
|
|
|
|
|
#include <google/protobuf/io/coded_stream.h>
|
|
|
#include <google/protobuf/io/zero_copy_stream_impl.h>
|
|
@@ -36,10 +38,14 @@ 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;
|
|
|
using ::asio::mutable_buffers_1;
|
|
|
+using ::testing::_;
|
|
|
+using ::testing::InvokeArgument;
|
|
|
using ::testing::Return;
|
|
|
using std::make_pair;
|
|
|
using std::string;
|
|
@@ -49,12 +55,47 @@ namespace pbio = pb::io;
|
|
|
|
|
|
namespace hdfs {
|
|
|
|
|
|
-class MockDNConnection : public MockConnectionBase {
|
|
|
- public:
|
|
|
+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<void(Status status, std::shared_ptr<DataNodeConnection> dn)>));
|
|
|
+
|
|
|
+ void async_read_some(const MutableBuffers &buf,
|
|
|
+ std::function<void (const asio::error_code & error,
|
|
|
+ std::size_t bytes_transferred) > handler) override {
|
|
|
+ this->MockConnectionBase::async_read_some(buf, handler);
|
|
|
+ }
|
|
|
+
|
|
|
+ void async_write_some(const ConstBuffers &buf,
|
|
|
+ std::function<void (const asio::error_code & error,
|
|
|
+ std::size_t bytes_transferred) > handler) override {
|
|
|
+ this->MockConnectionBase::async_write_some(buf, handler);
|
|
|
+ }
|
|
|
+};
|
|
|
+
|
|
|
+// Mocks AsyncReadPacket and AsyncRequestBlock but not AsyncReadBlock, so we
|
|
|
+// can test the logic of AsyncReadBlock
|
|
|
+class PartialMockReader : public BlockReaderImpl {
|
|
|
+public:
|
|
|
+ PartialMockReader() :
|
|
|
+ BlockReaderImpl(BlockReaderOptions(), std::shared_ptr<DataNodeConnection>()) {};
|
|
|
+
|
|
|
+ MOCK_METHOD2(
|
|
|
+ AsyncReadPacket,
|
|
|
+ void(const asio::mutable_buffers_1 &,
|
|
|
+ const std::function<void(const Status &, size_t transferred)> &));
|
|
|
+
|
|
|
+ MOCK_METHOD5(AsyncRequestBlock,
|
|
|
+ void(const std::string &client_name,
|
|
|
+ const hadoop::hdfs::ExtendedBlockProto *block,
|
|
|
+ uint64_t length, uint64_t offset,
|
|
|
+ const std::function<void(Status)> &handler));
|
|
|
};
|
|
|
+
|
|
|
+
|
|
|
}
|
|
|
|
|
|
static inline string ToDelimitedString(const pb::MessageLite *msg) {
|
|
@@ -94,20 +135,102 @@ static inline std::pair<error_code, string> ProducePacket(
|
|
|
return std::make_pair(error_code(), std::move(payload));
|
|
|
}
|
|
|
|
|
|
+TEST(RemoteBlockReaderTest, TestReadSingleTrunk) {
|
|
|
+ auto file_info = std::make_shared<struct FileInfo>();
|
|
|
+ LocatedBlocksProto blocks;
|
|
|
+ LocatedBlockProto block;
|
|
|
+ char buf[4096] = {
|
|
|
+ 0,
|
|
|
+ };
|
|
|
+
|
|
|
+ Status stat;
|
|
|
+ size_t read = 0;
|
|
|
+ PartialMockReader reader;
|
|
|
+ EXPECT_CALL(reader, AsyncRequestBlock(_, _, _, _, _))
|
|
|
+ .WillOnce(InvokeArgument<4>(Status::OK()));
|
|
|
+ EXPECT_CALL(reader, AsyncReadPacket(_, _))
|
|
|
+ .WillOnce(InvokeArgument<1>(Status::OK(), sizeof(buf)));
|
|
|
+
|
|
|
+ reader.AsyncReadBlock(
|
|
|
+ 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, AsyncRequestBlock(_, _, _, _, _))
|
|
|
+ .WillOnce(InvokeArgument<4>(Status::OK()));
|
|
|
+
|
|
|
+ EXPECT_CALL(reader, AsyncReadPacket(_, _))
|
|
|
+ .Times(4)
|
|
|
+ .WillRepeatedly(InvokeArgument<1>(Status::OK(), sizeof(buf) / 4));
|
|
|
+
|
|
|
+ reader.AsyncReadBlock(
|
|
|
+ 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, AsyncRequestBlock(_, _, _, _, _))
|
|
|
+ .WillOnce(InvokeArgument<4>(Status::OK()));
|
|
|
+
|
|
|
+ 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(
|
|
|
+ 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 <class Stream = MockDNConnection, class Handler>
|
|
|
-static std::shared_ptr<RemoteBlockReader<Stream>> ReadContent(
|
|
|
- Stream *conn, TokenProto *token, const ExtendedBlockProto &block,
|
|
|
- uint64_t length, uint64_t offset, const mutable_buffers_1 &buf,
|
|
|
- const Handler &handler) {
|
|
|
+static std::shared_ptr<BlockReaderImpl>
|
|
|
+ReadContent(std::shared_ptr<Stream> 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<RemoteBlockReader<Stream>>(options, conn);
|
|
|
+ auto reader = std::make_shared<BlockReaderImpl>(options, conn);
|
|
|
Status result;
|
|
|
- reader->async_connect("libhdfs++", token, &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_some(buf, handler);
|
|
|
+ reader->AsyncReadPacket(buf, handler);
|
|
|
}
|
|
|
});
|
|
|
return reader;
|
|
@@ -117,11 +240,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<MockDNConnection>(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,16 +253,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,
|
|
|
+ ReadContent(conn, block, kChunkSize, 0,
|
|
|
buffer(const_cast<char *>(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);
|
|
|
+ done = true;
|
|
|
io_service.stop();
|
|
|
});
|
|
|
io_service.run();
|
|
|
+ ASSERT_TRUE(done);
|
|
|
}
|
|
|
|
|
|
TEST(RemoteBlockReaderTest, TestReadWithinChunk) {
|
|
@@ -149,7 +275,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<MockDNConnection>(io_service);
|
|
|
BlockOpResponseProto block_op_resp;
|
|
|
ReadOpChecksumInfoProto *checksum_info =
|
|
|
block_op_resp.mutable_readopchecksuminfo();
|
|
@@ -159,7 +285,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,16 +294,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,
|
|
|
+ ReadContent(conn, block, data.size(), kOffset,
|
|
|
buffer(const_cast<char *>(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) {
|
|
@@ -185,11 +315,11 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) {
|
|
|
static const string kChunkData(kChunkSize, 'a');
|
|
|
|
|
|
::asio::io_service io_service;
|
|
|
- MockDNConnection conn(io_service);
|
|
|
+ auto conn = std::make_shared<MockDNConnection>(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)));
|
|
@@ -202,25 +332,22 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) {
|
|
|
string data(kChunkSize, 0);
|
|
|
mutable_buffers_1 buf = buffer(const_cast<char *>(data.c_str()), data.size());
|
|
|
BlockReaderOptions options;
|
|
|
- auto reader =
|
|
|
- std::make_shared<RemoteBlockReader<MockDNConnection>>(options, &conn);
|
|
|
+ auto reader = std::make_shared<BlockReaderImpl>(options, conn);
|
|
|
Status result;
|
|
|
- reader->async_connect(
|
|
|
- "libhdfs++", nullptr, &block, data.size(), 0,
|
|
|
+ reader->AsyncRequestBlock(
|
|
|
+ "libhdfs++", &block, data.size(), 0,
|
|
|
[buf, reader, &data, &io_service](const Status &stat) {
|
|
|
ASSERT_TRUE(stat.ok());
|
|
|
- reader->async_read_some(
|
|
|
- buf, [buf, reader, &data, &io_service](const Status &stat,
|
|
|
- size_t transferred) {
|
|
|
+ reader->AsyncReadPacket(
|
|
|
+ buf, [buf, reader, &data, &io_service](const Status &stat, size_t transferred) {
|
|
|
ASSERT_TRUE(stat.ok());
|
|
|
ASSERT_EQ(kChunkSize, transferred);
|
|
|
ASSERT_EQ(kChunkData, data);
|
|
|
data.clear();
|
|
|
data.resize(kChunkSize);
|
|
|
transferred = 0;
|
|
|
- reader->async_read_some(
|
|
|
- buf,
|
|
|
- [&data, &io_service](const Status &stat, size_t transferred) {
|
|
|
+ reader->AsyncReadPacket(
|
|
|
+ buf, [&data,&io_service](const Status &stat, size_t transferred) {
|
|
|
ASSERT_TRUE(stat.ok());
|
|
|
ASSERT_EQ(kChunkSize, transferred);
|
|
|
ASSERT_EQ(kChunkData, data);
|
|
@@ -234,12 +361,11 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) {
|
|
|
TEST(RemoteBlockReaderTest, TestSaslConnection) {
|
|
|
static const size_t kChunkSize = 512;
|
|
|
static const string kChunkData(kChunkSize, 'a');
|
|
|
- static const string kAuthPayload =
|
|
|
- "realm=\"0\",nonce=\"+GAWc+O6yEAWpew/"
|
|
|
- "qKah8qh4QZLoOLCDcTtEKhlS\",qop=\"auth\","
|
|
|
- "charset=utf-8,algorithm=md5-sess";
|
|
|
+ static const string kAuthPayload = "realm=\"0\",nonce=\"+GAWc+O6yEAWpew/"
|
|
|
+ "qKah8qh4QZLoOLCDcTtEKhlS\",qop=\"auth\","
|
|
|
+ "charset=utf-8,algorithm=md5-sess";
|
|
|
::asio::io_service io_service;
|
|
|
- MockDNConnection conn(io_service);
|
|
|
+ auto conn = std::make_shared<MockDNConnection>(io_service);
|
|
|
BlockOpResponseProto block_op_resp;
|
|
|
block_op_resp.set_status(::hadoop::hdfs::Status::SUCCESS);
|
|
|
|
|
@@ -252,23 +378,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<MockDNConnection> sasl_conn(&conn, "foo", "bar");
|
|
|
+ auto sasl_conn = std::make_shared<DataTransferSaslStream<MockDNConnection> >(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, block, kChunkSize, 0,
|
|
|
buffer(const_cast<char *>(data.c_str()), data.size()),
|
|
|
[&data, &io_service](const Status &stat, size_t transferred) {
|
|
|
ASSERT_TRUE(stat.ok());
|