HDFS-10247: libhdfs++: Datanode protocol version mismatch fix. Contributed by James Clampffer

This commit is contained in:
James 2016-04-13 10:22:30 -04:00 committed by James Clampffer
parent ed77d8d5df
commit 60c3437267
6 changed files with 88 additions and 42 deletions

View File

@ -27,6 +27,7 @@
#include <asio/read.hpp> #include <asio/read.hpp>
#include <asio/write.hpp> #include <asio/write.hpp>
#include <asio/ip/tcp.hpp> #include <asio/ip/tcp.hpp>
#include <memory>
namespace hdfs { namespace hdfs {
namespace asio_continuation { namespace asio_continuation {
@ -36,7 +37,7 @@ using namespace continuation;
template <class Stream, class MutableBufferSequence> template <class Stream, class MutableBufferSequence>
class ReadContinuation : public Continuation { class ReadContinuation : public Continuation {
public: public:
ReadContinuation(Stream *stream, const MutableBufferSequence &buffer) ReadContinuation(std::shared_ptr<Stream>& stream, const MutableBufferSequence &buffer)
: stream_(stream), buffer_(buffer) {} : stream_(stream), buffer_(buffer) {}
virtual void Run(const Next &next) override { virtual void Run(const Next &next) override {
auto handler = auto handler =
@ -45,14 +46,16 @@ public:
} }
private: private:
Stream *stream_; // prevent construction from raw ptr
ReadContinuation(Stream *stream, MutableBufferSequence &buffer);
std::shared_ptr<Stream> stream_;
MutableBufferSequence buffer_; MutableBufferSequence buffer_;
}; };
template <class Stream, class ConstBufferSequence> template <class Stream, class ConstBufferSequence>
class WriteContinuation : public Continuation { class WriteContinuation : public Continuation {
public: public:
WriteContinuation(Stream *stream, const ConstBufferSequence &buffer) WriteContinuation(std::shared_ptr<Stream>& stream, const ConstBufferSequence &buffer)
: stream_(stream), buffer_(buffer) {} : stream_(stream), buffer_(buffer) {}
virtual void Run(const Next &next) override { virtual void Run(const Next &next) override {
@ -62,7 +65,9 @@ public:
} }
private: private:
Stream *stream_; // prevent construction from raw ptr
WriteContinuation(Stream *stream, ConstBufferSequence &buffer);
std::shared_ptr<Stream> stream_;
ConstBufferSequence buffer_; ConstBufferSequence buffer_;
}; };
@ -117,13 +122,13 @@ private:
}; };
template <class Stream, class ConstBufferSequence> template <class Stream, class ConstBufferSequence>
static inline Continuation *Write(Stream *stream, static inline Continuation *Write(std::shared_ptr<Stream> stream,
const ConstBufferSequence &buffer) { const ConstBufferSequence &buffer) {
return new WriteContinuation<Stream, ConstBufferSequence>(stream, buffer); return new WriteContinuation<Stream, ConstBufferSequence>(stream, buffer);
} }
template <class Stream, class MutableBufferSequence> template <class Stream, class MutableBufferSequence>
static inline Continuation *Read(Stream *stream, static inline Continuation *Read(std::shared_ptr<Stream> stream,
const MutableBufferSequence &buffer) { const MutableBufferSequence &buffer) {
return new ReadContinuation<Stream, MutableBufferSequence>(stream, buffer); return new ReadContinuation<Stream, MutableBufferSequence>(stream, buffer);
} }

View File

@ -94,13 +94,14 @@ struct WriteDelimitedPBMessageContinuation : Continuation {
: stream_(stream), msg_(msg) {} : stream_(stream), msg_(msg) {}
virtual void Run(const Next &next) override { virtual void Run(const Next &next) override {
namespace pbio = google::protobuf::io; bool success = true;
int size = msg_->ByteSize(); buf_ = SerializeDelimitedProtobufMessage(msg_, &success);
buf_.reserve(pbio::CodedOutputStream::VarintSize32(size) + size);
pbio::StringOutputStream ss(&buf_); if(!success) {
pbio::CodedOutputStream os(&ss); next(Status::Error("Unable to serialize protobuf message."));
os.WriteVarint32(size); return;
msg_->SerializeToCodedStream(&os); }
asio::async_write(*stream_, 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)); } );
} }

View File

@ -18,12 +18,44 @@
#include "common/util.h" #include "common/util.h"
#include <google/protobuf/io/zero_copy_stream_impl_lite.h>
namespace hdfs { namespace hdfs {
bool ReadDelimitedPBMessage(::google::protobuf::io::CodedInputStream *in,
::google::protobuf::MessageLite *msg) {
uint32_t size = 0;
in->ReadVarint32(&size);
auto limit = in->PushLimit(size);
bool res = msg->ParseFromCodedStream(in);
in->PopLimit(limit);
return res;
}
std::string SerializeDelimitedProtobufMessage(const ::google::protobuf::MessageLite *msg,
bool *err) {
namespace pbio = ::google::protobuf::io;
std::string buf;
int size = msg->ByteSize();
buf.reserve(pbio::CodedOutputStream::VarintSize32(size) + size);
pbio::StringOutputStream ss(&buf);
pbio::CodedOutputStream os(&ss);
os.WriteVarint32(size);
if(err)
*err = msg->SerializeToCodedStream(&os);
return buf;
}
std::string GetRandomClientName() { std::string GetRandomClientName() {
unsigned char buf[6] = { unsigned char buf[6];
0,
};
RAND_pseudo_bytes(buf, sizeof(buf)); RAND_pseudo_bytes(buf, sizeof(buf));
std::stringstream ss; std::stringstream ss;

View File

@ -28,6 +28,7 @@
#include <google/protobuf/message_lite.h> #include <google/protobuf/message_lite.h>
#include <google/protobuf/io/coded_stream.h> #include <google/protobuf/io/coded_stream.h>
namespace hdfs { namespace hdfs {
static inline Status ToStatus(const ::asio::error_code &ec) { static inline Status ToStatus(const ::asio::error_code &ec) {
@ -38,32 +39,30 @@ static inline Status ToStatus(const ::asio::error_code &ec) {
} }
} }
static inline int DelimitedPBMessageSize( // Determine size of buffer that needs to be allocated in order to serialize msg
const ::google::protobuf::MessageLite *msg) { // in delimited format
static inline int DelimitedPBMessageSize(const ::google::protobuf::MessageLite *msg) {
size_t size = msg->ByteSize(); size_t size = msg->ByteSize();
return ::google::protobuf::io::CodedOutputStream::VarintSize32(size) + size; return ::google::protobuf::io::CodedOutputStream::VarintSize32(size) + size;
} }
static inline void ReadDelimitedPBMessage( // Construct msg from the input held in the CodedInputStream
::google::protobuf::io::CodedInputStream *in, // return false on failure, otherwise return true
::google::protobuf::MessageLite *msg) { bool ReadDelimitedPBMessage(::google::protobuf::io::CodedInputStream *in,
uint32_t size = 0; ::google::protobuf::MessageLite *msg);
in->ReadVarint32(&size);
auto limit = in->PushLimit(size); // Serialize msg into a delimited form (java protobuf compatible)
msg->ParseFromCodedStream(in); // err, if not null, will be set to false on failure
in->PopLimit(limit); std::string SerializeDelimitedProtobufMessage(const ::google::protobuf::MessageLite *msg,
} bool *err);
std::string Base64Encode(const std::string &src); std::string Base64Encode(const std::string &src);
/* // Return a new high-entropy client name
* Returns a new high-entropy client name
*/
std::string GetRandomClientName(); std::string GetRandomClientName();
/* Returns true if _someone_ is holding the lock (not necessarily this thread, // Returns true if _someone_ is holding the lock (not necessarily this thread,
* but a std::mutex doesn't track which thread is holding the lock) // but a std::mutex doesn't track which thread is holding the lock)
*/
template<class T> template<class T>
bool lock_held(T & mutex) { bool lock_held(T & mutex) {
bool result = !mutex.try_lock(); bool result = !mutex.try_lock();
@ -72,8 +71,6 @@ bool lock_held(T & mutex) {
return result; return result;
} }
} }
#endif #endif

View File

@ -20,6 +20,7 @@
#include "common/continuation/continuation.h" #include "common/continuation/continuation.h"
#include "common/continuation/asio.h" #include "common/continuation/asio.h"
#include "common/logging.h" #include "common/logging.h"
#include "common/util.h"
#include <future> #include <future>
@ -55,6 +56,9 @@ ReadBlockProto(const std::string &client_name, bool verify_checksum,
return p; return p;
} }
static int8_t unsecured_request_block_header[3] = {0, kDataTransferVersion, Operation::kReadBlock};
void BlockReaderImpl::AsyncRequestBlock( void BlockReaderImpl::AsyncRequestBlock(
const std::string &client_name, const std::string &client_name,
const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length, const hadoop::hdfs::ExtendedBlockProto *block, uint64_t length,
@ -78,17 +82,24 @@ void BlockReaderImpl::AsyncRequestBlock(
auto m = continuation::Pipeline<State>::Create(cancel_state_); auto m = continuation::Pipeline<State>::Create(cancel_state_);
State *s = &m->state(); State *s = &m->state();
s->header.insert(s->header.begin(), s->request = ReadBlockProto(client_name, options_.verify_checksum,
{0, kDataTransferVersion, Operation::kReadBlock}); dn_->token_.get(), block, length, offset);
s->request = std::move(ReadBlockProto(client_name, options_.verify_checksum,
dn_->token_.get(), block, length, offset)); s->header = std::string((const char*)unsecured_request_block_header, 3);
bool serialize_success = true;
s->header += SerializeDelimitedProtobufMessage(&s->request, &serialize_success);
if(!serialize_success) {
handler(Status::Error("Unable to serialize protobuf message"));
return;
}
auto read_pb_message = auto read_pb_message =
new continuation::ReadDelimitedPBMessageContinuation<AsyncStream, 16384>( new continuation::ReadDelimitedPBMessageContinuation<AsyncStream, 16384>(
dn_, &s->response); dn_, &s->response);
m->Push(asio_continuation::Write(dn_.get(), asio::buffer(s->header))) m->Push(asio_continuation::Write(dn_, asio::buffer(s->header)))
.Push(asio_continuation::WriteDelimitedPBMessage(dn_, &s->request))
.Push(read_pb_message); .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;

View File

@ -117,7 +117,7 @@ void DataTransferSaslStream<Stream>::Handshake(const Handler &next) {
DataTransferSaslStreamUtil::PrepareInitialHandshake(&s->req0); DataTransferSaslStreamUtil::PrepareInitialHandshake(&s->req0);
m->Push(Write(stream_.get(), kMagicNumberBuffer)) m->Push(Write(stream_, kMagicNumberBuffer))
.Push(WriteDelimitedPBMessage(stream_, &s->req0)) .Push(WriteDelimitedPBMessage(stream_, &s->req0))
.Push(new ReadSaslMessage(stream_, &s->resp0)) .Push(new ReadSaslMessage(stream_, &s->resp0))
.Push(new Authenticator(&authenticator_, &s->resp0, &s->req1)) .Push(new Authenticator(&authenticator_, &s->resp0, &s->req1))