|
@@ -20,6 +20,7 @@
|
|
|
|
|
|
#include "datatransfer.pb.h"
|
|
#include "datatransfer.pb.h"
|
|
#include "common/util.h"
|
|
#include "common/util.h"
|
|
|
|
+#include "common/cancel_tracker.h"
|
|
#include "reader/block_reader.h"
|
|
#include "reader/block_reader.h"
|
|
#include "reader/datatransfer.h"
|
|
#include "reader/datatransfer.h"
|
|
#include "reader/fileinfo.h"
|
|
#include "reader/fileinfo.h"
|
|
@@ -29,6 +30,8 @@
|
|
#include <gmock/gmock.h>
|
|
#include <gmock/gmock.h>
|
|
#include <gtest/gtest.h>
|
|
#include <gtest/gtest.h>
|
|
|
|
|
|
|
|
+#include <iostream>
|
|
|
|
+
|
|
using namespace hdfs;
|
|
using namespace hdfs;
|
|
|
|
|
|
using ::hadoop::common::TokenProto;
|
|
using ::hadoop::common::TokenProto;
|
|
@@ -58,14 +61,18 @@ namespace hdfs {
|
|
class MockDNConnection : public MockConnectionBase, public DataNodeConnection{
|
|
class MockDNConnection : public MockConnectionBase, public DataNodeConnection{
|
|
public:
|
|
public:
|
|
MockDNConnection(::asio::io_service &io_service)
|
|
MockDNConnection(::asio::io_service &io_service)
|
|
- : MockConnectionBase(&io_service) {}
|
|
|
|
|
|
+ : MockConnectionBase(&io_service), OnRead([](){}) {}
|
|
MOCK_METHOD0(Produce, ProducerResult());
|
|
MOCK_METHOD0(Produce, ProducerResult());
|
|
|
|
|
|
MOCK_METHOD1(Connect, void(std::function<void(Status status, std::shared_ptr<DataNodeConnection> dn)>));
|
|
MOCK_METHOD1(Connect, void(std::function<void(Status status, std::shared_ptr<DataNodeConnection> dn)>));
|
|
|
|
|
|
|
|
+ /* event handler to trigger side effects */
|
|
|
|
+ std::function<void(void)> OnRead;
|
|
|
|
+
|
|
void async_read_some(const MutableBuffers &buf,
|
|
void async_read_some(const MutableBuffers &buf,
|
|
std::function<void (const asio::error_code & error,
|
|
std::function<void (const asio::error_code & error,
|
|
std::size_t bytes_transferred) > handler) override {
|
|
std::size_t bytes_transferred) > handler) override {
|
|
|
|
+ this->OnRead();
|
|
this->MockConnectionBase::async_read_some(buf, handler);
|
|
this->MockConnectionBase::async_read_some(buf, handler);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -74,6 +81,10 @@ public:
|
|
std::size_t bytes_transferred) > handler) override {
|
|
std::size_t bytes_transferred) > handler) override {
|
|
this->MockConnectionBase::async_write_some(buf, handler);
|
|
this->MockConnectionBase::async_write_some(buf, handler);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ void Cancel() {
|
|
|
|
+ /* no-op, declared pure virtual */
|
|
|
|
+ }
|
|
};
|
|
};
|
|
|
|
|
|
// Mocks AsyncReadPacket and AsyncRequestBlock but not AsyncReadBlock, so we
|
|
// Mocks AsyncReadPacket and AsyncRequestBlock but not AsyncReadBlock, so we
|
|
@@ -81,7 +92,7 @@ public:
|
|
class PartialMockReader : public BlockReaderImpl {
|
|
class PartialMockReader : public BlockReaderImpl {
|
|
public:
|
|
public:
|
|
PartialMockReader() :
|
|
PartialMockReader() :
|
|
- BlockReaderImpl(BlockReaderOptions(), std::shared_ptr<DataNodeConnection>()) {};
|
|
|
|
|
|
+ BlockReaderImpl(BlockReaderOptions(), std::shared_ptr<DataNodeConnection>(), CancelTracker::New()) {};
|
|
|
|
|
|
MOCK_METHOD2(
|
|
MOCK_METHOD2(
|
|
AsyncReadPacket,
|
|
AsyncReadPacket,
|
|
@@ -221,9 +232,9 @@ template <class Stream = MockDNConnection, class Handler>
|
|
static std::shared_ptr<BlockReaderImpl>
|
|
static std::shared_ptr<BlockReaderImpl>
|
|
ReadContent(std::shared_ptr<Stream> conn, const ExtendedBlockProto &block,
|
|
ReadContent(std::shared_ptr<Stream> conn, const ExtendedBlockProto &block,
|
|
uint64_t length, uint64_t offset, const mutable_buffers_1 &buf,
|
|
uint64_t length, uint64_t offset, const mutable_buffers_1 &buf,
|
|
- const Handler &handler) {
|
|
|
|
|
|
+ const Handler &handler, CancelHandle cancel_handle = CancelTracker::New()) {
|
|
BlockReaderOptions options;
|
|
BlockReaderOptions options;
|
|
- auto reader = std::make_shared<BlockReaderImpl>(options, conn);
|
|
|
|
|
|
+ auto reader = std::make_shared<BlockReaderImpl>(options, conn, cancel_handle);
|
|
Status result;
|
|
Status result;
|
|
reader->AsyncRequestBlock("libhdfs++", &block, length, offset,
|
|
reader->AsyncRequestBlock("libhdfs++", &block, length, offset,
|
|
[buf, reader, handler](const Status &stat) {
|
|
[buf, reader, handler](const Status &stat) {
|
|
@@ -268,6 +279,59 @@ TEST(RemoteBlockReaderTest, TestReadWholeBlock) {
|
|
ASSERT_TRUE(done);
|
|
ASSERT_TRUE(done);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+/* used for cancelation tests, global to avoid cluttering capture lists */
|
|
|
|
+CancelHandle packet_canceller;
|
|
|
|
+
|
|
|
|
+TEST(RemoteBlockReaderTest, TestCancelWhileReceiving) {
|
|
|
|
+ packet_canceller = CancelTracker::New();
|
|
|
|
+
|
|
|
|
+ static const size_t kChunkSize = 512;
|
|
|
|
+ static const string kChunkData(kChunkSize, 'a');
|
|
|
|
+ ::asio::io_service io_service;
|
|
|
|
+ auto conn = std::make_shared<MockDNConnection>(io_service);
|
|
|
|
+ BlockOpResponseProto block_op_resp;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * async_read would normally get called 5 times here; once for each
|
|
|
|
+ * continuation in the pipeline. Cancel will be triggered on the
|
|
|
|
+ * fourth call to catch the pipeline mid-execution.
|
|
|
|
+ **/
|
|
|
|
+ int call_count = 0;
|
|
|
|
+ int trigger_at_count = 4;
|
|
|
|
+ auto cancel_trigger = [&call_count, &trigger_at_count]() {
|
|
|
|
+ call_count += 1;
|
|
|
|
+ std::cout << "read called " << call_count << " times" << std::endl;
|
|
|
|
+ if(call_count == trigger_at_count)
|
|
|
|
+ packet_canceller->set_canceled();
|
|
|
|
+ };
|
|
|
|
+
|
|
|
|
+ conn->OnRead = cancel_trigger;
|
|
|
|
+
|
|
|
|
+ block_op_resp.set_status(::hadoop::hdfs::Status::SUCCESS);
|
|
|
|
+ EXPECT_CALL(*conn, Produce())
|
|
|
|
+ .WillOnce(Return(Produce(ToDelimitedString(&block_op_resp))))
|
|
|
|
+ .WillOnce(Return(ProducePacket(kChunkData, "", 0, 1, true)));
|
|
|
|
+
|
|
|
|
+ ExtendedBlockProto block;
|
|
|
|
+ block.set_poolid("foo");
|
|
|
|
+ block.set_blockid(0);
|
|
|
|
+ block.set_generationstamp(0);
|
|
|
|
+
|
|
|
|
+ bool done = false;
|
|
|
|
+ std::string data(kChunkSize, 0);
|
|
|
|
+ ReadContent(conn, block, kChunkSize, 0,
|
|
|
|
+ buffer(const_cast<char *>(data.c_str()), data.size()),
|
|
|
|
+ [&data, &io_service, &done](const Status &stat, size_t transferred) {
|
|
|
|
+ ASSERT_EQ(stat.code(), Status::kOperationCanceled);
|
|
|
|
+ ASSERT_EQ(0, transferred);
|
|
|
|
+ done = true;
|
|
|
|
+ io_service.stop();
|
|
|
|
+ }, packet_canceller);
|
|
|
|
+
|
|
|
|
+ io_service.run();
|
|
|
|
+ ASSERT_TRUE(done);
|
|
|
|
+}
|
|
|
|
+
|
|
TEST(RemoteBlockReaderTest, TestReadWithinChunk) {
|
|
TEST(RemoteBlockReaderTest, TestReadWithinChunk) {
|
|
static const size_t kChunkSize = 1024;
|
|
static const size_t kChunkSize = 1024;
|
|
static const size_t kLength = kChunkSize / 4 * 3;
|
|
static const size_t kLength = kChunkSize / 4 * 3;
|
|
@@ -332,7 +396,7 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) {
|
|
string data(kChunkSize, 0);
|
|
string data(kChunkSize, 0);
|
|
mutable_buffers_1 buf = buffer(const_cast<char *>(data.c_str()), data.size());
|
|
mutable_buffers_1 buf = buffer(const_cast<char *>(data.c_str()), data.size());
|
|
BlockReaderOptions options;
|
|
BlockReaderOptions options;
|
|
- auto reader = std::make_shared<BlockReaderImpl>(options, conn);
|
|
|
|
|
|
+ auto reader = std::make_shared<BlockReaderImpl>(options, conn, CancelTracker::New());
|
|
Status result;
|
|
Status result;
|
|
reader->AsyncRequestBlock(
|
|
reader->AsyncRequestBlock(
|
|
"libhdfs++", &block, data.size(), 0,
|
|
"libhdfs++", &block, data.size(), 0,
|
|
@@ -358,6 +422,60 @@ TEST(RemoteBlockReaderTest, TestReadMultiplePacket) {
|
|
io_service.run();
|
|
io_service.run();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+TEST(RemoteBlockReaderTest, TestReadCancelBetweenPackets) {
|
|
|
|
+ packet_canceller = CancelTracker::New();
|
|
|
|
+
|
|
|
|
+ static const size_t kChunkSize = 1024;
|
|
|
|
+ static const string kChunkData(kChunkSize, 'a');
|
|
|
|
+
|
|
|
|
+ ::asio::io_service 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())
|
|
|
|
+ .WillOnce(Return(Produce(ToDelimitedString(&block_op_resp))))
|
|
|
|
+ .WillOnce(Return(ProducePacket(kChunkData, "", 0, 1, false)));
|
|
|
|
+ /* the second AsyncReadPacket should never attempt to read */
|
|
|
|
+
|
|
|
|
+ ExtendedBlockProto block;
|
|
|
|
+ block.set_poolid("foo");
|
|
|
|
+ block.set_blockid(0);
|
|
|
|
+ block.set_generationstamp(0);
|
|
|
|
+
|
|
|
|
+ string data(kChunkSize, 0);
|
|
|
|
+ mutable_buffers_1 buf = buffer(const_cast<char *>(data.c_str()), data.size());
|
|
|
|
+ BlockReaderOptions options;
|
|
|
|
+ auto reader = std::make_shared<BlockReaderImpl>(options, conn, packet_canceller);
|
|
|
|
+ Status result;
|
|
|
|
+ reader->AsyncRequestBlock(
|
|
|
|
+ "libhdfs++", &block, data.size(), 0,
|
|
|
|
+ [buf, reader, &data, &io_service](const Status &stat) {
|
|
|
|
+ ASSERT_TRUE(stat.ok());
|
|
|
|
+ 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;
|
|
|
|
+
|
|
|
|
+ /* Cancel the operation.*/
|
|
|
|
+ packet_canceller->set_canceled();
|
|
|
|
+
|
|
|
|
+ reader->AsyncReadPacket(
|
|
|
|
+ buf, [&data,&io_service](const Status &stat, size_t transferred) {
|
|
|
|
+ ASSERT_EQ(stat.code(), Status::kOperationCanceled);
|
|
|
|
+ ASSERT_EQ(0, transferred);
|
|
|
|
+ io_service.stop();
|
|
|
|
+ });
|
|
|
|
+ });
|
|
|
|
+ });
|
|
|
|
+ io_service.run();
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+
|
|
TEST(RemoteBlockReaderTest, TestSaslConnection) {
|
|
TEST(RemoteBlockReaderTest, TestSaslConnection) {
|
|
static const size_t kChunkSize = 512;
|
|
static const size_t kChunkSize = 512;
|
|
static const string kChunkData(kChunkSize, 'a');
|
|
static const string kChunkData(kChunkSize, 'a');
|