Create leveldb server via Thrift.

Summary:
First draft.
Unit tests pass.

Test Plan: unit tests attached

Reviewers: heyongqiang

Reviewed By: heyongqiang

Differential Revision: https://reviews.facebook.net/D3969
This commit is contained in:
Dhruba Borthakur 2012-07-02 22:45:59 -07:00
parent 22ee777f68
commit 80c663882a
216 changed files with 45837 additions and 10 deletions

View File

@ -27,6 +27,7 @@ CXXFLAGS += -I. -I./include $(PLATFORM_CXXFLAGS) $(OPT)
LDFLAGS += $(PLATFORM_LDFLAGS)
LIBOBJECTS = $(SOURCES:.cc=.o)
LIBOBJECTS += $(SOURCESCPP:.cpp=.o)
MEMENVOBJECTS = $(MEMENV_SOURCES:.cc=.o)
TESTUTIL = ./util/testutil.o
@ -51,7 +52,8 @@ TESTS = \
table_test \
version_edit_test \
version_set_test \
write_batch_test
write_batch_test \
leveldb_server_test
PROGRAMS = db_bench $(TESTS)
BENCHMARKS = db_bench_sqlite3 db_bench_tree_db
@ -71,20 +73,20 @@ SHARED2 = $(SHARED1).$(SHARED_MAJOR)
SHARED3 = $(SHARED1).$(SHARED_MAJOR).$(SHARED_MINOR)
SHARED = $(SHARED1) $(SHARED2) $(SHARED3)
$(SHARED3):
$(CXX) $(LDFLAGS) $(PLATFORM_SHARED_LDFLAGS)$(INSTALL_PATH)/$(SHARED2) $(CXXFLAGS) $(PLATFORM_SHARED_CFLAGS) $(SOURCES) -o $(SHARED3) $(EXEC_LDFLAGS_SHARED)
$(CXX) $(LDFLAGS) $(PLATFORM_SHARED_LDFLAGS)$(INSTALL_PATH)/$(SHARED2) $(CXXFLAGS) $(PLATFORM_SHARED_CFLAGS) $(SOURCES) $(SOURCESCPP) -o $(SHARED3) $(EXEC_LDFLAGS_SHARED)
$(SHARED2): $(SHARED3)
ln -fs $(SHARED3) $(SHARED2)
$(SHARED1): $(SHARED3)
ln -fs $(SHARED3) $(SHARED1)
endif
all: $(SHARED) $(LIBRARY)
all: $(SHARED) $(LIBRARY) $(THRIFTSERVER)
check: all $(PROGRAMS) $(TESTS)
for t in $(TESTS); do echo "***** Running $$t"; ./$$t || exit 1; done
clean:
-rm -f $(PROGRAMS) $(BENCHMARKS) $(LIBRARY) $(SHARED) $(MEMENVLIBRARY) */*.o */*/*.o ios-x86/*/*.o ios-arm/*/*.o build_config.mk
-rm -f $(PROGRAMS) $(BENCHMARKS) $(LIBRARY) $(SHARED) $(MEMENVLIBRARY) $(THRIFTSERVER) */*.o */*/*.o ios-x86/*/*.o ios-arm/*/*.o build_config.mk
-rm -rf ios-x86/* ios-arm/*
$(LIBRARY): $(LIBOBJECTS)
@ -161,6 +163,12 @@ $(MEMENVLIBRARY) : $(MEMENVOBJECTS)
memenv_test : helpers/memenv/memenv_test.o $(MEMENVLIBRARY) $(LIBRARY) $(TESTHARNESS)
$(CXX) helpers/memenv/memenv_test.o $(MEMENVLIBRARY) $(LIBRARY) $(TESTHARNESS) -o $@ $(LDFLAGS)
leveldb_server: thrift/server.o $(LIBRARY)
$(CXX) thrift/server.o $(LIBRARY) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS)
leveldb_server_test: thrift/test/simpletest.o $(LIBRARY)
$(CXX) thrift/test/simpletest.o $(LIBRARY) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS)
ifeq ($(PLATFORM), IOS)
# For iOS, create universal object files to be used on both the simulator and
# a device.

View File

@ -101,15 +101,21 @@ esac
# of all files matching either rule, so we need to append -print to make the
# prune take effect.
DIRS="util db table"
if test "$USE_THRIFT"; then
DIRS+=" thrift/gen-cpp thrift/server_utils.cpp"
THRIFTSERVER=leveldb_server
fi
set -f # temporarily disable globbing so that our patterns aren't expanded
PRUNE_TEST="-name *test*.cc -prune"
PRUNE_BENCH="-name *_bench.cc -prune"
PORTABLE_FILES=`find $DIRS $PRUNE_TEST -o $PRUNE_BENCH -o -name '*.cc' -print | sort | tr "\n" " "`
PORTABLE_CPP=`find $DIRS $PRUNE_TEST -o $PRUNE_BENCH -o -name '*.cpp' -print | sort | tr "\n" " "`
set +f # re-enable globbing
# The sources consist of the portable files, plus the platform-specific port
# file.
echo "SOURCES=$PORTABLE_FILES $PORT_FILE" >> $OUTPUT
echo "SOURCESCPP=$PORTABLE_CPP" >> $OUTPUT
echo "MEMENV_SOURCES=helpers/memenv/memenv.cc" >> $OUTPUT
if [ "$PLATFORM" = "OS_ANDROID_CROSSCOMPILE" ]; then
@ -136,7 +142,7 @@ EOF
EOF
if [ "$?" = 0 ]; then
COMMON_FLAGS="$COMMON_FLAGS -DSNAPPY"
PLATFORM_LDFLAGS="$PLATFORM_LDFLAGS -lsnappy"
PLATFORM_LDFLAGS="$PLATFORM_LDFLAGS -lsnappy -L./snappy/libs"
fi
# Test whether zlib library is installed
@ -183,6 +189,14 @@ if test "$USE_HDFS"; then
PLATFORM_LDFLAGS+=$HDFS_LDFLAGS
fi
# shall we build thrift server
if test "$USE_THRIFT"; then
THRIFT_CCFLAGS=" -I./thrift -I./thrift/gen-cpp -I./thrift/lib/cpp -I/usr/include -std=gnu++0x"
THRIFT_LDFLAGS=" -lserver -lthrift_base -ltransport -lthrift_exception -lutil -L./thrift/libs "
COMMON_FLAGS+=$THRIFT_CCFLAGS
PLATFORM_LDFLAGS+=$THRIFT_LDFLAGS
fi
PLATFORM_CCFLAGS="$PLATFORM_CCFLAGS $COMMON_FLAGS"
PLATFORM_CXXFLAGS="$PLATFORM_CXXFLAGS $COMMON_FLAGS"
@ -193,3 +207,4 @@ echo "PLATFORM_CXXFLAGS=$PLATFORM_CXXFLAGS" >> $OUTPUT
echo "PLATFORM_SHARED_CFLAGS=$PLATFORM_SHARED_CFLAGS" >> $OUTPUT
echo "PLATFORM_SHARED_EXT=$PLATFORM_SHARED_EXT" >> $OUTPUT
echo "PLATFORM_SHARED_LDFLAGS=$PLATFORM_SHARED_LDFLAGS" >> $OUTPUT
echo "THRIFTSERVER=$THRIFTSERVER" >> $OUTPUT

View File

@ -22,8 +22,13 @@ fi
SNAPPY_INCLUDE=" -I ./snappy"
SNAPPY_LIBS=" -L./snappy/libs"
# location of boost headers and libraries
THRIFT_INCLUDE=" -I $TOOLCHAIN_LIB_BASE/boost/default/eed002c/include -std=gnu++0x"
THRIFT_INCLUDE+=" -I./thrift -I./thrift/gen-cpp -I./thrift/lib/cpp"
THRIFT_LIBS=" -L $TOOLCHAIN_LIB_BASE/boost/default/eed002c/lib"
CC="$TOOLCHAIN_EXECUTABLES/gcc/gcc-4.6.2-glibc-2.13/bin/gcc"
CXX="$TOOLCHAIN_EXECUTABLES/gcc/gcc-4.6.2-glibc-2.13/bin/g++ $JINCLUDE $SNAPPY_INCLUDE"
CXX="$TOOLCHAIN_EXECUTABLES/gcc/gcc-4.6.2-glibc-2.13/bin/g++ $JINCLUDE $SNAPPY_INCLUDE $THRIFT_INCLUDE"
AR=$TOOLCHAIN_EXECUTABLES/binutils/binutils-2.21.1/da39a3e/bin/ar
RANLIB=$TOOLCHAIN_EXECUTABLES/binutils/binutils-2.21.1/da39a3e/bin/ranlib
@ -32,7 +37,7 @@ CFLAGS+=" -I $TOOLCHAIN_LIB_BASE/jemalloc/jemalloc-2.2.5/96de4f9/include -DHAVE_
EXEC_LDFLAGS=" -Wl,--whole-archive $TOOLCHAIN_LIB_BASE/jemalloc/jemalloc-2.2.4/96de4f9/lib/libjemalloc.a "
EXEC_LDFLAGS+="-Wl,--no-whole-archive $TOOLCHAIN_LIB_BASE/libunwind/libunwind-20100810/4bc2c16/lib/libunwind.a"
EXEC_LDFLAGS+="$HDFSLIB $SNAPPY_LIBS"
EXEC_LDFLAGS+="$HDFSLIB $SNAPPY_LIBS $THRIFT_LIBS "
EXEC_LDFLAGS_SHARED="$SNAPPY_LIBS $TOOLCHAIN_LIB_BASE/jemalloc/jemalloc-2.2.4/96de4f9/lib/libjemalloc.so"
export CC CXX AR RANLIB CFLAGS EXEC_LDFLAGS EXEC_LDFLAGS_SHARED

View File

@ -16,7 +16,7 @@ If you want to compile leveldb with hdfs support, please set the following
enviroment variables appropriately:
USE_HDFS=1
JAVA_HOME=/usr/local/jdk-6u22-64
LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$JAVA_HOME/jre/lib/amd64/server:$JAVA_HOME/jre/lib/amd64/:./snappy/libs
LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/usr/local/jdk-6u22-64/jre/lib/amd64/server:/usr/local/jdk-6u22-64/jre/lib/amd64/:./snappy/libs
make clean all db_bench
To run dbbench,

View File

@ -2,5 +2,5 @@ This is a pre-compiled version of snappy 1.0.5 from
http://code.google.com/p/snappy/downloads/detail?name=snappy-1.0.5.tar.gz
This is compiled using gcc-4.6.2-glibc-2.13 on centos5.2
and uses jemalloc. This is here so that one can compie leveldb to use
and uses jemalloc. This is here so that one can compile leveldb to use
snappy easily.

18
thrift/README Normal file
View File

@ -0,0 +1,18 @@
This directory has the thrift server code that exposes leveldb apis.
The thrift header files are in ./thrift/lib. These are part of
Apache Thrift code base and are needed for compilation of the leveldb
thrift server. The thrift libraries are copied into ./thrift/libs.
If you want to use a different version of thrift, please update these
directories with the corresponding thrift header files and the
compiled thrift libraries.
If you want to compile leveldb with thrift-server support, please set the following
enviroment variables appropriately:
USE_THRIFT=1
LD_LIBRARY_PATH=$LD_LIBRARY_PATH:./thrift/libs
make clean leveldb_server leveldb_server_test
You can run the leveldb server unit tests by
./leveldb_server_test

39
thrift/folly/Likely.h Normal file
View File

@ -0,0 +1,39 @@
/*
* Copyright 2012 Facebook, Inc.
*
* Licensed 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.
*/
/**
* Compiler hints to indicate the fast path of an "if" branch: whether
* the if condition is likely to be true or false.
*
* @author Tudor Bosman (tudorb@fb.com)
*/
#ifndef FOLLY_BASE_LIKELY_H_
#define FOLLY_BASE_LIKELY_H_
#undef LIKELY
#undef UNLIKELY
#if defined(__GNUC__) && __GNUC__ >= 4
#define LIKELY(x) (__builtin_expect((x), 1))
#define UNLIKELY(x) (__builtin_expect((x), 0))
#else
#define LIKELY(x) (x)
#define UNLIKELY(x) (x)
#endif
#endif /* FOLLY_BASE_LIKELY_H_ */

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,201 @@
/*
* Copyright 2012 Facebook, Inc.
*
* Licensed 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 FOLLY_IO_IOBUF_QUEUE_H
#define FOLLY_IO_IOBUF_QUEUE_H
#include "folly/experimental/io/IOBuf.h"
#include <stdexcept>
#include <string>
namespace folly {
/**
* An IOBufQueue encapsulates a chain of IOBufs and provides
* convenience functions to append data to the back of the chain
* and remove data from the front.
*/
class IOBufQueue {
public:
struct Options {
Options() : cacheChainLength(false) { }
bool cacheChainLength;
};
explicit IOBufQueue(const Options& options = Options());
/**
* Add a buffer or buffer chain to the end of this queue. The
* queue takes ownership of buf.
*/
void append(std::unique_ptr<folly::IOBuf>&& buf);
/**
* Add a queue to the end of this queue. The queue takes ownership of
* all buffers from the other queue.
*/
void append(IOBufQueue& other);
void append(IOBufQueue&& other) {
append(other); // call lvalue reference overload, above
}
/**
* Copy len bytes, starting at buf, to the end of this queue.
* The caller retains ownership of the source data.
*/
void append(const void* buf, size_t len);
/**
* Copy a string to the end of this queue.
* The caller retains ownership of the source data.
*/
void append(const std::string& buf) {
append(buf.data(), buf.length());
}
/**
* Append a chain of IOBuf objects that point to consecutive regions
* within buf.
*
* Just like IOBuf::wrapBuffer, this should only be used when the caller
* knows ahead of time and can ensure that all IOBuf objects that will point
* to this buffer will be destroyed before the buffer itself is destroyed;
* all other caveats from wrapBuffer also apply.
*
* Every buffer except for the last will wrap exactly blockSize bytes.
* Importantly, this method may be used to wrap buffers larger than 4GB.
*/
void wrapBuffer(const void* buf, size_t len,
uint32_t blockSize=(1U << 31)); // default block size: 2GB
/**
* Obtain a writable block of contiguous bytes at the end of this
* queue, allocating more space if necessary. The amount of space
* reserved will be between min and max, inclusive; the IOBufQueue
* implementation may pick a value in that range that makes efficient
* use of already-allocated internal space.
*
* If the caller subsequently writes anything into the returned space,
* it must call the postallocate() method.
*
* @return The starting address of the block and the length in bytes.
*
* @note The point of the preallocate()/postallocate() mechanism is
* to support I/O APIs such as Thrift's TAsyncSocket::ReadCallback
* that request a buffer from the application and then, in a later
* callback, tell the application how much of the buffer they've
* filled with data.
*/
std::pair<void*,uint32_t> preallocate(uint32_t min, uint32_t max);
/**
* Tell the queue that the caller has written data into the first n
* bytes provided by the previous preallocate() call.
*
* @note n should be less than or equal to the size returned by
* preallocate(). If n is zero, the caller may skip the call
* to postallocate(). If n is nonzero, the caller must not
* invoke any other non-const methods on this IOBufQueue between
* the call to preallocate and the call to postallocate().
*/
void postallocate(uint32_t n);
/**
* Obtain a writable block of n contiguous bytes, allocating more space
* if necessary, and mark it as used. The caller can fill it later.
*/
void* allocate(uint32_t n) {
void* p = preallocate(n, n).first;
postallocate(n);
return p;
}
/**
* Split off the first n bytes of the queue into a separate IOBuf chain,
* and transfer ownership of the new chain to the caller. The IOBufQueue
* retains ownership of everything after the split point.
*
* @warning If the split point lies in the middle of some IOBuf within
* the chain, this function may, as an implementation detail,
* clone that IOBuf.
*
* @throws std::underflow_error if n exceeds the number of bytes
* in the queue.
*/
std::unique_ptr<folly::IOBuf> split(size_t n);
/**
* Similar to IOBuf::trimStart, but works on the whole queue. Will
* pop off buffers that have been completely trimmed.
*/
void trimStart(size_t amount);
/**
* Similar to IOBuf::trimEnd, but works on the whole queue. Will
* pop off buffers that have been completely trimmed.
*/
void trimEnd(size_t amount);
/**
* Transfer ownership of the queue's entire IOBuf chain to the caller.
*/
std::unique_ptr<folly::IOBuf>&& move() {
chainLength_ = 0;
return std::move(head_);
}
/**
* Access
*/
const folly::IOBuf* front() const {
return head_.get();
}
/**
* Total chain length, only valid if cacheLength was specified in the
* constructor.
*/
size_t chainLength() const {
if (!options_.cacheChainLength) {
throw std::invalid_argument("IOBufQueue: chain length not cached");
}
return chainLength_;
}
const Options& options() const {
return options_;
}
/** Movable */
IOBufQueue(IOBufQueue&&);
IOBufQueue& operator=(IOBufQueue&&);
private:
static const size_t kChainLengthNotCached = (size_t)-1;
/** Not copyable */
IOBufQueue(const IOBufQueue&) = delete;
IOBufQueue& operator=(const IOBufQueue&) = delete;
Options options_;
size_t chainLength_;
/** Everything that has been appended but not yet discarded or moved out */
std::unique_ptr<folly::IOBuf> head_;
};
} // folly
#endif // FOLLY_IO_IOBUF_QUEUE_H

View File

@ -0,0 +1,2 @@
#define CHECK(a)
#define CHECK_LT(a, b)

4161
thrift/gen-cpp/DB.cpp Normal file

File diff suppressed because it is too large Load Diff

2225
thrift/gen-cpp/DB.h Normal file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,17 @@
/**
* Autogenerated by Thrift
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
#include "leveldb_constants.h"
namespace Tleveldb {
const leveldbConstants g_leveldb_constants;
leveldbConstants::leveldbConstants() {
}
} // namespace

View File

@ -0,0 +1,24 @@
/**
* Autogenerated by Thrift
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
#ifndef leveldb_CONSTANTS_H
#define leveldb_CONSTANTS_H
#include "leveldb_types.h"
namespace Tleveldb {
class leveldbConstants {
public:
leveldbConstants();
};
extern const leveldbConstants g_leveldb_constants;
} // namespace
#endif

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,867 @@
/**
* Autogenerated by Thrift
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
#ifndef leveldb_TYPES_H
#define leveldb_TYPES_H
#include <Thrift.h>
#include <TApplicationException.h>
#include <protocol/TProtocol.h>
#include <transport/TTransport.h>
namespace apache { namespace thrift { namespace reflection {
class Schema;
}}}
namespace Tleveldb {
enum CompressionType {
kNoCompression = 0,
kSnappyCompression = 1
};
extern const std::map<int, const char*> _CompressionType_VALUES_TO_NAMES;
extern const std::map<const char*, int, apache::thrift::ltstr> _CompressionType_NAMES_TO_VALUES;
enum Code {
kOk = 0,
kNotFound = 1,
kCorruption = 2,
kNotSupported = 3,
kInvalidArgument = 4,
kIOError = 5,
kEnd = 6
};
extern const std::map<int, const char*> _Code_VALUES_TO_NAMES;
extern const std::map<const char*, int, apache::thrift::ltstr> _Code_NAMES_TO_VALUES;
enum IteratorType {
seekToFirst = 0,
seekToLast = 1,
seekToKey = 2
};
extern const std::map<int, const char*> _IteratorType_VALUES_TO_NAMES;
extern const std::map<const char*, int, apache::thrift::ltstr> _IteratorType_NAMES_TO_VALUES;
typedef std::string Text;
typedef std::string Bytes;
class Slice {
public:
static const uint64_t _reflection_id = 2199896239461470156U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
Slice() : data(""), size(0) {
}
Slice(const Slice&) = default;
Slice& operator=(const Slice&) = default;
Slice(Slice&&) = default;
Slice& operator=(Slice&&) = default;
void __clear() {
data = "";
size = 0;
__isset.__clear();
}
virtual ~Slice() throw() {}
Text data;
int32_t size;
struct __isset {
__isset() { __clear(); }
void __clear() {
data = false;
size = false;
}
bool data;
bool size;
} __isset;
bool operator == (const Slice & rhs) const
{
if (!(this->data == rhs.data))
return false;
if (!(this->size == rhs.size))
return false;
return true;
}
bool operator != (const Slice &rhs) const {
return !(*this == rhs);
}
bool operator < (const Slice & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class Slice;
void swap(Slice &a, Slice &b);
class Range {
public:
static const uint64_t _reflection_id = 6170219570187881516U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
Range() {
}
Range(const Range&) = default;
Range& operator=(const Range&) = default;
Range(Range&&) = default;
Range& operator=(Range&&) = default;
void __clear() {
start.__clear();
limit.__clear();
__isset.__clear();
}
virtual ~Range() throw() {}
Slice start;
Slice limit;
struct __isset {
__isset() { __clear(); }
void __clear() {
start = false;
limit = false;
}
bool start;
bool limit;
} __isset;
bool operator == (const Range & rhs) const
{
if (!(this->start == rhs.start))
return false;
if (!(this->limit == rhs.limit))
return false;
return true;
}
bool operator != (const Range &rhs) const {
return !(*this == rhs);
}
bool operator < (const Range & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class Range;
void swap(Range &a, Range &b);
class DBOptions {
public:
static const uint64_t _reflection_id = 6731746507948871532U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
DBOptions() : create_if_missing(0), error_if_exists(0), write_buffer_size(0), max_open_files(0), block_size(0), block_restart_interval(0), compression(static_cast<CompressionType>(0)) {
}
DBOptions(const DBOptions&) = default;
DBOptions& operator=(const DBOptions&) = default;
DBOptions(DBOptions&&) = default;
DBOptions& operator=(DBOptions&&) = default;
void __clear() {
create_if_missing = 0;
error_if_exists = 0;
write_buffer_size = 0;
max_open_files = 0;
block_size = 0;
block_restart_interval = 0;
compression = static_cast<CompressionType>(0);
__isset.__clear();
}
virtual ~DBOptions() throw() {}
bool create_if_missing;
bool error_if_exists;
int32_t write_buffer_size;
int32_t max_open_files;
int32_t block_size;
int32_t block_restart_interval;
CompressionType compression;
struct __isset {
__isset() { __clear(); }
void __clear() {
create_if_missing = false;
error_if_exists = false;
write_buffer_size = false;
max_open_files = false;
block_size = false;
block_restart_interval = false;
compression = false;
}
bool create_if_missing;
bool error_if_exists;
bool write_buffer_size;
bool max_open_files;
bool block_size;
bool block_restart_interval;
bool compression;
} __isset;
bool operator == (const DBOptions & rhs) const
{
if (!(this->create_if_missing == rhs.create_if_missing))
return false;
if (!(this->error_if_exists == rhs.error_if_exists))
return false;
if (!(this->write_buffer_size == rhs.write_buffer_size))
return false;
if (!(this->max_open_files == rhs.max_open_files))
return false;
if (!(this->block_size == rhs.block_size))
return false;
if (!(this->block_restart_interval == rhs.block_restart_interval))
return false;
if (!(this->compression == rhs.compression))
return false;
return true;
}
bool operator != (const DBOptions &rhs) const {
return !(*this == rhs);
}
bool operator < (const DBOptions & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class DBOptions;
void swap(DBOptions &a, DBOptions &b);
class WriteOptions {
public:
static const uint64_t _reflection_id = 8830325115029814540U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
WriteOptions() : sync(0) {
}
WriteOptions(const WriteOptions&) = default;
WriteOptions& operator=(const WriteOptions&) = default;
WriteOptions(WriteOptions&&) = default;
WriteOptions& operator=(WriteOptions&&) = default;
void __clear() {
sync = 0;
__isset.__clear();
}
virtual ~WriteOptions() throw() {}
bool sync;
struct __isset {
__isset() { __clear(); }
void __clear() {
sync = false;
}
bool sync;
} __isset;
bool operator == (const WriteOptions & rhs) const
{
if (!(this->sync == rhs.sync))
return false;
return true;
}
bool operator != (const WriteOptions &rhs) const {
return !(*this == rhs);
}
bool operator < (const WriteOptions & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class WriteOptions;
void swap(WriteOptions &a, WriteOptions &b);
class Snapshot {
public:
static const uint64_t _reflection_id = 15670548806217660204U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
Snapshot() : snapshotid(0) {
}
Snapshot(const Snapshot&) = default;
Snapshot& operator=(const Snapshot&) = default;
Snapshot(Snapshot&&) = default;
Snapshot& operator=(Snapshot&&) = default;
void __clear() {
snapshotid = 0;
__isset.__clear();
}
virtual ~Snapshot() throw() {}
int64_t snapshotid;
struct __isset {
__isset() { __clear(); }
void __clear() {
snapshotid = false;
}
bool snapshotid;
} __isset;
bool operator == (const Snapshot & rhs) const
{
if (!(this->snapshotid == rhs.snapshotid))
return false;
return true;
}
bool operator != (const Snapshot &rhs) const {
return !(*this == rhs);
}
bool operator < (const Snapshot & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class Snapshot;
void swap(Snapshot &a, Snapshot &b);
class ReadOptions {
public:
static const uint64_t _reflection_id = 1092669993626789804U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
ReadOptions() : verify_checksums(0), fill_cache(0) {
}
ReadOptions(const ReadOptions&) = default;
ReadOptions& operator=(const ReadOptions&) = default;
ReadOptions(ReadOptions&&) = default;
ReadOptions& operator=(ReadOptions&&) = default;
void __clear() {
verify_checksums = 0;
fill_cache = 0;
snapshot.__clear();
__isset.__clear();
}
virtual ~ReadOptions() throw() {}
bool verify_checksums;
bool fill_cache;
Snapshot snapshot;
struct __isset {
__isset() { __clear(); }
void __clear() {
verify_checksums = false;
fill_cache = false;
snapshot = false;
}
bool verify_checksums;
bool fill_cache;
bool snapshot;
} __isset;
bool operator == (const ReadOptions & rhs) const
{
if (!(this->verify_checksums == rhs.verify_checksums))
return false;
if (!(this->fill_cache == rhs.fill_cache))
return false;
if (!(this->snapshot == rhs.snapshot))
return false;
return true;
}
bool operator != (const ReadOptions &rhs) const {
return !(*this == rhs);
}
bool operator < (const ReadOptions & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class ReadOptions;
void swap(ReadOptions &a, ReadOptions &b);
class DBHandle {
public:
static const uint64_t _reflection_id = 8973827971994157004U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
DBHandle() : dbname(""), handleid(0) {
}
DBHandle(const DBHandle&) = default;
DBHandle& operator=(const DBHandle&) = default;
DBHandle(DBHandle&&) = default;
DBHandle& operator=(DBHandle&&) = default;
void __clear() {
dbname = "";
handleid = 0;
__isset.__clear();
}
virtual ~DBHandle() throw() {}
Text dbname;
int64_t handleid;
struct __isset {
__isset() { __clear(); }
void __clear() {
dbname = false;
handleid = false;
}
bool dbname;
bool handleid;
} __isset;
bool operator == (const DBHandle & rhs) const
{
if (!(this->dbname == rhs.dbname))
return false;
if (!(this->handleid == rhs.handleid))
return false;
return true;
}
bool operator != (const DBHandle &rhs) const {
return !(*this == rhs);
}
bool operator < (const DBHandle & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class DBHandle;
void swap(DBHandle &a, DBHandle &b);
class Iterator {
public:
static const uint64_t _reflection_id = 11184146435197093740U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
Iterator() : iteratorid(0) {
}
Iterator(const Iterator&) = default;
Iterator& operator=(const Iterator&) = default;
Iterator(Iterator&&) = default;
Iterator& operator=(Iterator&&) = default;
void __clear() {
iteratorid = 0;
__isset.__clear();
}
virtual ~Iterator() throw() {}
int64_t iteratorid;
struct __isset {
__isset() { __clear(); }
void __clear() {
iteratorid = false;
}
bool iteratorid;
} __isset;
bool operator == (const Iterator & rhs) const
{
if (!(this->iteratorid == rhs.iteratorid))
return false;
return true;
}
bool operator != (const Iterator &rhs) const {
return !(*this == rhs);
}
bool operator < (const Iterator & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class Iterator;
void swap(Iterator &a, Iterator &b);
class kv {
public:
static const uint64_t _reflection_id = 16082992224095104076U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
kv() {
}
kv(const kv&) = default;
kv& operator=(const kv&) = default;
kv(kv&&) = default;
kv& operator=(kv&&) = default;
void __clear() {
key.__clear();
value.__clear();
__isset.__clear();
}
virtual ~kv() throw() {}
Slice key;
Slice value;
struct __isset {
__isset() { __clear(); }
void __clear() {
key = false;
value = false;
}
bool key;
bool value;
} __isset;
bool operator == (const kv & rhs) const
{
if (!(this->key == rhs.key))
return false;
if (!(this->value == rhs.value))
return false;
return true;
}
bool operator != (const kv &rhs) const {
return !(*this == rhs);
}
bool operator < (const kv & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class kv;
void swap(kv &a, kv &b);
class ResultItem {
public:
static const uint64_t _reflection_id = 13211316281207238796U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
ResultItem() : status(static_cast<Code>(0)) {
}
ResultItem(const ResultItem&) = default;
ResultItem& operator=(const ResultItem&) = default;
ResultItem(ResultItem&&) = default;
ResultItem& operator=(ResultItem&&) = default;
void __clear() {
status = static_cast<Code>(0);
value.__clear();
__isset.__clear();
}
virtual ~ResultItem() throw() {}
Code status;
Slice value;
struct __isset {
__isset() { __clear(); }
void __clear() {
status = false;
value = false;
}
bool status;
bool value;
} __isset;
bool operator == (const ResultItem & rhs) const
{
if (!(this->status == rhs.status))
return false;
if (!(this->value == rhs.value))
return false;
return true;
}
bool operator != (const ResultItem &rhs) const {
return !(*this == rhs);
}
bool operator < (const ResultItem & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class ResultItem;
void swap(ResultItem &a, ResultItem &b);
class ResultPair {
public:
static const uint64_t _reflection_id = 14875242256166808460U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
ResultPair() : status(static_cast<Code>(0)) {
}
ResultPair(const ResultPair&) = default;
ResultPair& operator=(const ResultPair&) = default;
ResultPair(ResultPair&&) = default;
ResultPair& operator=(ResultPair&&) = default;
void __clear() {
status = static_cast<Code>(0);
keyvalue.__clear();
__isset.__clear();
}
virtual ~ResultPair() throw() {}
Code status;
kv keyvalue;
struct __isset {
__isset() { __clear(); }
void __clear() {
status = false;
keyvalue = false;
}
bool status;
bool keyvalue;
} __isset;
bool operator == (const ResultPair & rhs) const
{
if (!(this->status == rhs.status))
return false;
if (!(this->keyvalue == rhs.keyvalue))
return false;
return true;
}
bool operator != (const ResultPair &rhs) const {
return !(*this == rhs);
}
bool operator < (const ResultPair & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class ResultPair;
void swap(ResultPair &a, ResultPair &b);
class ResultSnapshot {
public:
static const uint64_t _reflection_id = 16627180600575569004U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
ResultSnapshot() : status(static_cast<Code>(0)) {
}
ResultSnapshot(const ResultSnapshot&) = default;
ResultSnapshot& operator=(const ResultSnapshot&) = default;
ResultSnapshot(ResultSnapshot&&) = default;
ResultSnapshot& operator=(ResultSnapshot&&) = default;
void __clear() {
status = static_cast<Code>(0);
snapshot.__clear();
__isset.__clear();
}
virtual ~ResultSnapshot() throw() {}
Code status;
Snapshot snapshot;
struct __isset {
__isset() { __clear(); }
void __clear() {
status = false;
snapshot = false;
}
bool status;
bool snapshot;
} __isset;
bool operator == (const ResultSnapshot & rhs) const
{
if (!(this->status == rhs.status))
return false;
if (!(this->snapshot == rhs.snapshot))
return false;
return true;
}
bool operator != (const ResultSnapshot &rhs) const {
return !(*this == rhs);
}
bool operator < (const ResultSnapshot & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class ResultSnapshot;
void swap(ResultSnapshot &a, ResultSnapshot &b);
class ResultIterator {
public:
static const uint64_t _reflection_id = 595886977232564460U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
ResultIterator() : status(static_cast<Code>(0)) {
}
ResultIterator(const ResultIterator&) = default;
ResultIterator& operator=(const ResultIterator&) = default;
ResultIterator(ResultIterator&&) = default;
ResultIterator& operator=(ResultIterator&&) = default;
void __clear() {
status = static_cast<Code>(0);
iterator.__clear();
__isset.__clear();
}
virtual ~ResultIterator() throw() {}
Code status;
Iterator iterator;
struct __isset {
__isset() { __clear(); }
void __clear() {
status = false;
iterator = false;
}
bool status;
bool iterator;
} __isset;
bool operator == (const ResultIterator & rhs) const
{
if (!(this->status == rhs.status))
return false;
if (!(this->iterator == rhs.iterator))
return false;
return true;
}
bool operator != (const ResultIterator &rhs) const {
return !(*this == rhs);
}
bool operator < (const ResultIterator & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class ResultIterator;
void swap(ResultIterator &a, ResultIterator &b);
class LeveldbException : public apache::thrift::TException {
public:
static const uint64_t _reflection_id = 2551220192341843436U;
static void _reflection_register(::apache::thrift::reflection::Schema&);
LeveldbException() : message(""), errorCode(static_cast<Code>(0)) {
}
LeveldbException(const LeveldbException&) = default;
LeveldbException& operator=(const LeveldbException&) = default;
LeveldbException(LeveldbException&&) = default;
LeveldbException& operator=(LeveldbException&&) = default;
void __clear() {
message = "";
errorCode = static_cast<Code>(0);
__isset.__clear();
}
virtual ~LeveldbException() throw() {}
Text message;
Code errorCode;
struct __isset {
__isset() { __clear(); }
void __clear() {
message = false;
errorCode = false;
}
bool message;
bool errorCode;
} __isset;
bool operator == (const LeveldbException & rhs) const
{
if (!(this->message == rhs.message))
return false;
if (!(this->errorCode == rhs.errorCode))
return false;
return true;
}
bool operator != (const LeveldbException &rhs) const {
return !(*this == rhs);
}
bool operator < (const LeveldbException & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
virtual const char* what() const throw() {
return "LeveldbException";
}
};
class LeveldbException;
void swap(LeveldbException &a, LeveldbException &b);
} // namespace
#endif

View File

@ -0,0 +1,247 @@
/**
* Autogenerated by Thrift
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated SignedSource<<b50937b866513442f6f2690c8af84db0>>
*/
#ifndef reflection_TYPES_H
#define reflection_TYPES_H
#include <Thrift.h>
#include <TApplicationException.h>
#include <protocol/TProtocol.h>
#include <transport/TTransport.h>
#include <unordered_map>
#include <boost/interprocess/containers/flat_map.hpp>
namespace apache { namespace thrift { namespace reflection {
enum Type {
TYPE_VOID = 0,
TYPE_STRING = 1,
TYPE_BOOL = 2,
TYPE_BYTE = 3,
TYPE_I16 = 4,
TYPE_I32 = 5,
TYPE_I64 = 6,
TYPE_DOUBLE = 7,
TYPE_ENUM = 8,
TYPE_LIST = 9,
TYPE_SET = 10,
TYPE_MAP = 11,
TYPE_STRUCT = 12,
TYPE_SERVICE = 13,
TYPE_PROGRAM = 14
};
extern const std::map<int, const char*> _Type_VALUES_TO_NAMES;
extern const std::map<const char*, int, apache::thrift::ltstr> _Type_NAMES_TO_VALUES;
class StructField {
public:
StructField() : isRequired(0), type(0), name("") {
}
StructField(const StructField&) = default;
StructField& operator=(const StructField&) = default;
StructField(StructField&&) = default;
StructField& operator=(StructField&&) = default;
void __clear() {
isRequired = 0;
type = 0;
name = "";
annotations.clear();
__isset.__clear();
}
virtual ~StructField() throw() {}
bool isRequired;
int64_t type;
std::string name;
boost::container::flat_map<std::string, std::string> annotations;
struct __isset {
__isset() { __clear(); }
void __clear() {
isRequired = false;
type = false;
name = false;
annotations = false;
}
bool isRequired;
bool type;
bool name;
bool annotations;
} __isset;
bool operator == (const StructField & rhs) const
{
if (!(this->isRequired == rhs.isRequired))
return false;
if (!(this->type == rhs.type))
return false;
if (!(this->name == rhs.name))
return false;
if (__isset.annotations != rhs.__isset.annotations)
return false;
else if (__isset.annotations && !(annotations == rhs.annotations))
return false;
return true;
}
bool operator != (const StructField &rhs) const {
return !(*this == rhs);
}
bool operator < (const StructField & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class StructField;
void swap(StructField &a, StructField &b);
class DataType {
public:
DataType() : name(""), mapKeyType(0), valueType(0) {
}
DataType(const DataType&) = default;
DataType& operator=(const DataType&) = default;
DataType(DataType&&) = default;
DataType& operator=(DataType&&) = default;
void __clear() {
name = "";
fields.clear();
mapKeyType = 0;
valueType = 0;
enumValues.clear();
__isset.__clear();
}
virtual ~DataType() throw() {}
std::string name;
boost::container::flat_map<int16_t, StructField> fields;
int64_t mapKeyType;
int64_t valueType;
boost::container::flat_map<std::string, int32_t> enumValues;
struct __isset {
__isset() { __clear(); }
void __clear() {
name = false;
fields = false;
mapKeyType = false;
valueType = false;
enumValues = false;
}
bool name;
bool fields;
bool mapKeyType;
bool valueType;
bool enumValues;
} __isset;
bool operator == (const DataType & rhs) const
{
if (!(this->name == rhs.name))
return false;
if (__isset.fields != rhs.__isset.fields)
return false;
else if (__isset.fields && !(fields == rhs.fields))
return false;
if (__isset.mapKeyType != rhs.__isset.mapKeyType)
return false;
else if (__isset.mapKeyType && !(mapKeyType == rhs.mapKeyType))
return false;
if (__isset.valueType != rhs.__isset.valueType)
return false;
else if (__isset.valueType && !(valueType == rhs.valueType))
return false;
if (__isset.enumValues != rhs.__isset.enumValues)
return false;
else if (__isset.enumValues && !(enumValues == rhs.enumValues))
return false;
return true;
}
bool operator != (const DataType &rhs) const {
return !(*this == rhs);
}
bool operator < (const DataType & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class DataType;
void swap(DataType &a, DataType &b);
class Schema {
public:
Schema() {
}
Schema(const Schema&) = default;
Schema& operator=(const Schema&) = default;
Schema(Schema&&) = default;
Schema& operator=(Schema&&) = default;
void __clear() {
dataTypes.clear();
names.clear();
__isset.__clear();
}
virtual ~Schema() throw() {}
std::unordered_map<int64_t, DataType> dataTypes;
std::unordered_map<std::string, int64_t> names;
struct __isset {
__isset() { __clear(); }
void __clear() {
dataTypes = false;
names = false;
}
bool dataTypes;
bool names;
} __isset;
bool operator == (const Schema & rhs) const
{
if (!(this->dataTypes == rhs.dataTypes))
return false;
if (!(this->names == rhs.names))
return false;
return true;
}
bool operator != (const Schema &rhs) const {
return !(*this == rhs);
}
bool operator < (const Schema & ) const;
uint32_t read(apache::thrift::protocol::TProtocol* iprot);
uint32_t write(apache::thrift::protocol::TProtocol* oprot) const;
};
class Schema;
void swap(Schema &a, Schema &b);
}}} // namespace
#endif

180
thrift/leveldb.thrift Normal file
View File

@ -0,0 +1,180 @@
#!/usr/local/bin/thrift --gen cpp
namespace java Tleveldb
namespace cpp Tleveldb
namespace rb Tleveldb
namespace py Tleveldb
namespace perl Tleveldb
// Types
typedef binary Text
typedef binary Bytes
// A basic object needed for storing keys and values
struct Slice {
1:Text data;
2:i32 size
}
// Different compression types supported
enum CompressionType {
kNoCompression = 0x0,
kSnappyCompression = 0x1
}
// Error codes
enum Code {
kOk = 0,
kNotFound = 1,
kCorruption = 2,
kNotSupported = 3,
kInvalidArgument = 4,
kIOError = 5,
kEnd = 6
}
// A range object
struct Range {
1:Slice start; // Included in the range
2:Slice limit // Not included in the range
}
// Options to creating a database
struct DBOptions {
1:bool create_if_missing;
2:bool error_if_exists;
3:i32 write_buffer_size;
4:i32 max_open_files;
5:i32 block_size;
6:i32 block_restart_interval;
7:CompressionType compression
}
// Options for writing
struct WriteOptions {
1:bool sync
}
struct Snapshot {
1:i64 snapshotid // server generated
}
// Options for reading. If you do not have a
// snapshot, set snapshot.snapshotid = 0
struct ReadOptions {
1:bool verify_checksums;
2:bool fill_cache,
3:Snapshot snapshot
}
// Represents a open database object
struct DBHandle {
1:Text dbname; //name of the database
2:i64 handleid // server generated
}
struct Iterator {
1:i64 iteratorid // server generated
}
// flags for the iterator
enum IteratorType {
seekToFirst = 0,
seekToLast = 1,
seekToKey = 2
}
struct kv {
1:Slice key;
2:Slice value
}
// Return a single value from the Get call
struct ResultItem {
1:Code status;
2:Slice value
}
// Return a key,value from a Scan call
struct ResultPair {
1:Code status;
2:kv keyvalue
}
// Snapshot result
struct ResultSnapshot {
1:Code status;
2:Snapshot snapshot
}
// Iterator result
struct ResultIterator {
1:Code status;
2:Iterator iterator
}
exception LeveldbException {
1:Text message,
2:Code errorCode
}
// The Database service
service DB {
// opens the database
DBHandle Open(1:Text dbname, 2:DBOptions dboptions)
throws (1:LeveldbException se),
// closes the database
Code Close(1:DBHandle dbhandle, 2:Text dbname),
// puts a key in the database
Code Put(1:DBHandle dbhandle, 2:kv keyvalue, 3:WriteOptions options),
// deletes a key from the database
Code Delete(1:DBHandle dbhandle, 2:Slice key, 3:WriteOptions options),
// writes batch of keys into the database
Code Write(1:DBHandle dbhandle, 2:list<kv> batch, 3:WriteOptions options),
// fetch a key from the DB.
// ResultItem.status == kNotFound means key is non existant
// ResultItem.status == kOk means key is found
ResultItem Get(1:DBHandle dbhandle, 2:Slice inputkey,
3:ReadOptions options),
// start iteration over a set of keys. If iteratorType.seekToFirst
// is set, then position the iterator at the first key in the source.
// If iteratorType.seekToLast is set, then position at the last key in the
// source. If iteratorType.seekToKey is set, then position at the first
// key in the source that is at or past target.
// If any two of iteratorType.seekToFirst & iteratorType.seekToLast
// and iteratorType.seekToKey are set, then error.
// If either iteratorType.seekToFirst or iteratorType.seekToLast is set,
// then target is not used.
ResultIterator NewIterator(1:DBHandle dbhandle, 2:ReadOptions options,
3:IteratorType iteratorType,
4:Slice target),
// Release resources associated with an iterator allocated previously
// via a call to NewIterator. The call to this method may be skipped
// if the iterator had already traversed all the keys in the specified
// range. If the application wants to terminate a scan before processing
// all the resulting keys, then it is essential to invoke this method.
Code DeleteIterator(1:DBHandle dbhandle, 2:Iterator iterator),
// Return the previous/next from this iteration
ResultPair GetNext(1:DBHandle dbhandle, 2:Iterator iterator),
ResultPair GetPrev(1:DBHandle dbhandle, 2:Iterator iterator),
// Create snapshot.
ResultSnapshot GetSnapshot(1:DBHandle dbhandle),
// Release snapshots
Code ReleaseSnapshot(1:DBHandle dbhandle, 2:Snapshot snapshot),
// compact a range of keys
// begin.size == 0 to start at a range earlier than the first existing key
// end.size == 0 to end at a range later than the last existing key
Code CompactRange(1:DBHandle dbhandle, 2:Slice begin, 3:Slice end),
}

229
thrift/lib/cpp/ClientUtil.h Normal file
View File

@ -0,0 +1,229 @@
/*
* 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 THRIFT_CLIENTUTIL_H_
#define THRIFT_CLIENTUTIL_H_ 1
#include "thrift/lib/cpp/transport/TBufferTransports.h"
#include "thrift/lib/cpp/transport/TSocket.h"
#include "thrift/lib/cpp/protocol/TBinaryProtocol.h"
namespace apache { namespace thrift { namespace util {
/*
* Versions that accept a host and port
*/
template<typename ClientT, typename ProtocolT, typename TransportT>
ClientT* createClient(const std::string& host, uint16_t port) {
boost::shared_ptr<transport::TSocket> socket(
new transport::TSocket(host, port));
// We could specialize this to not create a wrapper transport when
// TransportT is TTransport or TSocket. However, everyone should always
// use a TFramedTransport or TBufferedTransport wrapper for performance
// reasons.
boost::shared_ptr<TransportT> transport(new TransportT(socket));
boost::shared_ptr<ProtocolT> protocol( new ProtocolT(transport));
transport->open();
return new ClientT(protocol);
}
template<typename ClientT, typename ProtocolT, typename TransportT>
boost::shared_ptr<ClientT> createClientPtr(const std::string& host,
uint16_t port) {
return boost::shared_ptr<ClientT>(
createClient<ClientT, ProtocolT, TransportT>(host, port));
}
template<typename ClientT, typename ProtocolT>
ClientT* createClient(const std::string& host,
uint16_t port,
bool useFramed = true) {
if (useFramed) {
return createClient<ClientT, ProtocolT, transport::TFramedTransport>(
host, port);
} else {
return createClient<ClientT, ProtocolT, transport::TBufferedTransport>(
host, port);
}
}
template<typename ClientT>
ClientT* createClient(const std::string& host,
uint16_t port,
bool useFramed = true) {
return createClient<ClientT,
protocol::TBinaryProtocolT<transport::TBufferBase> >(
host, port, useFramed);
}
template<typename ClientT, typename ProtocolT>
boost::shared_ptr<ClientT> createClientPtr(const std::string& host,
uint16_t port,
bool useFramed = true) {
return boost::shared_ptr<ClientT>(
createClient<ClientT, ProtocolT>(host, port, useFramed));
}
template<typename ClientT>
boost::shared_ptr<ClientT> createClientPtr(const std::string& host,
uint16_t port,
bool useFramed = true) {
return boost::shared_ptr<ClientT>(
createClient<ClientT>(host, port, useFramed));
}
/*
* Versions that accept TSocketAddress
*/
template<typename ClientT, typename ProtocolT, typename TransportT>
ClientT* createClient(const transport::TSocketAddress* address) {
boost::shared_ptr<transport::TSocket> socket(
new transport::TSocket(address));
// We could specialize this to not create a wrapper transport when
// TransportT is TTransport or TSocket. However, everyone should always
// use a TFramedTransport or TBufferedTransport wrapper for performance
// reasons.
boost::shared_ptr<TransportT> transport(new TransportT(socket));
boost::shared_ptr<ProtocolT> protocol( new ProtocolT(transport));
transport->open();
return new ClientT(protocol);
}
template<typename ClientT, typename ProtocolT, typename TransportT>
boost::shared_ptr<ClientT> createClientPtr(
const transport::TSocketAddress* address) {
return boost::shared_ptr<ClientT>(
createClient<ClientT, ProtocolT, TransportT>(address));
}
template<typename ClientT, typename ProtocolT>
ClientT* createClient(const transport::TSocketAddress* address,
bool useFramed = true) {
if (useFramed) {
return createClient<ClientT, ProtocolT, transport::TFramedTransport>(
address);
} else {
return createClient<ClientT, ProtocolT, transport::TBufferedTransport>(
address);
}
}
template<typename ClientT>
ClientT* createClient(const transport::TSocketAddress* address,
bool useFramed = true) {
return createClient<ClientT,
protocol::TBinaryProtocolT<transport::TBufferBase> >(
address, useFramed);
}
template<typename ClientT, typename ProtocolT>
boost::shared_ptr<ClientT> createClientPtr(
const transport::TSocketAddress* address,
bool useFramed = true) {
return boost::shared_ptr<ClientT>(
createClient<ClientT, ProtocolT>(address, useFramed));
}
template<typename ClientT>
boost::shared_ptr<ClientT> createClientPtr(
const transport::TSocketAddress* address,
bool useFramed = true) {
return boost::shared_ptr<ClientT>(
createClient<ClientT>(address, useFramed));
}
/*
* Versions that accept TSocketAddress and socket options
*/
template<typename ClientT, typename ProtocolT, typename TransportT>
ClientT* createClient(const transport::TSocketAddress* address,
const transport::TSocket::Options& options) {
boost::shared_ptr<transport::TSocket> socket(
new transport::TSocket(address));
socket->setSocketOptions(options);
// We could specialize this to not create a wrapper transport when
// TransportT is TTransport or TSocket. However, everyone should always
// use a TFramedTransport or TBufferedTransport wrapper for performance
// reasons.
boost::shared_ptr<TransportT> transport(new TransportT(socket));
boost::shared_ptr<ProtocolT> protocol( new ProtocolT(transport));
transport->open();
return new ClientT(protocol);
}
template<typename ClientT, typename ProtocolT, typename TransportT>
boost::shared_ptr<ClientT> createClientPtr(
const transport::TSocketAddress* address,
const transport::TSocket::Options& options) {
return boost::shared_ptr<ClientT>(
createClient<ClientT, ProtocolT, TransportT>(address), options);
}
template<typename ClientT, typename ProtocolT>
ClientT* createClient(const transport::TSocketAddress* address,
const transport::TSocket::Options& options,
bool useFramed = true) {
if (useFramed) {
return createClient<ClientT, ProtocolT, transport::TFramedTransport>(
address, options);
} else {
return createClient<ClientT, ProtocolT, transport::TBufferedTransport>(
address, options);
}
}
template<typename ClientT>
ClientT* createClient(const transport::TSocketAddress* address,
const transport::TSocket::Options& options,
bool useFramed = true
) {
return createClient<ClientT,
protocol::TBinaryProtocolT<transport::TBufferBase> >(
address, options, useFramed);
}
template<typename ClientT, typename ProtocolT>
boost::shared_ptr<ClientT> createClientPtr(
const transport::TSocketAddress* address,
const transport::TSocket::Options& options,
bool useFramed = true) {
return boost::shared_ptr<ClientT>(
createClient<ClientT, ProtocolT>(address, options, useFramed));
}
template<typename ClientT>
boost::shared_ptr<ClientT> createClientPtr(
const transport::TSocketAddress* address,
const transport::TSocket::Options& options,
bool useFramed = true) {
return boost::shared_ptr<ClientT>(
createClient<ClientT>(address, options, useFramed));
}
}}} // apache::thrift::util
#endif // THRIFT_CLIENTUTIL_H_

View File

@ -0,0 +1,311 @@
/*
* 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 THRIFT_EVENTHANDLERBASE_H_
#define THRIFT_EVENTHANDLERBASE_H_ 1
#include <string>
#include <vector>
#include <memory>
#include "thrift/lib/cpp/server/TConnectionContext.h"
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift {
using server::TConnectionContext;
/**
* Virtual interface class that can handle events from the processor. To
* use this you should subclass it and implement the methods that you care
* about. Your subclass can also store local data that you may care about,
* such as additional "arguments" to these methods (stored in the object
* instance's state).
*/
class TProcessorEventHandler {
public:
virtual ~TProcessorEventHandler() {}
/**
* Called before calling other callback methods.
* Expected to return some sort of context object.
* The return value is passed to all other callbacks
* for that function invocation.
*/
virtual void* getContext(const char* fn_name,
TConnectionContext* connectionContext) {
return NULL;
}
/**
* Expected to free resources associated with a context.
*/
virtual void freeContext(void* ctx, const char* fn_name) { }
/**
* Called before reading arguments.
*/
virtual void preRead(void* ctx, const char* fn_name) {}
/**
* Called between reading arguments and calling the handler.
*/
virtual void postRead(void* ctx, const char* fn_name, uint32_t bytes) {}
/**
* Called between calling the handler and writing the response.
*/
virtual void preWrite(void* ctx, const char* fn_name) {}
/**
* Called after writing the response.
*/
virtual void postWrite(void* ctx, const char* fn_name, uint32_t bytes) {}
/**
* Called when an async function call completes successfully.
*/
virtual void asyncComplete(void* ctx, const char* fn_name) {}
/**
* Called if the handler throws an undeclared exception.
*/
virtual void handlerError(void* ctx, const char* fn_name) {}
protected:
TProcessorEventHandler() {}
};
/**
* A helper class used by the generated code to free each context.
*/
class TProcessorContextFreer {
public:
TProcessorContextFreer(boost::shared_ptr<TProcessorEventHandler> handler,
void* context, const char* method) :
handler_(handler), context_(context), method_(method) {}
~TProcessorContextFreer() {
if (handler_ != NULL) {
handler_->freeContext(context_, method_);
}
}
void unregister() { handler_.reset(); }
private:
boost::shared_ptr<TProcessorEventHandler> handler_;
void* context_;
const char* method_;
};
class ContextStack {
friend class EventHandlerBase;
public:
ContextStack(
const std::vector<boost::shared_ptr<TProcessorEventHandler>>& handlers,
const char* method,
TConnectionContext* connectionContext)
: handlers_(handlers)
, method_(method) {
for (auto handler: handlers) {
ctxs.push_back(handler->getContext(method, connectionContext));
}
}
~ContextStack() {
for (size_t i = 0; i < handlers_.size(); i++) {
handlers_[i]->freeContext(ctxs[i], method_);
}
}
private:
std::vector<void*> ctxs;
std::vector<boost::shared_ptr<TProcessorEventHandler>> handlers_;
const char* method_;
};
class EventHandlerBase {
private:
int setEventHandlerPos_;
ContextStack* s_;
public:
EventHandlerBase()
: setEventHandlerPos_(-1)
, s_(NULL)
{}
void addEventHandler(
const boost::shared_ptr<TProcessorEventHandler>& handler) {
handlers_.push_back(handler);
}
void clearEventHandlers() {
handlers_.clear();
setEventHandlerPos_ = -1;
if (eventHandler_) {
setEventHandler(eventHandler_);
}
}
boost::shared_ptr<TProcessorEventHandler> getEventHandler() {
return eventHandler_;
}
void setEventHandler(boost::shared_ptr<TProcessorEventHandler> eventHandler) {
eventHandler_ = eventHandler;
if (setEventHandlerPos_ > 0) {
handlers_.erase(handlers_.begin() + setEventHandlerPos_);
}
setEventHandlerPos_ = handlers_.size();
handlers_.push_back(eventHandler);
}
/**
* These functions are only used in the client handler
* implementation. The server process functions maintain
* ContextStack on the stack and binds ctx in to the async calls.
*
* Clients are not thread safe, so using a member variable is okay.
* Client send_ and recv_ functions contain parameters based off of
* the function call, and adding a parameter there would change the
* function signature enough that other thrift users might break.
*
* The generated code should be the ONLY user of s_. All other functions
* should just use the ContextStack parameter.
*/
ContextStack* getContextStack() {
return s_;
}
// Context only freed by freer, this is only used across function calls.
void setContextStack(ContextStack* s) {
s_ = s;
}
protected:
std::unique_ptr<ContextStack> getContextStack(
const char* fn_name,
TConnectionContext* connectionContext) {
std::unique_ptr<ContextStack> ctx(
new ContextStack(handlers_, fn_name, connectionContext));
return ctx;
}
void preWrite(ContextStack* s, const char* fn_name) {
if (s) {
for (size_t i = 0; i < handlers_.size(); i++) {
handlers_[i]->preWrite(s->ctxs[i], fn_name);
}
}
}
void postWrite(ContextStack* s, const char* fn_name,
uint32_t bytes) {
if (s) {
for (size_t i = 0; i < handlers_.size(); i++) {
handlers_[i]->postWrite(s->ctxs[i], fn_name, bytes);
}
}
}
void preRead(ContextStack* s, const char* fn_name) {
if (s) {
for (size_t i = 0; i < handlers_.size(); i++) {
handlers_[i]->preRead(s->ctxs[i], fn_name);
}
}
}
void postRead(ContextStack* s, const char* fn_name,
uint32_t bytes) {
if (s) {
for (size_t i = 0; i < handlers_.size(); i++) {
handlers_[i]->postRead(s->ctxs[i], fn_name, bytes);
}
}
}
void handlerError(ContextStack* s, const char* fn_name) {
if (s) {
for (size_t i = 0; i < handlers_.size(); i++) {
handlers_[i]->handlerError(s->ctxs[i], fn_name);
}
}
}
void asyncComplete(ContextStack* s, const char* fn_name) {
if (s) {
for (size_t i = 0; i < handlers_.size(); i++) {
handlers_[i]->asyncComplete(s->ctxs[i], fn_name);
}
}
}
std::vector<boost::shared_ptr<TProcessorEventHandler>> handlers_;
boost::shared_ptr<TProcessorEventHandler> eventHandler_;
};
class TProcessorEventHandlerFactory {
public:
virtual boost::shared_ptr<TProcessorEventHandler> getEventHandler() = 0;
};
/**
* Base class for all thrift processors. Used to automatically attach event
* handlers to processors at creation time.
*/
class TProcessorBase : public EventHandlerBase {
public:
TProcessorBase();
static void addProcessorEventHandlerFactory(
boost::shared_ptr<TProcessorEventHandlerFactory> factory);
static void removeProcessorEventHandlerFactory(
boost::shared_ptr<TProcessorEventHandlerFactory> factory);
private:
static std::vector<boost::shared_ptr<TProcessorEventHandlerFactory>>
registeredHandlerFactories_;
};
/**
* Base class for all thrift clients. Used to automatically attach event
* handlers to clients at creation time.
*/
class TClientBase : public EventHandlerBase {
public:
TClientBase();
static void addClientEventHandlerFactory(
boost::shared_ptr<TProcessorEventHandlerFactory> factory);
static void removeClientEventHandlerFactory(
boost::shared_ptr<TProcessorEventHandlerFactory> factory);
private:
static std::vector<boost::shared_ptr<TProcessorEventHandlerFactory>>
registeredHandlerFactories_;
};
}} // apache::thrift
#endif // #ifndef THRIFT_EVENTHANDLERBASE_H_

67
thrift/lib/cpp/README Normal file
View File

@ -0,0 +1,67 @@
Thrift C++ Software Library
License
=======
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.
Using Thrift with C++
=====================
The Thrift C++ libraries are built using the GNU tools. Follow the instructions
in the top-level README, or run bootstrap.sh in this folder to generate the
Makefiles.
In case you do not want to open another README file, do this:
./bootstrap.sh
./configure (--with-boost=/usr/local)
make
sudo make install
Thrift is divided into two libraries.
libthrift
The core Thrift library contains all the core Thrift code. It requires
boost shared pointers, pthreads, and librt.
libthriftnb
This library contains the Thrift nonblocking server, which uses libevent.
To link this library you will also need to link libevent.
Linking Against Thrift
======================
After you build and install Thrift the libraries are installed to
/usr/local/lib by default. Make sure this is in your LDPATH.
On Linux, the best way to do this is to ensure that /usr/local/lib is in
your /etc/ld.so.conf and then run /sbin/ldconfig.
Depending upon whether you are linking dynamically or statically and how
your build environment it set up, you may need to include additional
libraries when linking against thrift, such as librt and/or libpthread. If
you are using libthriftnb you will also need libevent.
Dependencies
============
boost shared pointers
http://www.boost.org/libs/smart_ptr/smart_ptr.htm
libevent (for libthriftnb only)
http://monkey.org/~provos/libevent/

View File

@ -0,0 +1,40 @@
/**
* Copyright 2012 Facebook
* @author Tudor Bosman (tudorb@fb.com)
*/
#ifndef THRIFT_LIB_CPP_REFLECTION_H_
#define THRIFT_LIB_CPP_REFLECTION_H_
#include <cstddef>
#include <cstdint>
#include "reflection_types.h"
namespace apache {
namespace thrift {
namespace reflection {
namespace detail {
const size_t kTypeBits = 5;
const uint64_t kTypeMask = (1ULL << kTypeBits) - 1;
} // namespace detail
inline int64_t makeTypeId(Type type, uint64_t hash) {
return static_cast<int64_t>((hash & ~detail::kTypeMask) | type);
}
inline Type getType(int64_t typeId) {
return static_cast<Type>(typeId & detail::kTypeMask);
}
inline bool isBaseType(Type type) {
return type <= TYPE_DOUBLE;
}
} // namespace reflection
} // namespace thrift
} // namespace apache
#endif /* THRIFT_LIB_CPP_REFLECTION_H_ */

46
thrift/lib/cpp/TARGETS Normal file
View File

@ -0,0 +1,46 @@
# TARGETS file for thrift/lib/cpp
cpp_library (
name = "thrift_exception",
srcs = [
"TApplicationException.cpp",
],
external_deps = [
('boost', None),
],
)
cpp_library (
name = "thrift",
srcs = [
"VirtualProfiling.cpp",
],
deps = [
"@/thrift/lib/cpp:thrift_base",
"@/thrift/lib/cpp/concurrency",
"@/thrift/lib/cpp/processor",
"@/thrift/lib/cpp/protocol",
"@/thrift/lib/cpp/server",
"@/thrift/lib/cpp/transport",
"@/thrift/lib/cpp:thrift_exception",
]
)
cpp_library (
name = "thrift_base",
srcs = [
"Thrift.cpp",
"EventHandlerBase.cpp",
],
external_deps = [
('boost', None),
],
)
cpp_library(
name = "reflection",
srcs = [],
deps = [
"@/thrift/lib/thrift:reflection-cpp",
],
)

View File

@ -0,0 +1,108 @@
// Copyright (c) 2006- Facebook
// Distributed under the Thrift Software License
//
// See accompanying file LICENSE or visit the Thrift site at:
// http://developers.facebook.com/thrift/
#ifndef _THRIFT_TAPPLICATIONEXCEPTION_H_
#define _THRIFT_TAPPLICATIONEXCEPTION_H_ 1
#include "thrift/lib/cpp/Thrift.h"
namespace apache { namespace thrift {
namespace protocol {
class TProtocol;
}
/**
* This class is thrown when some high-level communication errors with
* the remote peer occur, and also when a server throws an unexpected
* exception from a handler method. Because of the latter case, this
* class can be serialized.
*/
class TApplicationException : public TException {
public:
/**
* Error codes for the various types of exceptions.
*/
enum TApplicationExceptionType
{ UNKNOWN = 0
, UNKNOWN_METHOD = 1
, INVALID_MESSAGE_TYPE = 2
, WRONG_METHOD_NAME = 3
, BAD_SEQUENCE_ID = 4
, MISSING_RESULT = 5
, INVALID_TRANSFORM = 6
, INVALID_PROTOCOL = 7
, UNSUPPORTED_CLIENT_TYPE = 8
};
TApplicationException() :
type_(UNKNOWN) {}
TApplicationException(TApplicationExceptionType type) :
type_(type) {}
TApplicationException(const std::string& message) :
message_(message),
type_(UNKNOWN) {}
TApplicationException(TApplicationExceptionType type,
const std::string& message) :
message_(message),
type_(type) {}
virtual ~TApplicationException() throw() {}
/**
* Returns an error code that provides information about the type of error
* that has occurred.
*
* @return Error code
*/
TApplicationExceptionType getType() {
return type_;
}
virtual const char* what() const throw() {
if (message_.empty()) {
switch (type_) {
case UNKNOWN : return "TApplicationException: Unknown application exception";
case UNKNOWN_METHOD : return "TApplicationException: Unknown method";
case INVALID_MESSAGE_TYPE : return "TApplicationException: Invalid message type";
case WRONG_METHOD_NAME : return "TApplicationException: Wrong method name";
case BAD_SEQUENCE_ID : return "TApplicationException: Bad sequence identifier";
case MISSING_RESULT : return "TApplicationException: Missing result";
case INVALID_TRANSFORM :
return "TApplicationException: Invalid transform";
case INVALID_PROTOCOL :
return "TApplicationException: Invalid protocol";
case UNSUPPORTED_CLIENT_TYPE:
return "TApplicationException: Unsupported client type";
default : return "TApplicationException: (Invalid exception type)";
};
} else {
return message_.c_str();
}
}
uint32_t read(protocol::TProtocol* iprot);
uint32_t write(protocol::TProtocol* oprot) const;
protected:
std::string message_;
/**
* Error code
*/
TApplicationExceptionType type_;
};
}} // apache::thrift
#endif // #ifndef _THRIFT_TAPPLICATIONEXCEPTION_H_

View File

@ -0,0 +1,145 @@
/*
* 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 _THRIFT_TDISPATCHPROCESSOR_H_
#define _THRIFT_TDISPATCHPROCESSOR_H_ 1
#include "thrift/lib/cpp/TProcessor.h"
namespace apache { namespace thrift {
using server::TConnectionContext;
/**
* TDispatchProcessor is a helper class to parse the message header then call
* another function to dispatch based on the function name.
*
* Subclasses must implement dispatchCall() to dispatch on the function name.
*/
template <class Protocol_>
class TDispatchProcessorT : public TProcessor {
public:
virtual bool process(boost::shared_ptr<protocol::TProtocol> in,
boost::shared_ptr<protocol::TProtocol> out,
TConnectionContext* connectionContext) {
protocol::TProtocol* inRaw = in.get();
protocol::TProtocol* outRaw = out.get();
// Try to dynamic cast to the template protocol type
Protocol_* specificIn = dynamic_cast<Protocol_*>(inRaw);
Protocol_* specificOut = dynamic_cast<Protocol_*>(outRaw);
if (specificIn && specificOut) {
return processFast(specificIn, specificOut, connectionContext);
}
// Log the fact that we have to use the slow path
T_GENERIC_PROTOCOL(this, inRaw, specificIn);
T_GENERIC_PROTOCOL(this, outRaw, specificOut);
std::string fname;
protocol::TMessageType mtype;
int32_t seqid;
inRaw->readMessageBegin(fname, mtype, seqid);
// If this doesn't look like a valid call, log an error and return false so
// that the server will close the connection.
//
// (The old generated processor code used to try to skip a T_STRUCT and
// continue. However, that seems unsafe.)
if (mtype != protocol::T_CALL && mtype != protocol::T_ONEWAY) {
GlobalOutput.printf("received invalid message type %d from client",
mtype);
return false;
}
return this->dispatchCall(inRaw, outRaw, fname, seqid, connectionContext);
}
protected:
bool processFast(Protocol_* in, Protocol_* out,
TConnectionContext* connectionContext) {
std::string fname;
protocol::TMessageType mtype;
int32_t seqid;
in->readMessageBegin(fname, mtype, seqid);
if (mtype != protocol::T_CALL && mtype != protocol::T_ONEWAY) {
GlobalOutput.printf("received invalid message type %d from client",
mtype);
return false;
}
return this->dispatchCallTemplated(in, out, fname,
seqid, connectionContext);
}
/**
* dispatchCall() methods must be implemented by subclasses
*/
virtual bool dispatchCall(apache::thrift::protocol::TProtocol* in,
apache::thrift::protocol::TProtocol* out,
const std::string& fname, int32_t seqid,
TConnectionContext* connectionContext) = 0;
virtual bool dispatchCallTemplated(Protocol_* in, Protocol_* out,
const std::string& fname, int32_t seqid,
TConnectionContext* connectionContext) = 0;
};
/**
* Non-templatized version of TDispatchProcessor, that doesn't bother trying to
* perform a dynamic_cast.
*/
class TDispatchProcessor : public TProcessor {
public:
virtual bool process(boost::shared_ptr<protocol::TProtocol> in,
boost::shared_ptr<protocol::TProtocol> out,
TConnectionContext* connectionContext) {
std::string fname;
protocol::TMessageType mtype;
int32_t seqid;
in->readMessageBegin(fname, mtype, seqid);
if (mtype != protocol::T_CALL && mtype != protocol::T_ONEWAY) {
GlobalOutput.printf("received invalid message type %d from client",
mtype);
return false;
}
return dispatchCall(in.get(), out.get(), fname, seqid, connectionContext);
}
protected:
virtual bool dispatchCall(apache::thrift::protocol::TProtocol* in,
apache::thrift::protocol::TProtocol* out,
const std::string& fname, int32_t seqid,
TConnectionContext* connectionContext) = 0;
};
// Specialize TDispatchProcessorT for TProtocol and TDummyProtocol just to use
// the generic TDispatchProcessor.
template <>
class TDispatchProcessorT<protocol::TDummyProtocol> :
public TDispatchProcessor {};
template <>
class TDispatchProcessorT<protocol::TProtocol> :
public TDispatchProcessor {};
}} // apache::thrift
#endif // _THRIFT_TDISPATCHPROCESSOR_H_

193
thrift/lib/cpp/TLogging.h Normal file
View File

@ -0,0 +1,193 @@
/*
* 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 THRIFT_TLOGGING_H
#define THRIFT_TLOGGING_H 1
#include "thrift/lib/cpp/thrift_config.h"
#include "thrift/lib/cpp/concurrency/Util.h"
#include <stdio.h>
/**
* Contains utility macros for debugging and logging.
*
*/
#ifdef THRIFT_HAVE_CLOCK_GETTIME
#include <sys/time.h>
#endif
#include <time.h>
#ifdef THRIFT_HAVE_STDINT_H
#include <stdint.h>
#endif
/**
* T_GLOBAL_DEBUGGING_LEVEL = 0: all debugging turned off, debug macros undefined
* T_GLOBAL_DEBUGGING_LEVEL = 1: all debugging turned on
*/
#ifndef T_GLOBAL_DEBUGGING_LEVEL
#define T_GLOBAL_DEBUGGING_LEVEL 0
#endif
/**
* T_GLOBAL_LOGGING_LEVEL = 0: all logging turned off, logging macros undefined
* T_GLOBAL_LOGGING_LEVEL = 1: all logging turned on
*/
#define T_GLOBAL_LOGGING_LEVEL 1
/**
* Standard wrapper around fprintf what will prefix the file name and line
* number to the line. Uses T_GLOBAL_DEBUGGING_LEVEL to control whether it is
* turned on or off.
*
* @param format_string
*/
#define T_DEBUG(format_string,...) \
T_DEBUG_L(0, format_string, ##__VA_ARGS__)
#define COMPUTE_TIME \
int64_t nowMs = apache::thrift::concurrency::Util::currentTime(); \
time_t nowSec = (time_t) (nowMs / 1000); \
nowMs -= nowSec * 1000; \
int ms = (int)nowMs; \
char dbgtime[26]; \
ctime_r(&nowSec, dbgtime); \
dbgtime[24] = '\0';
/**
* analogous to T_DEBUG but also prints the time
*
* @param string format_string input: printf style format string
*/
#define T_DEBUG_T(format_string,...) \
do { \
if (T_GLOBAL_DEBUGGING_LEVEL > 0) { \
COMPUTE_TIME \
fprintf(stderr, "[%s,%d] [%s, %d ms] " format_string " \n", \
__FILE__, __LINE__, dbgtime, ms, \
##__VA_ARGS__); \
} \
} while(0)
/**
* analogous to T_DEBUG but uses input level to determine whether or not the string
* should be logged.
*
* @param int level: specified debug level
* @param string format_string input: format string
*/
#define T_DEBUG_L(level, format_string, ...) \
do { \
if (T_GLOBAL_DEBUGGING_LEVEL > (level)) { \
COMPUTE_TIME \
fprintf(stderr, "[%s,%d] [%s, %d ms] " format_string " \n", \
__FILE__, __LINE__, dbgtime, ms, ##__VA_ARGS__); \
} \
} while (0)
/**
* Explicit error logging. Prints time, file name and line number
*
* @param string format_string input: printf style format string
*/
#define T_ERROR(format_string,...) \
{ \
COMPUTE_TIME \
fprintf(stderr,"[%s,%d] [%s, %d ms] ERROR: " format_string " \n", \
__FILE__, __LINE__,dbgtime, ms, \
##__VA_ARGS__); \
}
/**
* Analogous to T_ERROR, additionally aborting the process.
* WARNING: macro calls abort(), ending program execution
*
* @param string format_string input: printf style format string
*/
#define T_ERROR_ABORT(format_string,...) \
{ \
COMPUTE_TIME \
fprintf(stderr,"[%s,%d] [%s, %d ms] ERROR: Going to abort " \
format_string " \n", \
__FILE__, __LINE__,dbgtime, ms, \
##__VA_ARGS__); \
exit(1); \
}
/**
* Log input message
*
* @param string format_string input: printf style format string
*/
#if T_GLOBAL_LOGGING_LEVEL > 0
#define T_LOG_OPER(format_string,...) \
{ \
if (T_GLOBAL_LOGGING_LEVEL > 0) { \
COMPUTE_TIME \
fprintf(stderr,"[%s, %d ms] " format_string " \n", \
dbgtime, ms, ##__VA_ARGS__); \
} \
}
#else
#define T_LOG_OPER(format_string,...)
#endif
/**
* T_GLOBAL_DEBUG_VIRTUAL = 0 or unset: normal operation,
* virtual call debug messages disabled
* T_GLOBAL_DEBUG_VIRTUAL = 1: log a debug messages whenever an
* avoidable virtual call is made
* T_GLOBAL_DEBUG_VIRTUAL = 2: record detailed info that can be
* printed by calling
* apache::thrift::profile_print_info()
*/
#if T_GLOBAL_DEBUG_VIRTUAL > 1
#define T_VIRTUAL_CALL() \
::apache::thrift::profile_virtual_call(typeid(*this))
#define T_GENERIC_PROTOCOL(template_class, generic_prot, specific_prot) \
do { \
if (!(specific_prot)) { \
::apache::thrift::profile_generic_protocol( \
typeid(*template_class), typeid(*generic_prot)); \
} \
} while (0)
#elif T_GLOBAL_DEBUG_VIRTUAL == 1
#define T_VIRTUAL_CALL() \
fprintf(stderr,"[%s,%d] virtual call\n", __FILE__, __LINE__)
#define T_GENERIC_PROTOCOL(template_class, generic_prot, specific_prot) \
do { \
if (!(specific_prot)) { \
fprintf(stderr, \
"[%s,%d] failed to cast to specific protocol type\n", \
__FILE__, __LINE__); \
} \
} while (0)
#else
#define T_VIRTUAL_CALL()
#define T_GENERIC_PROTOCOL(template_class, generic_prot, specific_prot)
#endif
#endif // #ifndef THRIFT_TLOGGING_H

120
thrift/lib/cpp/TProcessor.h Normal file
View File

@ -0,0 +1,120 @@
/*
* 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 THRIFT_TPROCESSOR_H_
#define THRIFT_TPROCESSOR_H_ 1
#include <string>
#include <vector>
#include "thrift/lib/cpp/EventHandlerBase.h"
#include "thrift/lib/cpp/server/TConnectionContext.h"
#include "thrift/lib/cpp/protocol/TProtocol.h"
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift {
/**
* A processor is a generic object that acts upon two streams of data, one
* an input and the other an output. The definition of this object is loose,
* though the typical case is for some sort of server that either generates
* responses to an input stream or forwards data from one pipe onto another.
*
*/
class TProcessor : public TProcessorBase {
public:
virtual ~TProcessor() {}
virtual bool process(boost::shared_ptr<protocol::TProtocol> in,
boost::shared_ptr<protocol::TProtocol> out,
TConnectionContext* connectionContext) = 0;
bool process(boost::shared_ptr<apache::thrift::protocol::TProtocol> io,
TConnectionContext* connectionContext) {
return process(io, io, connectionContext);
}
protected:
TProcessor() {}
};
class TProcessorFactory {
public:
virtual ~TProcessorFactory() {}
/**
* Get the TProcessor to use for a particular connection.
*
* This method is always invoked in the same thread that the connection was
* accepted on. This generally means that this call does not need to be
* thread safe, as it will always be invoked from a single thread.
*/
virtual boost::shared_ptr<TProcessor> getProcessor(
server::TConnectionContext* ctx) = 0;
};
class TSingletonProcessorFactory : public TProcessorFactory {
public:
explicit TSingletonProcessorFactory(
const boost::shared_ptr<TProcessor>& processor) :
processor_(processor) {}
boost::shared_ptr<TProcessor> getProcessor(server::TConnectionContext*) {
return processor_;
}
boost::shared_ptr<TProcessor> getProcessor() {
return processor_;
}
private:
boost::shared_ptr<TProcessor> processor_;
};
/**
* This is a helper class to allow boost::shared_ptr to be used with handler
* pointers returned by the generated handler factories.
*
* The handler factory classes generated by the thrift compiler return raw
* pointers, and factory->releaseHandler() must be called when the handler is
* no longer needed.
*
* A ReleaseHandler object can be instantiated and passed as the second
* parameter to a shared_ptr, so that factory->releaseHandler() will be called
* when the object is no longer needed, instead of deleting the pointer.
*/
template<typename HandlerFactory_>
class ReleaseHandler {
public:
explicit ReleaseHandler(
const boost::shared_ptr<HandlerFactory_>& handlerFactory) :
handlerFactory_(handlerFactory) {}
void operator()(typename HandlerFactory_::Handler* handler) {
if (handler) {
handlerFactory_->releaseHandler(handler);
}
}
private:
boost::shared_ptr<HandlerFactory_> handlerFactory_;
};
}} // apache::thrift
#endif // #ifndef THRIFT_TPROCESSOR_H_

View File

@ -0,0 +1,96 @@
/*
* 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 _THRIFT_TREFLECTIONLOCAL_H_
#define _THRIFT_TREFLECTIONLOCAL_H_ 1
#include <stdint.h>
#include <cstring>
#include "thrift/lib/cpp/protocol/TProtocol.h"
/**
* Local Reflection is a blanket term referring to the the structure
* and generation of this particular representation of Thrift types.
* (It is called local because it cannot be serialized by Thrift).
*
*/
namespace apache { namespace thrift { namespace reflection { namespace local {
using apache::thrift::protocol::TType;
// We include this many bytes of the structure's fingerprint when serializing
// a top-level structure. Long enough to make collisions unlikely, short
// enough to not significantly affect the amount of memory used.
const int FP_PREFIX_LEN = 4;
struct FieldMeta {
int16_t tag;
bool is_optional;
};
struct TypeSpec {
TType ttype;
uint8_t fp_prefix[FP_PREFIX_LEN];
// Use an anonymous union here so we can fit two TypeSpecs in one cache line.
union {
struct {
// Use parallel arrays here for denser packing (of the arrays).
FieldMeta* metas;
TypeSpec** specs;
} tstruct;
struct {
TypeSpec *subtype1;
TypeSpec *subtype2;
} tcontainer;
};
// Static initialization of unions isn't really possible,
// so take the plunge and use constructors.
// Hopefully they'll be evaluated at compile time.
TypeSpec(TType ttype) : ttype(ttype) {
std::memset(fp_prefix, 0, FP_PREFIX_LEN);
}
TypeSpec(TType ttype,
const uint8_t* fingerprint,
FieldMeta* metas,
TypeSpec** specs) :
ttype(ttype)
{
std::memcpy(fp_prefix, fingerprint, FP_PREFIX_LEN);
tstruct.metas = metas;
tstruct.specs = specs;
}
TypeSpec(TType ttype, TypeSpec* subtype1, TypeSpec* subtype2) :
ttype(ttype)
{
std::memset(fp_prefix, 0, FP_PREFIX_LEN);
tcontainer.subtype1 = subtype1;
tcontainer.subtype2 = subtype2;
}
};
}}}} // apache::thrift::reflection::local
#endif // #ifndef _THRIFT_TREFLECTIONLOCAL_H_

261
thrift/lib/cpp/Thrift.h Normal file
View File

@ -0,0 +1,261 @@
/*
* 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 THRIFT_THRIFT_H_
#define THRIFT_THRIFT_H_
#include "thrift/lib/cpp/thrift_config.h"
#include <stdio.h>
#include <assert.h>
#include <sys/types.h>
#include <netinet/in.h>
#ifdef THRIFT_HAVE_INTTYPES_H
#include <inttypes.h>
#endif
#include <string>
#include <map>
#include <list>
#include <set>
#include <vector>
#include <exception>
#include <typeinfo>
#include <string.h>
#include "thrift/lib/cpp/TLogging.h"
namespace apache { namespace thrift {
struct ltstr {
bool operator()(const char* s1, const char* s2) const {
return strcmp(s1, s2) < 0;
}
};
/**
* Helper template class for enum<->string conversion.
*/
template<typename T>
struct TEnumTraits {
/**
* Finds the name of a given enum value, returning it or NULL on failure.
* Specialized implementations will be emitted as part of enum codegen.
*
* Example specialization:
* template<>
* const char* TEnumTraits<MyEnum>::findName(MyEnum value) {
* return findName(_MyEnum_VALUES_TO_NAMES, value);
* }
* Note the use of helper function 'findName(...)', below.
*/
static const char* findName(T value);
/**
* Attempts to find a value for a given name.
* Specialized implementations will be emitted as part of enum codegen.
*
* Example implementation:
* template<>
* bool TEnumTraits<MyEnum>::findValue(const char* name,
* MyEnum* outValue) {
* return findValue(_MyEnum_NAMES_TO_VALUES, name, outValue);
* }
* Note the use of helper function 'findValue(...)', below.
*/
static bool findValue(const char* name, T* outValue);
private:
/**
* Helper method used by codegen implementation of findName, Supports
* use with strict and non-strict enums by way of template parameter
* 'ValueType'.
*/
template<typename ValueType>
static const char* findName(const std::map<ValueType, const char*>& map,
T value) {
auto found = map.find(value);
if (found == map.end()) {
return NULL;
} else {
return found->second;
}
}
/**
* Helper method used by codegen implementation of findValue, Supports
* use with strict and non-strict enums by way of template parameter
* 'ValueType'.
*/
template<typename ValueType>
static bool findValue(const std::map<const char*, ValueType, ltstr>& map,
const char* name, T* out) {
auto found = map.find(name);
if (found == map.end()) {
return false;
} else {
*out = static_cast<T>(found->second);
return true;
}
}
};
template <typename T>
class TEnumIterator : public std::map<T, char*>::iterator {
public:
TEnumIterator(int n,
T* enums,
const char** names) :
ii_(0), n_(n), enums_(enums), names_(names) {
}
int operator ++() {
return ++ii_;
}
bool operator !=(const TEnumIterator<T>& end) {
assert(end.n_ == -1);
return (ii_ != n_);
}
std::pair<T, const char*> operator*() const {
return std::make_pair(enums_[ii_], names_[ii_]);
}
private:
int ii_;
const int n_;
T* enums_;
const char** names_;
};
template <typename T>
class TEnumInverseIterator : public std::map<T, char*>::iterator {
public:
TEnumInverseIterator(int n,
T* enums,
const char** names) :
ii_(0), n_(n), enums_(enums), names_(names) {
}
int operator ++() {
return ++ii_;
}
bool operator !=(const TEnumInverseIterator<T>& end) {
assert(end.n_ == -1);
return (ii_ != n_);
}
std::pair<const char*, T> operator*() const {
return std::make_pair(names_[ii_], enums_[ii_]);
}
private:
int ii_;
const int n_;
T* enums_;
const char** names_;
};
class TOutput {
public:
TOutput() : f_(&errorTimeWrapper) {}
inline void setOutputFunction(void (*function)(const char *)){
f_ = function;
}
inline void operator()(const char *message){
f_(message);
}
// It is important to have a const char* overload here instead of
// just the string version, otherwise errno could be corrupted
// if there is some problem allocating memory when constructing
// the string.
void perror(const char *message, int errno_copy);
inline void perror(const std::string &message, int errno_copy) {
perror(message.c_str(), errno_copy);
}
void printf(const char *message, ...);
inline static void errorTimeWrapper(const char* msg) {
time_t now;
char dbgtime[26];
time(&now);
ctime_r(&now, dbgtime);
dbgtime[24] = 0;
fprintf(stderr, "Thrift: %s %s\n", dbgtime, msg);
}
/** Just like strerror_r but returns a C++ string object. */
static std::string strerror_s(int errno_copy);
private:
void (*f_)(const char *);
};
extern TOutput GlobalOutput;
/**
* Base class for all Thrift exceptions.
* Should never be instantiated, only caught.
*/
class TException : public std::exception {
};
/**
* Base class for exceptions from the Thrift library, and occasionally
* from the generated code. This class should not be thrown by user code.
* Instances of this class are not meant to be serialized.
*/
class TLibraryException : public TException {
public:
TLibraryException() {}
explicit TLibraryException(const std::string& message) :
message_(message) {}
TLibraryException(const char* message, int errnoValue);
virtual ~TLibraryException() throw() {}
virtual const char* what() const throw() {
if (message_.empty()) {
return "Default TLibraryException.";
} else {
return message_.c_str();
}
}
protected:
std::string message_;
};
#if T_GLOBAL_DEBUG_VIRTUAL > 1
void profile_virtual_call(const std::type_info& info);
void profile_generic_protocol(const std::type_info& template_type,
const std::type_info& prot_type);
void profile_print_info(FILE *f);
void profile_print_info();
void profile_write_pprof(FILE* gen_calls_f, FILE* virtual_calls_f);
#endif
}} // apache::thrift
#endif // #ifndef THRIFT_THRIFT_H_

View File

@ -0,0 +1,98 @@
#ifndef _THRIFT_ASYNC_SIMPLECALLBACK_H_
#define _THRIFT_ASYNC_SIMPLECALLBACK_H_ 1
#include "thrift/lib/cpp/Thrift.h"
namespace apache { namespace thrift {
/**
* A template class for forming simple method callbacks with either an empty
* argument list or one argument of known type.
*
* For more efficiency where tr1::function is overkill.
*/
template<typename C, ///< class whose method we wish to wrap
typename A = void, ///< type of argument
typename R = void> ///< type of return value
class SimpleCallback {
typedef R (C::*cfptr_t)(A); ///< pointer-to-member-function type
cfptr_t fptr_; ///< the embedded function pointer
C* obj_; ///< object whose function we're wrapping
public:
/**
* Constructor for empty callback object.
*/
SimpleCallback() :
fptr_(NULL), obj_(NULL) {}
/**
* Construct callback wrapper for member function.
*
* @param fptr pointer-to-member-function
* @param "this" for object associated with callback
*/
SimpleCallback(cfptr_t fptr, const C* obj) :
fptr_(fptr), obj_(const_cast<C*>(obj))
{}
/**
* Make a call to the member function we've wrapped.
*
* @param i argument for the wrapped member function
* @return value from that function
*/
R operator()(A i) const {
(obj_->*fptr_)(i);
}
operator bool() const {
return obj_ != NULL && fptr_ != NULL;
}
~SimpleCallback() {}
};
/**
* Specialization of SimpleCallback for empty argument list.
*/
template<typename C, ///< class whose method we wish to wrap
typename R> ///< type of return value
class SimpleCallback<C, void, R> {
typedef R (C::*cfptr_t)(); ///< pointer-to-member-function type
cfptr_t fptr_; ///< the embedded function pointer
C* obj_; ///< object whose function we're wrapping
public:
/**
* Constructor for empty callback object.
*/
SimpleCallback() :
fptr_(NULL), obj_(NULL) {}
/**
* Construct callback wrapper for member function.
*
* @param fptr pointer-to-member-function
* @param obj "this" for object associated with callback
*/
SimpleCallback(cfptr_t fptr, const C* obj) :
fptr_(fptr), obj_(const_cast<C*>(obj))
{}
/**
* Make a call to the member function we've wrapped.
*
* @return value from that function
*/
R operator()() const {
(obj_->*fptr_)();
}
operator bool() const {
return obj_ != NULL && fptr_ != NULL;
}
~SimpleCallback() {}
};
}} // apache::thrift
#endif /* !_THRIFT_ASYNC_SIMPLECALLBACK_H_ */

View File

@ -0,0 +1,103 @@
# The base async library
cpp_library(
name = 'async_base',
srcs = [
'TAsyncServerSocket.cpp',
'TAsyncSignalHandler.cpp',
'TAsyncSocket.cpp',
'TAsyncTimeout.cpp',
'TBinaryAsyncChannel.cpp',
'THeaderAsyncChannel.cpp',
'TEventBase.cpp',
'TEventBaseManager.cpp',
'TEventHandler.cpp',
'TFramedAsyncChannel.cpp',
'TNotificationPipe.cpp',
'TUnframedAsyncChannel.cpp',
'THttpAsyncChannel.cpp',
],
deps = [
'@/folly/experimental/io',
'@/thrift/lib/cpp:thrift_base',
'@/thrift/lib/cpp/transport',
'@/thrift/lib/cpp/transport:header',
'@/thrift/lib/cpp/protocol',
'@/thrift/lib/cpp/server',
'@/thrift/lib/cpp/util:httpparser',
],
external_deps = [ ('libevent', None) ],
)
# TEventServer library (async name is used all over), now depends on ssl
cpp_library(
name = 'async',
srcs = [
'TEventConnection.cpp',
'TEventServer.cpp',
'TEventTask.cpp',
'TEventWorker.cpp',
],
deps = [
':async_ssl',
],
)
cpp_library (
name = "async_ssl",
srcs = [
'TAsyncSSLServerSocket.cpp',
'TAsyncSSLSocket.cpp',
],
deps = [
":async_base",
"@/thrift/lib/cpp/transport:transport_ssl",
],
external_deps = [ ("openssl", None, "ssl"),
("openssl", None, "crypto") ],
)
# This library is the same as async_ssl, except that it has the debug
# logging level set to 4, whereas async_ssl has debug logging disabled.
cpp_library (
name = "async_ssl_log",
srcs = [
'TAsyncSSLServerSocket.cpp',
'TAsyncSSLSocket.cpp',
],
deps = [
":async_base",
"@/thrift/lib/cpp/transport:transport_ssl",
],
external_deps = [ ("openssl", None, "ssl"),
("openssl", None, "crypto") ],
preprocessor_flags = [
"-DT_GLOBAL_DEBUGGING_LEVEL=4"
],
output_subdir = "async_ssl_log",
)
cpp_library(
name = 'zlib',
srcs = [
'TZlibAsyncChannel.cpp',
],
deps = [
':async',
'@/thrift/lib/cpp/transport:zlib',
],
)
# Parts of the async library that depend on C++11 features.
# This is separate from the main async library for now so that users who cannot
# depend on C++11 features yet will still be able to use the bulk of the async
# library.
cpp_library(
name = 'async_cxx11',
srcs = [
'TAsyncTimeoutSet.cpp',
],
deps = [
':async_base',
'@/folly',
],
)

View File

@ -0,0 +1,113 @@
// Copyright (c) 2006- Facebook
// Distributed under the Thrift Software License
//
// See accompanying file LICENSE or visit the Thrift site at:
// http://developers.facebook.com/thrift/
#ifndef THRIFT_ASYNC_TASYNCCHANNEL_H_
#define THRIFT_ASYNC_TASYNCCHANNEL_H_ 1
#include <tr1/functional>
#include "thrift/lib/cpp/Thrift.h"
#include "thrift/lib/cpp/transport/TTransportUtils.h"
namespace apache { namespace thrift { namespace async {
class TAsyncTransport;
/**
* TAsyncChannel defines an asynchronous API for message-based I/O.
*/
class TAsyncChannel {
public:
typedef std::tr1::function<void()> VoidCallback;
virtual ~TAsyncChannel() {}
// is the channel readable (possibly closed by the remote site)?
virtual bool readable() const = 0;
// is the channel in a good state?
virtual bool good() const = 0;
virtual bool error() const = 0;
virtual bool timedOut() const = 0;
/**
* Send a message over the channel.
*
* @return call "cob" on success, "errorCob" on fail. Caller must be ready
* for either cob to be called before return. Only one cob will be
* called and it will be called exactly once per invocation.
*/
virtual void sendMessage(
const VoidCallback& cob,
const VoidCallback& errorCob,
apache::thrift::transport::TMemoryBuffer* message) = 0;
/**
* Receive a message from the channel.
*
* @return call "cob" on success, "errorCob" on fail. Caller must be ready
* for either cob to be called before return. Only one cob will be
* called and it will be called exactly once per invocation.
*/
virtual void recvMessage(
const VoidCallback& cob,
const VoidCallback& errorCob,
apache::thrift::transport::TMemoryBuffer* message) = 0;
/**
* Send a message over the channel and receive a response.
*
* @return call "cob" on success, "errorCob" on fail. Caller must be ready
* for either cob to be called before return. Only one cob will be
* called and it will be called exactly once per invocation.
*/
virtual void sendAndRecvMessage(
const VoidCallback& cob,
const VoidCallback& errorCob,
apache::thrift::transport::TMemoryBuffer* sendBuf,
apache::thrift::transport::TMemoryBuffer* recvBuf) = 0;
/**
* Send a message over the channel, single cob version. (See above.)
*
* @return call "cob" on success or fail; channel status must be queried
* by the cob.
*/
void sendMessage(const VoidCallback& cob,
apache::thrift::transport::TMemoryBuffer* message) {
return sendMessage(cob, cob, message);
}
/**
* Receive a message from the channel, single cob version. (See above.)
*
* @return call "cob" on success or fail; channel status must be queried
* by the cob.
*/
void recvMessage(const VoidCallback& cob,
apache::thrift::transport::TMemoryBuffer* message) {
return recvMessage(cob, cob, message);
}
/**
* Send a message over the channel and receive response, single cob version.
* (See above.)
*
* @return call "cob" on success or fail; channel status must be queried
* by the cob.
*/
void sendAndRecvMessage(const VoidCallback& cob,
apache::thrift::transport::TMemoryBuffer* sendBuf,
apache::thrift::transport::TMemoryBuffer* recvBuf) {
return sendAndRecvMessage(cob, cob, sendBuf, recvBuf);
}
// TODO(dreiss): Make this nonvirtual when TFramedSocketAsyncChannel gets
// renamed to TFramedAsyncChannel.
virtual boost::shared_ptr<TAsyncTransport> getTransport() = 0;
};
}}} // apache::thrift::async
#endif // #ifndef THRIFT_ASYNC_TASYNCCHANNEL_H_

View File

@ -0,0 +1,175 @@
/*
* 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 THRIFT_ASYNC_TASYNCDISPATCHPROCESSOR_H_
#define THRIFT_ASYNC_TASYNCDISPATCHPROCESSOR_H_ 1
#include "thrift/lib/cpp/async/TAsyncProcessor.h"
namespace apache { namespace thrift { namespace async {
/**
* TAsyncDispatchProcessor is a helper class to parse the message header then
* call another function to dispatch based on the function name.
*
* Subclasses must implement dispatchCall() to dispatch on the function name.
*/
template <class Protocol_>
class TAsyncDispatchProcessorT : public TAsyncProcessor {
public:
virtual void process(std::tr1::function<void(bool success)> _return,
boost::shared_ptr<protocol::TProtocol> in,
boost::shared_ptr<protocol::TProtocol> out,
TConnectionContext* context) {
protocol::TProtocol* inRaw = in.get();
protocol::TProtocol* outRaw = out.get();
// Try to dynamic cast to the template protocol type
Protocol_* specificIn = dynamic_cast<Protocol_*>(inRaw);
Protocol_* specificOut = dynamic_cast<Protocol_*>(outRaw);
if (specificIn && specificOut) {
return processFast(_return, specificIn, specificOut, context);
}
// Log the fact that we have to use the slow path
T_GENERIC_PROTOCOL(this, inRaw, specificIn);
T_GENERIC_PROTOCOL(this, outRaw, specificOut);
std::string fname;
protocol::TMessageType mtype;
int32_t seqid;
try {
inRaw->readMessageBegin(fname, mtype, seqid);
} catch (const TException &ex) {
GlobalOutput.printf("received invalid message from client: %s",
ex.what());
_return(false);
return;
}
// If this doesn't look like a valid call, log an error and return false so
// that the server will close the connection.
//
// (The old generated processor code used to try to skip a T_STRUCT and
// continue. However, that seems unsafe.)
if (mtype != protocol::T_CALL && mtype != protocol::T_ONEWAY) {
GlobalOutput.printf("received invalid message type %d from client",
mtype);
_return(false);
return;
}
return this->dispatchCall(_return, inRaw, outRaw, fname, seqid, context);
}
void processFast(std::tr1::function<void(bool success)> _return,
Protocol_* in, Protocol_* out,
TConnectionContext* context) {
std::string fname;
protocol::TMessageType mtype;
int32_t seqid;
try {
in->readMessageBegin(fname, mtype, seqid);
} catch (const TException &ex) {
GlobalOutput.printf("received invalid message from client: %s",
ex.what());
_return(false);
return;
}
if (mtype != protocol::T_CALL && mtype != protocol::T_ONEWAY) {
GlobalOutput.printf("received invalid message type %d from client",
mtype);
_return(false);
return;
}
return this->dispatchCallTemplated(_return, in, out, fname, seqid, context);
}
virtual void dispatchCall(std::tr1::function<void(bool ok)> _return,
apache::thrift::protocol::TProtocol* in,
apache::thrift::protocol::TProtocol* out,
const std::string& fname, int32_t seqid,
TConnectionContext* context) = 0;
virtual void dispatchCallTemplated(std::tr1::function<void(bool ok)> _return,
Protocol_* in, Protocol_* out,
const std::string& fname, int32_t seqid,
TConnectionContext* context) = 0;
};
/**
* Non-templatized version of TAsyncDispatchProcessor,
* that doesn't bother trying to perform a dynamic_cast.
*/
class TAsyncDispatchProcessor : public TAsyncProcessor {
public:
virtual void process(std::tr1::function<void(bool success)> _return,
boost::shared_ptr<protocol::TProtocol> in,
boost::shared_ptr<protocol::TProtocol> out,
TConnectionContext* context) {
protocol::TProtocol* inRaw = in.get();
protocol::TProtocol* outRaw = out.get();
std::string fname;
protocol::TMessageType mtype;
int32_t seqid;
try {
inRaw->readMessageBegin(fname, mtype, seqid);
} catch (const TException &ex) {
GlobalOutput.printf("received invalid message from client: %s",
ex.what());
_return(false);
return;
}
// If this doesn't look like a valid call, log an error and return false so
// that the server will close the connection.
//
// (The old generated processor code used to try to skip a T_STRUCT and
// continue. However, that seems unsafe.)
if (mtype != protocol::T_CALL && mtype != protocol::T_ONEWAY) {
GlobalOutput.printf("received invalid message type %d from client",
mtype);
_return(false);
return;
}
return dispatchCall(_return, inRaw, outRaw, fname, seqid, context);
}
virtual void dispatchCall(std::tr1::function<void(bool ok)> _return,
apache::thrift::protocol::TProtocol* in,
apache::thrift::protocol::TProtocol* out,
const std::string& fname, int32_t seqid,
TConnectionContext* context) = 0;
};
// Specialize TAsyncDispatchProcessorT for TProtocol and TDummyProtocol just to
// use the generic TDispatchProcessor.
template <>
class TAsyncDispatchProcessorT<protocol::TDummyProtocol> :
public TAsyncDispatchProcessor {};
template <>
class TAsyncDispatchProcessorT<protocol::TProtocol> :
public TAsyncDispatchProcessor {};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TASYNCDISPATCHPROCESSOR_H_

View File

@ -0,0 +1,95 @@
/*
* 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 THRIFT_ASYNC_TASYNCEVENTCHANNEL_H_
#define THRIFT_ASYNC_TASYNCEVENTCHANNEL_H_ 1
#include "thrift/lib/cpp/async/TAsyncChannel.h"
#include "thrift/lib/cpp/async/TDelayedDestruction.h"
namespace apache { namespace thrift { namespace async {
class TEventBase;
/**
* TAsyncEventChannel defines an API for TAsyncChannel objects that are driven
* by TEventBase.
*/
class TAsyncEventChannel : public TAsyncChannel,
public TDelayedDestruction {
public:
/**
* Determine if this channel is idle (i.e., has no outstanding reads or
* writes).
*/
virtual bool isIdle() const = 0;
/**
* Attach the channel to a TEventBase.
*
* This may only be called if the channel is not currently attached to a
* TEventBase (by an earlier call to detachEventBase()).
*
* This method must be invoked in the TEventBase's thread.
*/
virtual void attachEventBase(TEventBase* eventBase) = 0;
/**
* Detach the channel from its TEventBase.
*
* This may only be called when the channel is idle and has no reads or
* writes pending. Once detached, the channel may not be used again until it
* is re-attached to a TEventBase by calling attachEventBase().
*
* This method must be called from the current TEventBase's thread.
*/
virtual void detachEventBase() = 0;
/**
* Get the receive timeout.
*
* @return Returns the current receive timeout, in milliseconds. A return
* value of 0 indicates that no timeout is set.
*/
virtual uint32_t getRecvTimeout() const = 0;
/**
* Set the timeout for receiving messages.
*
* When set to a non-zero value, the entire message must be received within
* the specified number of milliseconds, or the receive will fail and the
* channel will be closed.
*/
virtual void setRecvTimeout(uint32_t milliseconds) = 0;
protected:
/**
* Protected destructor.
*
* Users of TAsyncEventChannel must never delete it directly. Instead, invoke
* destroy() instead. (See the documentation in TDelayedDestruction.h for
* more details.)
*/
virtual ~TAsyncEventChannel() { }
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TASYNCEVENTCHANNEL_H_

View File

@ -0,0 +1,90 @@
// Copyright (c) 2006- Facebook
// Distributed under the Thrift Software License
//
// See accompanying file LICENSE or visit the Thrift site at:
// http://developers.facebook.com/thrift/
#ifndef THRIFT_TASYNCPROCESSOR_H
#define THRIFT_TASYNCPROCESSOR_H 1
#include <tr1/functional>
#include <boost/shared_ptr.hpp>
#include "thrift/lib/cpp/TProcessor.h"
#include "thrift/lib/cpp/server/TConnectionContext.h"
#include "thrift/lib/cpp/protocol/TProtocol.h"
using apache::thrift::server::TConnectionContext;
namespace apache { namespace thrift { namespace async {
/**
* Async version of a TProcessor. It is not expected to complete by the time
* the call to process returns. Instead, it calls a cob to signal completion.
*
* @author David Reiss <dreiss@facebook.com>
*/
class TEventServer; // forward declaration
class TAsyncProcessor : public TProcessorBase {
public:
virtual ~TAsyncProcessor() {}
virtual void process(std::tr1::function<void(bool success)> _return,
boost::shared_ptr<protocol::TProtocol> in,
boost::shared_ptr<protocol::TProtocol> out,
TConnectionContext* context = NULL) = 0;
void process(std::tr1::function<void(bool success)> _return,
boost::shared_ptr<apache::thrift::protocol::TProtocol> io) {
return process(_return, io, io);
}
const TEventServer* getAsyncServer() {
return asyncServer_;
}
protected:
TAsyncProcessor() {}
const TEventServer* asyncServer_;
private:
friend class TEventServer;
void setAsyncServer(const TEventServer* server) {
asyncServer_ = server;
}
};
class TAsyncProcessorFactory {
public:
virtual ~TAsyncProcessorFactory() {}
/**
* Get the TAsyncProcessor to use for a particular connection.
*/
virtual boost::shared_ptr<TAsyncProcessor> getProcessor(
server::TConnectionContext* ctx) = 0;
};
class TAsyncSingletonProcessorFactory : public TAsyncProcessorFactory {
public:
explicit TAsyncSingletonProcessorFactory(
const boost::shared_ptr<TAsyncProcessor>& processor) :
processor_(processor) {}
boost::shared_ptr<TAsyncProcessor> getProcessor(server::TConnectionContext*) {
return processor_;
}
private:
boost::shared_ptr<TAsyncProcessor> processor_;
};
}}} // apache::thrift::async
// XXX I'm lazy for now
namespace apache { namespace thrift {
using apache::thrift::async::TAsyncProcessor;
}}
#endif // #ifndef THRIFT_TASYNCPROCESSOR_H

View File

@ -0,0 +1,171 @@
/*
* 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 THRIFT_ASYNC_TASYNCSSLSERVERSOCKET_H_
#define THRIFT_ASYNC_TASYNCSSLSERVERSOCKET_H_ 1
#include "thrift/lib/cpp/async/TAsyncServerSocket.h"
#include "thrift/lib/cpp/transport/TSSLSocket.h"
namespace apache { namespace thrift {
namespace transport {
class TSocketAddress;
}
namespace async {
class TAsyncSSLSocket;
class TAsyncSSLServerSocket : public TDelayedDestruction,
private TAsyncServerSocket::AcceptCallback {
public:
class SSLAcceptCallback {
public:
virtual ~SSLAcceptCallback() {}
/**
* connectionAccepted() is called whenever a new client connection is
* received.
*
* The SSLAcceptCallback will remain installed after connectionAccepted()
* returns.
*
* @param sock The newly accepted client socket. The
* SSLAcceptCallback
* assumes ownership of this socket, and is responsible
* for closing it when done.
*/
virtual void connectionAccepted(
const boost::shared_ptr<TAsyncSSLSocket> &sock)
THRIFT_NOEXCEPT = 0;
/**
* acceptError() is called if an error occurs while accepting.
*
* The SSLAcceptCallback will remain installed even after an accept error.
* If the callback wants to uninstall itself and stop trying to accept new
* connections, it must explicit call setAcceptCallback(NULL).
*
* @param ex An exception representing the error.
*/
virtual void acceptError(const std::exception& ex) THRIFT_NOEXCEPT = 0;
};
/**
* Create a new TAsyncSSLServerSocket with the specified TEventBase.
*
* @param eventBase The TEventBase to use for driving the asynchronous I/O.
* If this parameter is NULL, attachEventBase() must be
* called before this socket can begin accepting
* connections. All TAsyncSSLSocket objects accepted by
* this server socket will be attached to this TEventBase
* when they are created.
*/
explicit TAsyncSSLServerSocket(
const boost::shared_ptr<apache::thrift::transport::SSLContext>& ctx,
TEventBase* eventBase = NULL);
/**
* Destroy the socket.
*
* destroy() must be called to destroy the socket. The normal destructor is
* private, and should not be invoked directly. This prevents callers from
* deleting a TAsyncSSLServerSocket while it is invoking a callback.
*/
virtual void destroy();
void bind(const transport::TSocketAddress& address) {
serverSocket_->bind(address);
}
void bind(uint16_t port) {
serverSocket_->bind(port);
}
void getAddress(transport::TSocketAddress* addressReturn) {
serverSocket_->getAddress(addressReturn);
}
void listen(int backlog) {
serverSocket_->listen(backlog);
}
/**
* Helper function to create a shared_ptr<TAsyncSSLServerSocket>.
*
* This passes in the correct destructor object, since TAsyncSSLServerSocket's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<TAsyncSSLServerSocket> newSocket(
const boost::shared_ptr<apache::thrift::transport::SSLContext>& ctx,
TEventBase* evb) {
return boost::shared_ptr<TAsyncSSLServerSocket>(
new TAsyncSSLServerSocket(ctx, evb),
Destructor());
}
/**
* Set the accept callback.
*
* This method may only be invoked from the TEventBase's loop thread.
*
* @param callback The callback to invoke when a new socket
* connection is accepted and a new TAsyncSSLSocket is
* created.
*
* Throws TTransportException on error.
*/
void setSSLAcceptCallback(SSLAcceptCallback* callback);
SSLAcceptCallback *getSSLAcceptCallback() const {
return sslCallback_;
}
void attachEventBase(TEventBase* eventBase);
void detachEventBase();
/**
* Returns the TEventBase that the handler is currently attached to.
*/
TEventBase* getEventBase() const {
return eventBase_;
}
protected:
/**
* Protected destructor.
*
* Invoke destroy() instead to destroy the TAsyncSSLServerSocket.
*/
virtual ~TAsyncSSLServerSocket();
private:
virtual void connectionAccepted(int fd,
const transport::TSocketAddress& clientAddr)
THRIFT_NOEXCEPT;
virtual void acceptError(const std::exception& ex) THRIFT_NOEXCEPT;
TEventBase* eventBase_;
TAsyncServerSocket* serverSocket_;
// SSL context
boost::shared_ptr<apache::thrift::transport::SSLContext> ctx_;
// The accept callback
SSLAcceptCallback* sslCallback_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TASYNCSSLSERVERSOCKET_H_

View File

@ -0,0 +1,357 @@
/*
* 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 THRIFT_ASYNC_TASYNCSSLSOCKET_H_
#define THRIFT_ASYNC_TASYNCSSLSOCKET_H_ 1
#include "thrift/lib/cpp/async/TAsyncSocket.h"
#include "thrift/lib/cpp/async/TAsyncTimeout.h"
#include "thrift/lib/cpp/transport/TSSLSocket.h"
#include "thrift/lib/cpp/transport/TTransportException.h"
namespace apache { namespace thrift {
namespace async {
class TSSLException: public apache::thrift::transport::TTransportException {
public:
TSSLException(int sslError, int errno_copy);
int getSSLError() const { return error_; }
protected:
int error_;
};
/**
* A class for performing asynchronous I/O on an SSL connection.
*
* TAsyncSSLSocket allows users to asynchronously wait for data on an
* SSL connection, and to asynchronously send data.
*
* The APIs for reading and writing are intentionally asymmetric.
* Waiting for data to read is a persistent API: a callback is
* installed, and is notified whenever new data is available. It
* continues to be notified of new events until it is uninstalled.
*
* TAsyncSSLSocket does not provide read timeout functionality,
* because it typically cannot determine when the timeout should be
* active. Generally, a timeout should only be enabled when
* processing is blocked waiting on data from the remote endpoint.
* For server connections, the timeout should not be active if the
* server is currently processing one or more outstanding requests for
* this connection. For client connections, the timeout should not be
* active if there are no requests pending on the connection.
* Additionally, if a client has multiple pending requests, it will
* ususally want a separate timeout for each request, rather than a
* single read timeout.
*
* The write API is fairly intuitive: a user can request to send a
* block of data, and a callback will be informed once the entire
* block has been transferred to the kernel, or on error.
* TAsyncSSLSocket does provide a send timeout, since most callers
* want to give up if the remote end stops responding and no further
* progress can be made sending the data.
*/
class TAsyncSSLSocket : public TAsyncSocket {
public:
#if THRIFT_HAVE_UNIQUE_PTR
typedef std::unique_ptr<TAsyncSSLSocket, Destructor> UniquePtr;
#endif
class HandshakeCallback {
public:
virtual ~HandshakeCallback() {}
/**
* handshakeSuccess() is called when a new SSL connection is
* established, i.e., after SSL_accept/connect() returns successfully.
*
* The HandshakeCallback will be uninstalled before handshakeSuccess()
* is called.
*
* @param sock SSL socket on which the handshake was initiated
*/
virtual void handshakeSuccess(TAsyncSSLSocket *sock) THRIFT_NOEXCEPT = 0;
/**
* handshakeError() is called if an error occurs while
* establishing the SSL connection.
*
* The HandshakeCallback will be uninstalled before handshakeError()
* is called.
*
* @param sock SSL socket on which the handshake was initiated
* @param ex An exception representing the error.
*/
virtual void handshakeError(
TAsyncSSLSocket *sock,
const apache::thrift::transport::TTransportException& ex)
THRIFT_NOEXCEPT = 0;
};
class HandshakeTimeout : public TAsyncTimeout {
public:
HandshakeTimeout(TAsyncSSLSocket* sslSocket, TEventBase* eventBase)
: TAsyncTimeout(eventBase)
, sslSocket_(sslSocket) {}
virtual void timeoutExpired() THRIFT_NOEXCEPT {
sslSocket_->timeoutExpired();
}
private:
TAsyncSSLSocket* sslSocket_;
};
/**
* Create a client TAsyncSSLSocket
*/
TAsyncSSLSocket(const boost::shared_ptr<transport::SSLContext> &ctx,
TEventBase* evb) :
TAsyncSocket(evb),
corked_(false),
server_(false),
sslState_(STATE_UNINIT),
ctx_(ctx),
handshakeCallback_(NULL),
ssl_(NULL),
sslSession_(NULL),
handshakeTimeout_(this, evb) {
}
/**
* Create a TAsyncSSLSocket from an already connected socket file descriptor.
*
* Note that while TAsyncSSLSocket enables TCP_NODELAY for sockets it creates
* when connecting, it does not change the socket options when given an
* existing file descriptor. If callers want TCP_NODELAY enabled when using
* this version of the constructor, they need to explicitly call
* setNoDelay(true) after the constructor returns.
*
* @param ctx SSL context for this connection.
* @param evb EventBase that will manage this socket.
* @param fd File descriptor to take over (should be a connected socket).
*/
TAsyncSSLSocket(const boost::shared_ptr<transport::
SSLContext>& ctx,
TEventBase* evb, int fd, bool server = true);
/**
* Helper function to create a shared_ptr<TAsyncSSLSocket>.
*/
static boost::shared_ptr<TAsyncSSLSocket> newSocket(
const boost::shared_ptr<transport::SSLContext>& ctx,
TEventBase* evb, int fd, bool server=true) {
return boost::shared_ptr<TAsyncSSLSocket>(
new TAsyncSSLSocket(ctx, evb, fd, server),
Destructor());
}
/**
* Helper function to create a shared_ptr<TAsyncSSLSocket>.
*/
static boost::shared_ptr<TAsyncSSLSocket> newSocket(
const boost::shared_ptr<transport::SSLContext>& ctx,
TEventBase* evb) {
return boost::shared_ptr<TAsyncSSLSocket>(
new TAsyncSSLSocket(ctx, evb),
Destructor());
}
/**
* TODO: implement support for SSL renegosiation.
*
* This involves proper handling of the SSL_ERROR_WANT_READ/WRITE
* code as a result of SSL_write/read(), instead of returning an
* error. In that case, the READ/WRITE event should be registered,
* and a flag (e.g., writeBlockedOnRead) should be set to indiciate
* the condition. In the next invocation of read/write callback, if
* the flag is on, performWrite()/performRead() should be called in
* addition to the normal call to performRead()/performWrite(), and
* the flag should be reset.
*/
// Inherit TAsyncTransport methods from TAsyncSocket except the
// following.
// See the documentation in TAsyncTransport.h
// TODO: implement graceful shutdown in close()
// TODO: implement detachSSL() that returns the SSL connection
virtual void closeNow();
virtual void shutdownWrite();
virtual void shutdownWriteNow();
virtual bool good() const;
virtual bool connecting() const;
/**
* Accept an SSL connection on the socket.
*
* The callback will be invoked and uninstalled when an SSL
* connection has been established on the underlying socket.
*
* @param callback callback object to invoke on success/failure
* @param timeout timeout for this function in milliseconds, or 0 for no
* timeout
*/
void sslAccept(HandshakeCallback* callback, uint32_t timeout = 0);
/**
* Invoke SSL accept following an asynchronous session cache lookup
*/
void restartSSLAccept();
/**
* Connect to the given address, invoking callback when complete or on error
*
* Note timeout applies to TCP + SSL connection time
*/
void connect(ConnectCallback* callback,
const transport::TSocketAddress& address,
int timeout = 0,
const OptionList &options = emptyOptionList) THRIFT_NOEXCEPT;
using TAsyncSocket::connect;
/**
* Initiate an SSL connection on the socket
* THe callback will be invoked and uninstalled when an SSL connection
* has been establshed on the underlying socket.
*
* @param callback callback object to invoke on success/failure
* @param timeout timeout for this function in milliseconds, or 0 for no
* timeout
*/
void sslConnect(HandshakeCallback *callback, uint64_t timeout = 0);
enum SSLStateEnum {
STATE_UNINIT,
STATE_ACCEPTING,
STATE_CACHE_LOOKUP,
STATE_CONNECTING,
STATE_ESTABLISHED,
STATE_REMOTE_CLOSED, /// remote end closed; we can still write
STATE_CLOSING, ///< close() called, but waiting on writes to complete
/// close() called with pending writes, before connect() has completed
STATE_CONNECTING_CLOSING,
STATE_CLOSED,
STATE_ERROR
};
SSLStateEnum getSSLState() const { return sslState_;}
/**
* Get a handle to the negotiated SSL session. This increments the session
* refcount and must be deallocated by the caller.
*/
SSL_SESSION *getSSLSession();
/**
* Set the SSL session to be used during sslConnect. TAsyncSSLSocket will
* hold a reference to the session until it is destroyed or released by the
* underlying SSL structure.
*
* @param takeOwnership if true, TAsyncSSLSocket will assume the caller's
* reference count to session.
*/
void setSSLSession(SSL_SESSION *session, bool takeOwnership = false);
#ifdef OPENSSL_NPN_NEGOTIATED
/**
* Get the name of the protocol selected by the client during
* Next Protocol Negotiation (NPN)
*
* @param protoName Name of the protocol (not guaranteed to be
* null terminated); will be set to NULL if
* the client did not negotiate a protocol.
* Note: the TAsyncSSLSocket retains ownership
* of this string.
* @param protoNameLen Length of the name.
*/
void getSelectedNextProtocol(const unsigned char** protoName,
unsigned* protoLen);
#endif // OPENSSL_NPN_NEGOTIATED
/**
* Determine if the session specified during setSSLSession was reused
* or if the server rejected it and issued a new session.
*/
bool getSSLSessionReused() const;
virtual void attachEventBase(TEventBase* eventBase) {
TAsyncSocket::attachEventBase(eventBase);
handshakeTimeout_.attachEventBase(eventBase);
}
virtual void detachEventBase() {
TAsyncSocket::detachEventBase();
handshakeTimeout_.detachEventBase();
}
void timeoutExpired() THRIFT_NOEXCEPT;
protected:
/**
* Protected destructor.
*
* Users of TAsyncSSLSocket must never delete it directly. Instead, invoke
* destroy() instead. (See the documentation in TDelayedDestruction.h for
* more details.)
*/
~TAsyncSSLSocket();
// Inherit event notification methods from TAsyncSocket except
// the following.
void handleRead() THRIFT_NOEXCEPT;
void handleWrite() THRIFT_NOEXCEPT;
void handleAccept() THRIFT_NOEXCEPT;
void handleConnect() THRIFT_NOEXCEPT;
void invalidState(HandshakeCallback* callback);
bool willBlock(int ret, int *errorOut) THRIFT_NOEXCEPT;
// TAsyncSocket calls this at the wrong time for SSL
void handleInitialReadWrite() THRIFT_NOEXCEPT {}
ssize_t performRead(void* buf, size_t buflen);
ssize_t performWrite(const iovec* vec, uint32_t count, bool haveMore,
uint32_t* countWritten, uint32_t* partialWritten);
// Inherit error handling methods from TAsyncSocket, plus the following.
void failHandshake(const char* fn, const transport::TTransportException& ex);
void invokeHandshakeCallback();
// Whether we've applied the TCP_CORK option to the socket
bool corked_;
// SSL related members.
bool server_;
SSLStateEnum sslState_;
boost::shared_ptr<transport::SSLContext> ctx_;
// Callback for SSL_accept() or SSL_connect()
HandshakeCallback* handshakeCallback_;
SSL* ssl_;
SSL_SESSION *sslSession_;
HandshakeTimeout handshakeTimeout_;
};
}}} // apache::thrift::async
#endif // #ifndef THRIFT_ASYNC_TASYNCSSLSOCKET_H_

View File

@ -0,0 +1,513 @@
/*
* 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 THRIFT_ASYNC_TASYNCSERVERSOCKET_H_
#define THRIFT_ASYNC_TASYNCSERVERSOCKET_H_ 1
#include "thrift/lib/cpp/thrift_config.h"
#include "thrift/lib/cpp/async/TDelayedDestruction.h"
#include "thrift/lib/cpp/async/TEventHandler.h"
#include <boost/shared_ptr.hpp>
#include <exception>
#include <vector>
#include <limits.h>
#include <stddef.h>
#include <memory>
#include <sys/socket.h>
namespace apache { namespace thrift {
namespace transport {
class TSocketAddress;
}
namespace async {
class TNotificationPipe;
/**
* A listening socket that asynchronously informs a callback whenever a new
* connection has been accepted.
*
* Unlike most async interfaces that always invoke their callback in the same
* TEventBase thread, TAsyncServerSocket is unusual in that it can distribute
* the callbacks across multiple TEventBase threads.
*
* This supports a common use case for network servers to distribute incoming
* connections across a number of TEventBase threads. (Servers typically run
* with one TEventBase thread per CPU.)
*
* Despite being able to invoke callbacks in multiple TEventBase threads,
* TAsyncServerSocket still has one "primary" TEventBase. Operations that
* modify the TAsyncServerSocket state may only be performed from the primary
* TEventBase thread.
*/
class TAsyncServerSocket : public TDelayedDestruction,
private TEventHandler {
public:
#if THRIFT_HAVE_UNIQUE_PTR
typedef std::unique_ptr<TAsyncServerSocket, Destructor> UniquePtr;
#endif
class AcceptCallback {
public:
virtual ~AcceptCallback() {}
/**
* connectionAccepted() is called whenever a new client connection is
* received.
*
* The AcceptCallback will remain installed after connectionAccepted()
* returns.
*
* @param fd The newly accepted client socket. The AcceptCallback
* assumes ownership of this socket, and is responsible
* for closing it when done. The newly accepted file
* descriptor will have already been put into
* non-blocking mode.
* @param clientAddr A reference to a TSocketAddress struct containing the
* client's address. This struct is only guaranteed to
* remain valid until connectionAccepted() returns.
*/
virtual void connectionAccepted(int fd,
const transport::TSocketAddress& clientAddr)
THRIFT_NOEXCEPT = 0;
/**
* acceptError() is called if an error occurs while accepting.
*
* The AcceptCallback will remain installed even after an accept error,
* as the errors are typically somewhat transient, such as being out of
* file descriptors. The server socket must be explicitly stopped if you
* wish to stop accepting after an error.
*
* @param ex An exception representing the error.
*/
virtual void acceptError(const std::exception& ex) THRIFT_NOEXCEPT = 0;
/**
* acceptStarted() will be called in the callback's TEventBase thread
* after this callback has been added to the TAsyncServerSocket.
*
* acceptStarted() will be called before any calls to connectionAccepted()
* or acceptError() are made on this callback.
*
* acceptStarted() makes it easier for callbacks to perform initialization
* inside the callback thread. (The call to addAcceptCallback() must
* always be made from the TAsyncServerSocket's primary TEventBase thread.
* acceptStarted() provides a hook that will always be invoked in the
* callback's thread.)
*
* Note that the call to acceptStarted() is made once the callback is
* added, regardless of whether or not the TAsyncServerSocket is actually
* accepting at the moment. acceptStarted() will be called even if the
* TAsyncServerSocket is paused when the callback is added (including if
* the initial call to startAccepting() on the TAsyncServerSocket has not
* been made yet).
*/
virtual void acceptStarted() THRIFT_NOEXCEPT {}
/**
* acceptStopped() will be called when this AcceptCallback is removed from
* the TAsyncServerSocket, or when the TAsyncServerSocket is destroyed,
* whichever occurs first.
*
* No more calls to connectionAccepted() or acceptError() will be made
* after acceptStopped() is invoked.
*/
virtual void acceptStopped() THRIFT_NOEXCEPT {}
};
static const uint32_t kDefaultMaxAcceptAtOnce = 30;
static const uint32_t kDefaultCallbackAcceptAtOnce = 5;
static const uint32_t kDefaultMaxMessagesInPipe = 0xffffffff;
/**
* Create a new TAsyncServerSocket with the specified TEventBase.
*
* @param eventBase The TEventBase to use for driving the asynchronous I/O.
* If this parameter is NULL, attachEventBase() must be
* called before this socket can begin accepting
* connections.
*/
explicit TAsyncServerSocket(TEventBase* eventBase = NULL);
/**
* Helper function to create a shared_ptr<TAsyncServerSocket>.
*
* This passes in the correct destructor object, since TAsyncServerSocket's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<TAsyncServerSocket> newSocket(TEventBase* evb) {
return boost::shared_ptr<TAsyncServerSocket>(new TAsyncServerSocket(evb),
Destructor());
}
/**
* Destroy the socket.
*
* TAsyncServerSocket::destroy() must be called to destroy the socket.
* The normal destructor is private, and should not be invoked directly.
* This prevents callers from deleting a TAsyncServerSocket while it is
* invoking a callback.
*
* destroy() must be invoked from the socket's primary TEventBase thread.
*
* If there are AcceptCallbacks still installed when destroy() is called,
* acceptStopped() will be called on these callbacks to notify them that
* accepting has stopped. Accept callbacks being driven by other TEventBase
* threads may continue to receive new accept callbacks for a brief period of
* time after destroy() returns. They will not receive any more callback
* invocations once acceptStopped() is invoked.
*/
virtual void destroy();
/**
* Attach this TAsyncServerSocket to its primary TEventBase.
*
* This may only be called if the TAsyncServerSocket is not already attached
* to a TEventBase. The TAsyncServerSocket must be attached to a TEventBase
* before it can begin accepting connections.
*/
void attachEventBase(TEventBase *eventBase);
/**
* Detach the TAsyncServerSocket from its primary TEventBase.
*
* detachEventBase() may only be called if the TAsyncServerSocket is not
* currently accepting connections.
*/
void detachEventBase();
/**
* Get the TEventBase used by this socket.
*/
TEventBase* getEventBase() const {
return eventBase_;
}
/**
* Create a TAsyncServerSocket from an existing socket file descriptor.
*
* useExistingSocket() will cause the TAsyncServerSocket to take ownership of
* the specified file descriptor, and use it to listen for new connections.
* The TAsyncServerSocket will close the file descriptor when it is
* destroyed.
*
* useExistingSocket() must be called before bind() or listen().
*
* The supplied file descriptor will automatically be put into non-blocking
* mode. The caller may have already directly called bind() and possibly
* listen on the file descriptor. If so the caller should skip calling the
* corresponding TAsyncServerSocket::bind() and listen() methods.
*
* On error a TTransportException will be thrown and the caller will retain
* ownership of the file descriptor.
*/
void useExistingSocket(int fd);
/**
* Return the underlying file descriptor
*/
int getSocket() const {
return socket_;
}
/**
* Bind to the specified address.
*
* This must be called from the primary TEventBase thread.
*
* Throws TTransportException on error.
*/
void bind(const transport::TSocketAddress& address);
/**
* Bind to the specified port.
*
* This must be called from the primary TEventBase thread.
*
* Throws TTransportException on error.
*/
void bind(uint16_t port);
/**
* Get the local address to which the socket is bound.
*
* Throws TTransportException on error.
*/
void getAddress(transport::TSocketAddress* addressReturn);
/**
* Begin listening for connections.
*
* This calls ::listen() with the specified backlog.
*
* Once listen() is invoked the socket will actually be open so that remote
* clients may establish connections. (Clients that attempt to connect
* before listen() is called will receive a connection refused error.)
*
* At least one callback must be set and startAccepting() must be called to
* actually begin notifying the accept callbacks of newly accepted
* connections. The backlog parameter controls how many connections the
* kernel will accept and buffer internally while the accept callbacks are
* paused (or if accepting is enabled but the callbacks cannot keep up).
*
* bind() must be called before calling listen().
* listen() must be called from the primary TEventBase thread.
*
* Throws TTransportException on error.
*/
void listen(int backlog);
/**
* Add an AcceptCallback.
*
* When a new socket is accepted, one of the AcceptCallbacks will be invoked
* with the new socket. The AcceptCallbacks are invoked in a round-robin
* fashion. This allows the accepted sockets to distributed among a pool of
* threads, each running its own TEventBase object. This is a common model,
* since most asynchronous-style servers typically run one TEventBase thread
* per CPU.
*
* The TEventBase object associated with each AcceptCallback must be running
* its loop. If the TEventBase loop is not running, sockets will still be
* scheduled for the callback, but the callback cannot actually get invoked
* until the loop runs.
*
* This method must be invoked from the TAsyncServerSocket's primary
* TEventBase thread.
*
* Note that startAccepting() must be called on the TAsyncServerSocket to
* cause it to actually start accepting sockets once callbacks have been
* installed.
*
* @param callback The callback to invoke.
* @param eventBase The TEventBase to use to invoke the callback. This
* parameter may be NULL, in which case the callback will be invoked in
* the TAsyncServerSocket's primary TEventBase.
* @param maxAtOnce The maximum number of connections to accept in this
* callback on a single iteration of the event base loop.
* This only takes effect when eventBase is non-NULL. When
* using a NULL eventBase for the callback, the
* setMaxAcceptAtOnce() method controls how many
* connections the main event base will accept at once.
*/
void addAcceptCallback(AcceptCallback *callback, TEventBase *eventBase,
uint32_t maxAtOnce = kDefaultCallbackAcceptAtOnce);
/**
* Remove an AcceptCallback.
*
* This allows a single AcceptCallback to be removed from the round-robin
* pool.
*
* This method must be invoked from the TAsyncServerSocket's primary
* TEventBase thread. Use TEventBase::runInEventBaseThread() to schedule the
* operation in the correct TEventBase if your code is not in the server
* socket's primary TEventBase.
*
* Given that the accept callback is being driven by a different TEventBase,
* the AcceptCallback may continue to be invoked for a short period of time
* after removeAcceptCallback() returns in this thread. Once the other
* TEventBase thread receives the notification to stop, it will call
* acceptStopped() on the callback to inform it that it is fully stopped and
* will not receive any new sockets.
*
* If the last accept callback is removed while the socket is accepting,
* the socket will implicitly pause accepting. If a callback is later added,
* it will resume accepting immediately, without requiring startAccepting()
* to be invoked.
*
* @param callback The callback to uninstall.
* @param eventBase The TEventBase associated with this callback. This must
* be the same TEventBase that was used when the callback was installed
* with addAcceptCallback().
*/
void removeAcceptCallback(AcceptCallback *callback, TEventBase *eventBase);
/**
* Begin accepting connctions on this socket.
*
* bind() and listen() must be called before calling startAccepting().
*
* When a TAsyncServerSocket is initially created, it will not begin
* accepting connections until at least one callback has been added and
* startAccepting() has been called. startAccepting() can also be used to
* resume accepting connections after a call to pauseAccepting().
*
* If startAccepting() is called when there are no accept callbacks
* installed, the socket will not actually begin accepting until an accept
* callback is added.
*
* This method may only be called from the primary TEventBase thread.
*/
void startAccepting();
/**
* Pause accepting connections.
*
* startAccepting() may be called to resume accepting.
*
* This method may only be called from the primary TEventBase thread.
* If there are AcceptCallbacks being driven by other TEventBase threads they
* may continue to receive callbacks for a short period of time after
* pauseAccepting() returns.
*
* Unlike removeAcceptCallback() or destroy(), acceptStopped() will not be
* called on the AcceptCallback objects simply due to a temporary pause. If
* the server socket is later destroyed while paused, acceptStopped() will be
* called all of the installed AcceptCallbacks.
*/
void pauseAccepting();
/**
* Get the maximum number of connections that will be accepted each time
* around the event loop.
*/
uint32_t getMaxAcceptAtOnce() const {
return maxAcceptAtOnce_;
}
/**
* Set the maximum number of connections that will be accepted each time
* around the event loop.
*
* This provides a very coarse-grained way of controlling how fast the
* TAsyncServerSocket will accept connections. If you find that when your
* server is overloaded TAsyncServerSocket accepts connections more quickly
* than your code can process them, you can try lowering this number so that
* fewer connections will be accepted each event loop iteration.
*
* For more explicit control over the accept rate, you can also use
* pauseAccepting() to temporarily pause accepting when your server is
* overloaded, and then use startAccepting() later to resume accepting.
*/
void setMaxAcceptAtOnce(uint32_t numConns) {
maxAcceptAtOnce_ = numConns;
}
/**
* Get the maximum number of unprocessed messages which a NotificationPipe
* can hold.
*/
uint32_t getMaxNumMessagesInPipe() const {
return maxNumMsgsInPipe_;
}
/**
* Set the maximum number of unprocessed messages in NotificationPipe.
* No new message will be sent to that NotificationPipe if there are more
* than such number of unprocessed messages in that pipe.
*/
void setMaxNumMessagesInPipe(uint32_t num) {
maxNumMsgsInPipe_ = num;
}
/**
* Get the speed of adjusting connection accept rate.
*/
double getAcceptRateAdjustSpeed() const {
return acceptRateAdjustSpeed_;
}
/**
* Set the speed of adjusting connection accept rate.
*/
void setAcceptRateAdjustSpeed(double speed) {
acceptRateAdjustSpeed_ = speed;
}
/**
* Get the number of connections dropped by the TAsyncServerSocket
*/
double getNumDroppedConnections() const {
return numDroppedConnections_;
}
protected:
/**
* Protected destructor.
*
* Invoke destroy() instead to destroy the TAsyncServerSocket.
*/
virtual ~TAsyncServerSocket();
private:
/**
* A struct to keep track of the callbacks associated with this server
* socket.
*/
struct CallbackInfo {
CallbackInfo(AcceptCallback *callback, TEventBase *eventBase)
: callback(callback),
eventBase(eventBase),
pipe(NULL) {}
AcceptCallback *callback;
TEventBase *eventBase;
// Note that the TNotificationPipe is actually owned by the RemoteAcceptor.
// The RemoteAcceptor will destroy the TNotificationPipe (and itself)
// once the pipe is closed by the TAsyncServerSocket.
TNotificationPipe *pipe;
};
class RemoteAcceptor;
enum MessageType {
MSG_NEW_CONN = 0,
MSG_ERROR = 1
};
class BackoffTimeout;
// Inherited from TEventHandler
virtual void handlerReady(uint16_t events) THRIFT_NOEXCEPT;
int createSocket(int family);
void setupSocket(int fd);
void dispatchSocket(int socket, const transport::TSocketAddress& address);
void dispatchError(const char *msg, int errnoValue);
void enterBackoff();
void backoffTimeoutExpired();
CallbackInfo* nextCallback() {
CallbackInfo* info = &callbacks_[callbackIndex_];
++callbackIndex_;
if (callbackIndex_ >= callbacks_.size()) {
callbackIndex_ = 0;
}
return info;
}
TEventBase *eventBase_;
sa_family_t addressFamily_;
int socket_;
bool accepting_;
uint32_t maxAcceptAtOnce_;
uint32_t maxNumMsgsInPipe_;
double acceptRateAdjustSpeed_; //0 to disable auto adjust
double acceptRate_;
int64_t lastAccepTimestamp_; // milliseconds
int64_t numDroppedConnections_;
uint32_t callbackIndex_;
BackoffTimeout *backoffTimeout_;
std::vector<CallbackInfo> callbacks_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TASYNCSERVERSOCKET_H_

View File

@ -0,0 +1,100 @@
/*
* 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 THRIFT_ASYNC_TASYNCSIGNALHANDLER_H_
#define THRIFT_ASYNC_TASYNCSIGNALHANDLER_H_ 1
#include "thrift/lib/cpp/thrift_config.h"
#include <event.h>
#include <map>
namespace apache { namespace thrift { namespace async {
class TEventBase;
/**
* A handler to receive notification about POSIX signals.
*
* TAsyncSignalHandler allows code to process signals from within a TEventBase
* loop. Standard signal handlers interrupt execution of the main thread, and
* are run while the main thread is paused. As a result, great care must be
* taken to avoid race conditions if the signal handler has to access or modify
* any data used by the main thread.
*
* TAsyncSignalHandler solves this problem by running the TAsyncSignalHandler
* callback in normal thread of execution, as a TEventBase callback.
*
* TAsyncSignalHandler may only be used in a single thread. It will only
* process signals received by the thread where the TAsyncSignalHandler is
* registered. It is the user's responsibility to ensure that signals are
* delivered to the desired thread in multi-threaded programs.
*/
class TAsyncSignalHandler {
public:
/**
* Create a new TAsyncSignalHandler.
*/
TAsyncSignalHandler(TEventBase* eventBase);
virtual ~TAsyncSignalHandler();
/**
* Register to receive callbacks about the specified signal.
*
* Once the handler has been registered for a particular signal,
* signalReceived() will be called each time this thread receives this
* signal.
*
* Throws a TException if an error occurs, or if this handler is already
* registered for this signal.
*/
void registerSignalHandler(int signum);
/**
* Unregister for callbacks about the specified signal.
*
* Throws a TException if an error occurs, or if this signal was not
* registered.
*/
void unregisterSignalHandler(int signum);
/**
* signalReceived() will called to indicate that the specified signal has
* been received.
*
* signalReceived() will always be invoked from the TEventBase loop (i.e.,
* after the main POSIX signal handler has returned control to the TEventBase
* thread).
*/
virtual void signalReceived(int signum) THRIFT_NOEXCEPT = 0;
private:
typedef std::map<int, struct event> SignalEventMap;
// Forbidden copy constructor and assignment operator
TAsyncSignalHandler(TAsyncSignalHandler const &);
TAsyncSignalHandler& operator=(TAsyncSignalHandler const &);
static void libeventCallback(int signum, short events, void* arg);
TEventBase* eventBase_;
SignalEventMap signalEvents_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TASYNCSIGNALHANDLER_H_

View File

@ -0,0 +1,551 @@
/*
* 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 THRIFT_ASYNC_TASYNCSOCKET_H_
#define THRIFT_ASYNC_TASYNCSOCKET_H_ 1
#include <sys/types.h>
#include <sys/socket.h>
#include "thrift/lib/cpp/async/TAsyncTimeout.h"
#include "thrift/lib/cpp/async/TAsyncTransport.h"
#include "thrift/lib/cpp/async/TDelayedDestruction.h"
#include "thrift/lib/cpp/async/TEventHandler.h"
#include <list>
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift { namespace async {
/**
* A class for performing asynchronous I/O on a socket.
*
* TAsyncSocket allows users to asynchronously wait for data on a socket, and
* to asynchronously send data.
*
* The APIs for reading and writing are intentionally asymmetric. Waiting for
* data to read is a persistent API: a callback is installed, and is notified
* whenever new data is available. It continues to be notified of new events
* until it is uninstalled.
*
* TAsyncSocket does not provide read timeout functionality, because it
* typically cannot determine when the timeout should be active. Generally, a
* timeout should only be enabled when processing is blocked waiting on data
* from the remote endpoint. For server sockets, the timeout should not be
* active if the server is currently processing one or more outstanding
* requests for this socket. For client sockets, the timeout should not be
* active if there are no requests pending on the socket. Additionally, if a
* client has multiple pending requests, it will ususally want a separate
* timeout for each request, rather than a single read timeout.
*
* The write API is fairly intuitive: a user can request to send a block of
* data, and a callback will be informed once the entire block has been
* transferred to the kernel, or on error. TAsyncSocket does provide a send
* timeout, since most callers want to give up if the remote end stops
* responding and no further progress can be made sending the data.
*/
class TAsyncSocket : public TAsyncTransport,
public TDelayedDestruction {
public:
#if THRIFT_HAVE_UNIQUE_PTR
typedef std::unique_ptr<TAsyncSocket, Destructor> UniquePtr;
#endif
class ConnectCallback {
public:
virtual ~ConnectCallback() {}
/**
* connectSuccess() will be invoked when the connection has been
* successfully established.
*/
virtual void connectSuccess() THRIFT_NOEXCEPT = 0;
/**
* connectError() will be invoked if the connection attempt fails.
*
* @param ex An exception describing the error that occurred.
*/
virtual void connectError(const transport::TTransportException& ex)
THRIFT_NOEXCEPT = 0;
};
/**
* Create a new unconnected TAsyncSocket.
*
* connect() must later be called on this socket to establish a connection.
*/
explicit TAsyncSocket(TEventBase* evb);
/**
* Create a new TAsyncSocket and begin the connection process.
*
* @param evb EventBase that will manage this socket.
* @param address The address to connect to.
* @param connectTimeout Optional timeout in milliseconds for the connection
* attempt.
*/
TAsyncSocket(TEventBase* evb,
const transport::TSocketAddress& address,
uint32_t connectTimeout = 0);
/**
* Create a new TAsyncSocket and begin the connection process.
*
* @param evb EventBase that will manage this socket.
* @param ip IP address to connect to (dotted-quad).
* @param port Destination port in host byte order.
* @param connectTimeout Optional timeout in milliseconds for the connection
* attempt.
*/
TAsyncSocket(TEventBase* evb,
const std::string& ip,
uint16_t port,
uint32_t connectTimeout = 0);
/**
* Create a TAsyncSocket from an already connected socket file descriptor.
*
* Note that while TAsyncSocket enables TCP_NODELAY for sockets it creates
* when connecting, it does not change the socket options when given an
* existing file descriptor. If callers want TCP_NODELAY enabled when using
* this version of the constructor, they need to explicitly call
* setNoDelay(true) after the constructor returns.
*
* @param evb EventBase that will manage this socket.
* @param fd File descriptor to take over (should be a connected socket).
*/
TAsyncSocket(TEventBase* evb, int fd);
/**
* Helper function to create a shared_ptr<TAsyncSocket>.
*
* This passes in the correct destructor object, since TAsyncSocket's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<TAsyncSocket> newSocket(TEventBase* evb) {
return boost::shared_ptr<TAsyncSocket>(new TAsyncSocket(evb),
Destructor());
}
/**
* Helper function to create a shared_ptr<TAsyncSocket>.
*/
static boost::shared_ptr<TAsyncSocket> newSocket(
TEventBase* evb,
const transport::TSocketAddress& address,
uint32_t connectTimeout = 0) {
return boost::shared_ptr<TAsyncSocket>(
new TAsyncSocket(evb, address, connectTimeout),
Destructor());
}
/**
* Helper function to create a shared_ptr<TAsyncSocket>.
*/
static boost::shared_ptr<TAsyncSocket> newSocket(
TEventBase* evb,
const std::string& ip,
uint16_t port,
uint32_t connectTimeout = 0) {
return boost::shared_ptr<TAsyncSocket>(
new TAsyncSocket(evb, ip, port, connectTimeout),
Destructor());
}
/**
* Helper function to create a shared_ptr<TAsyncSocket>.
*/
static boost::shared_ptr<TAsyncSocket> newSocket(TEventBase* evb, int fd) {
return boost::shared_ptr<TAsyncSocket>(new TAsyncSocket(evb, fd),
Destructor());
}
/**
* Destroy the socket.
*
* TAsyncSocket::destroy() must be called to destroy the socket.
* The normal destructor is private, and should not be invoked directly.
* This prevents callers from deleting a TAsyncSocket while it is invoking a
* callback.
*/
virtual void destroy();
/**
* Get the TEventBase used by this socket.
*/
virtual TEventBase* getEventBase() const {
return eventBase_;
}
/**
* Get the file descriptor used by the TAsyncSocket.
*/
int getFd() const {
return fd_;
}
/**
* Extract the file descriptor from the TAsyncSocket.
*
* This will immediately cause any installed callbacks to be invoked with an
* error. The TAsyncSocket may no longer be used after the file descriptor
* has been extracted.
*
* Returns the file descriptor. The caller assumes ownership of the
* descriptor, and it will not be closed when the TAsyncSocket is destroyed.
*/
int detachFd();
/**
* Class that consists of the input parameters for setsockopt().
*
* The memory referenced by optval should be valid throughout the
* life cycle of the SocketOption object.
*/
class SocketOption {
public:
SocketOption(): level_(0), optname_(0), optval_(NULL), size_(0) {}
template <class T>
SocketOption(int level, int optname, const T* optval):
level_(level), optname_(optname), optval_(optval), size_(sizeof(T)) {}
int apply(int fd) const {
return setsockopt(fd, level_, optname_, optval_, size_);
}
protected:
int level_;
int optname_;
const void *optval_;
size_t size_;
};
typedef std::list<SocketOption> OptionList;
static OptionList emptyOptionList;
/**
* Initiate a connection.
*
* @param callback The callback to inform when the connection attempt
* completes.
* @param address The address to connect to.
* @param timeout A timeout value, in milliseconds. If the connection
* does not succeed within this period,
* callback->connectError() will be invoked.
*/
virtual void connect(ConnectCallback* callback,
const transport::TSocketAddress& address,
int timeout = 0,
const OptionList &options = emptyOptionList) THRIFT_NOEXCEPT;
void connect(ConnectCallback* callback, const std::string& ip, uint16_t port,
int timeout = 00,
const OptionList &options = emptyOptionList) THRIFT_NOEXCEPT;
/**
* Set the send timeout.
*
* If write requests do not make any progress for more than the specified
* number of milliseconds, fail all pending writes and close the socket.
*
* If write requests are currently pending when setSendTimeout() is called,
* the timeout interval is immediately restarted using the new value.
*
* (See the comments for TAsyncSocket for an explanation of why TAsyncSocket
* provides setSendTimeout() but not setRecvTimeout().)
*
* @param milliseconds The timeout duration, in milliseconds. If 0, no
* timeout will be used.
*/
void setSendTimeout(uint32_t milliseconds);
/**
* Get the send timeout.
*
* @return Returns the current send timeout, in milliseconds. A return value
* of 0 indicates that no timeout is set.
*/
uint32_t getSendTimeout() const {
return sendTimeout_;
}
// Methods inherited from TAsyncTransport
// See the documentation in TAsyncTransport.h
virtual void setReadCallback(ReadCallback* callback);
virtual ReadCallback* getReadCallback() const;
virtual void write(WriteCallback* callback, const void* buf, size_t bytes);
virtual void writev(WriteCallback* callback, const iovec* vec, size_t count);
virtual void writeChain(WriteCallback* callback,
std::unique_ptr<folly::IOBuf>&& buf,
bool cork = false);
virtual void close();
virtual void closeNow();
virtual void shutdownWrite();
virtual void shutdownWriteNow();
virtual bool readable() const;
virtual bool good() const;
virtual bool error() const;
virtual void attachEventBase(TEventBase* eventBase);
virtual void detachEventBase();
virtual void getLocalAddress(transport::TSocketAddress* address) const;
virtual void getPeerAddress(transport::TSocketAddress* address) const;
virtual bool connecting() const {
return (state_ == STATE_CONNECTING);
}
// Methods controlling socket options
/**
* Force writes to be transmitted immediately.
*
* This controls the TCP_NODELAY socket option. When enabled, TCP segments
* are sent as soon as possible, even if it is not a full frame of data.
* When disabled, the data may be buffered briefly to try and wait for a full
* frame of data.
*
* By default, TCP_NODELAY is enabled for TAsyncSocket objects.
*
* This method will fail if the socket is not currently open.
*
* @return Returns 0 if the TCP_NODELAY flag was successfully updated,
* or a non-zero errno value on error.
*/
int setNoDelay(bool noDelay);
/**
* Generic API for reading a socket option.
*
* @param level same as the "level" parameter in getsockopt().
* @param optname same as the "optname" parameter in getsockopt().
* @param optval pointer to the variable in which the option value should
* be returned.
* @return same as the return value of getsockopt().
*/
template <typename T>
int getSockOpt(int level, int optname, T *optval) {
return getsockopt(fd_, level, optname, optval, sizeof(T));
}
/**
* Generic API for setting a socket option.
*
* @param level same as the "level" parameter in getsockopt().
* @param optname same as the "optname" parameter in getsockopt().
* @param optval the option value to set.
* @return same as the return value of setsockopt().
*/
template <typename T>
int setSockOpt(int level, int optname, const T *optval) {
return setsockopt(fd_, level, optname, optval, sizeof(T));
}
protected:
enum ReadResultEnum {
READ_EOF = 0,
READ_ERROR = -1,
READ_BLOCKING = -2,
};
/**
* Protected destructor.
*
* Users of TAsyncSocket must never delete it directly. Instead, invoke
* destroy() instead. (See the documentation in TDelayedDestruction.h for
* more details.)
*/
~TAsyncSocket();
enum StateEnum {
STATE_UNINIT,
STATE_CONNECTING,
STATE_ESTABLISHED,
STATE_CLOSED,
STATE_ERROR
};
enum ShutdownFlags {
/// shutdownWrite() called, but we are still waiting on writes to drain
SHUT_WRITE_PENDING = 0x01,
/// writes have been completely shut down
SHUT_WRITE = 0x02,
/**
* Reads have been shutdown.
*
* At the moment we don't distinguish between remote read shutdown
* (received EOF from the remote end) and local read shutdown. We can
* only receive EOF when a read callback is set, and we immediately inform
* it of the EOF. Therefore there doesn't seem to be any reason to have a
* separate state of "received EOF but the local side may still want to
* read".
*
* We also don't currently provide any API for only shutting down the read
* side of a socket. (This is a no-op as far as TCP is concerned, anyway.)
*/
SHUT_READ = 0x04,
};
class WriteRequest;
class WriteTimeout : public TAsyncTimeout {
public:
WriteTimeout(TAsyncSocket* socket, TEventBase* eventBase)
: TAsyncTimeout(eventBase)
, socket_(socket) {}
virtual void timeoutExpired() THRIFT_NOEXCEPT {
socket_->timeoutExpired();
}
private:
TAsyncSocket* socket_;
};
class IoHandler : public TEventHandler {
public:
IoHandler(TAsyncSocket* socket, TEventBase* eventBase)
: TEventHandler(eventBase, -1)
, socket_(socket) {}
IoHandler(TAsyncSocket* socket, TEventBase* eventBase, int fd)
: TEventHandler(eventBase, fd)
, socket_(socket) {}
virtual void handlerReady(uint16_t events) THRIFT_NOEXCEPT {
socket_->ioReady(events);
}
private:
TAsyncSocket* socket_;
};
void init();
// event notification methods
void ioReady(uint16_t events) THRIFT_NOEXCEPT;
virtual void handleInitialReadWrite() THRIFT_NOEXCEPT;
virtual void handleRead() THRIFT_NOEXCEPT;
virtual void handleWrite() THRIFT_NOEXCEPT;
virtual void handleConnect() THRIFT_NOEXCEPT;
void timeoutExpired() THRIFT_NOEXCEPT;
/**
* Attempt to read from the socket.
*
* @param buf The buffer to read data into.
* @param buflen The length of the buffer.
*
* @return Returns the number of bytes read, or READ_EOF on EOF, or
* READ_ERROR on error, or READ_BLOCKING if the operation will
* block.
*/
virtual ssize_t performRead(void* buf, size_t buflen);
/**
* Write as much data as possible to the socket without blocking,
* and queue up any leftover data to send when the socket can
* handle writes again.
*
* @param callback The callback to invoke when the write is completed.
* @param vec Array of buffers to write; this method will make a
* copy of the vector (but not the buffers themselves)
* if the write has to be completed asynchronously.
* @param count Number of elements in vec.
* @param buf The IOBuf that manages the buffers referenced by
* vec, or a pointer to NULL if the buffers are not
* associated with an IOBuf. Note that ownership of
* the IOBuf is transferred here; upon completion of
* the write, the TAsyncSocket deletes the IOBuf.
* @param cork Whether to delay the write until the next non-corked
* write operation. (Note: may not be supported in all
* subclasses or on all platforms.)
*/
void writeImpl(WriteCallback* callback, const iovec* vec, size_t count,
std::unique_ptr<folly::IOBuf>&& buf, bool cork = false);
/**
* Attempt to write to the socket.
*
* @param vec The iovec array pointing to the buffers to write.
* @param count The length of the iovec array.
* @param haveMore This flag is inherited from TAsyncSocket but is
* not handled here.
* @param countWritten On return, the value pointed to by this parameter
* will contain the number of iovec entries that were
* fully written.
* @param partialWritten On return, the value pointed to by this parameter
* will contain the number of bytes written in the
* partially written iovec entry.
*
* @return Returns the total number of bytes written, or -1 on error. If no
* data can be written immediately, 0 is returned.
*/
virtual ssize_t performWrite(const iovec* vec, uint32_t count,
bool haveMore, uint32_t* countWritten,
uint32_t* partialWritten);
bool updateEventRegistration();
/**
* Update event registration.
*
* @param enable Flags of events to enable. Set it to 0 if no events
* need to be enabled in this call.
* @param disable Flags of events
* to disable. Set it to 0 if no events need to be disabled in this
* call.
*
* @return true iff the update is successful.
*/
bool updateEventRegistration(uint16_t enable, uint16_t disable);
// error handling methods
void startFail();
void finishFail();
void fail(const char* fn, const transport::TTransportException& ex);
void failConnect(const char* fn, const transport::TTransportException& ex);
void failRead(const char* fn, const transport::TTransportException& ex);
void failWrite(const char* fn, WriteCallback* callback, size_t bytesWritten,
const transport::TTransportException& ex);
void failWrite(const char* fn, const transport::TTransportException& ex);
void failAllWrites(const transport::TTransportException& ex);
void invalidState(ConnectCallback* callback);
void invalidState(ReadCallback* callback);
void invalidState(WriteCallback* callback);
uint8_t state_; ///< StateEnum describing current state
uint8_t shutdownFlags_; ///< Shutdown state (ShutdownFlags)
uint16_t eventFlags_; ///< TEventBase::HandlerFlags settings
int fd_; ///< The socket file descriptor
uint32_t sendTimeout_; ///< The send timeout, in milliseconds
TEventBase* eventBase_; ///< The TEventBase
WriteTimeout writeTimeout_; ///< A timeout for connect and write
IoHandler ioHandler_; ///< A TEventHandler to monitor the fd
ConnectCallback* connectCallback_; ///< ConnectCallback
ReadCallback* readCallback_; ///< ReadCallback
WriteRequest* writeReqHead_; ///< Chain of WriteRequests
WriteRequest* writeReqTail_; ///< End of WriteRequest chain
};
}}} // apache::thrift::async
#endif // #ifndef THRIFT_ASYNC_TASYNCSOCKET_H_

View File

@ -0,0 +1,144 @@
/*
* 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 THRIFT_ASYNC_TASYNCTIMEOUT_H_
#define THRIFT_ASYNC_TASYNCTIMEOUT_H_ 1
#include "thrift/lib/cpp/thrift_config.h"
#include "thrift/lib/cpp/async/TEventUtil.h"
#include <boost/noncopyable.hpp>
namespace apache { namespace thrift { namespace async {
class TEventBase;
/**
* TAsyncTimeout is used to asynchronously wait for a timeout to occur.
*/
class TAsyncTimeout : private boost::noncopyable {
public:
enum InternalEnum {
INTERNAL,
NORMAL
};
/**
* Create a new TAsyncTimeout object, driven by the specified TEventBase.
*/
explicit TAsyncTimeout(TEventBase* eventBase);
/**
* Create a new internal TAsyncTimeout object.
*
* Internal timeouts are like regular timeouts, but will not stop the
* TEventBase loop from exiting if the only remaining events are internal
* timeouts.
*
* This is useful for implementing fallback timeouts to abort the TEventBase
* loop if the other events have not been processed within a specified time
* period: if the event loop takes too long the timeout will fire and can
* stop the event loop. However, if all other events complete, the event
* loop will exit even though the internal timeout is still installed.
*/
TAsyncTimeout(TEventBase* eventBase, InternalEnum internal);
/**
* Create a new TAsyncTimeout object, not yet assigned to a TEventBase.
*
* attachEventBase() must be called prior to scheduling the timeout.
*/
TAsyncTimeout();
/**
* TAsyncTimeout destructor.
*
* The timeout will be automatically cancelled if it is running.
*/
virtual ~TAsyncTimeout();
/**
* timeoutExpired() is invoked when the timeout period has expired.
*/
virtual void timeoutExpired() THRIFT_NOEXCEPT = 0;
/**
* Schedule the timeout to fire in the specified number of milliseconds.
*
* After the specified number of milliseconds has elapsed, timeoutExpired()
* will be invoked by the TEventBase's main loop.
*
* If the timeout is already running, it will be rescheduled with the
* new timeout value.
*
* @param milliseconds The timeout duration, in milliseconds.
*
* @return Returns true if the timeout was successfully scheduled,
* and false if an error occurred. After an error, the timeout is
* always unscheduled, even if scheduleTimeout() was just
* rescheduling an existing timeout.
*/
bool scheduleTimeout(uint32_t milliseconds);
/**
* Cancel the timeout, if it is running.
*/
void cancelTimeout();
/**
* Returns true if the timeout is currently scheduled.
*/
bool isScheduled() {
return TEventUtil::isEventRegistered(&event_);
}
/**
* Attach the timeout to a TEventBase.
*
* This may only be called if the timeout is not currently attached to a
* TEventBase (either by using the default constructor, or by calling
* detachEventBase()).
*
* This method must be invoked in the TEventBase's thread.
*
* The internal parameter specifies if this timeout should be treated as an
* internal event. TEventBase::loop() will return when there are no more
* non-internal events remaining.
*/
void attachEventBase(TEventBase* eventBase,
InternalEnum internal = NORMAL);
/**
* Detach the timeout from its TEventBase.
*
* This may only be called when the timeout is not running.
* Once detached, the timeout may not be scheduled again until it is
* re-attached to a TEventBase by calling attachEventBase().
*
* This method must be called from the current TEventBase's thread.
*/
void detachEventBase();
private:
static void libeventCallback(int fd, short events, void* arg);
struct event event_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TASYNCTIMEOUT_H_

View File

@ -0,0 +1,198 @@
/*
* 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 THRIFT_ASYNC_TASYNCTIMEOUTSET_H_
#define THRIFT_ASYNC_TASYNCTIMEOUTSET_H_ 1
#include "thrift/lib/cpp/async/TAsyncTimeout.h"
#include "thrift/lib/cpp/async/TDelayedDestruction.h"
#include <chrono>
#include <cstddef>
#include <memory>
namespace apache { namespace thrift { namespace async {
/**
* TAsyncTimeoutSet exists for efficiently managing a group of timeouts events
* that always have the same timeout interval.
*
* TAsyncTimeoutSet takes advantage of the fact that the timeouts are always
* scheduled in sorted order. (Since each timeout has the same interval, when
* a new timeout is scheduled it will always be the last timeout in the set.)
* This avoids the need to perform any additional sorting of the timeouts
* within a single TAsyncTimeoutSet.
*
* TAsyncTimeoutSet is useful whenever you have a large group of objects that
* each need their own timeout, but with the same interval for each object.
* For example, managing idle timeouts for thousands of connection, or
* scheduling health checks for a large group of servers.
*/
class TAsyncTimeoutSet : private TAsyncTimeout, public TDelayedDestruction {
public:
typedef std::unique_ptr<TAsyncTimeoutSet, Destructor> UniquePtr;
/**
* A callback to be notified when a timeout has expired.
*
* TAsyncTimeoutSet::Callback is very similar to TAsyncTimeout. The primary
* distinction is that TAsyncTimeout can choose its timeout interval each
* time it is scheduled. On the other hand, TAsyncTimeoutSet::Callback
* always uses the timeout interval defined by the TAsyncTimeoutSet where it
* is scheduled.
*/
class Callback {
public:
Callback()
: timeoutSet_(NULL),
expiration_(0),
prev_(NULL),
next_(NULL) {}
virtual ~Callback();
/**
* timeoutExpired() is invoked when the timeout has expired.
*/
virtual void timeoutExpired() THRIFT_NOEXCEPT = 0;
/**
* Cancel the timeout, if it is running.
*
* If the timeout is not scheduled, cancelTimeout() does nothing.
*/
void cancelTimeout() {
if (timeoutSet_ == NULL) {
// We're not scheduled, so there's nothing to do.
return;
}
cancelTimeoutImpl();
}
/**
* Return true if this timeout is currently scheduled, and false otherwise.
*/
bool isScheduled() const {
return timeoutSet_ != NULL;
}
private:
// Get the time remaining until this timeout expires
std::chrono::milliseconds getTimeRemaining(
std::chrono::milliseconds now) const {
if (now >= expiration_) {
return std::chrono::milliseconds(0);
}
return expiration_ - now;
}
void setScheduled(TAsyncTimeoutSet* timeoutSet, Callback* prev);
void cancelTimeoutImpl();
TAsyncTimeoutSet* timeoutSet_;
std::chrono::milliseconds expiration_;
Callback* prev_;
Callback* next_;
// Give TAsyncTimeoutSet direct access to our members so it can take care
// of scheduling/cancelling.
friend class TAsyncTimeoutSet;
};
/**
* Create a new TAsyncTimeoutSet with the specified interval.
*/
TAsyncTimeoutSet(TEventBase* eventBase,
std::chrono::milliseconds intervalMS,
std::chrono::milliseconds atMostEveryN =
std::chrono::milliseconds(0));
/**
* Destroy the TAsyncTimeoutSet.
*
* Normally a TAsyncTimeoutSet should only be destroyed when there are no
* more callbacks pending in the set. If there are timeout callbacks pending
* for this set, destroying the TAsyncTimeoutSet will automatically cancel
* them. If you destroy a TAsyncTimeoutSet with callbacks pending, your
* callback code needs to be aware that the callbacks will never be invoked.
*/
virtual void destroy();
/**
* Get the interval for this TAsyncTimeoutSet.
*
* Returns the timeout interval in milliseconds. All callbacks scheduled
* with scheduleTimeout() will be invoked after this amount of time has
* passed since the call to scheduleTimeout().
*/
std::chrono::milliseconds getInterval() const {
return interval_;
}
/**
* Schedule the specified Callback to be invoked after the TAsyncTimeoutSet's
* specified timeout interval.
*
* If the callback is already scheduled, this cancels the existing timeout
* before scheduling the new timeout.
*/
void scheduleTimeout(Callback* callback);
/**
* Limit how frequently this TAsyncTimeoutSet will fire.
*/
void fireAtMostEvery(const std::chrono::milliseconds& ms) {
atMostEveryN_ = ms;
}
/**
* Get a pointer to the next Callback scheduled to be invoked (may be null).
*/
Callback* front() { return head_; }
const Callback* front() const { return head_; }
protected:
/**
* Protected destructor.
*
* Use destroy() instead. See the comments in TDelayedDestruction for more
* details.
*/
virtual ~TAsyncTimeoutSet();
private:
// Forbidden copy constructor and assignment operator
TAsyncTimeoutSet(TAsyncTimeoutSet const &) = delete;
TAsyncTimeoutSet& operator=(TAsyncTimeoutSet const &) = delete;
// Private methods to be invoked by TAsyncTimeoutSet::Callback
void headChanged();
// Methods inherited from TAsyncTimeout
virtual void timeoutExpired() THRIFT_NOEXCEPT;
std::chrono::milliseconds interval_;
std::chrono::milliseconds atMostEveryN_;
bool inTimeoutExpired_;
Callback* head_;
Callback* tail_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TASYNCTIMEOUTSET_H_

View File

@ -0,0 +1,428 @@
/*
* 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 THRIFT_ASYNC_TASYNCTRANSPORT_H_
#define THRIFT_ASYNC_TASYNCTRANSPORT_H_ 1
#include "thrift/lib/cpp/thrift_config.h"
#include <sys/uio.h>
#include <inttypes.h>
#include <memory>
namespace folly {
class IOBuf;
}
namespace apache { namespace thrift {
namespace transport {
class TSocketAddress;
class TTransportException;
}
namespace async {
class TEventBase;
/**
* TAsyncTransport defines an asynchronous API for streaming I/O.
*
* This class provides an API to for asynchronously waiting for data
* on a streaming transport, and for asynchronously sending data.
*
* The APIs for reading and writing are intentionally asymmetric. Waiting for
* data to read is a persistent API: a callback is installed, and is notified
* whenever new data is available. It continues to be notified of new events
* until it is uninstalled.
*
* TAsyncTransport does not provide read timeout functionality, because it
* typically cannot determine when the timeout should be active. Generally, a
* timeout should only be enabled when processing is blocked waiting on data
* from the remote endpoint. For server-side applications, the timeout should
* not be active if the server is currently processing one or more outstanding
* requests on this transport. For client-side applications, the timeout
* should not be active if there are no requests pending on the transport.
* Additionally, if a client has multiple pending requests, it will ususally
* want a separate timeout for each request, rather than a single read timeout.
*
* The write API is fairly intuitive: a user can request to send a block of
* data, and a callback will be informed once the entire block has been
* transferred to the kernel, or on error. TAsyncTransport does provide a send
* timeout, since most callers want to give up if the remote end stops
* responding and no further progress can be made sending the data.
*/
class TAsyncTransport {
public:
class ReadCallback {
public:
virtual ~ReadCallback() {}
/**
* When data becomes available, getReadBuffer() will be invoked to get the
* buffer into which data should be read.
*
* This method allows the ReadCallback to delay buffer allocation until
* data becomes available. This allows applications to manage large
* numbers of idle connections, without having to maintain a separate read
* buffer for each idle connection.
*
* It is possible that in some cases, getReadBuffer() may be called
* multiple times before readDataAvailable() is invoked. In this case, the
* data will be written to the buffer returned from the most recent call to
* readDataAvailable(). If the previous calls to readDataAvailable()
* returned different buffers, the ReadCallback is responsible for ensuring
* that they are not leaked.
*
* If getReadBuffer() throws an exception, returns a NULL buffer, or
* returns a 0 length, the ReadCallback will be uninstalled and its
* readError() method will be invoked.
*
* getReadBuffer() is not allowed to change the transport state before it
* returns. (For example, it should never uninstall the read callback, or
* set a different read callback.)
*
* @param bufReturn getReadBuffer() should update *bufReturn to contain the
* address of the read buffer. This parameter will never
* be NULL.
* @param lenReturn getReadBuffer() should update *lenReturn to contain the
* maximum number of bytes that may be written to the read
* buffer. This parameter will never be NULL.
*/
virtual void getReadBuffer(void** bufReturn, size_t* lenReturn) = 0;
/**
* readDataAvailable() will be invoked when data has been successfully read
* into the buffer returned by the last call to getReadBuffer().
*
* The read callback remains installed after readDataAvailable() returns.
* It must be explicitly uninstalled to stop receiving read events.
* getReadBuffer() will be called at least once before each call to
* readDataAvailable(). getReadBuffer() will also be called before any
* call to readEOF().
*
* @param len The number of bytes placed in the buffer.
*/
virtual void readDataAvailable(size_t len) THRIFT_NOEXCEPT = 0;
/**
* readEOF() will be invoked when the transport is closed.
*
* The read callback will be automatically uninstalled immediately before
* readEOF() is invoked.
*/
virtual void readEOF() THRIFT_NOEXCEPT = 0;
/**
* readError() will be invoked if an error occurs reading from the
* transport.
*
* The read callback will be automatically uninstalled immediately before
* readError() is invoked.
*
* @param ex An exception describing the error that occurred.
*/
virtual void readError(const transport::TTransportException& ex)
THRIFT_NOEXCEPT = 0;
};
class WriteCallback {
public:
virtual ~WriteCallback() {}
/**
* writeSuccess() will be invoked when all of the data has been
* successfully written.
*
* Note that this mainly signals that the buffer containing the data to
* write is no longer needed and may be freed or re-used. It does not
* guarantee that the data has been fully transmitted to the remote
* endpoint. For example, on socket-based transports, writeSuccess() only
* indicates that the data has been given to the kernel for eventual
* transmission.
*/
virtual void writeSuccess() THRIFT_NOEXCEPT = 0;
/**
* writeError() will be invoked if an error occurs writing the data.
*
* @param bytesWritten The number of bytes that were successfull
* @param ex An exception describing the error that occurred.
*/
virtual void writeError(size_t bytesWritten,
const transport::TTransportException& ex)
THRIFT_NOEXCEPT = 0;
};
virtual ~TAsyncTransport() {}
/**
* Set the read callback.
*
* See the documentation for ReadCallback above for a description of how the
* callback will be invoked. Note that the callback remains installed until
* it is explicitly uninstalled, or until an error occurs.
*
* If a ReadCallback is already installed, it is replaced with the new
* callback.
*
* @param callback The callback to invoke when data is available.
* This parameter may be NULL to uninstall the current
* read callback.
*/
virtual void setReadCallback(ReadCallback* callback) = 0;
/**
* Get the currently installed read callback.
*
* @return Returns a pointer to the installed ReadCallback, or NULL if no
* ReadCallback is installed.
*/
virtual ReadCallback* getReadCallback() const = 0;
/**
* Write data to the transport.
*
* write() will always return immediately. The WriteCallback will later be
* invoked from the main TEventBase loop when the write has completed.
*
* Additional write attempts may be started before the first write completes.
* The subsequent write requests will be queued, and processed in the order
* in which they were called.
*
* @param callback The callback to invoke when the data has been written.
* The callback may not be NULL.
* @param buf The buffer containing the data to write. The caller is
* responsible for ensuring that this buffer remains valid
* until the callback is invoked. This parameter may not
* be NULL.
* @param bytes The number of bytes to write.
*/
virtual void write(WriteCallback* callback,
const void* buf, size_t bytes) = 0;
/**
* Write non-contiguous data to the transport.
*
* writev() will always return immediately. The WriteCallback will later be
* invoked from the main TEventBase loop when the write has completed.
*
* Additional write attempts may be started before the first write completes.
* The subsequent write requests will be queued, and processed in the order
* in which they were called.
*
* @param callback The callback to invoke when the data has been written.
* The callback may not be NULL.
* @param vec A pointer to an array of iovec objects. The caller is
* responsible for ensuring that the buffers remain valid
* until the callback is invoked. This parameter may not
* be NULL.
* @param count The number of iovec objects in the vec array.
*/
virtual void writev(WriteCallback* callback,
const iovec* vec, size_t count) = 0;
/**
* Write a chain of IOBufs to the transport.
*
* writeChain() will always return immediately. The WriteCallback will
* later be invoked from the main TEventBase loop when the write has
* completed.
*
* Additional write attempts may be started before the first write completes.
* The subsequent write requests will be queued, and processed in the order
* in which they were called.
*
* @param callback The callback to invoke when the data has been written.
* The callback may not be NULL.
* @param iob The head of an IOBuf chain. The TAsyncTransport
* will take ownership of this chain and delete it
* after writing.
* @param cork Whether to delay the write until the next non-corked
* write operation. (Note: may not be supported in all
* subclasses or on all platforms.)
*/
virtual void writeChain(WriteCallback* callback,
std::unique_ptr<folly::IOBuf>&& iob,
bool cork = false) = 0;
/**
* Close the transport.
*
* This gracefully closes the transport, waiting for all pending write
* requests to complete before actually closing the underlying transport.
*
* If a read callback is set, readEOF() will be called immediately. If there
* are outstanding write requests, the close will be delayed until all
* remaining writes have completed. No new writes may be started after
* close() has been called.
*/
virtual void close() = 0;
/**
* Close the transport immediately.
*
* This closes the transport immediately, dropping any outstanding data
* waiting to be written.
*
* If a read callback is set, readEOF() will be called immediately.
* If there are outstanding write requests, these requests will be aborted
* and writeError() will be invoked immediately on all outstanding write
* callbacks.
*/
virtual void closeNow() = 0;
/**
* Perform a half-shutdown of the write side of the transport.
*
* The caller should not make any more calls to write() or writev() after
* shutdownWrite() is called. Any future write attempts will fail
* immediately.
*
* Not all transport types support half-shutdown. If the underlying
* transport does not support half-shutdown, it will fully shutdown both the
* read and write sides of the transport. (Fully shutting down the socket is
* better than doing nothing at all, since the caller may rely on the
* shutdownWrite() call to notify the other end of the connection that no
* more data can be read.)
*
* If there is pending data still waiting to be written on the transport,
* the actual shutdown will be delayed until the pending data has been
* written.
*
* Note: There is no corresponding shutdownRead() equivalent. Simply
* uninstall the read callback if you wish to stop reading. (On TCP sockets
* at least, shutting down the read side of the socket is a no-op anyway.)
*/
virtual void shutdownWrite() = 0;
/**
* Perform a half-shutdown of the write side of the transport.
*
* shutdownWriteNow() is identical to shutdownWrite(), except that it
* immediately performs the shutdown, rather than waiting for pending writes
* to complete. Any pending write requests will be immediately failed when
* shutdownWriteNow() is called.
*/
virtual void shutdownWriteNow() = 0;
/**
* Determine if transport is open and ready to read or write.
*
* Note that this function returns false on EOF; you must also call error()
* to distinguish between an EOF and an error.
*
* @return true iff the transport is open and ready, false otherwise.
*/
virtual bool good() const = 0;
/**
* Determine if the transport is readable or not.
*
* @return true iff the transport is readable, false otherwise.
*/
virtual bool readable() const = 0;
/**
* Determine if transport is connected to the endpoint
*
* @return false iff the transport is connected, otherwise true
*/
virtual bool connecting() const = 0;
/**
* Determine if an error has occurred with this transport.
*
* @return true iff an error has occurred (not EOF).
*/
virtual bool error() const = 0;
/**
* Attach the transport to a TEventBase.
*
* This may only be called if the transport is not currently attached to a
* TEventBase (by an earlier call to detachEventBase()).
*
* This method must be invoked in the TEventBase's thread.
*/
virtual void attachEventBase(TEventBase* eventBase) = 0;
/**
* Detach the transport from its TEventBase.
*
* This may only be called when the transport is idle and has no reads or
* writes pending. Once detached, the transport may not be used again until
* it is re-attached to a TEventBase by calling attachEventBase().
*
* This method must be called from the current TEventBase's thread.
*/
virtual void detachEventBase() = 0;
/**
* Get the TEventBase used by this transport.
*
* Returns NULL if this transport is not currently attached to a TEventBase.
*/
virtual TEventBase* getEventBase() const = 0;
/**
* Set the send timeout.
*
* If write requests do not make any progress for more than the specified
* number of milliseconds, fail all pending writes and close the transport.
*
* If write requests are currently pending when setSendTimeout() is called,
* the timeout interval is immediately restarted using the new value.
*
* @param milliseconds The timeout duration, in milliseconds. If 0, no
* timeout will be used.
*/
virtual void setSendTimeout(uint32_t milliseconds) = 0;
/**
* Get the send timeout.
*
* @return Returns the current send timeout, in milliseconds. A return value
* of 0 indicates that no timeout is set.
*/
virtual uint32_t getSendTimeout() const = 0;
/**
* Get the address of the local endpoint of this transport.
*
* This function may throw TTransportException on error.
*
* @param address The local address will be stored in the specified
* TSocketAddress.
*/
virtual void getLocalAddress(transport::TSocketAddress* address) const = 0;
/**
* Get the address of the remote endpoint to which this transport is
* connected.
*
* This function may throw TTransportException on error.
*
* @param address The remote endpoint's address will be stored in the
* specified TSocketAddress.
*/
virtual void getPeerAddress(transport::TSocketAddress* address) const = 0;
};
}}} // apache::thrift::async
#endif // #ifndef THRIFT_ASYNC_TASYNCTRANSPORT_H_

View File

@ -0,0 +1,146 @@
/*
* 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 THRIFT_ASYNC_TBINARYASYNCCHANNEL_H_
#define THRIFT_ASYNC_TBINARYASYNCCHANNEL_H_ 1
#include "thrift/lib/cpp/async/TUnframedAsyncChannel.h"
namespace apache { namespace thrift { namespace async {
namespace detail {
/**
* A class to determine the end of a raw TBinaryProtocol message.
*/
class TBinaryACProtocolTraits {
public:
TBinaryACProtocolTraits() : strictRead_(true) {}
// Methods required by TUnframedACReadState
bool getMessageLength(uint8_t* buffer,
uint32_t bufferLength,
uint32_t* messageLength);
// Methods specific to TBinaryAsyncChannel
void setStrictRead(bool strictRead) {
strictRead_ = strictRead;
}
bool getStrictRead() const {
return strictRead_;
}
private:
bool strictRead_;
};
} // namespace detail
/**
* TBinaryAsyncChannel
*
* This is a TAsyncChannel implementation that reads and writes raw (unframed)
* messages encoded using TBinaryProtocol.
*/
class TBinaryAsyncChannel :
public TUnframedAsyncChannel<detail::TBinaryACProtocolTraits> {
private:
typedef TUnframedAsyncChannel<detail::TBinaryACProtocolTraits> Parent;
public:
TBinaryAsyncChannel(const boost::shared_ptr<TAsyncTransport>& transport)
: Parent(transport) {}
/**
* Helper function to create a shared_ptr<TBinaryAsyncChannel>.
*
* This passes in the correct destructor object, since TBinaryAsyncChannel's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<TBinaryAsyncChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
return boost::shared_ptr<TBinaryAsyncChannel>(
new TBinaryAsyncChannel(transport), Destructor());
}
// Note that we inherit setMaxMessageSize() and getMaxMessageSize()
// from TUnframedAsyncChannel.
void setStrictRead(bool strictRead) {
readState_.getProtocolTraits()->setStrictRead(strictRead);
}
bool getStrictRead() const {
return readState_.getProtocolTraits()->getStrictRead();
}
protected:
/**
* Protected destructor.
*
* Users of TBinaryAsyncChannel must never delete it directly. Instead,
* invoke destroy().
*/
virtual ~TBinaryAsyncChannel() { }
};
class TBinaryAsyncChannelFactory : public TStreamAsyncChannelFactory {
public:
TBinaryAsyncChannelFactory()
: maxMessageSize_(0x7fffffff)
, recvTimeout_(0)
, sendTimeout_(0)
, strictRead_(true) {}
void setMaxMessageSize(uint32_t bytes) {
maxMessageSize_ = bytes;
}
void setRecvTimeout(uint32_t milliseconds) {
recvTimeout_ = milliseconds;
}
void setSendTimeout(uint32_t milliseconds) {
sendTimeout_ = milliseconds;
}
void setStrictRead(bool strict) {
strictRead_ = strict;
}
virtual boost::shared_ptr<TAsyncEventChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
boost::shared_ptr<TBinaryAsyncChannel> channel(
TBinaryAsyncChannel::newChannel(transport));
transport->setSendTimeout(sendTimeout_);
channel->setMaxMessageSize(maxMessageSize_);
channel->setRecvTimeout(recvTimeout_);
channel->setStrictRead(strictRead_);
return channel;
}
private:
uint32_t maxMessageSize_;
uint32_t recvTimeout_;
uint32_t sendTimeout_;
bool strictRead_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TBINARYASYNCCHANNEL_H_

View File

@ -0,0 +1,178 @@
/*
* 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 THRIFT_ASYNC_TDELAYEDDESTRUCTION_H_
#define THRIFT_ASYNC_TDELAYEDDESTRUCTION_H_ 1
#include <boost/noncopyable.hpp>
#include <inttypes.h>
#include <assert.h>
namespace apache { namespace thrift { namespace async {
/**
* TDelayedDestruction is a helper class to ensure objects are not deleted
* while they still have functions executing in a higher stack frame.
*
* This is useful for objects that invoke callback functions, to ensure that a
* callback does not destroy the calling object.
*
* Classes needing this functionality should:
* - derive from TDelayedDestruction
* - make their destructor private or protected, so it cannot be called
* directly
* - create a DestructorGuard object on the stack in each public method that
* may invoke a callback
*
* TDelayedDestruction does not perform any locking. It is intended to be used
* only from a single thread.
*/
class TDelayedDestruction : private boost::noncopyable {
public:
/**
* Helper class to allow TDelayedDestruction classes to be used with
* boost::shared_ptr.
*
* This class can be specified as the destructor argument when creating the
* shared_ptr, and it will destroy the guarded class properly when all
* shared_ptr references are released.
*/
class Destructor {
public:
void operator()(TDelayedDestruction* dd) const {
dd->destroy();
}
};
/**
* destroy() requests destruction of the object.
*
* This method will destroy the object after it has no more functions running
* higher up on the stack. (i.e., No more DestructorGuard objects exist for
* this object.) This method must be used instead of the destructor.
*/
virtual void destroy() {
// If guardCount_ is not 0, just set destroyPending_ to delay
// actual destruction.
if (guardCount_ != 0) {
destroyPending_ = true;
} else {
destroyNow(false);
}
}
/**
* Classes should create a DestructorGuard object on the stack in any
* function that may invoke callback functions.
*
* The DestructorGuard prevents the guarded class from being destroyed while
* it exists. Without this, the callback function could delete the guarded
* object, causing problems when the callback function returns and the
* guarded object's method resumes execution.
*/
class DestructorGuard {
public:
explicit DestructorGuard(TDelayedDestruction* dd) : dd_(dd) {
++dd_->guardCount_;
assert(dd_->guardCount_ > 0); // check for wrapping
}
~DestructorGuard() {
assert(dd_->guardCount_ > 0);
--dd_->guardCount_;
if (dd_->guardCount_ == 0 && dd_->destroyPending_) {
dd_->destroyNow(true);
}
}
private:
TDelayedDestruction* dd_;
};
protected:
/**
* destroyNow() is invoked to actually destroy the object, after destroy()
* has been called and no more DestructorGuard objects exist. By default it
* calls "delete this", but subclasses may override this behavior.
*
* @param delayed This parameter is true if destruction was delayed because
* of a DestructorGuard object, or false if destroyNow() is
* being called directly from destroy().
*/
virtual void destroyNow(bool delayed) {
delete this;
(void)delayed; // prevent unused variable warnings
}
TDelayedDestruction()
: guardCount_(0)
, destroyPending_(false) {}
/**
* Protected destructor.
*
* Making this protected ensures that users cannot delete TDelayedDestruction
* objects directly, and that everyone must use destroy() instead.
* Subclasses of TDelayedDestruction must also define their destructors as
* protected or private in order for this to work.
*
* This also means that TDelayedDestruction objects cannot be created
* directly on the stack; they must always be dynamically allocated on the
* heap.
*
* In order to use a TDelayedDestruction object with a shared_ptr, create the
* shared_ptr using a TDelayedDestruction::Destructor as the second argument
* to the shared_ptr constructor.
*/
virtual ~TDelayedDestruction() {}
/**
* Get the number of DestructorGuards currently protecting this object.
*
* This is primarily intended for debugging purposes, such as asserting
* that an object has at least 1 guard.
*/
uint32_t getDestructorGuardCount() const {
return guardCount_;
}
private:
/**
* guardCount_ is incremented by DestructorGuard, to indicate that one of
* the TDelayedDestruction object's methods is currently running.
*
* If destroy() is called while guardCount_ is non-zero, destruction will
* be delayed until guardCount_ drops to 0. This allows TDelayedDestruction
* objects to invoke callbacks without having to worry about being deleted
* before the callback returns.
*/
uint32_t guardCount_;
/**
* destroyPending_ is set to true if destoy() is called while guardCount_ is
* non-zero.
*
* If destroyPending_ is true, the object will be destroyed the next time
* guardCount_ drops to 0.
*/
bool destroyPending_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TDELAYEDDESTRUCTION_H_

View File

@ -0,0 +1,397 @@
// Copyright (c) 2006- Facebook
// Distributed under the Thrift Software License
//
// See accompanying file LICENSE or visit the Thrift site at:
// http://developers.facebook.com/thrift/
#ifndef THRIFT_ASYNC_TEVENTBASE_H_
#define THRIFT_ASYNC_TEVENTBASE_H_ 1
#include "thrift/lib/cpp/Thrift.h"
#include "thrift/lib/cpp/async/TAsyncTimeout.h"
#include "thrift/lib/cpp/server/TServer.h"
#include "thrift/lib/cpp/transport/TTransportUtils.h"
#include "thrift/lib/cpp/concurrency/ThreadManager.h"
#include <memory>
#include <stack>
#include <list>
#include <queue>
#include <cstdlib>
#include <set>
#include <utility>
#include <boost/intrusive/list.hpp>
#include <boost/utility.hpp>
#include <tr1/functional>
#include <event.h> // libevent
#include <errno.h>
#include <math.h>
namespace apache { namespace thrift { namespace async {
typedef std::tr1::function<void()> Cob;
template <typename MessageT>
class TNotificationQueue;
/**
* This class is a wrapper for all asynchronous I/O processing functionality
* used in thrift.
*
* TEventBase provides a main loop that notifies TEventHandler callback objects
* when I/O is ready on a file descriptor, and notifies TAsyncTimeout objects
* when a specified timeout has expired. More complex, higher-level callback
* mechanisms can then be built on top of TEventHandler and TAsyncTimeout.
*
* A TEventBase object can only drive an event loop for a single thread. To
* take advantage of multiple CPU cores, most asynchronous I/O servers have one
* thread per CPU, and use a separate TEventBase for each thread.
*
* In general, most TEventBase methods may only be called from the thread
* running the TEventBase's loop. There are a few exceptions to this rule, for
* methods that are explicitly intended to allow communication with a
* TEventBase from other threads. When it is safe to call a method from
* another thread it is explicitly listed in the method comments.
*/
class TEventBase : private boost::noncopyable {
public:
/**
* A callback interface to use with runInLoop()
*
* Derive from this class if you need to delay some code execution until the
* next iteration of the event loop. This allows you to schedule code to be
* invoked from the top-level of the loop, after your immediate callers have
* returned.
*
* If a LoopCallback object is destroyed while it is scheduled to be run in
* the next loop iteration, it will automatically be cancelled.
*/
class LoopCallback {
public:
virtual ~LoopCallback() {}
virtual void runLoopCallback() THRIFT_NOEXCEPT = 0;
void cancelLoopCallback() {
hook_.unlink();
}
bool isLoopCallbackScheduled() const {
return hook_.is_linked();
}
private:
typedef boost::intrusive::list_member_hook<
boost::intrusive::link_mode<boost::intrusive::auto_unlink> > ListHook;
ListHook hook_;
typedef boost::intrusive::list<
LoopCallback,
boost::intrusive::member_hook<LoopCallback, ListHook,
&LoopCallback::hook_>,
boost::intrusive::constant_time_size<false> > List;
// TEventBase needs access to LoopCallbackList (and therefore to hook_)
friend class TEventBase;
};
/**
* Create a new TEventBase object.
*/
TEventBase();
/**
* Create a new TEventBase object that will use the specified libevent
* event_base object to drive the event loop.
*
* The TEventBase will take ownership of this event_base, and will call
* event_base_free(evb) when the TEventBase is destroyed.
*/
explicit TEventBase(event_base* evb);
~TEventBase();
/**
* Runs the event loop.
*
* loop() will loop waiting for I/O or timeouts and invoking TEventHandler
* and TAsyncTimeout callbacks as their events become ready. loop() will
* only return when there are no more events remaining to process, or after
* terminateLoopSoon() has been called.
*
* loop() may be called again to restart event processing after a previous
* call to loop() or loopForever() has returned.
*
* Returns true if the loop completed normally (if it processed all
* outstanding requests, or if terminateLoopSoon() was called). If an error
* occurs waiting for events, false will be returned.
*/
bool loop();
/**
* Runs the event loop.
*
* loopForever() behaves like loop(), except that it keeps running even if
* when there are no more user-supplied TEventHandlers or TAsyncTimeouts
* registered. It will only return after terminateLoopSoon() has been
* called.
*
* This is useful for callers that want to wait for other threads to call
* runInEventBaseThread(), even when there are no other scheduled events.
*
* loopForever() may be called again to restart event processing after a
* previous call to loop() or loopForever() has returned.
*
* Throws a TLibraryException if an error occurs.
*/
void loopForever();
/**
* Causes the event loop to exit soon.
*
* This will cause an existing call to loop() or loopForever() to stop event
* processing and return, even if there are still events remaining to be
* processed.
*
* It is safe to call terminateLoopSoon() from another thread to cause loop()
* to wake up and return in the TEventBase loop thread. terminateLoopSoon()
* may also be called from the loop thread itself (for example, a
* TEventHandler or TAsyncTimeout callback may call terminateLoopSoon() to
* cause the loop to exit after the callback returns.)
*
* Note that the caller is responsible for ensuring that cleanup of all event
* callbacks occurs properly. Since terminateLoopSoon() causes the loop to
* exit even when there are pending events present, there may be remaining
* callbacks present waiting to be invoked. If the loop is later restarted
* pending events will continue to be processed normally, however if the
* TEventBase is destroyed after calling terminateLoopSoon() it is the
* caller's responsibility to ensure that cleanup happens properly even if
* some outstanding events are never processed.
*/
void terminateLoopSoon();
/**
* Adds the given callback to a queue of things run after the current pass
* through the event loop completes. Note that if this callback calls
* runInLoop() the new callback won't be called until the main event loop
* has gone through a cycle.
*
* This method may only be called from the TEventBase's thread. This
* essentially allows an event handler to schedule an additional callback to
* be invoked after it returns.
*
* Use runInEventBaseThread() to schedule functions from another thread.
*/
void runInLoop(LoopCallback* callback);
/**
* Convenience function to call runInLoop() with a tr1::function.
*
* This creates a LoopCallback object to wrap the tr1::function, and invoke
* the tr1::function when the loop callback fires. This is slightly more
* expensive than defining your own LoopCallback, but more convenient in
* areas that aren't performance sensitive where you just want to use
* tr1::bind. (tr1::bind is fairly slow on even by itself.)
*
* This method may only be called from the TEventBase's thread. This
* essentially allows an event handler to schedule an additional callback to
* be invoked after it returns.
*
* Use runInEventBaseThread() to schedule functions from another thread.
*/
void runInLoop(const Cob& c);
/**
* Run the specified function in the TEventBase's thread.
*
* This method is thread-safe, and may be called from another thread.
*
* If runInEventBaseThread() is called when the TEventBase loop is not
* running, the function call will be delayed until the next time the loop is
* started.
*
* If runInEventBaseThread() returns true the function has successfully been
* scheduled to run in the loop thread. However, if the loop is terminated
* (and never later restarted) before it has a chance to run the requested
* function, the function may never be run at all. The caller is responsible
* for handling this situation correctly if they may terminate the loop with
* outstanding runInEventBaseThread() calls pending.
*
* If two calls to runInEventBaseThread() are made from the same thread, the
* functions will always be run in the order that they were scheduled.
* Ordering between functions scheduled from separate threads is not
* guaranteed.
*
* @param fn The function to run. The function must not throw any
* exceptions.
* @param arg An argument to pass to the function.
*
* @return Returns true if the function was successfully scheduled, or false
* if there was an error scheduling the function.
*/
template<typename T>
bool runInEventBaseThread(void (*fn)(T*), T* arg) {
return runInEventBaseThread(reinterpret_cast<void (*)(void*)>(fn),
reinterpret_cast<void*>(arg));
}
bool runInEventBaseThread(void (*fn)(void*), void* arg);
/**
* Run the specified function in the TEventBase's thread
*
* This version of runInEventBaseThread() takes a tr1::function object.
* Note that this is less efficient than the version that takes a plain
* function pointer and void* argument, as it has to allocate memory to copy
* the tr1::function object.
*
* If the TEventBase loop is terminated before it has a chance to run this
* function, the allocated memory will be leaked. The caller is responsible
* for ensuring that the TEventBase loop is not terminated before this
* function can run.
*
* The function must not throw any exceptions.
*/
bool runInEventBaseThread(const std::tr1::function<void()>& fn);
/**
* Runs the given Cob at some time after the specified number of
* milliseconds. (No guarantees exactly when.)
*
* @return true iff the cob was successfully registered.
*/
bool runAfterDelay(const Cob& c, int milliseconds);
/**
* Set the maximum desired latency in us and provide a callback which will be
* called when that latency is exceeded.
*/
void setMaxLatency(int64_t maxLatency, const Cob& maxLatencyCob) {
maxLatency_ = maxLatency;
maxLatencyCob_ = maxLatencyCob;
}
/**
* Set smoothing coefficient for loop load average; # of milliseconds
* for exp(-1) (1/2.71828...) decay.
*/
void setLoadAvgMsec(uint32_t ms);
/**
* Get the average loop time in microseconds (an exponentially-smoothed ave)
*/
double getAvgLoopTime() const {
return avgLoopTime_;
}
/**
* Verify that current thread is the TEventBase thread, if the TEventBase is
* running.
*
* This is primarily intended for debugging, to assert that functions that
* register or unregister events are only ever invoked in the TEventBase's
* thread.
*/
bool isInEventBaseThread() const {
return !running_ || pthread_equal(loopThread_, pthread_self());
}
// --------- interface to underlying libevent base ------------
// Avoid using these functions if possible. These functions are not
// guaranteed to always be present if we ever provide alternative TEventBase
// implementations that do not use libevent internally.
event_base* getLibeventBase() const { return evb_; }
static const char* getLibeventVersion() { return event_get_version(); }
static const char* getLibeventMethod() { return event_get_method(); }
private:
// --------- libevent callbacks (not for client use) ------------
/**
* Called after a delay to break out of an idle event loop. We need to
* use this instead of event_base_loopexit() since the latter installs
* an event within libevent which is queued until it expires. Installing
* our own timed event lets us delete it when another event causes the
* loop to exit earlier..
*/
static void loopTimerCallback(int fd, short which, void* arg);
static void runTr1FunctionPtr(std::tr1::function<void()>* fn);
// small object used as a callback arg with enough info to execute the
// appropriate client-provided Cob
class CobTimeout : public TAsyncTimeout {
public:
CobTimeout(TEventBase* b, const Cob& c) : TAsyncTimeout(b), cob_(c) {}
virtual void timeoutExpired() THRIFT_NOEXCEPT;
private:
Cob cob_;
public:
typedef boost::intrusive::list_member_hook<
boost::intrusive::link_mode<boost::intrusive::auto_unlink> > ListHook;
ListHook hook;
typedef boost::intrusive::list<
CobTimeout,
boost::intrusive::member_hook<CobTimeout, ListHook, &CobTimeout::hook>,
boost::intrusive::constant_time_size<false> > List;
};
typedef LoopCallback::List LoopCallbackList;
class FunctionRunner;
// executes any callbacks queued by runInLoop()
void runLoopCallbacks();
void initNotificationQueue();
CobTimeout::List pendingCobTimeouts_;
LoopCallbackList loopCallbacks_;
// stop_ is set by terminateLoopSoon() and is used by the main loop
// to determine if it should exit
bool stop_;
// running_ is set to true while loop() is running
bool running_;
// The ID of the thread running the main loop.
// Only valid while running_ is true.
pthread_t loopThread_;
// pointer to underlying event_base class doing the heavy lifting
event_base* evb_;
// A notification queue for runInEventBaseThread() to use
// to send function requests to the TEventBase thread.
std::unique_ptr<TNotificationQueue<std::pair<void (*)(void*), void*>>> queue_;
std::unique_ptr<FunctionRunner> fnRunner_;
// limit for latency in microseconds (0 disables)
int64_t maxLatency_;
// smoothed loop time used to invoke latency callbacks; differs from
// avgLoopTime_ in that it's scaled down after triggering a callback
// to reduce spamminess
double maxLatencyLoopTime_;
// exponentially-smoothed average loop time for latency-limiting
double avgLoopTime_;
// set to true if the event_base_loop(EVLOOP_ONCE) returned because
// the loop timeout fired, rather than because it found events to process
bool loopTimedOut_;
// factor used for exponential smoothing of load average
double expCoeff_;
// callback called when latency limit is exceeded
Cob maxLatencyCob_;
// we'll wait this long before running deferred callbacks if the event
// loop is idle.
static const int kDEFAULT_IDLE_WAIT_USEC = 20000; // 20ms
};
}}} // apache::thrift::async
#endif // #ifndef THRIFT_ASYNC_TEVENTBASE_H_

View File

@ -0,0 +1,121 @@
/*
* 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 THRIFT_ASYNC_TEVENTBASEMANAGER_H
#define THRIFT_ASYNC_TEVENTBASEMANAGER_H 1
#include "thrift/lib/cpp/concurrency/ThreadLocal.h"
namespace apache { namespace thrift { namespace async {
class TEventBase;
/**
* Manager for per-thread TEventBase objects.
* This class will find or create a TEventBase for the current
* thread, associated with thread-specific storage for that thread.
* Although a typical application will generally only have one
* TEventBaseManager, there is no restriction on multiple instances;
* the TEventBases belong to one instance are isolated from those of
* another.
*/
class TEventBaseManager {
public:
TEventBaseManager() {}
~TEventBaseManager() {}
/**
* Get the TEventBase for this thread, or create one if none exists yet.
*
* If no TEventBase exists for this thread yet, a new one will be created and
* returned. May throw std::bad_alloc if allocation fails.
*/
TEventBase* getEventBase() const {
// localStore_.get() will never return NULL.
// InfoManager::allocate() will throw an exception instead if it cannot
// allocate a new EventBaseInfo or TEventBase.
return localStore_.get()->eventBase;
}
/**
* Get the TEventBase for this thread.
*
* Returns NULL if no TEventBase has been created for this thread yet.
*/
TEventBase* getExistingEventBase() const {
EventBaseInfo* info = localStore_.getNoAlloc();
if (info == NULL) {
return NULL;
}
return info->eventBase;
}
/**
* Set the TEventBase to be used by this thread.
*
* This may only be called if no TEventBase has been defined for this thread
* yet. If a TEventBase is already defined for this thread, a
* TLibraryException is thrown. std::bad_alloc may also be thrown if
* allocation fails while setting the TEventBase.
*
* This should typically be invoked by the code that will call loop() on the
* TEventBase, to make sure the TEventBaseManager points to the correct
* TEventBase that is actually running in this thread.
*/
void setEventBase(TEventBase *eventBase, bool takeOwnership);
/**
* Clear the TEventBase for this thread.
*
* This can be used if the code driving the TEventBase loop() has finished
* the loop and new events should no longer be added to the TEventBase.
*/
void clearEventBase();
private:
struct EventBaseInfo {
EventBaseInfo(TEventBase *evb, bool owned)
: eventBase(evb),
owned(owned) {}
TEventBase *eventBase;
bool owned;
};
class InfoManager {
public:
EventBaseInfo* allocate();
void destroy(EventBaseInfo* info);
void replace(EventBaseInfo* oldInfo, EventBaseInfo* newInfo) {
if (oldInfo != newInfo) {
destroy(oldInfo);
}
}
};
// Forbidden copy constructor and assignment opererator
TEventBaseManager(TEventBaseManager const &);
TEventBaseManager& operator=(TEventBaseManager const &);
concurrency::ThreadLocal<EventBaseInfo, InfoManager> localStore_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TEVENTBASEMANAGER_H

View File

@ -0,0 +1,239 @@
// Copyright (c) 2006- Facebook
// Distributed under the Thrift Software License
//
// See accompanying file LICENSE or visit the Thrift site at:
// http://developers.facebook.com/thrift/
#ifndef THRIFT_ASYNC_TEVENTCONNECTION_H_
#define THRIFT_ASYNC_TEVENTCONNECTION_H_ 1
#include "thrift/lib/cpp/server/TConnectionContext.h"
#include "thrift/lib/cpp/transport/TSocketAddress.h"
#include "thrift/lib/cpp/async/TEventServer.h"
#include <boost/shared_ptr.hpp>
#include <boost/noncopyable.hpp>
namespace apache { namespace thrift {
class TProcessor;
namespace protocol {
class TProtocol;
}
namespace server {
class TServerEventHandler;
}
namespace transport {
class TMemoryBuffer;
}
namespace async {
class TAsyncEventChannel;
class TAsyncProcessor;
class TEventWorker;
class TAsyncSocket;
class TaskCompletionMessage;
/**
* Represents a connection that is handled via libevent. This connection
* essentially encapsulates a socket that has some associated libevent state.
*/
class TEventConnection : private boost::noncopyable,
public TEventBase::LoopCallback {
public:
/**
* Constructor for TEventConnection.
*
* @param asyncSocket shared pointer to the async socket
* @param address the peer address of this connection
* @param worker the worker instance that is handling this connection
*/
TEventConnection(boost::shared_ptr<TAsyncSocket> asyncSocket,
const transport::TSocketAddress* address,
TEventWorker* worker, TEventServer::TransportType transport);
/**
* (Re-)Initialize a TEventConnection. We break this out from the
* constructor to allow for pooling.
*
* @param asyncSocket shared pointer to the async socket
* @param address the peer address of this connection
* @param worker the worker instance that is handling this connection
*/
void init(boost::shared_ptr<TAsyncSocket> asyncSocket,
const transport::TSocketAddress* address,
TEventWorker* worker, TEventServer::TransportType transport);
/// First cause -- starts i/o on connection
void start();
/// Shut down the connection even if it's OK; used for load reduction.
void stop() {
shutdown_ = true;
}
/// Return a pointer to the worker that owns us
TEventWorker* getWorker() const {
return worker_;
}
/// cause the notification callback to occur within the appropriate context
bool notifyCompletion(TaskCompletionMessage &&msg);
/// Run scheduled read when there are too many reads on the stack
void runLoopCallback() THRIFT_NOEXCEPT;
boost::shared_ptr<apache::thrift::TProcessor> getProcessor() const {
return processor_;
}
boost::shared_ptr<apache::thrift::protocol::TProtocol>
getInputProtocol() const {
return inputProtocol_;
}
boost::shared_ptr<apache::thrift::protocol::TProtocol>
getOutputProtocol() const {
return outputProtocol_;
}
/// Get the per-server event handler set for this server, if any
boost::shared_ptr<apache::thrift::server::TServerEventHandler>
getServerEventHandler() const {
return serverEventHandler_;
}
/// Get the TConnectionContext for this connection
server::TConnectionContext* getConnectionContext() {
return &context_;
}
/// Destructor -- close down the connection.
~TEventConnection();
/**
* Check the size of our memory buffers and resize if needed. Do not call
* when a call is in progress.
*/
void checkBufferMemoryUsage();
private:
class ConnContext : public server::TConnectionContext {
public:
void init(const transport::TSocketAddress* address,
boost::shared_ptr<protocol::TProtocol> inputProtocol,
boost::shared_ptr<protocol::TProtocol> outputProtocol) {
address_ = *address;
inputProtocol_ = inputProtocol;
outputProtocol_ = outputProtocol;
}
virtual const transport::TSocketAddress* getPeerAddress() const {
return &address_;
}
void reset() {
address_.reset();
cleanupUserData();
}
// TODO(dsanduleac): implement the virtual getInputProtocol() & such
virtual boost::shared_ptr<protocol::TProtocol> getInputProtocol() const {
// from TEventConnection
return inputProtocol_;
}
virtual boost::shared_ptr<protocol::TProtocol> getOutputProtocol() const {
return outputProtocol_;
}
private:
transport::TSocketAddress address_;
boost::shared_ptr<protocol::TProtocol> inputProtocol_;
boost::shared_ptr<protocol::TProtocol> outputProtocol_;
};
void readNextRequest();
void handleReadSuccess();
void handleReadFailure();
void handleAsyncTaskComplete(bool success);
void handleSendSuccess();
void handleSendFailure();
void handleEOF();
void handleFailure(const char* msg);
void cleanup();
//! The worker instance handling this connection.
TEventWorker* worker_;
//! This connection's socket.
boost::shared_ptr<TAsyncSocket> asyncSocket_;
//! Transport that the processor reads from.
boost::shared_ptr<apache::thrift::transport::TMemoryBuffer> inputTransport_;
//! Transport that the processor writes to.
boost::shared_ptr<apache::thrift::transport::TMemoryBuffer> outputTransport_;
/// Largest size of read buffer seen since buffer was constructed
size_t largestReadBufferSize_;
/// Largest size of write buffer seen since buffer was constructed
size_t largestWriteBufferSize_;
/// Count of the number of calls for use with getResizeBufferEveryN().
int32_t callsForResize_;
//! Protocol decoder.
boost::shared_ptr<apache::thrift::protocol::TProtocol> inputProtocol_;
//! Protocol encoder.
boost::shared_ptr<apache::thrift::protocol::TProtocol> outputProtocol_;
//! Channel that actually performs the socket I/O and callbacks.
TAsyncEventChannel* asyncChannel_;
/// Count of outstanding processor callbacks (generally 0 or 1).
int32_t processorActive_;
//! Count of the number of handleReadSuccess frames on the stack
int32_t readersActive_;
/// Sync processor if we're in queuing mode
boost::shared_ptr<apache::thrift::TProcessor> processor_;
/// Flag used to shut down connection (used for load-shedding mechanism).
bool shutdown_;
/// Flag indicating that we have deferred closing down (processor was active)
bool closing_;
/// The per-server event handler set for this erver, if any
boost::shared_ptr<apache::thrift::server::TServerEventHandler>
serverEventHandler_;
/// per-connection context
ConnContext context_;
/// Our processor
boost::shared_ptr<TAsyncProcessor> asyncProcessor_;
/// So that TEventWorker can call handleAsyncTaskComplete();
friend class TEventWorker;
/// Make the server a friend so it can manage tasks when overloaded
friend class TEventServer;
/// Make an async task a friend so it can communicate a cleanup() to us.
friend class TEventTask;
};
}}} // apache::thrift::async
#endif // #ifndef THRIFT_ASYNC_TEVENTCONNECTION_H_

View File

@ -0,0 +1,182 @@
/*
* 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 THRIFT_ASYNC_TEVENTHANDLER_H_
#define THRIFT_ASYNC_TEVENTHANDLER_H_ 1
#include "thrift/lib/cpp/thrift_config.h"
#include "thrift/lib/cpp/async/TEventUtil.h"
#include <boost/noncopyable.hpp>
#include <stddef.h>
namespace apache { namespace thrift { namespace async {
class TEventBase;
/**
* The TEventHandler class is used to asynchronously wait for events on a file
* descriptor.
*
* Users that wish to wait on I/O events should derive from TEventHandler and
* implement the handlerReady() method.
*/
class TEventHandler : private boost::noncopyable {
public:
enum EventFlags {
NONE = 0,
READ = EV_READ,
WRITE = EV_WRITE,
READ_WRITE = (READ | WRITE),
PERSIST = EV_PERSIST
};
/**
* Create a new TEventHandler object.
*
* @param eventBase The TEventBase to use to drive this event handler.
* This may be NULL, in which case the TEventBase must be
* set separately using initHandler() or attachEventBase()
* before the handler can be registered.
* @param fd The file descriptor that this TEventHandler will
* monitor. This may be -1, in which case the file
* descriptor must be set separately using initHandler() or
* changeHandlerFD() before the handler can be registered.
*/
explicit TEventHandler(TEventBase* eventBase = NULL, int fd = -1);
/**
* TEventHandler destructor.
*
* The event will be automatically unregistered if it is still registered.
*/
virtual ~TEventHandler();
/**
* handlerReady() is invoked when the handler is ready.
*
* @param events A bitset indicating the events that are ready.
*/
virtual void handlerReady(uint16_t events) THRIFT_NOEXCEPT = 0;
/**
* Register the handler.
*
* If the handler is already registered, the registration will be updated
* to wait on the new set of events.
*
* @param events A bitset specifying the events to monitor.
* If the PERSIST bit is set, the handler will remain
* registered even after handlerReady() is called.
*
* @return Returns true if the handler was successfully registered,
* or false if an error occurred. After an error, the handler is
* always unregistered, even if it was already registered prior to
* this call to registerHandler().
*/
bool registerHandler(uint16_t events) {
return registerImpl(events, false);
}
/**
* Unregister the handler, if it is registered.
*/
void unregisterHandler();
/**
* Returns true if the handler is currently registered.
*/
bool isHandlerRegistered() const {
return TEventUtil::isEventRegistered(&event_);
}
/**
* Attach the handler to a TEventBase.
*
* This may only be called if the handler is not currently attached to a
* TEventBase (either by using the default constructor, or by calling
* detachEventBase()).
*
* This method must be invoked in the TEventBase's thread.
*/
void attachEventBase(TEventBase* eventBase);
/**
* Detach the handler from its TEventBase.
*
* This may only be called when the handler is not currently registered.
* Once detached, the handler may not be registered again until it is
* re-attached to a TEventBase by calling attachEventBase().
*
* This method must be called from the current TEventBase's thread.
*/
void detachEventBase();
/**
* Change the file descriptor that this handler is associated with.
*
* This may only be called when the handler is not currently registered.
*/
void changeHandlerFD(int fd);
/**
* Attach the handler to a TEventBase, and change the file descriptor.
*
* This method may only be called if the handler is not currently attached to
* a TEventBase. This is primarily intended to be used to initialize
* TEventHandler objects created using the default constructor.
*/
void initHandler(TEventBase* eventBase, int fd);
/**
* Return the set of events that we're currently registered for.
*/
uint16_t getRegisteredEvents() const {
return (isHandlerRegistered()) ?
event_.ev_events : 0;
}
/**
* Register the handler as an internal event.
*
* This event will not count as an active event for determining if the
* TEventBase loop has more events to process. The TEventBase loop runs
* only as long as there are active TEventHandlers, however "internal" event
* handlers are not counted. Therefore this event handler will not prevent
* TEventBase loop from exiting with no more work to do if there are no other
* non-internal event handlers registered.
*
* This is intended to be used only in very rare cases by the internal
* TEventBase code. This API is not guaranteed to remain stable or portable
* in the future.
*/
bool registerInternalHandler(uint16_t events) {
return registerImpl(events, true);
}
private:
bool registerImpl(uint16_t events, bool internal);
void ensureNotRegistered(const char* fn);
static void libeventCallback(int fd, short events, void* arg);
struct event event_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TEVENTHANDLER_H_

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,49 @@
#ifndef _THRIFT_TEVENTTASK_H_
#define _THRIFT_TEVENTTASK_H_ 1
#include "thrift/lib/cpp/Thrift.h"
#include "thrift/lib/cpp/server/TServer.h"
#include "thrift/lib/cpp/transport/TBufferTransports.h"
#include "thrift/lib/cpp/concurrency/ThreadManager.h"
#include "thrift/lib/cpp/async/TEventConnection.h"
#include <tr1/functional>
#include <boost/shared_ptr.hpp>
#include "thrift/lib/cpp/TProcessor.h"
#include "thrift/lib/cpp/protocol/TProtocol.h"
namespace apache { namespace thrift { namespace async {
class TEventTask : public apache::thrift::concurrency::Runnable {
public:
explicit TEventTask(TEventConnection* connection);
void run();
TEventConnection* getConnection() const {
return connection_;
}
private:
boost::shared_ptr<apache::thrift::server::TProcessor> processor_;
boost::shared_ptr<apache::thrift::protocol::TProtocol> input_;
boost::shared_ptr<apache::thrift::protocol::TProtocol> output_;
TEventConnection* connection_;
TConnectionContext* connectionContext_;
};
class TaskCompletionMessage {
public:
explicit TaskCompletionMessage(TEventConnection *inConnection)
: connection(inConnection) {}
TaskCompletionMessage(TaskCompletionMessage &&msg)
: connection(msg.connection) {
msg.connection = NULL;
}
TEventConnection *connection;
};
} } } // namespace apache::thrift::async
#endif // !_THRIFT_TEVENTTASK_H_

View File

@ -0,0 +1,43 @@
/*
* 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 THRIFT_ASYNC_TEVENTUTIL_H_
#define THRIFT_ASYNC_TEVENTUTIL_H_ 1
#include <event.h> // libevent
namespace apache { namespace thrift { namespace async {
/**
* low-level libevent utility functions
*/
class TEventUtil {
public:
static bool isEventRegistered(const struct event* ev) {
// If any of these flags are set, the event is registered.
enum {
EVLIST_REGISTERED = (EVLIST_INSERTED | EVLIST_ACTIVE |
EVLIST_TIMEOUT | EVLIST_SIGNAL)
};
return (ev->ev_flags & EVLIST_REGISTERED);
}
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TEVENTUTIL_H_

View File

@ -0,0 +1,264 @@
// Copyright (c) 2006- Facebook
// Distributed under the Thrift Software License
//
// See accompanying file LICENSE or visit the Thrift site at:
// http://developers.facebook.com/thrift/
#ifndef THRIFT_SERVER_TEVENTWORKER_H_
#define THRIFT_SERVER_TEVENTWORKER_H_ 1
#include "thrift/lib/cpp/async/TAsyncServerSocket.h"
#include "thrift/lib/cpp/async/TAsyncSSLSocket.h"
#include "thrift/lib/cpp/async/TEventServer.h"
#include "thrift/lib/cpp/async/TEventBase.h"
#include "thrift/lib/cpp/async/TEventHandler.h"
#include "thrift/lib/cpp/async/TNotificationQueue.h"
#include "thrift/lib/cpp/server/TServer.h"
#include <ext/hash_map>
#include <list>
#include <stack>
namespace apache { namespace thrift { namespace async {
// Forward declaration of classes
class TAsyncProcessorFactory;
class TEventConnection;
class TEventServer;
class TaskCompletionMessage;
/**
* TEventWorker drives the actual I/O for TEventServer connections.
*
* The TEventServer itself accepts incoming connections, then hands off each
* connection to a TEventWorker running in another thread. There should
* typically be around one TEventWorker thread per core.
*/
class TEventWorker :
public apache::thrift::server::TServer,
public TAsyncServerSocket::AcceptCallback,
public TAsyncSSLSocket::HandshakeCallback,
public TNotificationQueue<TaskCompletionMessage>::Consumer {
private:
/// Object that processes requests.
boost::shared_ptr<TAsyncProcessorFactory> asyncProcessorFactory_;
/// The mother ship.
TEventServer* server_;
/// An instance's TEventBase for I/O.
TEventBase eventBase_;
/// Our ID in [0:nWorkers).
uint32_t workerID_;
/// Pipe that task completion notifications are sent over
TNotificationQueue<TaskCompletionMessage> notificationQueue_;
/**
* A stack of idle TEventConnection objects for reuse.
* When we close a connection, we place it on this stack so that the
* object can be reused later, rather than freeing the memory and
* reallocating a new object later.
*/
std::stack<TEventConnection*> connectionStack_;
/// Transport type to use
TEventServer::TransportType transportType_;
/**
* Called when the connection is fully accepted (after SSL accept if needed)
*/
void finishConnectionAccepted(TAsyncSocket *asyncSock);
/**
* Create or reuse a TEventConnection initialized for the given socket FD.
*
* @param socket the FD of a freshly-connected socket.
* @param address the peer address of the socket.
* @return pointer to a TConenction object for that socket.
*/
TEventConnection* createConnection(
boost::shared_ptr<TAsyncSocket> asyncSocket,
const transport::TSocketAddress* address);
/**
* Handler called when a new connection may be available.
*/
void acceptConnections();
void makeCompletionCallback();
/**
* Initialize our TEventBase to generate incoming connection events.
* Note that this is called once before the main loop is executed and
* sets up a READ event on the output of the listener's socktpair.
*/
void registerEvents();
/**
* Callback used when loop latency exceeds the requested threshold.
*/
void maxLatencyCob();
typedef std::list<TEventConnection*> ConnectionList;
// pointer hash functor
static const uint64_t kPtrHashMult = 5700357409661599291LL;
static const uint64_t kPtrHashShift = 3;
template<typename T>
struct hash { };
template<typename T>
struct hash<T*> {
size_t operator()(T* p) const {
return ((size_t)p ^ ((size_t)p >> kPtrHashShift)) * kPtrHashMult;
}
size_t operator()(const T* p) const {
return ((size_t)p ^ ((size_t)p >> kPtrHashShift)) * kPtrHashMult;
}
};
typedef __gnu_cxx::hash_map<const TEventConnection*,
ConnectionList::iterator,
hash<TEventConnection*> > ConnectionMap;
/**
* The list of active connections (used to allow the oldest connections
* to be shed during overload).
*/
ConnectionList activeConnectionList_;
/**
* A hash map used to map connections to their place in the connection list
*/
ConnectionMap activeConnectionMap_;
public:
/**
* TEventWorker is the actual server object for existing connections.
* One or more of these should be created by TEventServer (one per
* CPU core is recommended).
*
* @param processorFactory a TAsyncProcessorFactory object as
* obtained from the generated Thrift code (the user service
* is integrated through this).
* @param inputProtocolFactory the TProtocolFactory class supporting
* inbound Thrift requests.
* @param outputProtocolFactory the TProtocolFactory class supporting
* responses (if any) after processing completes.
* @param server the TEventServer which created us.
* @param workerID the ID assigned to this worker
*/
TEventWorker(boost::shared_ptr<TAsyncProcessorFactory> processorFactory,
boost::shared_ptr<apache::thrift::server::TDuplexProtocolFactory>
protocolFactory,
TEventServer* server,
uint32_t workerID) :
TServer(boost::shared_ptr<apache::thrift::server::TProcessor>()),
asyncProcessorFactory_(processorFactory),
server_(server),
eventBase_(),
workerID_(workerID),
transportType_(TEventServer::FRAMED) {
setDuplexProtocolFactory(protocolFactory);
transportType_ = server->getTransportType();
if (transportType_ == TEventServer::FRAMED) {
// do the dynamic cast once rather than per connection
if (dynamic_cast<apache::thrift::protocol::THeaderProtocolFactory*>(
protocolFactory.get())) {
transportType_ = TEventServer::HEADER;
}
}
}
/**
* Destroy a TEventWorker. We've use boost::scoped_ptr<> to take care
* of freeing up memory, so nothing to be done here but release the
* connection stack.
*/
virtual ~TEventWorker();
/**
* Get my TAsyncProcessorFactory object.
*
* @returns pointer to my TAsyncProcessorFactory object.
*/
boost::shared_ptr<TAsyncProcessorFactory> getAsyncProcessorFactory() {
return asyncProcessorFactory_;
}
/**
* Get my TEventBase object.
*
* @returns pointer to my TEventBase object.
*/
TEventBase* getEventBase() {
return &eventBase_;
}
/**
* Get underlying server.
*
* @returns pointer to TEventServer
*/
TEventServer* getServer() const {
return server_;
}
/**
* Get my numeric worker ID (for diagnostics).
*
* @return integer ID of this worker
*/
int32_t getID() {
return workerID_;
}
/**
* Dispose of a TEventConnection object.
* Will add to a pool of these objects or destroy as necessary.
*
* @param connection a now-idle connection.
*/
void returnConnection(TEventConnection* connection);
/**
* Cause a completion callback for the requested connection to occur
* within that connection's context.
*
* @param msg task completion message
* @return true if notification was sent, false if it failed.
*/
bool notifyCompletion(TaskCompletionMessage &&msg);
/**
* Task completed (called in this worker's thread)
*/
void messageAvailable(TaskCompletionMessage &&msg);
virtual void connectionAccepted(int fd,
const transport::TSocketAddress& clientAddr)
THRIFT_NOEXCEPT;
virtual void acceptError(const std::exception& ex) THRIFT_NOEXCEPT;
virtual void acceptStopped() THRIFT_NOEXCEPT;
/**
* TAsyncSSLSocket::HandshakeCallback interface
*/
void handshakeSuccess(TAsyncSSLSocket *sock) THRIFT_NOEXCEPT;
void handshakeError(TAsyncSSLSocket *sock,
const transport::TTransportException& ex) THRIFT_NOEXCEPT;
/**
* Enter event loop and serve.
*/
void serve();
/**
* Exit event loop.
*/
void shutdown();
};
}}} // apache::thrift::async
#endif // #ifndef THRIFT_SERVER_TEVENTWORKER_H_

View File

@ -0,0 +1,204 @@
/*
* 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 THRIFT_ASYNC_TFRAMEDASYNCCHANNEL_H_
#define THRIFT_ASYNC_TFRAMEDASYNCCHANNEL_H_ 1
#include "thrift/lib/cpp/async/TStreamAsyncChannel.h"
namespace apache { namespace thrift { namespace async {
namespace detail {
/**
* Encapsulation of one outstanding write request on a TFramedAsyncChannel.
*/
class TFramedACWriteRequest :
public TAsyncChannelWriteRequestBase<TFramedACWriteRequest> {
public:
TFramedACWriteRequest(const VoidCallback& callback,
const VoidCallback& errorCallback,
transport::TMemoryBuffer* message,
TAsyncEventChannel* channel);
void write(TAsyncTransport* transport,
TAsyncTransport::WriteCallback* callback) THRIFT_NOEXCEPT;
void writeSuccess() THRIFT_NOEXCEPT;
void writeError(size_t bytesWritten,
const transport::TTransportException& ex) THRIFT_NOEXCEPT;
private:
union {
uint32_t frameSize_;
char frameSizeBuf_[sizeof(uint32_t)];
};
};
/**
* Read state for TFramedAsyncChannel
*/
class TFramedACReadState {
public:
typedef std::tr1::function<void()> VoidCallback;
TFramedACReadState();
// Methods required by TStreamAsyncChannel
void setCallbackBuffer(transport::TMemoryBuffer* buffer) {
buffer_ = buffer;
bytesRead_ = 0;
}
void unsetCallbackBuffer() {
buffer_ = NULL;
}
bool hasReadAheadData() {
assert(bytesRead_ == 0);
return false;
}
bool hasPartialMessage() {
return (bytesRead_ > 0);
}
void getReadBuffer(void** bufReturn, size_t* lenReturn);
bool readDataAvailable(size_t len);
// Other methods specifict to TFramedAsyncChannel
void setMaxFrameSize(uint32_t size) {
maxFrameSize_ = size;
}
uint32_t getMaxFrameSize() const {
return maxFrameSize_;
}
private:
/// maximum frame size accepted
uint32_t maxFrameSize_;
union {
uint32_t frameSize_;
char frameSizeBuf_[sizeof(uint32_t)];
};
/**
* The number of bytes read.
*
* This includes the bytes in the frame size. When bytesRead_ is less than
* sizeof(uint32_t), we are still reading the frame size. Otherwise, we have
* read bytesRead_ - sizeof(uint32_t) bytes of the body.
*/
uint32_t bytesRead_;
apache::thrift::transport::TMemoryBuffer* buffer_;
};
} // namespace detail
/**
* TFramedAsyncChannel
*
* This is a TAsyncChannel implementation that reads and writes messages
* prefixed with a 4-byte frame length.
*
* Its messages are compatible with TFramedTransport.
*/
class TFramedAsyncChannel :
public TStreamAsyncChannel<detail::TFramedACWriteRequest,
detail::TFramedACReadState> {
private:
typedef TStreamAsyncChannel<detail::TFramedACWriteRequest,
detail::TFramedACReadState> Parent;
public:
explicit TFramedAsyncChannel(
const boost::shared_ptr<TAsyncTransport>& transport
)
: Parent(transport) {}
/**
* Helper function to create a shared_ptr<TFramedAsyncChannel>.
*
* This passes in the correct destructor object, since TFramedAsyncChannel's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<TFramedAsyncChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
return boost::shared_ptr<TFramedAsyncChannel>(
new TFramedAsyncChannel(transport), Destructor());
}
/// size in bytes beyond which we'll reject a given frame size.
void setMaxFrameSize(uint32_t size) {
readState_.setMaxFrameSize(size);
}
uint32_t getMaxFrameSize() const {
return readState_.getMaxFrameSize();
}
protected:
/**
* Protected destructor.
*
* Users of TFramedAsyncChannel must never delete it directly. Instead,
* invoke destroy().
*/
virtual ~TFramedAsyncChannel() { }
};
class TFramedAsyncChannelFactory : public TStreamAsyncChannelFactory {
public:
TFramedAsyncChannelFactory()
: maxFrameSize_(0x7fffffff)
, recvTimeout_(0)
, sendTimeout_(0) {}
void setMaxFrameSize(uint32_t bytes) {
maxFrameSize_ = bytes;
}
void setRecvTimeout(uint32_t milliseconds) {
recvTimeout_ = milliseconds;
}
void setSendTimeout(uint32_t milliseconds) {
sendTimeout_ = milliseconds;
}
virtual boost::shared_ptr<TAsyncEventChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
boost::shared_ptr<TFramedAsyncChannel> channel(
TFramedAsyncChannel::newChannel(transport));
transport->setSendTimeout(sendTimeout_);
channel->setMaxFrameSize(maxFrameSize_);
channel->setRecvTimeout(recvTimeout_);
return channel;
}
private:
uint32_t maxFrameSize_;
uint32_t recvTimeout_;
uint32_t sendTimeout_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TFRAMEDASYNCCHANNEL_H_

View File

@ -0,0 +1,137 @@
/*
* 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 THRIFT_ASYNC_THEADERASYNCCHANNEL_H_
#define THRIFT_ASYNC_THEADERASYNCCHANNEL_H_ 1
#include "thrift/lib/cpp/async/TUnframedAsyncChannel.h"
namespace apache { namespace thrift { namespace async {
namespace detail {
/**
* A class to determine the end of a THeaderProtocol message. This is not as
* sophisticated as the logic in THeaderTransport, so it cannot yet handle any
* unframed transports, just THeader and TFramed. However, the previous
* implementation used TFramedAsyncChannel, so the limitation is not new.
*/
class THeaderACProtocolTraits {
public:
THeaderACProtocolTraits()
: maxMessageSize_(0x7ffffff) {}
// Methods required by TUnframedACReadState
bool getMessageLength(uint8_t* buffer,
uint32_t bufferLength,
uint32_t* messageLength);
void setMaxMessageSize(uint32_t maxMessageSize) {
maxMessageSize_ = maxMessageSize;
}
private:
uint32_t maxMessageSize_;
};
} // namespace detail
/**
* THeaderAsyncChannel
*
* This is a TAsyncChannel implementation that reads and writes
* messages encoded using THeaderProtocol.
*/
class THeaderAsyncChannel :
public TUnframedAsyncChannel<detail::THeaderACProtocolTraits> {
private:
typedef TUnframedAsyncChannel<detail::THeaderACProtocolTraits> Parent;
public:
explicit THeaderAsyncChannel(
const boost::shared_ptr<TAsyncTransport>& transport)
: Parent(transport) {}
/**
* Helper function to create a shared_ptr<THeaderAsyncChannel>.
*
* This passes in the correct destructor object, since THeaderAsyncChannel's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<THeaderAsyncChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
return boost::shared_ptr<THeaderAsyncChannel>(
new THeaderAsyncChannel(transport), Destructor());
}
void setMaxMessageSize(uint32_t size) {
Parent::setMaxMessageSize(size);
readState_.getProtocolTraits()->setMaxMessageSize(size);
}
// Note that we inherit getMaxMessageSize() from TUnframedAsyncChannel.
protected:
/**
* Protected destructor.
*
* Users of THeaderAsyncChannel must never delete it directly. Instead,
* invoke destroy().
*/
virtual ~THeaderAsyncChannel() { }
};
class THeaderAsyncChannelFactory : public TStreamAsyncChannelFactory {
public:
THeaderAsyncChannelFactory()
: maxMessageSize_(0x7fffffff)
, recvTimeout_(0)
, sendTimeout_(0) {}
void setMaxMessageSize(uint32_t bytes) {
maxMessageSize_ = bytes;
}
void setRecvTimeout(uint32_t milliseconds) {
recvTimeout_ = milliseconds;
}
void setSendTimeout(uint32_t milliseconds) {
sendTimeout_ = milliseconds;
}
virtual boost::shared_ptr<TAsyncEventChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
boost::shared_ptr<THeaderAsyncChannel> channel(
THeaderAsyncChannel::newChannel(transport));
transport->setSendTimeout(sendTimeout_);
channel->setMaxMessageSize(maxMessageSize_);
channel->setRecvTimeout(recvTimeout_);
return channel;
}
private:
uint32_t maxMessageSize_;
uint32_t recvTimeout_;
uint32_t sendTimeout_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_THEADERASYNCCHANNEL_H_

View File

@ -0,0 +1,204 @@
/*
* 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 THRIFT_ASYNC_THTTPASYNCCHANNEL_H_
#define THRIFT_ASYNC_THTTPASYNCCHANNEL_H_ 1
#include "thrift/lib/cpp/async/TStreamAsyncChannel.h"
#include "thrift/lib/cpp/util/THttpParser.h"
namespace apache { namespace thrift { namespace async {
class THttpAsyncChannel;
namespace detail {
/**
* Encapsulation of one outstanding write request on a THttpAsyncChannel.
*/
class THttpACWriteRequest :
public TAsyncChannelWriteRequestBase<THttpACWriteRequest> {
public:
typedef std::tr1::function<void()> VoidCallback;
THttpACWriteRequest(const VoidCallback& callback,
const VoidCallback& errorCallback,
transport::TMemoryBuffer* message,
TAsyncEventChannel* channel);
void write(TAsyncTransport* transport,
TAsyncTransport::WriteCallback* callback) THRIFT_NOEXCEPT;
void writeSuccess() THRIFT_NOEXCEPT;
void writeError(size_t bytesWritten,
const transport::TTransportException& ex) THRIFT_NOEXCEPT;
private:
char lengthBuf_[64];
THttpAsyncChannel* channel_;
};
/**
* Read state for THttpAsyncChannel
*/
class THttpACReadState {
public:
typedef std::tr1::function<void()> VoidCallback;
THttpACReadState() {
}
// Methods required by TStreamAsyncChannel
void setCallbackBuffer(transport::TMemoryBuffer* buffer) {
parser_->setDataBuffer(buffer);
}
void unsetCallbackBuffer() {
parser_->unsetDataBuffer();
}
bool hasReadAheadData() {
return parser_->hasReadAheadData();
}
bool hasPartialMessage() {
return parser_->hasPartialMessage();
}
void getReadBuffer(void** bufReturn, size_t* lenReturn);
bool readDataAvailable(size_t len);
// Other methods specific to THttpAsyncChannel
void setParser(boost::shared_ptr<apache::thrift::util::THttpParser> parser) {
parser_ = parser;
}
private:
boost::shared_ptr<apache::thrift::util::THttpParser> parser_;
};
} // namespace detail
/**
* THttpAsyncChannel
*
* This is a TAsyncChannel implementation that reads and writes messages
* encapuated in HTTP.
*
* Its messages are compatible with THttpTransport.
*/
class THttpAsyncChannel :
public TStreamAsyncChannel<detail::THttpACWriteRequest,
detail::THttpACReadState> {
private:
typedef TStreamAsyncChannel<detail::THttpACWriteRequest,
detail::THttpACReadState> Parent;
boost::shared_ptr<apache::thrift::util::THttpParser> parser_;
public:
explicit THttpAsyncChannel(
const boost::shared_ptr<TAsyncTransport>& transport)
: Parent(transport) {
}
/**
* Helper function to create a shared_ptr<THttpAsyncChannel>.
*
* This passes in the correct destructor object, since THttpAsyncChannel's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<THttpAsyncChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
return boost::shared_ptr<THttpAsyncChannel>(
new THttpAsyncChannel(transport), Destructor());
}
/// size in bytes beyond which we'll reject a given http size.
void setMaxHttpSize(uint32_t size) {
parser_->setMaxSize(size);
}
uint32_t getMaxHttpSize() const {
return parser_->getMaxSize();
}
void setParser(boost::shared_ptr<apache::thrift::util::THttpParser> parser) {
parser_ = parser;
readState_.setParser(parser);
}
boost::shared_ptr<apache::thrift::util::THttpParser> getParser() const {
return parser_;
}
int constructHeader(iovec* ops,
int opsLen,
int contentLength,
char* contentLengthBuf) {
return parser_->constructHeader(ops,
opsLen,
contentLength,
contentLengthBuf);
}
protected:
/**
* Protected destructor.
*
* Users of THttpAsyncChannel must never delete it directly. Instead,
* invoke destroy().
*/
virtual ~THttpAsyncChannel() { }
};
class THttpAsyncChannelFactory : public TStreamAsyncChannelFactory {
public:
THttpAsyncChannelFactory()
: maxHttpSize_(0x7fffffff)
, recvTimeout_(0)
, sendTimeout_(0) {}
void setMaxHttpSize(uint32_t bytes) {
maxHttpSize_ = bytes;
}
void setRecvTimeout(uint32_t milliseconds) {
recvTimeout_ = milliseconds;
}
void setSendTimeout(uint32_t milliseconds) {
sendTimeout_ = milliseconds;
}
virtual boost::shared_ptr<TAsyncEventChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
boost::shared_ptr<THttpAsyncChannel> channel(
THttpAsyncChannel::newChannel(transport));
transport->setSendTimeout(sendTimeout_);
channel->setMaxHttpSize(maxHttpSize_);
channel->setRecvTimeout(recvTimeout_);
return channel;
}
private:
uint32_t maxHttpSize_;
uint32_t recvTimeout_;
uint32_t sendTimeout_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_THTTPASYNCCHANNEL_H_

View File

@ -0,0 +1,337 @@
/*
* 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 THRIFT_ASYNC_TNOTIFICATIONPIPE_H
#define THRIFT_ASYNC_TNOTIFICATIONPIPE_H 1
#include "thrift/lib/cpp/async/TDelayedDestruction.h"
#include "thrift/lib/cpp/async/TEventBase.h"
#include "thrift/lib/cpp/async/TEventHandler.h"
#include "thrift/lib/cpp/concurrency/Mutex.h"
#include <boost/shared_ptr.hpp>
#include <exception>
#include <limits.h>
namespace apache { namespace thrift { namespace async {
/**
* A simple notification pipe for sending messages to a TEventBase thread.
*
* TNotificationPipe is a unidirectional pipe for sending small, atomic
* messages.
*
* TNotificationPipe cannot be send messages larger than a fixed size.
* TNotificationPipe::kMaxMessageSize defines the maximum message size
* supported. If you need to pass larger amounts of data between threads,
* consider just passing a pointer to the data over the pipe, and using some
* external mechanism to synchronize management of the memory.
*
*
* TNotificationPipe provides two parallel APIs for writing and closing the
* pipe: a thread-safe version and a non-thread-safe version. Which version to
* use depends on how the caller uses the pipe:
*
* - If there is only a single writer thread, you can use the non-thread-safe
* versions of trySendMessage() and close(). This guarantees close() is
* never called by one thread while another thread is attempting to send a
* message.
*
* - If there are multiple writers, but the pipe is never closed by the
* writers, you can use the non-thread-safe version of trySendMessage().
* Multiple simultaneous trySendMessage() calls will not interfere with each
* other. Since none of the writer threads call close, a call to close()
* cannot be running simultaneously with a write attempt. (With this model,
* the TNotificationPipe is never closed until it is destroyed. It is up to
* the caller to ensure the TNotificationPipe is not destroyed while write
* threads still have a pointer or reference to it.)
*
* In other circumstances (if one thread may call close while another thread is
* simultaneously trying to write), the thread-safe versions
* trySendMessageSync() and closeSync() must be used.
*/
class TNotificationPipe : public TDelayedDestruction,
private TEventHandler,
private TEventBase::LoopCallback {
public:
/**
* A callback interface for receiving notification of messages from the pipe.
*/
class Callback {
public:
virtual ~Callback() {}
/**
* notificationMessage() will be invoked whenever a new
* message is available from the pipe.
*/
virtual void notificationMessage(const void *msg, uint32_t msgSize) = 0;
/**
* notificationPipeError() will be invoked if an error occurs while reading
* from the pipe. Before notificationPipeError() is invoked, the read
* callback will automatically be uninstalled and the pipe will be closed.
*/
virtual void notificationPipeError(const std::exception& ex) = 0;
/**
* notificationPipeClosed() is invoked in the read thread after the write
* end of the pipe is closed.
*/
virtual void notificationPipeClosed() = 0;
};
/**
* Helper function to create a new shared_ptr<TNotificationPipe>.
*
* This simply sets the correct destructor to call destroy() instead of
* directly deleting the TNotificationPipe.
*/
static boost::shared_ptr<TNotificationPipe> newPipe(TEventBase *base) {
return boost::shared_ptr<TNotificationPipe>(new TNotificationPipe(base),
Destructor());
}
/**
* Create a new TNotificationPipe.
*
* @param eventBase The TEventBase to use for receiving read notifications
* from this pipe. All read events will be processed in this
* TEventBase's thread. trySendMessage() may be called from any thread.
*/
TNotificationPipe(TEventBase *eventBase);
/**
* Destroy this TNotificationPipe.
*
* This method may only be called from the read thread.
*
* This will automatically close the pipe if it is not already closed.
*/
virtual void destroy();
/**
* Close the pipe.
*
* This version of close() is not thread-safe. It should only be used if the
* caller is sure no other thread is attempting to write a message at the
* same time.
*
* Use closeSync() if other threads may be attempting to send a message
* simultaneously. The other threads must use also use the thread-safe
* trySendMessageSync() or trySendFrameSync() calls.
*/
void close();
/**
* A thread-safe version of close().
*/
void closeSync();
/**
* Send a message over the pipe.
*
* trySendMessage() is best-effort. It will either immediately succeed to
* send the message, or it will fail immediately if the pipe reader is too
* busy and it's backlog of unread messages is too large.
*
* trySendMessage() also does not support arbitrarily large messages.
* It will also fail immediately if msgSize is larger than (PIPE_BUF - 4).
*
* If trySendMessage() succeeds, the message is guaranteed to be delivered to
* the pipe reader, except in the case where the pipe reader explicitly stops
* reading and destroys the pipe before processing all of its messages.
*
* On failure a TTransportException is thrown. The error code will be
* TTransportException::BAD_ARGS if the message is too large,
* TTransportException::TIMED_OUT if the message cannot be sent right now
* because the pipe is full, or TTransportException::NOT_OPEN if the pipe has
* already been closed.
*
* This method is thread safe with other simultaneous trySendMessage() calls,
* but not with close() calls. Use trySendMessageSync() and closeSync() if a
* close may occur simultaneously on another thread.
*/
void trySendMessage(const void *msg, uint32_t msgSize);
/**
* A thread-safe version of trySendMessage().
*
* This may be called simultaneously with closeSync().
*/
void trySendMessageSync(const void *msg, uint32_t msgSize);
/**
* Send a message over the pipe.
*
* This is identical to trySendMessage(), except that the caller must provide
* 4 bytes at the beginning of the message where we can write a frame length.
* This allows us to avoid copying the message into a new buffer.
* (trySendMessage() always has to make a copy of the message.)
*
* @param frame A pointer to the frame buffer. trySendFrame() will
* overwrite the first 4 bytes of this buffer. When the read callback
* receives the message, it will not see these first 4 bytes.
* @param frameSize The full size of the frame buffer. This must be at
* least 4 bytes long. The actual message size that will be sent is
* frameSize - 4.
*/
void trySendFrame(void *frame, uint32_t frameSize);
/**
* A thread-safe version of trySendFrame().
*
* This may be called simultaneously with closeSync().
*/
void trySendFrameSync(void *frame, uint32_t frameSize);
/**
* Get the number of messages which haven't been processed.
*/
int64_t getNumNotProcessed() const {
return numInputs_ - numOutputs_;
}
/**
* Set the callback to receive read notifications from this pipe.
*
* This method must be invoked from the pipe's read thread.
*
* May throw TLibraryException on error. The callback will always be unset
* (NULL) after an error.
*/
void setReadCallback(Callback *callback);
/**
* Mark the pipe read event handler as an "internal" event handler.
*
* This causes the notification pipe not to be counted when determining if
* the TEventBase has any more active events to wait on. This is intended to
* be used only be internal TEventBase code. This API is not guaranteed to
* remain stable or portable in the future.
*
* May throw TLibraryException if it fails to re-register its event handler
* with the correct flags.
*/
void setInternal(bool internal);
/**
* Get the maximum number of messages that will be read on a single iteration
* of the event loop.
*/
uint32_t getMaxReadAtOnce() const {
return maxReadAtOnce_;
}
/**
* Set the maximum number of messages to read each iteration of the event
* loop.
*
* If messages are being received faster than they can be processed, this
* helps limit the rate at which they will be read. This can be used to
* prevent the notification pipe reader from starving other users of the
* event loop.
*/
void setMaxReadAtOnce(uint32_t numMessages) {
maxReadAtOnce_ = numMessages;
}
/**
* The maximum message size that can be sent over a TNotificationPipe.
*
* This restriction ensures that trySendMessage() can send all messages
* atomically. This is (PIPE_BUF - 4) bytes. (On Linux, this is 4092
* bytes.)
*/
static const uint32_t kMaxMessageSize = PIPE_BUF - 4;
/**
* The default maximum number of messages that will be read each time around
* the event loop.
*
* This value used for each TNotificationPipe can be changed using the
* setMaxReadAtOnce() method.
*/
static const uint32_t kDefaultMaxReadAtOnce = 10;
private:
enum ReadAction {
kDoNothing,
kContinue,
kWaitForRead,
kRunInNextLoop,
};
// Forbidden copy constructor and assignment opererator
TNotificationPipe(TNotificationPipe const &);
TNotificationPipe& operator=(TNotificationPipe const &);
// TEventHandler methods
virtual void handlerReady(uint16_t events) THRIFT_NOEXCEPT;
// TEventBase::LoopCallback methods
virtual void runLoopCallback() THRIFT_NOEXCEPT;
void initPipe();
void registerPipeEvent();
void readMessages(ReadAction action);
ReadAction performRead();
ReadAction processReadData(uint32_t* messagesProcessed);
ReadAction handleError(const char* fmt, ...)
__attribute__((format(printf, 2, 3)));
void checkMessage(uint32_t msgSize);
void writeFrame(const void *frame, uint32_t frameSize);
TEventBase *eventBase_;
Callback *readCallback_;
int readPipe_;
int writePipe_;
bool internal_;
uint32_t maxReadAtOnce_;
int64_t numInputs_;
int64_t numOutputs_;
/**
* Mutex for guarding numInputs_
*/
concurrency::Mutex numInputsMutex_;
/**
* A mutex that guards writePipe_.
*
* This is used by closeSync(), trySendMessageSync(), and trySendFrameSync(),
* since trySendMessageSync() and trySendFrameSync() read writePipe_
* and closeSync() resets it to -1.
*/
concurrency::NoStarveReadWriteMutex writePipeMutex_;
/**
* A pointer to the end of valid read data in the read buffer.
*/
uint8_t *readPtr_;
/**
* An internal read buffer
*
* This is large enough to contain the maximum possible message plus the
* mssage length.
*/
uint8_t readBuffer_[kMaxMessageSize + 4];
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TNOTIFICATIONPIPE_H

View File

@ -0,0 +1,592 @@
/*
* 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 THRIFT_ASYNC_TNOTIFICATIONQUEUE_H_
#define THRIFT_ASYNC_TNOTIFICATIONQUEUE_H_ 1
#include "thrift/lib/cpp/Thrift.h"
#include "thrift/lib/cpp/async/TEventBase.h"
#include "thrift/lib/cpp/async/TEventHandler.h"
#include "external/google_base/spinlock.h"
#include "external/glog/logging.h"
#include <deque>
#include "folly/eventfd.h"
namespace apache { namespace thrift { namespace async {
/**
* An exception class to be thrown when a TNotificationQueue is full.
*/
class TQueueFullException : public TLibraryException {
public:
TQueueFullException() :
TLibraryException("unable to add message to TNotificationQueue: "
"queue is full") {}
};
/**
* A producer-consumer queue for passing messages between TEventBase threads.
*
* Messages can be added to the queue from any thread. Multiple consumers may
* listen to the queue from multiple TEventBase threads.
*
* A TNotificationQueue may not be destroyed while there are still consumers
* registered to receive events from the queue. It is the user's
* responsibility to ensure that all consumers are unregistered before the
* queue is destroyed.
*
* MessageT should be MoveConstructible (i.e., must support either a move
* constructor or a copy constructor, or both). Ideally it's move constructor
* (or copy constructor if no move constructor is provided) should never throw
* exceptions. If the constructor may throw, the consumers could end up
* spinning trying to move a message off the queue and failing, and then
* retrying.
*/
template<typename MessageT>
class TNotificationQueue {
public:
/**
* A callback interface for consuming messages from the queue as they arrive.
*/
class Consumer : private TEventHandler {
public:
enum : uint16_t { kDefaultMaxReadAtOnce = 10 };
Consumer()
: queue_(NULL),
destroyedFlagPtr_(NULL),
maxReadAtOnce_(kDefaultMaxReadAtOnce) {}
virtual ~Consumer();
/**
* messageAvailable() will be invoked whenever a new
* message is available from the pipe.
*/
virtual void messageAvailable(MessageT&& message) = 0;
/**
* Begin consuming messages from the specified queue.
*
* messageAvailable() will be called whenever a message is available. This
* consumer will continue to consume messages until stopConsuming() is
* called.
*
* A Consumer may only consume messages from a single TNotificationQueue at
* a time. startConsuming() should not be called if this consumer is
* already consuming.
*/
void startConsuming(TEventBase* eventBase, TNotificationQueue* queue) {
init(eventBase, queue);
registerHandler(READ | PERSIST);
}
/**
* Same as above but registers this event handler as internal so that it
* doesn't count towards the pending reader count for the IOLoop.
*/
void startConsumingInternal(
TEventBase* eventBase, TNotificationQueue* queue) {
init(eventBase, queue);
registerInternalHandler(READ | PERSIST);
}
/**
* Stop consuming messages.
*
* startConsuming() may be called again to resume consumption of messages
* at a later point in time.
*/
void stopConsuming();
/**
* Get the TNotificationQueue that this consumer is currently consuming
* messages from. Returns NULL if the consumer is not currently consuming
* events from any queue.
*/
TNotificationQueue* getCurrentQueue() const {
return queue_;
}
/**
* Set a limit on how many messages this consumer will read each iteration
* around the event loop.
*
* This helps rate-limit how much work the Consumer will do each event loop
* iteration, to prevent it from starving other event handlers.
*
* A limit of 0 means no limit will be enforced. If unset, the limit
* defaults to kDefaultMaxReadAtOnce (defined to 10 above).
*/
void setMaxReadAtOnce(uint32_t maxAtOnce) {
maxReadAtOnce_ = maxAtOnce;
}
uint32_t getMaxReadAtOnce() const {
return maxReadAtOnce_;
}
private:
void init(TEventBase* eventBase, TNotificationQueue* queue);
virtual void handlerReady(uint16_t events) THRIFT_NOEXCEPT;
TNotificationQueue* queue_;
bool* destroyedFlagPtr_;
uint32_t maxReadAtOnce_;
};
enum class FdType {
EVENTFD,
PIPE
};
/**
* Create a new TNotificationQueue.
*
* If the maxSize parameter is specified, this sets the maximum queue size
* that will be enforced by tryPutMessage(). (This size is advisory, and may
* be exceeded if producers explicitly use putMessage() instead of
* tryPutMessage().)
*
* The fdType parameter determines the type of file descriptor used
* internally to signal message availability. The default (eventfd) is
* preferable for performance and because it won't fail when the queue gets
* too long. It is not available on on older and non-linux kernels, however.
* In this case the code will fall back to using a pipe, the parameter is
* mostly for testing purposes.
*/
explicit TNotificationQueue(uint32_t maxSize = 0,
FdType fdType = FdType::EVENTFD)
: spinlock_(),
eventfd_(-1),
pipeFds_({-1, -1}),
advisoryMaxQueueSize_(maxSize),
queue_() {
if (fdType == FdType::EVENTFD) {
eventfd_ = folly::eventfd(0, EFD_CLOEXEC | EFD_NONBLOCK | EFD_SEMAPHORE);
if (eventfd_ == -1) {
if (errno == ENOSYS || errno == EINVAL) {
// eventfd not availalble
T_ERROR("failed to create eventfd for TNotificationQueue: %d, "
"falling back to pipe mode", errno);
fdType = FdType::PIPE;
} else {
// some other error
throw TLibraryException("Failed to create eventfd for "
"TNotificationQueue", errno);
}
}
}
if (fdType == FdType::PIPE) {
if (pipe(pipeFds_)) {
throw TLibraryException("Failed to create pipe for TNotificationQueue",
errno);
}
try {
// put both ends of the pipe into non-blocking mode
if (fcntl(pipeFds_[0], F_SETFL, O_RDONLY | O_NONBLOCK) != 0) {
throw TLibraryException("failed to put TNotificationQueue pipe read "
"endpoint into non-blocking mode", errno);
}
if (fcntl(pipeFds_[1], F_SETFL, O_WRONLY | O_NONBLOCK) != 0) {
throw TLibraryException("failed to put TNotificationQueue pipe write "
"endpoint into non-blocking mode", errno);
}
} catch (...) {
::close(pipeFds_[0]);
::close(pipeFds_[1]);
throw;
}
}
}
~TNotificationQueue() {
if (eventfd_ >= 0) {
::close(eventfd_);
eventfd_ = -1;
}
if (pipeFds_[0] >= 0) {
::close(pipeFds_[0]);
pipeFds_[0] = -1;
}
if (pipeFds_[1] >= 0) {
::close(pipeFds_[1]);
pipeFds_[1] = -1;
}
}
/**
* Set the advisory maximum queue size.
*
* This maximum queue size affects calls to tryPutMessage(). Message
* producers can still use the putMessage() call to unconditionally put a
* message on the queue, ignoring the configured maximum queue size. This
* can cause the queue size to exceed the configured maximum.
*/
void setMaxQueueSize(uint32_t max) {
advisoryMaxQueueSize_ = max;
}
/**
* Attempt to put a message on the queue if the queue is not already full.
*
* If the queue is full, a TQueueFullException will be thrown. The
* setMaxQueueSize() function controls the maximum queue size.
*
* This method may contend briefly on a spinlock if many threads are
* concurrently accessing the queue, but for all intents and purposes it will
* immediately place the message on the queue and return.
*
* tryPutMessage() may throw std::bad_alloc if memory allocation fails, and
* may throw any other exception thrown by the MessageT move/copy
* constructor.
*/
void tryPutMessage(MessageT&& message) {
putMessageImpl(std::move(message), advisoryMaxQueueSize_);
}
void tryPutMessage(const MessageT& message) {
putMessageImpl(message, advisoryMaxQueueSize_);
}
/**
* Unconditionally put a message on the queue.
*
* This method is like tryPutMessage(), but ignores the maximum queue size
* and always puts the message on the queue, even if the maximum queue size
* would be exceeded.
*
* putMessage() may throw std::bad_alloc if memory allocation fails, and may
* throw any other exception thrown by the MessageT move/copy constructor.
*/
void putMessage(MessageT&& message) {
putMessageImpl(std::move(message), 0);
}
void putMessage(const MessageT& message) {
putMessageImpl(message, 0);
}
/**
* Put several messages on the queue.
*/
template<typename InputIteratorT>
void putMessages(InputIteratorT first, InputIteratorT last) {
typedef typename std::iterator_traits<InputIteratorT>::iterator_category
IterCategory;
putMessagesImpl(first, last, IterCategory());
}
/**
* Try to immediately pull a message off of the queue, without blocking.
*
* If a message is immediately available, the result parameter will be
* updated to contain the message contents and true will be returned.
*
* If no message is available, false will be returned and result will be left
* unmodified.
*/
bool tryConsume(MessageT& result) {
if (!tryConsumeEvent()) {
return false;
}
try {
facebook::SpinLockHolder guard(&spinlock_);
result = std::move(queue_.front());
queue_.pop_front();
} catch (...) {
// Handle an exception if the assignment operator happens to throw.
// We consumed an event but weren't able to pop the message off the
// queue. Signal the event again since the message is still in the
// queue.
signalEvent(1);
throw;
}
return true;
}
private:
// Forbidden copy constructor and assignment operator
TNotificationQueue(TNotificationQueue const &) = delete;
TNotificationQueue& operator=(TNotificationQueue const &) = delete;
inline void checkQueueSize(size_t maxSize) const {
assert(spinlock_.IsHeld());
if (maxSize > 0 && queue_.size() >= maxSize) {
throw TQueueFullException();
}
}
inline void signalEvent(uint64_t numAdded = 1) const {
static const uint8_t kPipeMessage[] = {
1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1
};
ssize_t bytes_written = 0;
ssize_t bytes_expected = 0;
if (eventfd_ >= 0) {
bytes_expected = static_cast<ssize_t>(sizeof(numAdded));
bytes_written = ::write(eventfd_, &numAdded, sizeof(numAdded));
} else {
// pipe semantics, add one message for each numAdded
bytes_expected = numAdded;
do {
size_t messageSize = std::min(numAdded, sizeof(kPipeMessage));
ssize_t rc = ::write(pipeFds_[1], kPipeMessage, messageSize);
if (rc < 0) {
// TODO: if the pipe is full, write will fail with EAGAIN.
// See task #1044651 for how this could be handled
break;
}
numAdded -= rc;
bytes_written += rc;
} while (numAdded > 0);
}
if (bytes_written != bytes_expected) {
throw TLibraryException("failed to signal TNotificationQueue after "
"write", errno);
}
}
bool tryConsumeEvent() {
uint64_t value = 0;
ssize_t rc = -1;
if (eventfd_ >= 0) {
rc = ::read(eventfd_, &value, sizeof(value));
} else {
uint8_t value8;
rc = ::read(pipeFds_[0], &value8, sizeof(value8));
value = value8;
}
if (rc < 0) {
// EAGAIN should pretty much be the only error we can ever get.
// This means someone else already processed the only available message.
assert(errno == EAGAIN);
return false;
}
assert(value == 1);
return true;
}
void putMessageImpl(MessageT&& message, size_t maxSize) {
{
facebook::SpinLockHolder guard(&spinlock_);
checkQueueSize(maxSize);
queue_.push_back(std::move(message));
}
signalEvent();
}
void putMessageImpl(const MessageT& message, size_t maxSize) {
{
facebook::SpinLockHolder guard(&spinlock_);
checkQueueSize(maxSize);
queue_.push_back(message);
}
signalEvent();
}
template<typename InputIteratorT>
void putMessagesImpl(InputIteratorT first, InputIteratorT last,
std::input_iterator_tag) {
uint64_t numAdded = 0;
{
facebook::SpinLockHolder guard(&spinlock_);
while (first != last) {
queue_.push_back(*first);
++first;
++numAdded;
}
}
signalEvent(numAdded);
}
template<typename InputIteratorT>
void putMessagesImpl(InputIteratorT first, InputIteratorT last,
std::forward_iterator_tag) {
uint64_t numAdded = std::distance(first, last);
{
facebook::SpinLockHolder guard(&spinlock_);
queue_.insert(queue_.end(), first, last);
}
signalEvent(numAdded);
}
facebook::SpinLock spinlock_;
int eventfd_;
int pipeFds_[2]; // to fallback to on older/non-linux systems
uint32_t advisoryMaxQueueSize_;
std::deque<MessageT> queue_;
};
template<typename MessageT>
TNotificationQueue<MessageT>::Consumer::~Consumer() {
// If we are in the middle of a call to handlerReady(), destroyedFlagPtr_
// will be non-NULL. Mark the value that it points to, so that
// handlerReady() will know the callback is destroyed, and that it cannot
// access any member variables anymore.
if (destroyedFlagPtr_) {
*destroyedFlagPtr_ = true;
}
}
template<typename MessageT>
void TNotificationQueue<MessageT>::Consumer::handlerReady(uint16_t events)
THRIFT_NOEXCEPT {
uint32_t numProcessed = 0;
while (true) {
// Try to decrement the eventfd.
//
// We decrement the eventfd before checking the queue, and only pop a
// message off the queue if we read from the eventfd.
//
// Reading the eventfd first allows us to not have to hold the spinlock
// while accessing the eventfd. If we popped from the queue first, we
// would have to hold the lock while reading from or writing to the
// eventfd. (Multiple consumers may be woken up from a single eventfd
// notification. If we popped from the queue first, we could end up
// popping a message from the queue before the eventfd has been notified by
// the producer, unless the consumer and producer both held the spinlock
// around the entire operation.)
if (!queue_->tryConsumeEvent()) {
// no message available right now
return;
}
// Now pop the message off of the queue.
// We successfully consumed the eventfd notification.
// There should be a message available for us to consume.
//
// We have to manually acquire and release the spinlock here, rather than
// using SpinLockHolder since the MessageT has to be constructed while
// holding the spinlock and available after we release it. SpinLockHolder
// unfortunately doesn't provide a release() method. (We can't construct
// MessageT first since we have no guarantee that MessageT has a default
// constructor.
queue_->spinlock_.Lock();
bool locked = true;
try {
// The eventfd is incremented once for every message, and only
// decremented when a message is popped off. There should always be a
// message here to read.
CHECK(!queue_->queue_.empty());
// Pull a message off the queue.
MessageT msg(std::move(queue_->queue_.front()));
queue_->queue_.pop_front();
// Check to see if the queue is empty now.
// We use this as an optimization to see if we should bother trying to
// loop again and read another message after invoking this callback.
bool wasEmpty = queue_->queue_.empty();
// Now unlock the spinlock before we invoke the callback.
queue_->spinlock_.Unlock();
locked = false;
// Call the callback
bool callbackDestroyed = false;
CHECK(destroyedFlagPtr_ == NULL);
destroyedFlagPtr_ = &callbackDestroyed;
messageAvailable(std::move(msg));
// If the callback was destroyed before it returned, we are done
if (callbackDestroyed) {
return;
}
destroyedFlagPtr_ = NULL;
// If the callback is no longer installed, we are done.
if (queue_ == NULL) {
return;
}
// If we have hit maxReadAtOnce_, we are done.
++numProcessed;
if (maxReadAtOnce_ > 0 && numProcessed >= maxReadAtOnce_) {
return;
}
// If the queue was empty before we invoked the callback, it's probable
// that it is still empty now. Just go ahead and return, rather than
// looping again and trying to re-read from the eventfd. (If a new
// message had in fact arrived while we were invoking the callback, we
// will simply be woken up the next time around the event loop and will
// process the message then.)
if (wasEmpty) {
return;
}
} catch (const std::exception& ex) {
// This catch block is really just to handle the case where the MessageT
// constructor throws. The messageAvailable() callback itself is
// declared as noexcept and should never throw.
//
// If the MessageT constructor does throw we try to handle it as best as
// we can, but we can't work miracles. We will just ignore the error for
// now and return. The next time around the event loop we will end up
// trying to read the message again. If MessageT continues to throw we
// will never make forward progress and will keep trying each time around
// the event loop.
if (locked) {
// Unlock the spinlock.
queue_->spinlock_.Unlock();
// Push a notification back on the eventfd since we didn't actually
// read the message off of the queue.
queue_->signalEvent(1);
}
return;
}
}
}
template<typename MessageT>
void TNotificationQueue<MessageT>::Consumer::init(
TEventBase* eventBase,
TNotificationQueue* queue) {
assert(eventBase->isInEventBaseThread());
assert(queue_ == NULL);
assert(!isHandlerRegistered());
queue_ = queue;
if (queue_->eventfd_ >= 0) {
initHandler(eventBase, queue_->eventfd_);
} else {
initHandler(eventBase, queue_->pipeFds_[0]);
}
}
template<typename MessageT>
void TNotificationQueue<MessageT>::Consumer::stopConsuming() {
if (queue_ == NULL) {
assert(!isHandlerRegistered());
return;
}
assert(isHandlerRegistered());
unregisterHandler();
detachEventBase();
queue_ = NULL;
}
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TNOTIFICATIONQUEUE_H_

View File

@ -0,0 +1,70 @@
#ifndef _THRIFT_TQUEUINGASYNCPROCESSOR_H_
#define _THRIFT_TQUEUINGASYNCPROCESSOR_H_ 1
#include <tr1/functional>
#include <boost/shared_ptr.hpp>
#include "thrift/lib/cpp/TProcessor.h"
#include "thrift/lib/cpp/async/TAsyncProcessor.h"
#include "thrift/lib/cpp/async/TEventTask.h"
#include "thrift/lib/cpp/concurrency/Exception.h"
namespace apache { namespace thrift { namespace async {
/**
* Adapter to allow a TProcessor to be used as a TAsyncProcessor.
*
* Note: this is not intended for use outside of TEventConnection since the
* callback mechanism used in TEventTask will invoke handleAsyncTaskComplete()
* regardless of what is passed in as the cob.
*
* Uses a per-server task queue for all calls.
*/
class TQueuingAsyncProcessor : public TAsyncProcessor {
public:
TQueuingAsyncProcessor(
boost::shared_ptr<apache::thrift::server::TProcessor> processor,
boost::shared_ptr<apache::thrift::concurrency::ThreadManager> threadManager,
int64_t taskExpireTime,
TEventConnection* connection)
: processor_(processor)
, threadManager_(threadManager)
, taskExpireTime_(taskExpireTime)
, connection_(connection)
{}
virtual void process(
std::tr1::function<void(bool success)> cob,
boost::shared_ptr<apache::thrift::protocol::TProtocol> in,
boost::shared_ptr<apache::thrift::protocol::TProtocol> out,
TConnectionContext* context) {
boost::shared_ptr<apache::thrift::concurrency::Runnable> task =
boost::shared_ptr<apache::thrift::concurrency::Runnable>(
new TEventTask(connection_));
try {
threadManager_->add(task, 0LL, taskExpireTime_);
} catch (apache::thrift::concurrency::IllegalStateException & ise) {
T_ERROR("IllegalStateException: TQueuingAsyncProcessor::process() %s",
ise.what());
// no task will be making a callback
return cob(false);
}
}
private:
boost::shared_ptr<apache::thrift::server::TProcessor> processor_;
/// For processing via thread pool
boost::shared_ptr<apache::thrift::concurrency::ThreadManager> threadManager_;
/// Time in milliseconds before an unperformed task expires (0 == infinite).
int64_t taskExpireTime_;
/// The worker that started us
TEventConnection* connection_;
};
}}} // apache::thrift::async
#endif // #ifndef _THRIFT_TQUEUINGASYNCPROCESSOR_H_

View File

@ -0,0 +1,441 @@
/*
* 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 THRIFT_ASYNC_TSTREAMASYNCCHANNEL_H_
#define THRIFT_ASYNC_TSTREAMASYNCCHANNEL_H_ 1
#include "thrift/lib/cpp/async/TAsyncEventChannel.h"
#include "thrift/lib/cpp/async/TAsyncTransport.h"
#include "thrift/lib/cpp/async/TAsyncTimeout.h"
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift { namespace async {
class TAsyncTransport;
template <class Subclass_>
class TAsyncChannelWriteRequestBase {
public:
typedef std::tr1::function<void()> VoidCallback;
TAsyncChannelWriteRequestBase(const VoidCallback& callback,
const VoidCallback& errorCallback,
transport::TMemoryBuffer* message)
: buffer_(message),
next_(NULL),
callback_(callback),
errorCallback_(errorCallback) {
// The WriteRequest's buffer consumes all of the data in message,
// so we don't attempt to resend data; yet is also an observer
// which prevents consumed data from being overwritten while it's pending
// for the transport
uint32_t len = message->available_read();
message->borrow(NULL, &len);
message->consume(len);
}
virtual ~TAsyncChannelWriteRequestBase() {
}
void setNext(Subclass_* next) {
assert(next_ == NULL);
next_ = next;
}
Subclass_* getNext() const {
return next_;
}
protected:
apache::thrift::transport::TMemoryBuffer buffer_;
void invokeCallback() {
// unlink the buffer before invoking the callback, since we are
// now done with it. Not strictly required but faster.
buffer_.unlink();
callback_();
}
void invokeErrorCallback() {
// unlink the buffer before invoking the callback, since we are
// now done with it. Not strictly required but faster.
buffer_.unlink();
errorCallback_();
}
private:
TAsyncChannelWriteRequestBase();
Subclass_* next_;
VoidCallback callback_;
VoidCallback errorCallback_;
};
/**
* TStreamAsyncChannel is a helper class for channel implementations that use
* TAsyncTransport underneath.
*
* TStreamAsyncChannel provides the basic functionality for implementing a
* message-based asynchronous channel on top of a streaming TAsyncTransport.
*
* It requires two template arguments that control how the stream is broken up
* into messagess:
*
* WriteRequest_:
*
* This template parameter controls how messages are written to the
* underlying stream. It must implement the following methods:
*
* - WriteRequest_(const VoidCallback& callback,
* const VoidCallback& errorCallback,
* transport::TMemoryBuffer* message);
*
* The WriteRequest_ constructor accepts the success and error callbacks,
* and the TMemoryBuffer containing the data to send. The WriteRequest_
* may consume data from the message, but does not own the TMemoryBuffer
* (i.e., it should not delete the TMemoryBuffer.)
*
* - void setNext(WriteRequest_* next);
* - WriteRequest_* getNext() const;
*
* These two methods support chaining together a list of WriteRequest_
* objects. This is used when multiple write requests are pending on the
* channel.
*
* - void write(TAsyncTransport* transport,
* TAsyncTransport::WriteCallback* callback) THRIFT_NOEXCEPT;
*
* This method will be called to schedule the write. The WriteRequest_
* should invoke the transport's write() or writev() method with the data
* to send, and set the specified callback as the transport callback.
*
* Note that this API requires the WriteRequest_ to write the entire
* message with a single write() or writev() call. This allows the code
* to let the TAsyncTransport perform the write queuing when multiple
* messages are pending. (If needed we could rewrite this API in the
* future to relax this restriction.)
*
* - void writeSuccess() THRIFT_NOEXCEPT;
* - void writeError(size_t bytesWritten,
* const TTransportException& ex) THRIFT_NOEXCEPT;
*
* Either writeSuccess() or writeError() will be invoked once the message
* write has completed.
*
* ReadState_:
*
* This template parameter controls how the incoming stream is broken up into
* individual messages. It must implement the following methods:
*
* - ReadState_();
*
* The ReadState_ constructor takes no arguments.
*
* - void setCallbackBuffer(transport::TMemoryBuffer* buffer);
*
* When a new read is started, setCallbackBuffer() is called to set the
* buffer into which the message data should be placed.
*
* - void unsetCallbackBuffer();
*
* unsetCallbackBuffer() is called to clear the callback buffer when after
* a full message has been read.
*
* - bool hasReadAheadData();
*
* Some ReadState_ implementations may perform read-ahead, and read past
* the end of the message when reading from the underlying transport.
* hasReadAheadData() is called when a new read starts, to see if the
* ReadState_ has pending data for a new message that has already been read
* from the transport.
*
* If hasReadAheadData() returns true, readDataAvailable(0) will be called
* immediately, rather than waiting for new data from the transport.
*
* - bool hasPartialMessage();
*
* When EOF is read from the underlying transport, hasPartialMessage() is
* called to see if the EOF should be treated as an error or a normal
* close. (It is an error if hasPartialMessage() returns true.)
*
* - void getReadBuffer(void** bufReturn, size_t* lenReturn);
*
* When data becomes available on the underlying transport, getReadBuffer()
* is called to get the buffer where the data should be placed.
*
* - bool readDataAvailable(size_t len);
*
* readDataAvailable() is called when new data has been read from the
* underlying transport. The data will have been placed in the buffer
* returned by the previous getReadBuffer() call.
*/
template<typename WriteRequest_, typename ReadState_>
class TStreamAsyncChannel : public TAsyncEventChannel,
protected TAsyncTransport::ReadCallback,
protected TAsyncTransport::WriteCallback,
protected TAsyncTimeout {
public:
TStreamAsyncChannel(const boost::shared_ptr<TAsyncTransport>& transport);
/**
* Helper function to create a shared_ptr<TStreamAsyncChannel>.
*
* This passes in the correct destructor object, since TStreamAsyncChannel's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<TStreamAsyncChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
return boost::shared_ptr<TStreamAsyncChannel>(
new TStreamAsyncChannel(transport), Destructor());
}
/**
* Destroy the channel.
*
* destroy() must be called to destroy the channel. The normal destructor
* is private, and should not be invoked directly. This prevents callers
* from deleting a TStreamAsyncChannel while it is invoking a callback.
*/
virtual void destroy();
// Methods inherited from TAsyncEventChannel
virtual bool readable() const;
virtual bool good() const;
virtual bool error() const;
virtual bool timedOut() const;
/**
* Send a message to the channel; note that "errorCob" will be called
* after a partial write as well as other errors. We will call "errorCob"
* immediately (before return) if the channel is unusable for some reason,
* and "cob" immediately if we're able to perform the write without delay.
*/
virtual void sendMessage(const VoidCallback& cob,
const VoidCallback& errorCob,
apache::thrift::transport::TMemoryBuffer* message);
/**
* Receive a message from the channel; note that "errorCob" will be called
* after a partial read as well as other errors. We will call "errorCob"
* immediately (before return) if the channel is unusable for some reason,
* and "cob" immediately if we're able to perform the read without delay.
*
* Note that an EOF is considered normal, so "cob" will be called although
* "good()" will be false.
*/
virtual void recvMessage(const VoidCallback& cob,
const VoidCallback& errorCob,
apache::thrift::transport::TMemoryBuffer* message);
/**
* Send a message to the channel and receive the response; note that the
* "errorCob: will be called after a write error and no receive is attempted.
* Also, a partial write or read will result in errorCob being called.
* We call "errorCob" before return if the channel is unusable for some
* reason. It is conceivable that "cob" will be called before return if data
* is somehow available in the channel when a read is first attempted.
*/
virtual void sendAndRecvMessage(
const VoidCallback& cob,
const VoidCallback& errorCob,
transport::TMemoryBuffer* sendBuf,
transport::TMemoryBuffer* recvBuf);
/**
* Close this channel.
*
* This gracefully closes the channel, waiting for all pending send
* requests to complete before actually closing the underlying transport.
*
* If a recvMessage() call is pending, it will be immediately failed.
*/
void close();
/**
* Close the channel immediately.
*
* This closes the channel immediately, dropping any outstanding messages
* waiting to be sent.
*
* If a recvMessage() call is pending, it will be immediately failed.
*/
void closeNow();
/**
* Attach the channel to a TEventBase.
*
* This may only be called if the channel is not currently attached to a
* TEventBase (by an earlier call to detachEventBase()).
*
* This method must be invoked in the TEventBase's thread.
*/
void attachEventBase(TEventBase* eventBase);
/**
* Detach the channel from its TEventBase.
*
* This may only be called when the channel is idle and has no reads or
* writes pending. Once detached, the channel may not be used again until it
* is re-attached to a TEventBase by calling attachEventBase().
*
* This method must be called from the current TEventBase's thread.
*/
void detachEventBase();
/**
* Get the TEventBase used by this channel.
*/
TEventBase* getEventBase() const;
/**
* Set the timeout for receiving messages.
*
* When set to a non-zero value, the entire message must be received within
* the specified number of milliseconds, or the receive will fail and the
* channel will be closed.
*
* If setRecvTimeout() is invoked while a recvMessage() call is currently in
* progress, the timeout will be restarted using the new value.
*/
void setRecvTimeout(uint32_t milliseconds);
/**
* Get the receive timeout.
*
* @return Returns the current receive timeout, in milliseconds. A return
* value of 0 indicates that no timeout is set.
*/
uint32_t getRecvTimeout() const {
return recvTimeout_;
}
/**
* Get the TAsyncTransport used by this channel.
*/
virtual boost::shared_ptr<TAsyncTransport> getTransport() {
return transport_;
}
/**
* Determine if this channel is idle (i.e., has no outstanding reads or
* writes).
*/
bool isIdle() const {
return (writeReqHead_ == NULL) && (!readCallback_) &&
!transport_->connecting();
}
protected:
struct ReadQueueEntry {
ReadQueueEntry(const VoidCallback& cob,
const VoidCallback& errorCob,
apache::thrift::transport::TMemoryBuffer* message) {
readCallback = cob;
readErrorCallback = errorCob;
readBuffer = message;
}
VoidCallback readCallback;
VoidCallback readErrorCallback;
transport::TMemoryBuffer *readBuffer;
int64_t startTime;
};
/**
* Protected destructor.
*
* Users of TStreamAsyncChannel must never delete it directly. Instead,
* invoke destroy().
*/
virtual ~TStreamAsyncChannel() {}
// callbacks from TAsyncTransport
void getReadBuffer(void** bufReturn, size_t* lenReturn);
void readDataAvailable(size_t len) THRIFT_NOEXCEPT;
void readEOF() THRIFT_NOEXCEPT;
void readError(const transport::TTransportException& ex) THRIFT_NOEXCEPT;
void writeSuccess() THRIFT_NOEXCEPT;
void writeError(size_t bytesWritten,
const transport::TTransportException& ex) THRIFT_NOEXCEPT;
// callback from TAsyncTimeout
void timeoutExpired() THRIFT_NOEXCEPT;
bool invokeReadDataAvailable(size_t len) THRIFT_NOEXCEPT;
void processReadEOF() THRIFT_NOEXCEPT;
void invokeReadCallback(VoidCallback cb,
char const* callbackName) THRIFT_NOEXCEPT;
void pushWriteRequest(WriteRequest_* req) {
if (writeReqTail_ == NULL) {
assert(writeReqHead_ == NULL);
writeReqHead_ = req;
} else {
writeReqTail_->setNext(req);
}
writeReqTail_ = req;
}
WriteRequest_* popWriteRequest() {
assert(writeReqHead_ != NULL);
WriteRequest_* req = writeReqHead_;
writeReqHead_ = req->getNext();
if (writeReqHead_ == NULL) {
assert(writeReqTail_ == req);
writeReqTail_ = NULL;
}
return req;
}
void failAllReads();
boost::shared_ptr<TAsyncTransport> transport_;
WriteRequest_* writeReqHead_;
WriteRequest_* writeReqTail_;
ReadState_ readState_;
VoidCallback readCallback_;
VoidCallback readErrorCallback_;
std::list<ReadQueueEntry> readCallbackQ_;
uint32_t recvTimeout_;
// true if a timeout has occurred
bool timedOut_;
private:
// Forbidden copy constructor and assignment opererator
TStreamAsyncChannel(TStreamAsyncChannel const &);
TStreamAsyncChannel& operator=(TStreamAsyncChannel const &);
};
class TStreamAsyncChannelFactory {
public:
virtual ~TStreamAsyncChannelFactory() {}
virtual boost::shared_ptr<TAsyncEventChannel> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) = 0;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TSTREAMASYNCCHANNEL_H_

View File

@ -0,0 +1,488 @@
/*
* 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 THRIFT_ASYNC_TSTREAMASYNCCHANNEL_TCC_
#define THRIFT_ASYNC_TSTREAMASYNCCHANNEL_TCC_ 1
#include "thrift/lib/cpp/async/TStreamAsyncChannel.h"
#include "thrift/lib/cpp/transport/TSocketAddress.h"
namespace apache { namespace thrift { namespace async {
template<typename WriteRequest_, typename ReadState_>
TStreamAsyncChannel<WriteRequest_, ReadState_>::TStreamAsyncChannel(
const boost::shared_ptr<TAsyncTransport>& transport)
: TAsyncTimeout(transport->getEventBase())
, transport_(transport)
, writeReqHead_(NULL)
, writeReqTail_(NULL)
, readState_()
, readCallback_()
, readErrorCallback_()
, recvTimeout_(0)
, timedOut_(false) {
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::destroy() {
// When destroy is called, close the channel immediately
closeNow();
// Then call TDelayedDestruction::destroy() to take care of
// whether or not we need immediate or delayed destruction
TDelayedDestruction::destroy();
}
template<typename WriteRequest_, typename ReadState_>
bool TStreamAsyncChannel<WriteRequest_, ReadState_>::readable() const {
return transport_->readable();
}
template<typename WriteRequest_, typename ReadState_>
bool TStreamAsyncChannel<WriteRequest_, ReadState_>::good() const {
return transport_->good();
}
template<typename WriteRequest_, typename ReadState_>
bool TStreamAsyncChannel<WriteRequest_, ReadState_>::error() const {
return (timedOut_ || transport_->error());
}
template<typename WriteRequest_, typename ReadState_>
bool TStreamAsyncChannel<WriteRequest_, ReadState_>::timedOut() const {
return timedOut_;
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::sendMessage(
const VoidCallback& cob,
const VoidCallback& errorCob,
transport::TMemoryBuffer* message) {
assert(message);
DestructorGuard dg(this);
if (!good()) {
T_DEBUG_T("sendMessage: transport went bad, bailing out.");
return errorCob();
}
if (message->available_read() == 0) {
T_ERROR("sendMessage: buffer is empty");
return errorCob();
}
WriteRequest_* writeReq;
try {
writeReq = new WriteRequest_(cob, errorCob, message, this);
} catch (const std::exception& ex) {
T_ERROR("sendMessage: failed to allocate new write request object");
errorCob();
return;
}
pushWriteRequest(writeReq);
writeReq->write(transport_.get(), this);
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::recvMessage(
const VoidCallback& cob,
const VoidCallback& errorCob,
transport::TMemoryBuffer* message) {
assert(message);
DestructorGuard dg(this);
if (!good()) {
T_DEBUG_T("recvMessage: transport went bad, bailing out.");
return errorCob();
}
if (message->available_read() != 0) {
T_ERROR("recvMessage: buffer is not empty.");
return errorCob();
}
if (readCallbackQ_.empty() && readCallback_ == NULL) {
readState_.setCallbackBuffer(message);
readCallback_ = cob;
readErrorCallback_ = errorCob;
} else {
readCallbackQ_.push_back(ReadQueueEntry(cob, errorCob, message));
return;
}
// Some ReadState implementations perform read-ahead,
// and they may already have data waiting to be processed.
// If so, we need to invoke readDataAvailable() immediately, rather than
// waiting for new data from the transport.
if (readState_.hasReadAheadData()) {
if (invokeReadDataAvailable(0)) {
// We already invoked the callback
return;
}
}
// start the read timeout
if (recvTimeout_ > 0) {
scheduleTimeout(recvTimeout_);
}
// start reading from the transport
// Note that setReadCallback() may invoke our read callback methods
// immediately, so the read may complete before setReadCallback() returns.
transport_->setReadCallback(this);
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::sendAndRecvMessage(
const VoidCallback& cob,
const VoidCallback& errorCob,
transport::TMemoryBuffer* sendBuf,
transport::TMemoryBuffer* recvBuf) {
// TODO: it would be better to perform this bind once, rather than
// each time sendAndRecvMessage() is called.
const VoidCallback& send_done =
std::tr1::bind(&TStreamAsyncChannel::recvMessage, this, cob, errorCob,
recvBuf);
return sendMessage(send_done, errorCob, sendBuf);
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::close() {
DestructorGuard dg(this); // transport::close can invoke callbacks
transport_->setReadCallback(NULL);
transport_->close();
if (readCallback_) {
processReadEOF();
}
// no need to free the write-queue here. The underlying transport will
// drain the writes first
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::closeNow() {
DestructorGuard dg(this); // transport::closeNow can invoke callbacks
transport_->setReadCallback(NULL);
transport_->closeNow();
if (readCallback_) {
processReadEOF();
}
// no need to free the write-queue here. The underlying transport will
// fail pending writes first
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::attachEventBase(
TEventBase* eventBase) {
TAsyncTimeout::attachEventBase(eventBase);
transport_->attachEventBase(eventBase);
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::detachEventBase() {
// detachEventBase() may not be called while in the middle of reading or
// writing a message. Make sure there are no read callbacks
assert(!readCallback_ && readCallbackQ_.empty());
// Even though readCallback_ is unset, the read timeout might still be
// installed. This happens when detachEventBase() is invoked by the
// recvMessage() callback, because invokeReadDataAvailable() optimizes and
// leaves the timeout and transport read callback installed while invoking
// the recvMessage() callback. Make sure we cancel the read timeout before
// detaching from the event base.
if (transport_->getReadCallback() == this) {
cancelTimeout();
transport_->setReadCallback(NULL);
}
TAsyncTimeout::detachEventBase();
transport_->detachEventBase();
}
template<typename WriteRequest_, typename ReadState_>
TEventBase*
TStreamAsyncChannel<WriteRequest_, ReadState_>::getEventBase() const {
return transport_->getEventBase();
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::setRecvTimeout(
uint32_t milliseconds) {
recvTimeout_ = milliseconds;
// If we are currently reading, update the timeout
if (transport_->getReadCallback() == this) {
if (milliseconds > 0) {
scheduleTimeout(milliseconds);
} else {
cancelTimeout();
}
}
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::getReadBuffer(
void** bufReturn, size_t* lenReturn) {
readState_.getReadBuffer(bufReturn, lenReturn);
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::readDataAvailable(
size_t len) THRIFT_NOEXCEPT {
invokeReadDataAvailable(len);
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::readEOF() THRIFT_NOEXCEPT {
// readCallback_ may be NULL if readEOF() is invoked while the read callback
// is already running inside invokeReadDataAvailable(), since
// invokeReadDataAvailable() leaves the transport read callback installed
// while calling the channel read callback.
if (readCallback_) {
processReadEOF();
}
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::readError(
const transport::TTransportException& ex) THRIFT_NOEXCEPT {
// readCallback_ may be NULL if readEOF() is invoked while the read callback
// is already running inside invokeReadDataAvailable(), since
// invokeReadDataAvailable() leaves the transport read callback installed
// while calling the channel read callback.
if (!readCallback_) {
return;
}
DestructorGuard dg(this);
cancelTimeout();
failAllReads();
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::writeSuccess()
THRIFT_NOEXCEPT {
DestructorGuard dg(this);
WriteRequest_* req = popWriteRequest();
req->writeSuccess();
delete req;
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::writeError(
size_t bytesWritten,
const transport::TTransportException& ex) THRIFT_NOEXCEPT {
DestructorGuard dg(this);
if (ex.getType() == transport::TTransportException::TIMED_OUT) {
timedOut_ = true;
}
WriteRequest_* req = popWriteRequest();
req->writeError(bytesWritten, ex);
delete req;
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::timeoutExpired()
THRIFT_NOEXCEPT {
DestructorGuard dg(this);
timedOut_ = true;
// Close the transport. It isn't usable anymore, since we are leaving
// it in a state with a partial message outstanding.
transport_->setReadCallback(NULL);
transport_->close();
// TODO: It would be nice not to have to always log an error message here;
// ideally the callback should decide if this is worth logging or not.
// Unfortunately the TAsyncChannel API doesn't allow us to pass any error
// info back to the callback.
T_ERROR("TStreamAsyncChannel: read timeout");
failAllReads();
}
template<typename WriteRequest_, typename ReadState_>
bool TStreamAsyncChannel<WriteRequest_, ReadState_>::invokeReadDataAvailable(
size_t len) THRIFT_NOEXCEPT {
DestructorGuard dg(this);
assert(readCallback_);
bool readDone;
try {
readDone = readState_.readDataAvailable(len);
} catch (const std::exception& ex) {
// The channel is in an unknown state after an error processing read data.
// Close the channel to ensure that callers cannot try to read from this
// channel again.
//
// Make sure we do this after clearing our callbacks, so that the
// channel won't call our readEOF() method.
cancelTimeout();
transport_->setReadCallback(NULL);
std::string addressStr;
try {
transport::TSocketAddress addr;
transport_->getPeerAddress(&addr);
addressStr = addr.describe();
} catch (const std::exception& e) {
addressStr = "unknown";
}
T_ERROR("error reading message from %s: %s", addressStr.c_str(), ex.what());
failAllReads();
return true;
}
if (!readDone) {
// We read some data, but didn't finish reading a full message.
if (recvTimeout_ > 0) {
// Reset the timeout whenever we receive any data.
// TODO: This matches the old TAsyncChannel behavior, but it seems like
// it would make more sense to have the timeout apply to the entire
// message as a whole. Eventually we should remove this code that resets
// the timeout.
scheduleTimeout(recvTimeout_);
}
return false;
}
TEventBase* ourEventBase = transport_->getEventBase();
// We read a full message. Invoke the read callback.
invokeReadCallback(readCallback_, "read callback");
// Note that we cleared readCallback_ and readErrorCallback_ before invoking
// the callback, but left ourself installed as the TAsyncTransport read
// callback.
//
// This allows us to avoid changing the TAsyncTransport read callback if the
// channel read callback immediately called recvMessage() again. This is
// fairly common, and we avoid 2 unnecessary epoll_ctl() calls by not
// changing the transport read callback. This results in a noticeable
// performance improvement.
//
// If readCallback_ is set again after the callback returns, we're still
// reading. recvMessage() will have taken care of reseting the receive
// timeout, so we have nothing else to do.
//
// If readCallback_ is unset, recvMessage() wasn't called again and we need
// to stop reading. If our TEventBase has changed, detachEventBase() will
// have already stopped reading. (Note that if the TEventBase has changed,
// it's possible that readCallback_ has already been set again to start
// reading in the other thread.)
if (transport_->getEventBase() == ourEventBase && !readCallback_) {
if (readCallbackQ_.empty()) {
cancelTimeout();
transport_->setReadCallback(NULL);
} else {
// There are queued readers, pop one. This block should have the same
// effect as if recvMessage were called
const ReadQueueEntry &qentry = readCallbackQ_.front();
readCallback_ = qentry.readCallback;
readErrorCallback_ = qentry.readErrorCallback;
readState_.setCallbackBuffer(qentry.readBuffer);
readCallbackQ_.pop_front();
if (readState_.hasReadAheadData()) {
return invokeReadDataAvailable(0);
} else if (recvTimeout_ > 0) {
scheduleTimeout(recvTimeout_);
}
}
}
return true;
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::failAllReads() {
invokeReadCallback(readErrorCallback_, "read error callback");
while (!readCallbackQ_.empty()) {
const ReadQueueEntry &qentry = readCallbackQ_.front();
invokeReadCallback(qentry.readErrorCallback, "read error callback");
readCallbackQ_.pop_front();
}
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::processReadEOF()
THRIFT_NOEXCEPT {
DestructorGuard dg(this);
assert(readCallback_);
VoidCallback cb;
const char* cbName;
if (readState_.hasPartialMessage()) {
cb = readErrorCallback_;
cbName = "read error callback";
} else {
// We call the normal (non-error) callback if no data has been received yet
// when EOF occurs.
//
// TODO: It would be nicer to have a mechanism to indicate to the caller
// that EOF was received, instead of treating this just like 0-sized
// message.
cb = readCallback_;
cbName = "read callback";
}
cancelTimeout();
invokeReadCallback(cb, cbName);
// Any queued reads should be notified like the else case above as only
// the first reader can have partial data.
while (!readCallbackQ_.empty()) {
const ReadQueueEntry &qentry = readCallbackQ_.front();
invokeReadCallback(qentry.readCallback, cbName);
readCallbackQ_.pop_front();
}
}
template<typename WriteRequest_, typename ReadState_>
void TStreamAsyncChannel<WriteRequest_, ReadState_>::invokeReadCallback(
VoidCallback cb, char const* callbackName) THRIFT_NOEXCEPT {
readState_.unsetCallbackBuffer();
readCallback_ = VoidCallback();
readErrorCallback_ = VoidCallback();
try {
cb();
} catch (const std::exception& ex) {
T_ERROR("TAsyncChannel: %s threw %s exception: %s",
callbackName, typeid(ex).name(), ex.what());
abort();
} catch (...) {
T_ERROR("TAsyncChannel: %s threw exception", callbackName);
abort();
}
}
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TSTREAMASYNCCHANNEL_TCC_

View File

@ -0,0 +1,42 @@
// Copyright (c) 2006- Facebook
// Distributed under the Thrift Software License
//
// See accompanying file LICENSE or visit the Thrift site at:
// http://developers.facebook.com/thrift/
#ifndef _THRIFT_TSYNCTOASYNCPROCESSOR_H_
#define _THRIFT_TSYNCTOASYNCPROCESSOR_H_ 1
#include <tr1/functional>
#include <boost/shared_ptr.hpp>
#include "thrift/lib/cpp/TProcessor.h"
#include "thrift/lib/cpp/async/TAsyncProcessor.h"
namespace apache { namespace thrift { namespace async {
/**
* Adapter to allow a TProcessor to be used as a TAsyncProcessor.
*
* Note that this should only be used for handlers that return quickly without
* blocking, since async servers can be stalled by a single blocking operation.
*/
class TSyncToAsyncProcessor : public TAsyncProcessor {
public:
TSyncToAsyncProcessor(boost::shared_ptr<TProcessor> processor)
: processor_(processor)
{}
virtual void process(std::tr1::function<void(bool success)> _return,
boost::shared_ptr<protocol::TProtocol> in,
boost::shared_ptr<protocol::TProtocol> out,
TConnectionContext* context) {
return _return(processor_->process(in, out, context));
}
private:
boost::shared_ptr<TProcessor> processor_;
};
}}} // apache::thrift::async
#endif // #ifndef _THRIFT_TSYNCTOASYNCPROCESSOR_H_

View File

@ -0,0 +1,117 @@
/*
* 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 THRIFT_ASYNC_TUNDELAYEDDESTRUCTION_H_
#define THRIFT_ASYNC_TUNDELAYEDDESTRUCTION_H_ 1
#include <cstdlib>
#include <type_traits>
#include <utility>
#include <cassert>
namespace apache { namespace thrift { namespace async {
/**
* A helper class to allow a TDelayedDestruction object to be instantiated on
* the stack.
*
* This class derives from an existing TDelayedDestruction type and makes the
* destructor public again. This allows objects of this type to be declared on
* the stack or directly inside another class. Normally TDelayedDestruction
* objects must be dynamically allocated on the heap.
*
* However, the trade-off is that you lose some of the protections provided by
* TDelayedDestruction::destroy(). TDelayedDestruction::destroy() will
* automatically delay destruction of the object until it is safe to do so.
* If you use TUndelayedDestruction, you become responsible for ensuring that
* you only destroy the object where it is safe to do so. Attempting to
* destroy a TUndelayedDestruction object while it has a non-zero destructor
* guard count will abort the program.
*/
template<typename TDD>
class TUndelayedDestruction : public TDD {
public:
// We want to expose all constructors provided by the parent class.
// C++11 adds constructor inheritance to support this. Unfortunately gcc
// does not implement constructor inheritance yet, so we have to fake it with
// variadic templates.
#if THRIFT_HAVE_CONSTRUCTOR_INHERITANCE
using TDD::TDD;
#else
// We unfortunately can't simulate constructor inheritance as well as I'd
// like.
//
// Ideally we would use std::enable_if<> and std::is_constructible<> to
// provide only constructor methods that are valid for our parent class.
// Unfortunately std::is_constructible<> doesn't work for types that aren't
// destructible. In gcc-4.6 it results in a compiler error. In the latest
// gcc code it looks like it has been fixed to return false. (The language
// in the standard seems to indicate that returning false is the correct
// behavior for non-destructible types, which is unfortunate.)
template<typename ...Args>
explicit TUndelayedDestruction(Args&& ...args)
: TDD(std::forward<Args>(args)...) {}
#endif
/**
* Public destructor.
*
* The caller is responsible for ensuring that the object is only destroyed
* where it is safe to do so. (i.e., when the destructor guard count is 0).
*
* The exact conditions for meeting this may be dependant upon your class
* semantics. Typically you are only guaranteed that it is safe to destroy
* the object directly from the event loop (e.g., directly from a
* TEventBase::LoopCallback), or when the event loop is stopped.
*/
virtual ~TUndelayedDestruction() {
// Crash if the caller is destroying us with outstanding destructor guards.
if (this->getDestructorGuardCount() != 0) {
abort();
}
// Invoke destroy. This is necessary since our base class may have
// implemented custom behavior in destroy().
this->destroy();
}
protected:
/**
* Override our parent's destroy() method to make it protected.
* Callers should use the normal destructor instead of destroy
*/
virtual void destroy() {
this->TDD::destroy();
}
virtual void destroyNow(bool delayed) {
// Do nothing. This will always be invoked from the call to destroy inside
// our destructor.
assert(!delayed);
// prevent unused variable warnings when asserts are compiled out.
(void)delayed;
}
private:
// Forbidden copy constructor and assignment operator
TUndelayedDestruction(TUndelayedDestruction const &) = delete;
TUndelayedDestruction& operator=(TUndelayedDestruction const &) = delete;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TUNDELAYEDDESTRUCTION_H_

View File

@ -0,0 +1,168 @@
/*
* 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 THRIFT_ASYNC_TUNFRAMEDASYNCCHANNEL_H_
#define THRIFT_ASYNC_TUNFRAMEDASYNCCHANNEL_H_ 1
#include "thrift/lib/cpp/async/TStreamAsyncChannel.h"
namespace apache { namespace thrift { namespace async {
namespace detail {
/**
* Encapsulation of one outstanding write request on a TUnframedAsyncChannel.
*/
class TUnframedACWriteRequest :
public TAsyncChannelWriteRequestBase<TUnframedACWriteRequest> {
public:
typedef std::tr1::function<void()> VoidCallback;
TUnframedACWriteRequest(const VoidCallback& callback,
const VoidCallback& errorCallback,
transport::TMemoryBuffer* message,
TAsyncEventChannel* channel);
void write(TAsyncTransport* transport,
TAsyncTransport::WriteCallback* callback) THRIFT_NOEXCEPT;
void writeSuccess() THRIFT_NOEXCEPT;
void writeError(size_t bytesWritten,
const transport::TTransportException& ex) THRIFT_NOEXCEPT;
};
/**
* Read state for TUnframedAsyncChannel
*/
template<typename ProtocolTraits_>
class TUnframedACReadState {
public:
typedef std::tr1::function<void()> VoidCallback;
typedef ProtocolTraits_ ProtocolTraits;
TUnframedACReadState();
~TUnframedACReadState();
// Methods required by TStreamAsyncChannel
void setCallbackBuffer(transport::TMemoryBuffer* buffer) {
callbackBuffer_ = buffer;
}
void unsetCallbackBuffer() {
callbackBuffer_ = NULL;
}
bool hasReadAheadData() {
return (memBuffer_.available_read() > 0);
}
bool hasPartialMessage() {
return (memBuffer_.available_read() > 0);
}
void getReadBuffer(void** bufReturn, size_t* lenReturn);
bool readDataAvailable(size_t len);
// Methods specific to TUnframedACReadState
void setMaxMessageSize(uint32_t size) {
maxMessageSize_ = size;
}
uint32_t getMaxMessageSize() const {
return maxMessageSize_;
}
ProtocolTraits_* getProtocolTraits() {
return &protocolTraits_;
}
const ProtocolTraits_* getProtocolTraits() const {
return &protocolTraits_;
}
private:
bool getMessageLength(uint8_t* buffer,
uint32_t bufferLength,
uint32_t* messageLength);
/// maximum frame size accepted
uint32_t maxMessageSize_;
apache::thrift::transport::TMemoryBuffer memBuffer_;
apache::thrift::transport::TMemoryBuffer* callbackBuffer_;
ProtocolTraits_ protocolTraits_;
};
} // namespace detail
/**
* TUnframedAsyncChannel
*
* This is a TAsyncChannel implementation that reads and writes raw (unframed)
* messages. When reading messages, ProtocolTraits_ is used to determine the
* end of a message.
*/
template<typename ProtocolTraits_>
class TUnframedAsyncChannel :
public TStreamAsyncChannel<detail::TUnframedACWriteRequest,
detail::TUnframedACReadState<ProtocolTraits_> > {
private:
typedef TStreamAsyncChannel<detail::TUnframedACWriteRequest,
detail::TUnframedACReadState<ProtocolTraits_> >
Parent;
typedef TUnframedAsyncChannel<ProtocolTraits_> Self;
public:
explicit TUnframedAsyncChannel(
const boost::shared_ptr<TAsyncTransport>& transport
)
: Parent(transport) {}
/**
* Helper function to create a shared_ptr<TUnframedAsyncChannel>.
*
* This passes in the correct destructor object, since TUnframedAsyncChannel's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<Self> newChannel(
const boost::shared_ptr<TAsyncTransport>& transport) {
return boost::shared_ptr<Self>(new Self(transport),
typename Self::Destructor());
}
/// size in bytes beyond which we'll reject a given message.
void setMaxMessageSize(uint32_t size) {
this->readState_.setMaxMessageSize(size);
}
uint32_t getMaxMessageSize() const {
return this->readState_.getMaxMessageSize();
}
protected:
/**
* Protected destructor.
*
* Users of TUnframedAsyncChannel must never delete it directly. Instead,
* invoke destroy().
*/
virtual ~TUnframedAsyncChannel() { }
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TUNFRAMEDASYNCCHANNEL_H_

View File

@ -0,0 +1,126 @@
/*
* 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 THRIFT_ASYNC_TUNFRAMEDASYNCCHANNEL_TCC_
#define THRIFT_ASYNC_TUNFRAMEDASYNCCHANNEL_TCC_ 1
#include "thrift/lib/cpp/async/TUnframedAsyncChannel.h"
#include "thrift/lib/cpp/transport/TBufferTransports.h"
namespace {
const uint32_t kInitialBufferSize = 4096;
}
namespace apache { namespace thrift { namespace async { namespace detail {
template<typename ProtocolTraits_>
TUnframedACReadState<ProtocolTraits_>::TUnframedACReadState()
: maxMessageSize_(0x7fffffff)
, memBuffer_(kInitialBufferSize)
, callbackBuffer_(NULL)
, protocolTraits_() {
}
template<typename ProtocolTraits_>
TUnframedACReadState<ProtocolTraits_>::~TUnframedACReadState() {
}
template<typename ProtocolTraits_>
void TUnframedACReadState<ProtocolTraits_>::getReadBuffer(void** bufReturn,
size_t* lenReturn) {
uint32_t bytesAvailable = memBuffer_.available_write();
if (bytesAvailable > 0) {
// If there is room available in the buffer, just return it.
*lenReturn = bytesAvailable;
*bufReturn = memBuffer_.getWritePtr(bytesAvailable);
return;
}
uint32_t bufferSize = memBuffer_.getBufferSize();
uint32_t available_read = memBuffer_.available_read();
// we get this much without growing the buffer capacity
uint32_t additionalSpace = bufferSize - available_read;
if (additionalSpace == 0) {
// We need more room. memBuffer_ will at least double it's capacity when
// asked for even a single byte.
additionalSpace = kInitialBufferSize;
}
// Don't allow more than maxMessageSize_.
// Be careful not to over- or underflow uint32_t when checking.
//
// readDataAvailable() fails the read when we've already read maxMessageSize_
// bytes, so available_read should always be less than maxMessageSize_ here.
// (Unless maxMessageSize_ is 0, but that's a programmer bug.)
assert(available_read < maxMessageSize_);
if (available_read > maxMessageSize_ - additionalSpace) {
// Don't ask for more than maxMessageSize_ total (but we might get more)
additionalSpace = maxMessageSize_ - available_read;
}
try {
uint8_t* newBuffer = memBuffer_.getWritePtr(additionalSpace);
*lenReturn = memBuffer_.available_write();
*bufReturn = newBuffer;
} catch (std::exception &ex) {
T_ERROR("TUnframedAsyncChannel: failed to allocate larger read buffer: %s",
ex.what());
*lenReturn = 0;
*bufReturn = NULL;
}
}
template<typename ProtocolTraits_>
bool TUnframedACReadState<ProtocolTraits_>::readDataAvailable(size_t len) {
assert(memBuffer_.available_read() + len <= memBuffer_.getBufferSize());
memBuffer_.wroteBytes(len);
uint32_t messageLength = 0;
uint32_t bytesRead = memBuffer_.available_read();
uint8_t *buffer = (uint8_t *)memBuffer_.borrow(NULL, &bytesRead);
if (!protocolTraits_.getMessageLength(buffer, bytesRead, &messageLength)) {
// We're not at the end of the message yet.
//
// If we've hit maxMessageSize_ already, fail now instead of waiting until
// getReadBuffer() is called again.
if (bytesRead >= maxMessageSize_) {
throw transport::TTransportException(
transport::TTransportException::CORRUPTED_DATA,
"TUnframedAsyncChannel: max message size exceeded");
}
return false;
}
// We've read a full message.
// Swap the data into the callback's buffer.
// Note that we may have actually read more than one message,
// so we have to make sure to save any remaining data after the end of the
// message.
assert(messageLength <= bytesRead);
callbackBuffer_->link(&memBuffer_, messageLength);
memBuffer_.consume(messageLength);
// We've put a new message in callbackBuffer_
return true;
}
}}}} // apache::thrift::async::detail
#endif // THRIFT_ASYNC_TUNFRAMEDASYNCCHANNEL_TCC_

View File

@ -0,0 +1,164 @@
/*
* 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 THRIFT_ASYNC_TZLIBASYNCCHANNEL_H_
#define THRIFT_ASYNC_TZLIBASYNCCHANNEL_H_ 1
#include "thrift/lib/cpp/async/TAsyncEventChannel.h"
#include "thrift/lib/cpp/transport/TZlibTransport.h"
namespace apache { namespace thrift { namespace async {
class TZlibAsyncChannel : public TAsyncEventChannel {
public:
TZlibAsyncChannel(const boost::shared_ptr<TAsyncEventChannel>& channel);
/**
* Helper function to create a shared_ptr<TZlibAsyncChannel>.
*
* This passes in the correct destructor object, since TZlibAsyncChannel's
* destructor is protected and cannot be invoked directly.
*/
static boost::shared_ptr<TZlibAsyncChannel> newChannel(
const boost::shared_ptr<TAsyncEventChannel>& channel) {
return boost::shared_ptr<TZlibAsyncChannel>(
new TZlibAsyncChannel(channel), Destructor());
}
virtual bool readable() const {
return channel_->readable();
}
virtual bool good() const {
return channel_->good();
}
virtual bool error() const {
return channel_->error();
}
virtual bool timedOut() const {
return channel_->timedOut();
}
virtual bool isIdle() const {
return channel_->isIdle();
}
virtual void sendMessage(const VoidCallback& cob,
const VoidCallback& errorCob,
transport::TMemoryBuffer* message);
virtual void recvMessage(const VoidCallback& cob,
const VoidCallback& errorCob,
transport::TMemoryBuffer* message);
virtual void sendAndRecvMessage(const VoidCallback& cob,
const VoidCallback& errorCob,
transport::TMemoryBuffer* sendBuf,
transport::TMemoryBuffer* recvBuf);
virtual boost::shared_ptr<TAsyncTransport> getTransport() {
return channel_->getTransport();
}
virtual void attachEventBase(TEventBase* eventBase) {
channel_->attachEventBase(eventBase);
}
virtual void detachEventBase() {
channel_->detachEventBase();
}
virtual uint32_t getRecvTimeout() const {
return channel_->getRecvTimeout();
}
virtual void setRecvTimeout(uint32_t milliseconds) {
channel_->setRecvTimeout(milliseconds);
}
protected:
/**
* Protected destructor.
*
* Users of TZlibAsyncChannel must never delete it directly. Instead,
* invoke destroy().
*/
virtual ~TZlibAsyncChannel() { }
private:
class SendRequest {
public:
SendRequest();
bool isSet() const {
return static_cast<bool>(callback_);
}
void set(const VoidCallback& callback,
const VoidCallback& errorCallback,
transport::TMemoryBuffer* message);
void send(TAsyncEventChannel* channel);
private:
void invokeCallback(VoidCallback callback);
void sendSuccess();
void sendError();
boost::shared_ptr<transport::TMemoryBuffer> compressedBuffer_;
transport::TZlibTransport zlibTransport_;
VoidCallback sendSuccess_;
VoidCallback sendError_;
VoidCallback callback_;
VoidCallback errorCallback_;
};
class RecvRequest {
public:
RecvRequest();
bool isSet() const {
return static_cast<bool>(callback_);
}
void set(const VoidCallback& callback,
const VoidCallback& errorCallback,
transport::TMemoryBuffer* message);
void recv(TAsyncEventChannel* channel);
private:
void invokeCallback(VoidCallback callback);
void recvSuccess();
void recvError();
boost::shared_ptr<transport::TMemoryBuffer> compressedBuffer_;
transport::TZlibTransport zlibTransport_;
VoidCallback recvSuccess_;
VoidCallback recvError_;
VoidCallback callback_;
VoidCallback errorCallback_;
transport::TMemoryBuffer *callbackBuffer_;
};
boost::shared_ptr<TAsyncEventChannel> channel_;
// TODO: support multiple pending send requests
SendRequest sendRequest_;
RecvRequest recvRequest_;
};
}}} // apache::thrift::async
#endif // THRIFT_ASYNC_TZLIBASYNCCHANNEL_H_

View File

@ -0,0 +1,64 @@
/*
* 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 _THRIFT_CONCURRENCY_EXCEPTION_H_
#define _THRIFT_CONCURRENCY_EXCEPTION_H_ 1
#include <exception>
#include "thrift/lib/cpp/Thrift.h"
namespace apache { namespace thrift { namespace concurrency {
class NoSuchTaskException : public apache::thrift::TLibraryException {};
class UncancellableTaskException : public apache::thrift::TLibraryException {};
class InvalidArgumentException : public apache::thrift::TLibraryException {};
class IllegalStateException : public apache::thrift::TLibraryException {
public:
IllegalStateException() {}
IllegalStateException(const std::string& message) : TLibraryException(message) {}
};
class TimedOutException : public apache::thrift::TLibraryException {
public:
TimedOutException():TLibraryException("TimedOutException"){};
TimedOutException(const std::string& message ) :
TLibraryException(message) {}
};
class TooManyPendingTasksException : public apache::thrift::TLibraryException {
public:
TooManyPendingTasksException():TLibraryException("TooManyPendingTasksException"){};
TooManyPendingTasksException(const std::string& message ) :
TLibraryException(message) {}
};
class SystemResourceException : public apache::thrift::TLibraryException {
public:
SystemResourceException() {}
SystemResourceException(const std::string& message) :
TLibraryException(message) {}
};
}}} // apache::thrift::concurrency
#endif // #ifndef _THRIFT_CONCURRENCY_EXCEPTION_H_

View File

@ -0,0 +1,131 @@
/*
* 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 _THRIFT_CONCURRENCY_FUNCTION_RUNNER_H
#define _THRIFT_CONCURRENCY_FUNCTION_RUNNER_H 1
#include <tr1/functional>
#include "thrift/lib/cpp/concurrency/Thread.h"
namespace apache { namespace thrift { namespace concurrency {
/**
* Convenient implementation of Runnable that will execute arbitrary callbacks.
* Interfaces are provided to accept both a generic 'void(void)' callback, and
* a 'void* (void*)' pthread_create-style callback.
*
* Example use:
* void* my_thread_main(void* arg);
* shared_ptr<ThreadFactory> factory = ...;
* // To create a thread that executes my_thread_main once:
* shared_ptr<Thread> thread = factory->newThread(
* FunctionRunner::create(my_thread_main, some_argument));
* thread->start();
*
* bool A::foo();
* A* a = new A();
* // To create a thread that executes a.foo() every 100 milliseconds:
* factory->newThread(FunctionRunner::create(
* std::tr1::bind(&A::foo, a), 100))->start();
*
*/
class FunctionRunner : public Runnable {
public:
// This is the type of callback 'pthread_create()' expects.
typedef void* (*PthreadFuncPtr)(void *arg);
// This a fully-generic void(void) callback for custom bindings.
typedef std::tr1::function<void()> VoidFunc;
typedef std::tr1::function<bool()> BoolFunc;
/**
* Syntactic sugar to make it easier to create new FunctionRunner
* objects wrapped in shared_ptr.
*/
static boost::shared_ptr<FunctionRunner> create(const VoidFunc& cob) {
return boost::shared_ptr<FunctionRunner>(new FunctionRunner(cob));
}
static boost::shared_ptr<FunctionRunner> create(PthreadFuncPtr func,
void* arg) {
return boost::shared_ptr<FunctionRunner>(new FunctionRunner(func, arg));
}
static boost::shared_ptr<FunctionRunner> create(const BoolFunc& cob,
int intervalMs) {
return boost::shared_ptr<FunctionRunner>(new FunctionRunner(cob,
intervalMs));
}
/**
* Given a 'pthread_create' style callback, this FunctionRunner will
* execute the given callback. Note that the 'void*' return value is ignored.
*/
FunctionRunner(PthreadFuncPtr func, void* arg)
: func_(std::tr1::bind(func, arg)), repFunc_(0), initFunc_(0)
{ }
/**
* Given a generic callback, this FunctionRunner will execute it.
*/
FunctionRunner(const VoidFunc& cob)
: func_(cob), repFunc_(0), initFunc_(0)
{ }
/**
* Given a bool foo(...) type callback, FunctionRunner will execute
* the callback repeatedly with 'intervalMs' milliseconds between the calls,
* until it returns false. Note that the actual interval between calls will
* be intervalMs plus execution time of the callback.
*/
FunctionRunner(const BoolFunc& cob, int intervalMs)
: func_(0), repFunc_(cob), intervalMs_(intervalMs), initFunc_(0)
{ }
/**
* Set a callback to be called when the thread is started.
*/
void setInitFunc(const VoidFunc& initFunc) {
initFunc_ = initFunc;
}
void run() {
if (initFunc_) {
initFunc_();
}
if (repFunc_) {
while(repFunc_()) {
usleep(intervalMs_*1000);
}
} else {
func_();
}
}
private:
VoidFunc func_;
BoolFunc repFunc_;
int intervalMs_;
VoidFunc initFunc_;
};
}}} // apache::thrift::concurrency
#endif // #ifndef _THRIFT_CONCURRENCY_FUNCTION_RUNNER_H

View File

@ -0,0 +1,124 @@
/*
* 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 _THRIFT_CONCURRENCY_MONITOR_H_
#define _THRIFT_CONCURRENCY_MONITOR_H_ 1
#include "thrift/lib/cpp/concurrency/Exception.h"
#include "thrift/lib/cpp/concurrency/Mutex.h"
#include <boost/utility.hpp>
namespace apache { namespace thrift { namespace concurrency {
/**
* A monitor is a combination mutex and condition-event. Waiting and
* notifying condition events requires that the caller own the mutex. Mutex
* lock and unlock operations can be performed independently of condition
* events. This is more or less analogous to java.lang.Object multi-thread
* operations.
*
* Note the Monitor can create a new, internal mutex; alternatively, a
* separate Mutex can be passed in and the Monitor will re-use it without
* taking ownership. It's the user's responsibility to make sure that the
* Mutex is not deallocated before the Monitor.
*
* Note that all methods are const. Monitors implement logical constness, not
* bit constness. This allows const methods to call monitor methods without
* needing to cast away constness or change to non-const signatures.
*
* @version $Id:$
*/
class Monitor : boost::noncopyable {
public:
/** Creates a new mutex, and takes ownership of it. */
Monitor();
/** Uses the provided mutex without taking ownership. */
explicit Monitor(Mutex* mutex);
/** Uses the mutex inside the provided Monitor without taking ownership. */
explicit Monitor(Monitor* monitor);
/** Deallocates the mutex only if we own it. */
virtual ~Monitor();
Mutex& mutex() const;
virtual void lock() const;
virtual void unlock() const;
/**
* Waits a maximum of the specified timeout in milliseconds for the condition
* to occur, or waits forever if timeout_ms == 0.
*
* Returns 0 if condition occurs, ETIMEDOUT on timeout, or an error code.
*/
int waitForTimeRelative(int64_t timeout_ms) const;
/**
* Waits until the absolute time specified using struct timespec.
* Returns 0 if condition occurs, ETIMEDOUT on timeout, or an error code.
*/
int waitForTime(const timespec* abstime) const;
/**
* Waits forever until the condition occurs.
* Returns 0 if condition occurs, or an error code otherwise.
*/
int waitForever() const;
/**
* Exception-throwing version of waitForTimeRelative(), called simply
* wait(int64) for historical reasons. Timeout is in milliseconds.
*
* If the condition occurs, this function returns cleanly; on timeout or
* error an exception is thrown.
*/
void wait(int64_t timeout_ms = 0LL) const;
/** Wakes up one thread waiting on this monitor. */
virtual void notify() const;
/** Wakes up all waiting threads on this monitor. */
virtual void notifyAll() const;
private:
class Impl;
Impl* impl_;
};
class Synchronized {
public:
Synchronized(const Monitor* monitor) : g(monitor->mutex()) { }
Synchronized(const Monitor& monitor) : g(monitor.mutex()) { }
private:
Guard g;
};
}}} // apache::thrift::concurrency
#endif // #ifndef _THRIFT_CONCURRENCY_MONITOR_H_

View File

@ -0,0 +1,276 @@
/*
* 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 THRIFT_CONCURRENCY_MUTEX_H_
#define THRIFT_CONCURRENCY_MUTEX_H_ 1
#include <boost/shared_ptr.hpp>
#include <boost/noncopyable.hpp>
namespace apache { namespace thrift { namespace concurrency {
#ifndef THRIFT_NO_CONTENTION_PROFILING
/**
* Determines if the Thrift Mutex and ReadWriteMutex classes will attempt to
* profile their blocking acquire methods. If this value is set to non-zero,
* Thrift will attempt to invoke the callback once every profilingSampleRate
* times. However, as the sampling is not synchronized the rate is not
* guaranteed, and could be subject to big bursts and swings. Please ensure
* your sampling callback is as performant as your application requires.
*
* The callback will get called with the wait time taken to lock the mutex in
* usec and a (void*) that uniquely identifies the Mutex (or ReadWriteMutex)
* being locked.
*
* The enableMutexProfiling() function is unsynchronized; calling this function
* while profiling is already enabled may result in race conditions. On
* architectures where a pointer assignment is atomic, this is safe but there
* is no guarantee threads will agree on a single callback within any
* particular time period.
*/
typedef void (*MutexWaitCallback)(const void* id, int64_t waitTimeMicros);
void enableMutexProfiling(int32_t profilingSampleRate,
MutexWaitCallback callback);
#endif
/**
* A simple mutex class
*
* @version $Id:$
*/
class Mutex {
public:
typedef void (*Initializer)(void*);
// Specifying the type of the mutex with one of the static Initializer
// methods defined in this class.
explicit Mutex(Initializer init = DEFAULT_INITIALIZER);
// Specifying the type of the mutex with an integer. The value has
// to be supported by the underlying implementation, currently
// pthread_mutex. So the possible values are PTHREAD_MUTEX_NORMAL,
// PTHREAD_MUTEX_ERRORCHECK, PTHREAD_MUTEX_RECURSIVE and
// PTHREAD_MUTEX_DEFAULT.
explicit Mutex(int type);
virtual ~Mutex() {}
virtual void lock() const;
virtual bool trylock() const;
virtual bool timedlock(int64_t milliseconds) const;
virtual void unlock() const;
/**
* Determine if the mutex is locked.
*
* This is intended to be used primarily as a debugging aid, and is not
* guaranteed to be a fast operation. For example, a common use case is to
* assert(mutex.isLocked()) in functions that may only be called with a
* particular mutex already locked.
*
* TODO: This method currently always returns false for recursive mutexes.
* Avoid calling this method on recursive mutexes.
*/
virtual bool isLocked() const;
void* getUnderlyingImpl() const;
static void DEFAULT_INITIALIZER(void*);
static void ADAPTIVE_INITIALIZER(void*);
static void RECURSIVE_INITIALIZER(void*);
private:
class impl;
boost::shared_ptr<impl> impl_;
};
class ReadWriteMutex {
public:
ReadWriteMutex();
virtual ~ReadWriteMutex() {}
// these get the lock and block until it is done successfully
virtual void acquireRead() const;
virtual void acquireWrite() const;
// these get the lock and block until it is done successfully
// or run out of time
virtual bool timedRead(int64_t milliseconds) const;
virtual bool timedWrite(int64_t milliseconds) const;
// these attempt to get the lock, returning false immediately if they fail
virtual bool attemptRead() const;
virtual bool attemptWrite() const;
// this releases both read and write locks
virtual void release() const;
private:
class impl;
boost::shared_ptr<impl> impl_;
};
/**
* A ReadWriteMutex that guarantees writers will not be starved by readers:
* When a writer attempts to acquire the mutex, all new readers will be
* blocked from acquiring the mutex until the writer has acquired and
* released it. In some operating systems, this may already be guaranteed
* by a regular ReadWriteMutex.
*/
class NoStarveReadWriteMutex : public ReadWriteMutex {
public:
NoStarveReadWriteMutex();
virtual void acquireRead() const;
virtual void acquireWrite() const;
// these get the lock and block until it is done successfully
// or run out of time
virtual bool timedRead(int64_t milliseconds) const;
virtual bool timedWrite(int64_t milliseconds) const;
private:
Mutex mutex_;
mutable volatile bool writerWaiting_;
};
class Guard : boost::noncopyable {
public:
explicit Guard(const Mutex& value, int64_t timeout = 0) : mutex_(&value) {
if (timeout == 0) {
value.lock();
} else if (timeout < 0) {
if (!value.trylock()) {
mutex_ = NULL;
}
} else {
if (!value.timedlock(timeout)) {
mutex_ = NULL;
}
}
}
~Guard() {
if (mutex_) {
mutex_->unlock();
}
}
/*
* This is really operator bool. However, implementing it to return
* bool is actually harmful. See
* www.artima.com/cppsource/safebool.html for the details; in brief,
* converting to bool allows a lot of nonsensical operations in
* addition to simple testing. To avoid that, we return a pointer to
* member which can only be used for testing.
*/
typedef const Mutex*const Guard::*const pBoolMember;
inline operator pBoolMember() const {
return mutex_ != NULL ? &Guard::mutex_ : NULL;
}
private:
const Mutex* mutex_;
};
// Can be used as second argument to RWGuard to make code more readable
// as to whether we're doing acquireRead() or acquireWrite().
enum RWGuardType {
RW_READ = 0,
RW_WRITE = 1,
};
class RWGuard : boost::noncopyable {
public:
explicit RWGuard(const ReadWriteMutex& value, bool write = false,
int64_t timeout=0)
: rw_mutex_(value), locked_(true) {
if (write) {
if (timeout) {
locked_ = rw_mutex_.timedWrite(timeout);
} else {
rw_mutex_.acquireWrite();
}
} else {
if (timeout) {
locked_ = rw_mutex_.timedRead(timeout);
} else {
rw_mutex_.acquireRead();
}
}
}
RWGuard(const ReadWriteMutex& value, RWGuardType type, int64_t timeout = 0)
: rw_mutex_(value), locked_(true) {
if (type == RW_WRITE) {
if (timeout) {
locked_ = rw_mutex_.timedWrite(timeout);
} else {
rw_mutex_.acquireWrite();
}
} else {
if (timeout) {
locked_ = rw_mutex_.timedRead(timeout);
} else {
rw_mutex_.acquireRead();
}
}
}
~RWGuard() {
if (locked_) {
rw_mutex_.release();
}
}
typedef const bool RWGuard::*const pBoolMember;
operator pBoolMember() const {
return locked_ ? &RWGuard::locked_ : NULL;
}
bool operator!() const {
return !locked_;
}
bool release() {
if (!locked_) return false;
rw_mutex_.release();
locked_ = false;
return true;
}
private:
const ReadWriteMutex& rw_mutex_;
mutable bool locked_;
};
// A little hack to prevent someone from trying to do "Guard(m);"
// Such a use is invalid because the temporary Guard object is
// destroyed at the end of the line, releasing the lock.
// Sorry for polluting the global namespace, but I think it's worth it.
#define Guard(m) incorrect_use_of_Guard(m)
#define RWGuard(m) incorrect_use_of_RWGuard(m)
}}} // apache::thrift::concurrency
#endif // #ifndef THRIFT_CONCURRENCY_MUTEX_H_

View File

@ -0,0 +1,137 @@
/*
* 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 THRIFT_CONCURRENCY_POSIXTHREADFACTORY_H_
#define THRIFT_CONCURRENCY_POSIXTHREADFACTORY_H_ 1
#include <set>
#include "thrift/lib/cpp/concurrency/Thread.h"
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift { namespace concurrency {
void getLiveThreadIds(std::set<pthread_t>* tids);
/**
* A thread factory to create posix threads
*
* @version $Id:$
*/
class PosixThreadFactory : public ThreadFactory {
public:
/**
* POSIX Thread scheduler policies
*/
enum POLICY {
OTHER,
FIFO,
ROUND_ROBIN
};
/**
* POSIX Thread scheduler relative priorities,
*
* Absolute priority is determined by scheduler policy and OS. This
* enumeration specifies relative priorities such that one can specify a
* priority within a giving scheduler policy without knowing the absolute
* value of the priority.
*/
enum PRIORITY {
LOWEST = 0,
LOWER = 1,
LOW = 2,
NORMAL = 3,
HIGH = 4,
HIGHER = 5,
HIGHEST = 6,
INCREMENT = 7,
DECREMENT = 8
};
/**
* Posix thread (pthread) factory. All threads created by a factory are reference-counted
* via boost::shared_ptr and boost::weak_ptr. The factory guarantees that threads and
* the Runnable tasks they host will be properly cleaned up once the last strong reference
* to both is given up.
*
* Threads are created with the specified policy, priority, stack-size and detachable-mode
* detached means the thread is free-running and will release all system resources the
* when it completes. A detachable thread is not joinable. The join method
* of a detachable thread will return immediately with no error.
*
* By default threads are not joinable.
*/
explicit PosixThreadFactory(POLICY policy=ROUND_ROBIN,
PRIORITY priority=NORMAL,
int stackSize=1,
bool detached=true);
// From ThreadFactory;
boost::shared_ptr<Thread> newThread(boost::shared_ptr<Runnable> runnable) const;
// From ThreadFactory;
Thread::id_t getCurrentThreadId() const;
/**
* Gets stack size for created threads
*
* @return int size in megabytes
*/
virtual int getStackSize() const;
/**
* Sets stack size for created threads
*
* @param value size in megabytes
*/
virtual void setStackSize(int value);
/**
* Gets priority relative to current policy
*/
virtual PRIORITY getPriority() const;
/**
* Sets priority relative to current policy
*/
virtual void setPriority(PRIORITY priority);
/**
* Sets detached mode of threads
*/
virtual void setDetached(bool detached);
/**
* Gets current detached mode
*/
virtual bool isDetached() const;
private:
class Impl;
boost::shared_ptr<Impl> impl_;
};
}}} // apache::thrift::concurrency
#endif // #ifndef _THRIFT_CONCURRENCY_POSIXTHREADFACTORY_H_

View File

@ -0,0 +1,17 @@
# TARGETS file for thrift/lib/cpp/concurrency
cpp_library (
name = "concurrency",
srcs = [
"Util.cpp",
"Monitor.cpp",
"Mutex.cpp",
"PosixThreadFactory.cpp",
"ThreadManager.cpp",
"TimerManager.cpp"
],
deps = [ '@/common/base:profiler',
'@/thrift/lib/cpp:thrift_base' ],
)

View File

@ -0,0 +1,125 @@
/*
* 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 _THRIFT_CONCURRENCY_THREAD_H_
#define _THRIFT_CONCURRENCY_THREAD_H_ 1
#include <stdint.h>
#include <boost/shared_ptr.hpp>
#include <boost/weak_ptr.hpp>
namespace apache { namespace thrift { namespace concurrency {
class Thread;
/**
* Minimal runnable class. More or less analogous to java.lang.Runnable.
*
* @version $Id:$
*/
class Runnable {
public:
virtual ~Runnable() {};
virtual void run() = 0;
/**
* Gets the thread object that is hosting this runnable object - can return
* an empty boost::shared pointer if no references remain on the thread object
*/
virtual boost::shared_ptr<Thread> thread() { return thread_.lock(); }
/**
* Sets the thread that is executing this object. This is only meant for
* use by concrete implementations of Thread.
*/
virtual void thread(boost::shared_ptr<Thread> value) { thread_ = value; }
private:
boost::weak_ptr<Thread> thread_;
};
/**
* Minimal thread class. Returned by thread factory bound to a Runnable object
* and ready to start execution. More or less analogous to java.lang.Thread
* (minus all the thread group, priority, mode and other baggage, since that
* is difficult to abstract across platforms and is left for platform-specific
* ThreadFactory implementations to deal with
*
* @see apache::thrift::concurrency::ThreadFactory)
*/
class Thread {
public:
typedef uint64_t id_t;
virtual ~Thread() {};
/**
* Starts the thread. Does platform specific thread creation and
* configuration then invokes the run method of the Runnable object bound
* to this thread.
*/
virtual void start() = 0;
/**
* Join this thread. Current thread blocks until this target thread
* completes.
*/
virtual void join() = 0;
/**
* Gets the thread's platform-specific ID
*/
virtual id_t getId() = 0;
/**
* Gets the runnable object this thread is hosting
*/
virtual boost::shared_ptr<Runnable> runnable() const { return _runnable; }
protected:
virtual void runnable(boost::shared_ptr<Runnable> value) { _runnable = value; }
private:
boost::shared_ptr<Runnable> _runnable;
};
/**
* Factory to create platform-specific thread object and bind them to Runnable
* object for execution
*/
class ThreadFactory {
public:
virtual ~ThreadFactory() {}
virtual boost::shared_ptr<Thread> newThread(boost::shared_ptr<Runnable> runnable) const = 0;
/** Gets the current thread id or unknown_thread_id if the current thread is not a thrift thread */
static const Thread::id_t unknown_thread_id;
virtual Thread::id_t getCurrentThreadId() const = 0;
};
}}} // apache::thrift::concurrency
#endif // #ifndef _THRIFT_CONCURRENCY_THREAD_H_

View File

@ -0,0 +1,202 @@
/*
* 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 THRIFT_CONCURRENCY_THREADLOCAL_H_
#define THRIFT_CONCURRENCY_THREADLOCAL_H_ 1
#include "thrift/lib/cpp/Thrift.h"
#include <pthread.h>
namespace apache { namespace thrift { namespace concurrency {
template <typename T>
class DefaultThreadLocalManager;
/**
* ThreadLocal manages thread-local storage for a particular object type.
*
* Each ThreadLocal object contains a separate instance of an object for each
* thread that accesses the ThreadLocal object.
*
* Note that you should avoid creating too many ThreadLocal objects (e.g., such
* as keeping a ThreadLocal member variable in commonly allocated objects).
* The number of ThreadLocal objects cannot be larger than the value of
* PTHREAD_KEYS_MAX, which is 1024 on many systems.
*
* The ManagerT template parameter controls how object allocation and
* deallocation should be performed. When get() is called from a thread that
* does not already have an instance of the object, Manager::allocate() is
* called. When a thread exits, Manager::destroy() is called if the thread has
* an instance of this object.
*/
template <typename T, typename ManagerT = DefaultThreadLocalManager<T> >
class ThreadLocal {
public:
typedef T DataType;
typedef ManagerT Manager;
/**
* Create a new ThreadLocal object.
*/
ThreadLocal() {
int ret = pthread_key_create(&key_, &ThreadLocal::onThreadExit);
if (ret != 0) {
throw TLibraryException("failed to allocate new thread-local key", ret);
}
}
/**
* Access this thread's local instance of the object.
*
* If there is no instance of the object in this thread, Manager::allocate()
* will be called to allocate a new instance. (Though some Manager
* implementations may return NULL, if each thread's instance must be
* expilcitly initialized.)
*/
T *get() const {
T *obj = getNoAlloc();
if (obj == NULL) {
Manager m;
obj = m.allocate();
if (obj != NULL) {
setImpl(obj);
}
}
return obj;
}
/**
* Access this thread's local instance of the object.
*
* If there is no instance of the object in this thread, NULL will be
* returned. Manager::allocate() will never be called.
*/
T *getNoAlloc() const {
return static_cast<T*>(pthread_getspecific(key_));
}
/**
* Operator overload to perform get()
*/
T *operator->() const {
return get();
}
/**
* Operator overload to perform get()
*/
T &operator*() const {
return *get();
}
/**
* Set the instance of the object to be used by this thread.
*/
void set(T* obj) {
T *old = getNoAlloc();
Manager m;
m.replace(old, obj);
setImpl(obj);
}
/**
* Clear the instance of the object used by this thread.
*
* If this thread had a non-NULL object, Manager::destroy() will be called.
*/
void clear() {
T *obj = getNoAlloc();
if (obj != NULL) {
Manager m;
m.destroy(obj);
setImpl(NULL);
}
}
private:
void setImpl(T* obj) const {
int ret = pthread_setspecific(key_, obj);
if (ret != 0) {
throw TLibraryException("failed to update thread-local key", ret);
}
}
static void onThreadExit(void* arg) {
T *obj = static_cast<T*>(arg);
if (obj != NULL) {
Manager m;
m.destroy(obj);
}
}
pthread_key_t key_;
};
template <typename T>
class DefaultThreadLocalManager {
public:
T* allocate() {
return new T;
}
void destroy(T* t) {
delete t;
}
void replace(T* oldObj, T* newObj) {
if (oldObj != newObj) {
delete oldObj;
}
}
};
template <typename T>
class DestroyOnlyThreadLocalManager {
public:
T* allocate() {
return NULL;
}
void destroy(T* t) {
delete t;
}
void replace(T* oldObj, T* newObj) {
if (oldObj != newObj) {
delete oldObj;
}
}
};
template <typename T>
class NoopThreadLocalManager {
public:
T* allocate() {
return NULL;
}
void destroy(T*) {
}
void replace(T*, T*) {
}
};
}}} // apache::thrift::concurrency
#endif // THRIFT_CONCURRENCY_THREADLOCAL_H_

View File

@ -0,0 +1,216 @@
/*
* 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 _THRIFT_CONCURRENCY_THREADMANAGER_H_
#define _THRIFT_CONCURRENCY_THREADMANAGER_H_ 1
#include <boost/shared_ptr.hpp>
#include <tr1/functional>
#include <sys/types.h>
namespace apache { namespace thrift { namespace concurrency {
class Runnable;
class ThreadFactory;
/**
* ThreadManager class
*
* This class manages a pool of threads. It uses a ThreadFactory to create
* threads. It never actually creates or destroys worker threads, rather
* It maintains statistics on number of idle threads, number of active threads,
* task backlog, and average wait and service times and informs the PoolPolicy
* object bound to instances of this manager of interesting transitions. It is
* then up the PoolPolicy object to decide if the thread pool size needs to be
* adjusted and call this object addWorker and removeWorker methods to make
* changes.
*
* This design allows different policy implementations to used this code to
* handle basic worker thread management and worker task execution and focus on
* policy issues. The simplest policy, StaticPolicy, does nothing other than
* create a fixed number of threads.
*/
class ThreadManager {
protected:
ThreadManager() {}
public:
class Task;
typedef std::tr1::function<void(boost::shared_ptr<Runnable>)> ExpireCallback;
typedef std::tr1::function<void()> InitCallback;
virtual ~ThreadManager() {}
/**
* Starts the thread manager. Verifies all attributes have been properly
* initialized, then allocates necessary resources to begin operation
*/
virtual void start() = 0;
/**
* Stops the thread manager. Aborts all remaining unprocessed task, shuts
* down all created worker threads, and releases all allocated resources.
* This method blocks for all worker threads to complete, thus it can
* potentially block forever if a worker thread is running a task that
* won't terminate.
*/
virtual void stop() = 0;
/**
* Joins the thread manager. This is the same as stop, except that it will
* wait until all the tasks have finished, rather than aborting the tasks.
*/
virtual void join() = 0;
enum STATE {
UNINITIALIZED,
STARTING,
STARTED,
JOINING,
STOPPING,
STOPPED
};
virtual const STATE state() const = 0;
virtual boost::shared_ptr<ThreadFactory> threadFactory() const = 0;
virtual void threadFactory(boost::shared_ptr<ThreadFactory> value) = 0;
virtual void addWorker(size_t value=1) = 0;
virtual void removeWorker(size_t value=1) = 0;
/**
* Gets the current number of idle worker threads
*/
virtual size_t idleWorkerCount() const = 0;
/**
* Gets the current number of total worker threads
*/
virtual size_t workerCount() const = 0;
/**
* Gets the current number of pending tasks
*/
virtual size_t pendingTaskCount() const = 0;
/**
* Gets the current number of pending and executing tasks
*/
virtual size_t totalTaskCount() const = 0;
/**
* Gets the maximum pending task count. 0 indicates no maximum
*/
virtual size_t pendingTaskCountMax() const = 0;
/**
* Gets the number of tasks which have been expired without being run.
*/
virtual size_t expiredTaskCount() = 0;
/**
* Adds a task to be executed at some time in the future by a worker thread.
*
* This method will block if pendingTaskCountMax() in not zero and
* pendingTaskCount() is greater than or equal to pendingTaskCountMax(). If
* this method is called in the context of a ThreadManager worker thread it
* will throw a TooManyPendingTasksException
*
* @param task The task to queue for execution
*
* @param timeout Time to wait in milliseconds to add a task when a
* pending-task-count is specified. Specific cases:
* timeout = 0 : Wait forever to queue task.
* timeout = -1 : Return immediately if pending task count exceeds specified
* max
* @param expiration when nonzero, the number of milliseconds the task is
* valid to be run; if exceeded, the task will be dropped off the queue and
* not run.
*
* @throws TooManyPendingTasksException Pending task count exceeds max
* pending task count
*/
virtual void add(boost::shared_ptr<Runnable>task,
int64_t timeout=0LL,
int64_t expiration=0LL) = 0;
/**
* Removes a pending task
*/
virtual void remove(boost::shared_ptr<Runnable> task) = 0;
/**
* Remove the next pending task which would be run.
*
* @return the task removed.
*/
virtual boost::shared_ptr<Runnable> removeNextPending() = 0;
/**
* Set a callback to be called when a task is expired and not run.
*
* @param expireCallback a function called with the shared_ptr<Runnable> for
* the expired task.
*/
virtual void setExpireCallback(ExpireCallback expireCallback) = 0;
/**
* Set a callback to be called when a worker thread is created.
*/
virtual void setThreadInitCallback(InitCallback initCallback) = 0;
static boost::shared_ptr<ThreadManager> newThreadManager();
/**
* Creates a simple thread manager the uses count number of worker threads
* and has a pendingTaskCountMax maximum pending tasks. The default, 0,
* specified no limit on pending tasks
*/
static boost::shared_ptr<ThreadManager>
newSimpleThreadManager(size_t count = 4,
size_t pendingTaskCountMax = 0,
bool enableTaskStats = false);
/**
* Get an internal statistics.
*
* @param waitTimeUs - average time (us) task spent in a queue
* @param runTimeUs - average time (us) task spent running
* @param maxItems - max items collected for stats
*/
virtual void getStats(int64_t& waitTimeUs, int64_t& runTimeUs,
int64_t maxItems) {
waitTimeUs = 0;
runTimeUs = 0;
}
class Task;
class Worker;
class Impl;
};
}}} // apache::thrift::concurrency
#endif // #ifndef _THRIFT_CONCURRENCY_THREADMANAGER_H_

View File

@ -0,0 +1,122 @@
/*
* 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 _THRIFT_CONCURRENCY_TIMERMANAGER_H_
#define _THRIFT_CONCURRENCY_TIMERMANAGER_H_ 1
#include "thrift/lib/cpp/concurrency/Exception.h"
#include "thrift/lib/cpp/concurrency/Monitor.h"
#include "thrift/lib/cpp/concurrency/Thread.h"
#include <boost/shared_ptr.hpp>
#include <map>
#include <time.h>
namespace apache { namespace thrift { namespace concurrency {
/**
* Timer Manager
*
* This class dispatches timer tasks when they fall due.
*
* @version $Id:$
*/
class TimerManager {
public:
TimerManager();
virtual ~TimerManager();
virtual boost::shared_ptr<const ThreadFactory> threadFactory() const;
virtual void threadFactory(boost::shared_ptr<const ThreadFactory> value);
/**
* Starts the timer manager service
*
* @throws IllegalArgumentException Missing thread factory attribute
*/
virtual void start();
/**
* Stops the timer manager service
*/
virtual void stop();
virtual size_t taskCount() const ;
/**
* Adds a task to be executed at some time in the future by a worker thread.
*
* @param task The task to execute
* @param timeout Time in milliseconds to delay before executing task
*/
virtual void add(boost::shared_ptr<Runnable> task, int64_t timeout);
/**
* Adds a task to be executed at some time in the future by a worker thread.
*
* @param task The task to execute
* @param timeout Absolute time in the future to execute task.
*/
virtual void add(boost::shared_ptr<Runnable> task, const struct timespec& timeout);
/**
* Removes a pending task
*
* @throws NoSuchTaskException Specified task doesn't exist. It was either
* processed already or this call was made for a
* task that was never added to this timer
*
* @throws UncancellableTaskException Specified task is already being
* executed or has completed execution.
*/
virtual void remove(boost::shared_ptr<Runnable> task);
enum STATE {
UNINITIALIZED,
STARTING,
STARTED,
STOPPING,
STOPPED
};
virtual const STATE state() const;
private:
boost::shared_ptr<const ThreadFactory> threadFactory_;
class Task;
friend class Task;
std::multimap<int64_t, boost::shared_ptr<Task> > taskMap_;
size_t taskCount_;
Monitor monitor_;
STATE state_;
class Dispatcher;
friend class Dispatcher;
boost::shared_ptr<Dispatcher> dispatcher_;
boost::shared_ptr<Thread> dispatcherThread_;
typedef std::multimap<int64_t, boost::shared_ptr<TimerManager::Task> >::iterator task_iterator;
typedef std::pair<task_iterator, task_iterator> task_range;
};
}}} // apache::thrift::concurrency
#endif // #ifndef _THRIFT_CONCURRENCY_TIMERMANAGER_H_

View File

@ -0,0 +1,167 @@
/*
* 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 _THRIFT_CONCURRENCY_UTIL_H_
#define _THRIFT_CONCURRENCY_UTIL_H_ 1
#include <assert.h>
#include <stddef.h>
#include <stdint.h>
#include <time.h>
#include <sys/time.h>
namespace apache { namespace thrift { namespace concurrency {
/**
* Utility methods
*
* This class contains basic utility methods for converting time formats,
* and other common platform-dependent concurrency operations.
* It should not be included in API headers for other concurrency library
* headers, since it will, by definition, pull in all sorts of horrid
* platform dependent crap. Rather it should be included directly in
* concurrency library implementation source.
*
* @version $Id:$
*/
class Util {
public:
static const int64_t NS_PER_S = 1000000000LL;
static const int64_t US_PER_S = 1000000LL;
static const int64_t MS_PER_S = 1000LL;
static const int64_t NS_PER_MS = NS_PER_S / MS_PER_S;
static const int64_t NS_PER_US = NS_PER_S / US_PER_S;
static const int64_t US_PER_MS = US_PER_S / MS_PER_S;
/**
* Converts millisecond timestamp into a timespec struct
*
* @param struct timespec& result
* @param time or duration in milliseconds
*/
static void toTimespec(struct timespec& result, int64_t value) {
result.tv_sec = value / MS_PER_S; // ms to s
result.tv_nsec = (value % MS_PER_S) * NS_PER_MS; // ms to ns
}
static void toTimeval(struct timeval& result, int64_t value) {
result.tv_sec = value / MS_PER_S; // ms to s
result.tv_usec = (value % MS_PER_S) * US_PER_MS; // ms to us
}
static const void toTicks(int64_t& result, int64_t secs, int64_t oldTicks,
int64_t oldTicksPerSec, int64_t newTicksPerSec) {
result = secs * newTicksPerSec;
result += oldTicks * newTicksPerSec / oldTicksPerSec;
int64_t oldPerNew = oldTicksPerSec / newTicksPerSec;
if (oldPerNew && ((oldTicks % oldPerNew) >= (oldPerNew / 2))) {
++result;
}
}
/**
* Converts struct timespec to arbitrary-sized ticks since epoch
*/
static const void toTicks(int64_t& result,
const struct timespec& value,
int64_t ticksPerSec) {
return toTicks(result, value.tv_sec, value.tv_nsec, NS_PER_S, ticksPerSec);
}
/**
* Converts struct timeval to arbitrary-sized ticks since epoch
*/
static const void toTicks(int64_t& result,
const struct timeval& value,
int64_t ticksPerSec) {
return toTicks(result, value.tv_sec, value.tv_usec, US_PER_S, ticksPerSec);
}
/**
* Converts struct timespec to milliseconds
*/
static const void toMilliseconds(int64_t& result,
const struct timespec& value) {
return toTicks(result, value, MS_PER_S);
}
/**
* Converts struct timeval to milliseconds
*/
static const void toMilliseconds(int64_t& result,
const struct timeval& value) {
return toTicks(result, value, MS_PER_S);
}
/**
* Converts struct timespec to microseconds
*/
static const void toUsec(int64_t& result, const struct timespec& value) {
return toTicks(result, value, US_PER_S);
}
/**
* Converts struct timeval to microseconds
*/
static const void toUsec(int64_t& result, const struct timeval& value) {
return toTicks(result, value, US_PER_S);
}
/**
* Get current time as a number of arbitrary-size ticks from epoch
*/
static const int64_t currentTimeTicks(int64_t ticksPerSec);
/**
* Get current time as milliseconds from epoch
*/
static const int64_t currentTime() { return currentTimeTicks(MS_PER_S); }
/**
* Get current time as micros from epoch
*/
static const int64_t currentTimeUsec() { return currentTimeTicks(US_PER_S); }
/**
* Get monotonic time as a number of arbitrary-size ticks from some
* unspecified starting point.
*
* This may fall back to the current time (potentially non-monotonic) on
* systems that do not support monotonic time.
*/
static const int64_t monotonicTimeTicks(int64_t ticksPerSec);
/**
* Get monotonic time as milliseconds.
*/
static const int64_t monotonicTime() { return monotonicTimeTicks(MS_PER_S); }
/**
* Get current time as micros from epoch
*/
static const int64_t monotonicTimeUsec() {
return monotonicTimeTicks(US_PER_S);
}
};
}}} // apache::thrift::concurrency
#endif // #ifndef _THRIFT_CONCURRENCY_UTIL_H_

View File

@ -0,0 +1,45 @@
# TARGETS file for thrift/lib/cpp/concurrency/test
cpp_binary (
name = "test",
srcs = [
"Tests.cpp"
],
deps = [
"@/thrift/lib/cpp/concurrency"
]
)
cpp_unittest (
name = "test_mutex",
srcs = [
"RWMutexTest.cpp"
],
deps = [
"@/thrift/lib/cpp/concurrency",
"@/thrift/lib/cpp:thrift",
],
external_deps = [
('boost', '>= 1.37', 'boost_unit_test_framework'),
],
type = 'boost',
owner = 'putivsky',
emails = ['adback-dev@lists.facebook.com',
'thrift-team@lists.facebook.com'],
)
cpp_unittest(
name = 'ThreadManagerTests',
srcs = [
'ThreadManagerTests.cpp',
],
deps = [
'@/thrift/lib/cpp/concurrency',
],
external_deps = [
('boost', '>= 1.37', 'boost_unit_test_framework'),
],
type = 'boost',
emails = ['thrift-team@lists.facebook.com'],
)

View File

@ -0,0 +1,354 @@
/*
* 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 "thrift/lib/cpp/config.h"
#include "thrift/lib/cpp/concurrency/Thread.h"
#include "thrift/lib/cpp/concurrency/PosixThreadFactory.h"
#include "thrift/lib/cpp/concurrency/Monitor.h"
#include "thrift/lib/cpp/concurrency/Util.h"
#include <assert.h>
#include <unistd.h>
#include <iostream>
#include <set>
namespace apache { namespace thrift { namespace concurrency { namespace test {
using boost::shared_ptr;
using namespace apache::thrift::concurrency;
/**
* ThreadManagerTests class
*
* @version $Id:$
*/
class ThreadFactoryTests {
public:
static const double ERROR;
class Task: public Runnable {
public:
Task() {}
void run() {
std::cout << "\t\t\tHello World" << std::endl;
}
};
/**
* Hello world test
*/
bool helloWorldTest() {
PosixThreadFactory threadFactory = PosixThreadFactory();
shared_ptr<Task> task = shared_ptr<Task>(new ThreadFactoryTests::Task());
shared_ptr<Thread> thread = threadFactory.newThread(task);
thread->start();
thread->join();
std::cout << "\t\t\tSuccess!" << std::endl;
return true;
}
/**
* Reap N threads
*/
class ReapNTask: public Runnable {
public:
ReapNTask(Monitor& monitor, int& activeCount) :
_monitor(monitor),
_count(activeCount) {}
void run() {
Synchronized s(_monitor);
_count--;
//std::cout << "\t\t\tthread count: " << _count << std::endl;
if (_count == 0) {
_monitor.notify();
}
}
Monitor& _monitor;
int& _count;
};
bool reapNThreads(int loop=1, int count=10) {
PosixThreadFactory threadFactory = PosixThreadFactory();
Monitor* monitor = new Monitor();
for(int lix = 0; lix < loop; lix++) {
int* activeCount = new int(count);
std::set<shared_ptr<Thread> > threads;
int tix;
for (tix = 0; tix < count; tix++) {
try {
threads.insert(threadFactory.newThread(shared_ptr<Runnable>(new ReapNTask(*monitor, *activeCount))));
} catch(SystemResourceException& e) {
std::cout << "\t\t\tfailed to create " << lix * count + tix << " thread " << e.what() << std::endl;
throw e;
}
}
tix = 0;
for (std::set<shared_ptr<Thread> >::const_iterator thread = threads.begin(); thread != threads.end(); tix++, ++thread) {
try {
(*thread)->start();
} catch(SystemResourceException& e) {
std::cout << "\t\t\tfailed to start " << lix * count + tix << " thread " << e.what() << std::endl;
throw e;
}
}
{
Synchronized s(*monitor);
while (*activeCount > 0) {
monitor->wait(1000);
}
}
std::cout << "\t\t\treaped " << lix * count << " threads" << std::endl;
}
std::cout << "\t\t\tSuccess!" << std::endl;
return true;
}
class SynchStartTask: public Runnable {
public:
enum STATE {
UNINITIALIZED,
STARTING,
STARTED,
STOPPING,
STOPPED
};
SynchStartTask(Monitor& monitor, volatile STATE& state) :
_monitor(monitor),
_state(state) {}
void run() {
{
Synchronized s(_monitor);
if (_state == SynchStartTask::STARTING) {
_state = SynchStartTask::STARTED;
_monitor.notify();
}
}
{
Synchronized s(_monitor);
while (_state == SynchStartTask::STARTED) {
_monitor.wait();
}
if (_state == SynchStartTask::STOPPING) {
_state = SynchStartTask::STOPPED;
_monitor.notifyAll();
}
}
}
private:
Monitor& _monitor;
volatile STATE& _state;
};
bool synchStartTest() {
Monitor monitor;
SynchStartTask::STATE state = SynchStartTask::UNINITIALIZED;
shared_ptr<SynchStartTask> task = shared_ptr<SynchStartTask>(new SynchStartTask(monitor, state));
PosixThreadFactory threadFactory = PosixThreadFactory();
shared_ptr<Thread> thread = threadFactory.newThread(task);
if (state == SynchStartTask::UNINITIALIZED) {
state = SynchStartTask::STARTING;
thread->start();
}
{
Synchronized s(monitor);
while (state == SynchStartTask::STARTING) {
monitor.wait();
}
}
assert(state != SynchStartTask::STARTING);
{
Synchronized s(monitor);
try {
monitor.wait(100);
} catch(TimedOutException& e) {
}
if (state == SynchStartTask::STARTED) {
state = SynchStartTask::STOPPING;
monitor.notify();
}
while (state == SynchStartTask::STOPPING) {
monitor.wait();
}
}
assert(state == SynchStartTask::STOPPED);
bool success = true;
std::cout << "\t\t\t" << (success ? "Success" : "Failure") << "!" << std::endl;
return true;
}
/** See how accurate monitor timeout is. */
bool monitorTimeoutTest(size_t count=1000, int64_t timeout=10) {
Monitor monitor;
int64_t startTime = Util::currentTime();
for (size_t ix = 0; ix < count; ix++) {
{
Synchronized s(monitor);
try {
monitor.wait(timeout);
} catch(TimedOutException& e) {
}
}
}
int64_t endTime = Util::currentTime();
double error = ((endTime - startTime) - (count * timeout)) / (double)(count * timeout);
if (error < 0.0) {
error *= 1.0;
}
bool success = error < ThreadFactoryTests::ERROR;
std::cout << "\t\t\t" << (success ? "Success" : "Failure") << "! expected time: " << count * timeout << "ms elapsed time: "<< endTime - startTime << "ms error%: " << error * 100.0 << std::endl;
return success;
}
class FloodTask : public Runnable {
public:
FloodTask(const size_t id) :_id(id) {}
~FloodTask(){
if(_id % 1000 == 0) {
std::cout << "\t\tthread " << _id << " done" << std::endl;
}
}
void run(){
if(_id % 1000 == 0) {
std::cout << "\t\tthread " << _id << " started" << std::endl;
}
usleep(1);
}
const size_t _id;
};
void foo(PosixThreadFactory *tf) {
}
bool floodNTest(size_t loop=1, size_t count=100000) {
bool success = false;
for(size_t lix = 0; lix < loop; lix++) {
PosixThreadFactory threadFactory = PosixThreadFactory();
threadFactory.setDetached(true);
for(size_t tix = 0; tix < count; tix++) {
try {
shared_ptr<FloodTask> task(new FloodTask(lix * count + tix ));
shared_ptr<Thread> thread = threadFactory.newThread(task);
thread->start();
usleep(1);
} catch (TException& e) {
std::cout << "\t\t\tfailed to start " << lix * count + tix << " thread " << e.what() << std::endl;
return success;
}
}
std::cout << "\t\t\tflooded " << (lix + 1) * count << " threads" << std::endl;
success = true;
}
return success;
}
};
const double ThreadFactoryTests::ERROR = .20;
}}}} // apache::thrift::concurrency::test

View File

@ -0,0 +1,142 @@
/*
* 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 "thrift/lib/cpp/concurrency/TimerManager.h"
#include "thrift/lib/cpp/concurrency/PosixThreadFactory.h"
#include "thrift/lib/cpp/concurrency/Monitor.h"
#include "thrift/lib/cpp/concurrency/Util.h"
#include <assert.h>
#include <iostream>
namespace apache { namespace thrift { namespace concurrency { namespace test {
using namespace apache::thrift::concurrency;
/**
* ThreadManagerTests class
*
* @version $Id:$
*/
class TimerManagerTests {
public:
static const double ERROR;
class Task: public Runnable {
public:
Task(Monitor& monitor, int64_t timeout) :
_timeout(timeout),
_startTime(Util::currentTime()),
_monitor(monitor),
_success(false),
_done(false) {}
~Task() { std::cerr << this << std::endl; }
void run() {
_endTime = Util::currentTime();
// Figure out error percentage
int64_t delta = _endTime - _startTime;
delta = delta > _timeout ? delta - _timeout : _timeout - delta;
float error = delta / _timeout;
if(error < ERROR) {
_success = true;
}
_done = true;
std::cout << "\t\t\tTimerManagerTests::Task[" << this << "] done" << std::endl; //debug
{Synchronized s(_monitor);
_monitor.notifyAll();
}
}
int64_t _timeout;
int64_t _startTime;
int64_t _endTime;
Monitor& _monitor;
bool _success;
bool _done;
};
/**
* This test creates two tasks and waits for the first to expire within 10%
* of the expected expiration time. It then verifies that the timer manager
* properly clean up itself and the remaining orphaned timeout task when the
* manager goes out of scope and its destructor is called.
*/
bool test00(int64_t timeout=1000LL) {
shared_ptr<TimerManagerTests::Task> orphanTask = shared_ptr<TimerManagerTests::Task>(new TimerManagerTests::Task(_monitor, 10 * timeout));
{
TimerManager timerManager;
timerManager.threadFactory(shared_ptr<PosixThreadFactory>(new PosixThreadFactory()));
timerManager.start();
assert(timerManager.state() == TimerManager::STARTED);
shared_ptr<TimerManagerTests::Task> task = shared_ptr<TimerManagerTests::Task>(new TimerManagerTests::Task(_monitor, timeout));
{
Synchronized s(_monitor);
timerManager.add(orphanTask, 10 * timeout);
timerManager.add(task, timeout);
_monitor.wait();
}
assert(task->_done);
std::cout << "\t\t\t" << (task->_success ? "Success" : "Failure") << "!" << std::endl;
}
// timerManager.stop(); This is where it happens via destructor
assert(!orphanTask->_done);
return true;
}
friend class TestTask;
Monitor _monitor;
};
const double TimerManagerTests::ERROR = .20;
}}}} // apache::thrift::concurrency

316
thrift/lib/cpp/config.h Normal file
View File

@ -0,0 +1,316 @@
/* config.h. Generated by configure. */
/*
* Generated by simpkins on 2009-11-18,
* on an x86_64 CentOS 5.2 system.
*/
/* config.hin. Generated from configure.ac by autoheader. */
/* Define if the AI_ADDRCONFIG symbol is unavailable */
/* #undef AI_ADDRCONFIG */
/* Possible value for SIGNED_RIGHT_SHIFT_IS */
#define ARITHMETIC_RIGHT_SHIFT 1
/* Define to one of `_getb67', `GETB67', `getb67' for Cray-2 and Cray-YMP
systems. This function is required for `alloca.c' support on those systems.
*/
/* #undef CRAY_STACKSEG_END */
/* Define to 1 if using `alloca.c'. */
/* #undef C_ALLOCA */
/* Define to 1 if you have `alloca', as a function or macro. */
#define HAVE_ALLOCA 1
/* Define to 1 if you have <alloca.h> and it should be used (not on Ultrix).
*/
#define HAVE_ALLOCA_H 1
/* Define to 1 if you have the <arpa/inet.h> header file. */
#define HAVE_ARPA_INET_H 1
/* define if the Boost library is available */
#define HAVE_BOOST
/* Define to 1 if you have the `bzero' function. */
#define HAVE_BZERO 1
/* Define to 1 if you have the `clock_gettime' function. */
#define HAVE_CLOCK_GETTIME 1
/* Define to 1 if you have the declaration of `strerror_r', and to 0 if you
don't. */
#define HAVE_DECL_STRERROR_R 1
/* Define to 1 if you have the <dlfcn.h> header file. */
#define HAVE_DLFCN_H 1
/* Define to 1 if you don't have `vprintf' but do have `_doprnt.' */
/* #undef HAVE_DOPRNT */
/* Define to 1 if you have the <endian.h> header file. */
#define HAVE_ENDIAN_H 1
/* Define to 1 if you have the <fcntl.h> header file. */
#define HAVE_FCNTL_H 1
/* Define to 1 if you have the `ftruncate' function. */
#define HAVE_FTRUNCATE 1
/* Define to 1 if you have the `gethostbyname' function. */
#define HAVE_GETHOSTBYNAME 1
/* Define to 1 if you have the `gettimeofday' function. */
#define HAVE_GETTIMEOFDAY 1
/* Define to 1 if you have the <inttypes.h> header file. */
#define HAVE_INTTYPES_H 1
/* define if libevent is available */
#define HAVE_LIBEVENT
/* Define to 1 if you have the <libintl.h> header file. */
#define HAVE_LIBINTL_H 1
/* Define to 1 if you have the `pthread' library (-lpthread). */
#define HAVE_LIBPTHREAD 1
/* Define to 1 if you have the `rt' library (-lrt). */
#define HAVE_LIBRT 1
/* Define to 1 if you have the `socket' library (-lsocket). */
/* #undef HAVE_LIBSOCKET */
/* Define to 1 if you have the <limits.h> header file. */
#define HAVE_LIMITS_H 1
/* Define to 1 if your system has a GNU libc compatible `malloc' function, and
to 0 otherwise. */
#define HAVE_MALLOC 1
/* Define to 1 if you have the <malloc.h> header file. */
#define HAVE_MALLOC_H 1
/* Define to 1 if you have the `memmove' function. */
#define HAVE_MEMMOVE 1
/* Define to 1 if you have the <memory.h> header file. */
#define HAVE_MEMORY_H 1
/* Define to 1 if you have the `memset' function. */
#define HAVE_MEMSET 1
/* Define to 1 if you have the `mkdir' function. */
#define HAVE_MKDIR 1
/* Define to 1 if you have the <netdb.h> header file. */
#define HAVE_NETDB_H 1
/* Define to 1 if you have the <netinet/in.h> header file. */
#define HAVE_NETINET_IN_H 1
/* Define to 1 if you have the <pthread.h> header file. */
#define HAVE_PTHREAD_H 1
/* Define to 1 if the system has the type `ptrdiff_t'. */
#define HAVE_PTRDIFF_T 1
/* Define to 1 if your system has a GNU libc compatible `realloc' function,
and to 0 otherwise. */
#define HAVE_REALLOC 1
/* Define to 1 if you have the `realpath' function. */
#define HAVE_REALPATH 1
/* Define to 1 if you have the `sched_get_priority_max' function. */
#define HAVE_SCHED_GET_PRIORITY_MAX 1
/* Define to 1 if you have the `sched_get_priority_min' function. */
#define HAVE_SCHED_GET_PRIORITY_MIN 1
/* Define to 1 if you have the `select' function. */
#define HAVE_SELECT 1
/* Define to 1 if you have the `socket' function. */
#define HAVE_SOCKET 1
/* Define to 1 if you have the `sqrt' function. */
#define HAVE_SQRT 1
/* Define to 1 if `stat' has the bug that it succeeds when given the
zero-length file name argument. */
/* #undef HAVE_STAT_EMPTY_STRING_BUG */
/* Define to 1 if stdbool.h conforms to C99. */
#define HAVE_STDBOOL_H 1
/* Define to 1 if you have the <stddef.h> header file. */
#define HAVE_STDDEF_H 1
/* Define to 1 if you have the <stdint.h> header file. */
#define HAVE_STDINT_H 1
/* Define to 1 if you have the <stdlib.h> header file. */
#define HAVE_STDLIB_H 1
/* Define to 1 if you have the `strchr' function. */
#define HAVE_STRCHR 1
/* Define to 1 if you have the `strdup' function. */
#define HAVE_STRDUP 1
/* Define to 1 if you have the `strerror' function. */
#define HAVE_STRERROR 1
/* Define to 1 if you have the `strerror_r' function. */
#define HAVE_STRERROR_R 1
/* Define to 1 if you have the `strftime' function. */
#define HAVE_STRFTIME 1
/* Define to 1 if you have the <strings.h> header file. */
#define HAVE_STRINGS_H 1
/* Define to 1 if you have the <string.h> header file. */
#define HAVE_STRING_H 1
/* Define to 1 if you have the `strstr' function. */
#define HAVE_STRSTR 1
/* Define to 1 if you have the `strtol' function. */
#define HAVE_STRTOL 1
/* Define to 1 if you have the `strtoul' function. */
#define HAVE_STRTOUL 1
/* Define to 1 if you have the <sys/select.h> header file. */
#define HAVE_SYS_SELECT_H 1
/* Define to 1 if you have the <sys/socket.h> header file. */
#define HAVE_SYS_SOCKET_H 1
/* Define to 1 if you have the <sys/stat.h> header file. */
#define HAVE_SYS_STAT_H 1
/* Define to 1 if you have the <sys/time.h> header file. */
#define HAVE_SYS_TIME_H 1
/* Define to 1 if you have the <sys/types.h> header file. */
#define HAVE_SYS_TYPES_H 1
/* Define to 1 if you have the <unistd.h> header file. */
#define HAVE_UNISTD_H 1
/* Define to 1 if you have the `vprintf' function. */
#define HAVE_VPRINTF 1
/* define if zlib is available */
#define HAVE_ZLIB
/* Define to 1 if the system has the type `_Bool'. */
/* #undef HAVE__BOOL */
/* Possible value for SIGNED_RIGHT_SHIFT_IS */
#define LOGICAL_RIGHT_SHIFT 2
/* Define to 1 if `lstat' dereferences a symlink specified with a trailing
slash. */
#define LSTAT_FOLLOWS_SLASHED_SYMLINK 1
/* Name of package */
#define PACKAGE "thrift"
/* Define to the address where bug reports for this package should be sent. */
#define PACKAGE_BUGREPORT ""
/* Define to the full name of this package. */
#define PACKAGE_NAME "thrift"
/* Define to the full name and version of this package. */
#define PACKAGE_STRING "thrift 20080411"
/* Define to the one symbol short name of this package. */
#define PACKAGE_TARNAME "thrift"
/* Define to the version of this package. */
#define PACKAGE_VERSION "20080411"
/* Define to the type of arg 1 for `select'. */
#define SELECT_TYPE_ARG1 int
/* Define to the type of args 2, 3 and 4 for `select'. */
#define SELECT_TYPE_ARG234 (fd_set *)
/* Define to the type of arg 5 for `select'. */
#define SELECT_TYPE_ARG5 (struct timeval *)
/* Indicates the effect of the right shift operator on negative signed
integers */
#define SIGNED_RIGHT_SHIFT_IS 1
/* If using the C implementation of alloca, define if you know the
direction of stack growth for your system; otherwise it will be
automatically deduced at run-time.
STACK_DIRECTION > 0 => grows toward higher addresses
STACK_DIRECTION < 0 => grows toward lower addresses
STACK_DIRECTION = 0 => direction of growth unknown */
/* #undef STACK_DIRECTION */
/* Define to 1 if you have the ANSI C header files. */
#define STDC_HEADERS 1
/* Define to 1 if strerror_r returns char *. */
#define STRERROR_R_CHAR_P 1
/* Define to 1 if you can safely include both <sys/time.h> and <time.h>. */
#define TIME_WITH_SYS_TIME 1
/* Define to 1 if your <sys/time.h> declares `struct tm'. */
/* #undef TM_IN_SYS_TIME */
/* Possible value for SIGNED_RIGHT_SHIFT_IS */
#define UNKNOWN_RIGHT_SHIFT 3
/* Version number of package */
#define VERSION "20080411"
/* Define to 1 if `lex' declares `yytext' as a `char *' by default, not a
`char[]'. */
#define YYTEXT_POINTER 1
/* Calls Google Perftools ProfilerRegisterThread() on pthread creation
Set define to 0. It is exposing a linux 6.12 posix bug when an
application receives a SIGKILL, which causes the machine to crash.
*/
#ifndef GOOGLE_PERFTOOLS_REGISTER_THREAD
# define GOOGLE_PERFTOOLS_REGISTER_THREAD 0
#endif
/* Define to empty if `const' does not conform to ANSI C. */
/* #undef const */
/* Define to `__inline__' or `__inline' if that's what the C compiler
calls it, or to nothing if 'inline' is not supported under any name. */
#ifndef __cplusplus
/* #undef inline */
#endif
/* Define to rpl_malloc if the replacement function should be used. */
/* #undef malloc */
/* Define to `int' if <sys/types.h> does not define. */
/* #undef mode_t */
/* Define to `long' if <sys/types.h> does not define. */
/* #undef off_t */
/* Define to rpl_realloc if the replacement function should be used. */
/* #undef realloc */
/* Define to `unsigned' if <sys/types.h> does not define. */
/* #undef size_t */
/* Define to empty if the keyword `volatile' does not work. Warning: valid
code using `volatile' can become incorrect without. Disable with care. */
/* #undef volatile */

View File

@ -0,0 +1,82 @@
/*
* 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 PEEKPROCESSOR_H
#define PEEKPROCESSOR_H
#include <string>
#include "thrift/lib/cpp/TProcessor.h"
#include "thrift/lib/cpp/transport/TTransport.h"
#include "thrift/lib/cpp/transport/TTransportUtils.h"
#include "thrift/lib/cpp/transport/TBufferTransports.h"
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift { namespace processor {
namespace server {
class TConnectionContext;
}
/*
* Class for peeking at the raw data that is being processed by another processor
* and gives the derived class a chance to change behavior accordingly
*
*/
class PeekProcessor : public apache::thrift::TProcessor {
public:
PeekProcessor();
virtual ~PeekProcessor();
// Input here: actualProcessor - the underlying processor
// protocolFactory - the protocol factory used to wrap the memory buffer
// transportFactory - this TPipedTransportFactory is used to wrap the source transport
// via a call to getPipedTransport
void initialize(boost::shared_ptr<apache::thrift::TProcessor> actualProcessor,
boost::shared_ptr<apache::thrift::protocol::TProtocolFactory> protocolFactory,
boost::shared_ptr<apache::thrift::transport::TPipedTransportFactory> transportFactory);
boost::shared_ptr<apache::thrift::transport::TTransport> getPipedTransport(boost::shared_ptr<apache::thrift::transport::TTransport> in);
void setTargetTransport(boost::shared_ptr<apache::thrift::transport::TTransport> targetTransport);
virtual bool process(boost::shared_ptr<apache::thrift::protocol::TProtocol> in,
boost::shared_ptr<apache::thrift::protocol::TProtocol> out,
TConnectionContext* connectionContext);
// The following three functions can be overloaded by child classes to
// achieve desired peeking behavior
virtual void peekName(const std::string& fname);
virtual void peekBuffer(uint8_t* buffer, uint32_t size);
virtual void peek(boost::shared_ptr<apache::thrift::protocol::TProtocol> in,
apache::thrift::protocol::TType ftype,
int16_t fid);
virtual void peekEnd();
private:
boost::shared_ptr<apache::thrift::TProcessor> actualProcessor_;
boost::shared_ptr<apache::thrift::protocol::TProtocol> pipedProtocol_;
boost::shared_ptr<apache::thrift::transport::TPipedTransportFactory> transportFactory_;
boost::shared_ptr<apache::thrift::transport::TMemoryBuffer> memoryBuffer_;
boost::shared_ptr<apache::thrift::transport::TTransport> targetTransport_;
};
}}} // apache::thrift::processor
#endif

View File

@ -0,0 +1,266 @@
/*
* 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 STATSPROCESSOR_H
#define STATSPROCESSOR_H
#include <boost/shared_ptr.hpp>
#include "thrift/lib/cpp/transport/TTransport.h"
#include "thrift/lib/cpp/protocol/TProtocol.h"
#include "thrift/lib/cpp/TProcessor.h"
namespace apache { namespace thrift { namespace processor {
/*
* Class for keeping track of function call statistics and printing them if desired
*
*/
class StatsProcessor : public apache::thrift::TProcessor {
public:
StatsProcessor(bool print, bool frequency)
: print_(print),
frequency_(frequency)
{}
virtual ~StatsProcessor() {};
virtual bool process(boost::shared_ptr<apache::thrift::protocol::TProtocol> piprot,
boost::shared_ptr<apache::thrift::protocol::TProtocol> poprot,
void* serverContext) {
piprot_ = piprot;
std::string fname;
apache::thrift::protocol::TMessageType mtype;
int32_t seqid;
piprot_->readMessageBegin(fname, mtype, seqid);
if (mtype != apache::thrift::protocol::T_CALL) {
if (print_) {
printf("Unknown message type\n");
}
throw apache::thrift::TLibraryException("Unexpected message type");
}
if (print_) {
printf("%s (", fname.c_str());
}
if (frequency_) {
if (frequency_map_.find(fname) != frequency_map_.end()) {
frequency_map_[fname]++;
} else {
frequency_map_[fname] = 1;
}
}
apache::thrift::protocol::TType ftype;
int16_t fid;
while (true) {
piprot_->readFieldBegin(fname, ftype, fid);
if (ftype == apache::thrift::protocol::T_STOP) {
break;
}
printAndPassToBuffer(ftype);
if (print_) {
printf(", ");
}
}
if (print_) {
printf("\b\b)\n");
}
return true;
}
const std::map<std::string, int64_t>& get_frequency_map() {
return frequency_map_;
}
protected:
void printAndPassToBuffer(apache::thrift::protocol::TType ftype) {
switch (ftype) {
case apache::thrift::protocol::T_BOOL:
{
bool boolv;
piprot_->readBool(boolv);
if (print_) {
printf("%d", boolv);
}
}
break;
case apache::thrift::protocol::T_BYTE:
{
int8_t bytev;
piprot_->readByte(bytev);
if (print_) {
printf("%d", bytev);
}
}
break;
case apache::thrift::protocol::T_I16:
{
int16_t i16;
piprot_->readI16(i16);
if (print_) {
printf("%d", i16);
}
}
break;
case apache::thrift::protocol::T_I32:
{
int32_t i32;
piprot_->readI32(i32);
if (print_) {
printf("%d", i32);
}
}
break;
case apache::thrift::protocol::T_I64:
{
int64_t i64;
piprot_->readI64(i64);
if (print_) {
printf("%ld", i64);
}
}
break;
case apache::thrift::protocol::T_DOUBLE:
{
double dub;
piprot_->readDouble(dub);
if (print_) {
printf("%f", dub);
}
}
break;
case apache::thrift::protocol::T_STRING:
{
std::string str;
piprot_->readString(str);
if (print_) {
printf("%s", str.c_str());
}
}
break;
case apache::thrift::protocol::T_STRUCT:
{
std::string name;
int16_t fid;
apache::thrift::protocol::TType ftype;
piprot_->readStructBegin(name);
if (print_) {
printf("<");
}
while (true) {
piprot_->readFieldBegin(name, ftype, fid);
if (ftype == apache::thrift::protocol::T_STOP) {
break;
}
printAndPassToBuffer(ftype);
if (print_) {
printf(",");
}
piprot_->readFieldEnd();
}
piprot_->readStructEnd();
if (print_) {
printf("\b>");
}
}
break;
case apache::thrift::protocol::T_MAP:
{
apache::thrift::protocol::TType keyType;
apache::thrift::protocol::TType valType;
uint32_t i, size;
piprot_->readMapBegin(keyType, valType, size);
if (print_) {
printf("{");
}
for (i = 0; i < size; i++) {
printAndPassToBuffer(keyType);
if (print_) {
printf("=>");
}
printAndPassToBuffer(valType);
if (print_) {
printf(",");
}
}
piprot_->readMapEnd();
if (print_) {
printf("\b}");
}
}
break;
case apache::thrift::protocol::T_SET:
{
apache::thrift::protocol::TType elemType;
uint32_t i, size;
piprot_->readSetBegin(elemType, size);
if (print_) {
printf("{");
}
for (i = 0; i < size; i++) {
printAndPassToBuffer(elemType);
if (print_) {
printf(",");
}
}
piprot_->readSetEnd();
if (print_) {
printf("\b}");
}
}
break;
case apache::thrift::protocol::T_LIST:
{
apache::thrift::protocol::TType elemType;
uint32_t i, size;
piprot_->readListBegin(elemType, size);
if (print_) {
printf("[");
}
for (i = 0; i < size; i++) {
printAndPassToBuffer(elemType);
if (print_) {
printf(",");
}
}
piprot_->readListEnd();
if (print_) {
printf("\b]");
}
}
break;
default:
break;
}
}
boost::shared_ptr<apache::thrift::protocol::TProtocol> piprot_;
std::map<std::string, int64_t> frequency_map_;
bool print_;
bool frequency_;
};
}}} // apache::thrift::processor
#endif

View File

@ -0,0 +1,9 @@
# TARGETS file for thrift/lib/cpp/processor
cpp_library (
name = "processor",
srcs = [
"PeekProcessor.cpp"
],
deps = [ "@/thrift/lib/cpp/transport" ]
)

View File

@ -0,0 +1,94 @@
/*
* 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 _THRIFT_TEST_EVENTLOG_H_
#define _THRIFT_TEST_EVENTLOG_H_ 1
#include "thrift/lib/cpp/concurrency/Monitor.h"
namespace apache { namespace thrift { namespace test {
// Initially I made EventType an enum, but using char* results
// in much more readable error messages when there is a mismatch.
// It also lets users of EventLog easily define their own new types.
// Comparing the literal pointer values should be safe, barring any strange
// linking setup that results in duplicate symbols.
typedef const char* EventType;
struct Event {
Event(EventType type, uint32_t connectionId, uint32_t callId,
const std::string& message) :
type(type),
connectionId(connectionId),
callId(callId),
message(message) {}
EventType type;
uint32_t connectionId;
uint32_t callId;
std::string message;
};
class EventLog {
public:
static EventType ET_LOG_END;
static EventType ET_CONN_CREATED;
static EventType ET_CONN_DESTROYED;
static EventType ET_CALL_STARTED;
static EventType ET_CALL_FINISHED;
static EventType ET_PROCESS;
static EventType ET_PRE_READ;
static EventType ET_POST_READ;
static EventType ET_PRE_WRITE;
static EventType ET_POST_WRITE;
static EventType ET_ASYNC_COMPLETE;
static EventType ET_HANDLER_ERROR;
static EventType ET_CALL_INCREMENT_GENERATION;
static EventType ET_CALL_GET_GENERATION;
static EventType ET_CALL_ADD_STRING;
static EventType ET_CALL_GET_STRINGS;
static EventType ET_CALL_GET_DATA_WAIT;
static EventType ET_CALL_ONEWAY_WAIT;
static EventType ET_CALL_UNEXPECTED_EXCEPTION_WAIT;
static EventType ET_CALL_EXCEPTION_WAIT;
static EventType ET_WAIT_RETURN;
static EventType ET_CALL_SET_VALUE;
static EventType ET_CALL_GET_VALUE;
EventLog();
void append(EventType type, uint32_t connectionId, uint32_t callId,
const std::string& message = "");
Event waitForEvent(int64_t timeout = 500);
Event waitForConnEvent(uint32_t connId, int64_t timeout = 500);
protected:
typedef std::list<Event> EventList;
concurrency::Monitor monitor_;
EventList events_;
uint32_t id_;
static uint32_t nextId_;
};
}}} // apache::thrift::test
#endif // _THRIFT_TEST_EVENTLOG_H_

View File

@ -0,0 +1,311 @@
/*
* 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 _THRIFT_PROCESSOR_TEST_HANDLERS_H_
#define _THRIFT_PROCESSOR_TEST_HANDLERS_H_ 1
#include "thrift/lib/cpp/processor/test/EventLog.h"
#include "thrift/lib/cpp/processor/test/gen-cpp/ChildService.h"
#include "thrift/lib/cpp/server/TConnectionContext.h"
namespace apache { namespace thrift { namespace test {
class ParentHandler : virtual public ParentServiceIf {
public:
ParentHandler(const boost::shared_ptr<EventLog>& log) :
triggerMonitor(&mutex_),
generation_(0),
wait_(false),
log_(log) { }
int32_t incrementGeneration() {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_INCREMENT_GENERATION, 0, 0);
return ++generation_;
}
int32_t getGeneration() {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_GET_GENERATION, 0, 0);
return generation_;
}
void addString(const std::string& s) {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_ADD_STRING, 0, 0);
strings_.push_back(s);
}
void getStrings(std::vector<std::string>& _return) {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_GET_STRINGS, 0, 0);
_return = strings_;
}
void getDataWait(std::string& _return, int32_t length) {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_GET_DATA_WAIT, 0, 0);
blockUntilTriggered();
_return.append(length, 'a');
}
void onewayWait() {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_ONEWAY_WAIT, 0, 0);
blockUntilTriggered();
}
void exceptionWait(const std::string& message) {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_EXCEPTION_WAIT, 0, 0);
blockUntilTriggered();
MyError e;
e.message = message;
throw e;
}
void unexpectedExceptionWait(const std::string& message) {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_UNEXPECTED_EXCEPTION_WAIT, 0, 0);
blockUntilTriggered();
MyError e;
e.message = message;
throw e;
}
/**
* After prepareTriggeredCall() is invoked, calls to any of the *Wait()
* functions won't return until triggerPendingCalls() is invoked
*
* This has to be a separate function invoked by the main test thread
* in order to to avoid race conditions.
*/
void prepareTriggeredCall() {
concurrency::Guard g(mutex_);
wait_ = true;
}
/**
* Wake up all calls waiting in blockUntilTriggered()
*/
void triggerPendingCalls() {
concurrency::Guard g(mutex_);
wait_ = false;
triggerMonitor.notifyAll();
}
protected:
/**
* blockUntilTriggered() won't return until triggerPendingCalls() is invoked
* in another thread.
*
* This should only be called when already holding mutex_.
*/
void blockUntilTriggered() {
while (wait_) {
triggerMonitor.waitForever();
}
// Log an event when we return
log_->append(EventLog::ET_WAIT_RETURN, 0, 0);
}
concurrency::Mutex mutex_;
concurrency::Monitor triggerMonitor;
int32_t generation_;
bool wait_;
std::vector<std::string> strings_;
boost::shared_ptr<EventLog> log_;
};
class ChildHandler : public ParentHandler, virtual public ChildServiceIf {
public:
ChildHandler(const boost::shared_ptr<EventLog>& log) :
ParentHandler(log),
value_(0) {}
int32_t setValue(int32_t value) {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_SET_VALUE, 0, 0);
int32_t oldValue = value_;
value_ = value;
return oldValue;
}
int32_t getValue() {
concurrency::Guard g(mutex_);
log_->append(EventLog::ET_CALL_GET_VALUE, 0, 0);
return value_;
}
protected:
int32_t value_;
};
struct ConnContext {
public:
ConnContext(server::TConnectionContext* ctx, uint32_t id) :
ctx(ctx),
id(id) {}
server::TConnectionContext* ctx;
uint32_t id;
};
struct CallContext {
public:
CallContext(ConnContext *context, uint32_t id, const std::string& name) :
connContext(context),
name(name),
id(id) {}
ConnContext *connContext;
std::string name;
uint32_t id;
};
class ServerEventHandler : public server::TServerEventHandler {
public:
ServerEventHandler(const boost::shared_ptr<EventLog>& log) :
nextId_(1),
log_(log) {}
virtual void preServe(const transport::TSocketAddress*) {}
virtual void newConnection(server::TConnectionContext* ctx) {
ConnContext* context = new ConnContext(ctx, nextId_);
++nextId_;
ctx->setUserData(context);
log_->append(EventLog::ET_CONN_CREATED, context->id, 0);
}
virtual void connectionDestroyed(server::TConnectionContext* ctx) {
ConnContext* context = static_cast<ConnContext*>(ctx->getUserData());
if (ctx != context->ctx) {
abort();
}
log_->append(EventLog::ET_CONN_DESTROYED, context->id, 0);
delete context;
}
protected:
uint32_t nextId_;
boost::shared_ptr<EventLog> log_;
};
class ProcessorEventHandler : public TProcessorEventHandler {
public:
ProcessorEventHandler(const boost::shared_ptr<EventLog>& log) :
nextId_(1),
log_(log) {}
void* getContext(const char* fnName, TConnectionContext* serverContext) {
ConnContext* connContext =
reinterpret_cast<ConnContext*>(serverContext->getUserData());
CallContext* context = new CallContext(connContext, nextId_, fnName);
++nextId_;
log_->append(EventLog::ET_CALL_STARTED, connContext->id, context->id,
fnName);
return context;
}
void freeContext(void* ctx, const char* fnName) {
CallContext* context = reinterpret_cast<CallContext*>(ctx);
checkName(context, fnName);
log_->append(EventLog::ET_CALL_FINISHED, context->connContext->id,
context->id, fnName);
delete context;
}
void preRead(void* ctx, const char* fnName) {
CallContext* context = reinterpret_cast<CallContext*>(ctx);
checkName(context, fnName);
log_->append(EventLog::ET_PRE_READ, context->connContext->id, context->id,
fnName);
}
void postRead(void* ctx, const char* fnName, uint32_t bytes) {
CallContext* context = reinterpret_cast<CallContext*>(ctx);
checkName(context, fnName);
log_->append(EventLog::ET_POST_READ, context->connContext->id, context->id,
fnName);
}
void preWrite(void* ctx, const char* fnName) {
CallContext* context = reinterpret_cast<CallContext*>(ctx);
checkName(context, fnName);
log_->append(EventLog::ET_PRE_WRITE, context->connContext->id, context->id,
fnName);
}
void postWrite(void* ctx, const char* fnName, uint32_t bytes) {
CallContext* context = reinterpret_cast<CallContext*>(ctx);
checkName(context, fnName);
log_->append(EventLog::ET_POST_WRITE, context->connContext->id,
context->id, fnName);
}
void asyncComplete(void* ctx, const char* fnName) {
CallContext* context = reinterpret_cast<CallContext*>(ctx);
checkName(context, fnName);
log_->append(EventLog::ET_ASYNC_COMPLETE, context->connContext->id,
context->id, fnName);
}
void handlerError(void* ctx, const char* fnName) {
CallContext* context = reinterpret_cast<CallContext*>(ctx);
checkName(context, fnName);
log_->append(EventLog::ET_HANDLER_ERROR, context->connContext->id,
context->id, fnName);
}
protected:
void checkName(const CallContext* context, const char* fnName) {
// Note: we can't use BOOST_CHECK_EQUAL here, since the handler runs in a
// different thread from the test functions. Just abort if the names are
// different
if (context->name != fnName) {
fprintf(stderr, "call context name mismatch: \"%s\" != \"%s\"\n",
context->name.c_str(), fnName);
fflush(stderr);
abort();
}
}
uint32_t nextId_;
boost::shared_ptr<EventLog> log_;
};
}}} // apache::thrift::test
#endif // _THRIFT_PROCESSOR_TEST_HANDLERS_H_

View File

@ -0,0 +1,143 @@
/*
* 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 _THRIFT_TEST_SERVERTHREAD_H_
#define _THRIFT_TEST_SERVERTHREAD_H_ 1
#include "thrift/lib/cpp/TProcessor.h"
#include "thrift/lib/cpp/protocol/TProtocol.h"
#include "thrift/lib/cpp/server/TServer.h"
#include "thrift/lib/cpp/transport/TTransport.h"
#include "thrift/lib/cpp/processor/test/EventLog.h"
namespace apache { namespace thrift { namespace test {
/**
* A helper class to tell ServerThread how to create the server
*/
class ServerState {
public:
virtual ~ServerState() {}
/**
* Create a server to listen on the specified port.
*
* If the server returned fails to bind to the specified port when serve() is
* called on it, createServer() may be called again on a different port.
*/
virtual boost::shared_ptr<server::TServer> createServer(uint16_t port) = 0;
/**
* Get the TServerEventHandler to set on the server.
*
* This is only called after the server successfully binds and is about to
* start serving traffic. It is invoked from the server thread, rather than
* the main thread.
*/
virtual boost::shared_ptr<server::TServerEventHandler>
getServerEventHandler() {
return boost::shared_ptr<server::TServerEventHandler>();
}
/**
* This method is called in the server thread after server binding succeeds.
*
* Subclasses may override this method if they wish to record the final
* port that was used for the server.
*/
virtual void bindSuccessful(uint16_t port) {
}
};
/**
* ServerThread starts a thrift server running in a separate thread.
*/
class ServerThread {
public:
ServerThread(const boost::shared_ptr<ServerState>& state, bool autoStart) :
helper_(new Helper(this)),
port_(0),
running_(false),
serving_(false),
error_(false),
serverState_(state) {
if (autoStart) {
start();
}
}
void start();
void stop();
uint16_t getPort() const {
return port_;
}
~ServerThread() {
if (running_) {
try {
stop();
} catch (...) {
GlobalOutput.printf("error shutting down server");
}
}
}
protected:
// Annoying. thrift forces us to use shared_ptr, so we have to use
// a helper class that we can allocate on the heap and give to thrift.
// It would be simpler if we could just make Runnable and TServerEventHandler
// private base classes of ServerThread.
class Helper : public concurrency::Runnable,
public server::TServerEventHandler {
public:
Helper(ServerThread* serverThread)
: serverThread_(serverThread) {}
void run() {
serverThread_->run();
}
void preServe(const transport::TSocketAddress* address) {
serverThread_->preServe(address);
}
private:
ServerThread* serverThread_;
};
void run();
void preServe(const transport::TSocketAddress* address);
boost::shared_ptr<Helper> helper_;
uint16_t port_;
bool running_;
bool serving_;
bool error_;
concurrency::Monitor serverMonitor_;
boost::shared_ptr<ServerState> serverState_;
boost::shared_ptr<server::TServer> server_;
boost::shared_ptr<concurrency::Thread> thread_;
};
}}} // apache::thrift::test
#endif // _THRIFT_TEST_SERVERTHREAD_H_

View File

@ -0,0 +1,24 @@
cpp_unittest(
name = 'ProcessorTest',
srcs = [
'EventLog.cpp',
'ProcessorTest.cpp',
'ServerThread.cpp',
],
deps = [
':thrift',
'@/thrift/lib/cpp/server:nonblocking',
],
external_deps = [
# Versions of boost lower than 1.37 don't support BOOST_AUTO_TEST_CASE.
('boost', '>= 1.37', 'boost_unit_test_framework'),
],
type = 'boost',
emails = ['thrift-team@lists.facebook.com'],
)
cpp_library(
name = 'thrift',
thrift_srcs = { 'proc.thrift' : ['ParentService', 'ChildService']},
thrift_cpp_options = 'templates',
)

View File

@ -0,0 +1,22 @@
namespace cpp apache.thrift.test
exception MyError {
1: string message
}
service ParentService {
i32 incrementGeneration()
i32 getGeneration()
void addString(1: string s)
list<string> getStrings()
binary getDataWait(1: i32 length)
oneway void onewayWait()
void exceptionWait(1: string message) throws (2: MyError error)
void unexpectedExceptionWait(1: string message)
}
service ChildService extends ParentService {
i32 setValue(1: i32 value)
i32 getValue()
}

View File

@ -0,0 +1,18 @@
# TARGETS file for thrift/lib/cpp/protocol
cpp_library (
name = "protocol",
srcs = [
"TBase64Utils.cpp",
"TDebugProtocol.cpp",
"TJSONProtocol.cpp",
"TSimpleJSONProtocol.cpp",
"THeaderProtocol.cpp",
"TPhpSerializeProtocol.cpp",
],
deps = [
"@/thrift/lib/cpp/transport",
"@/thrift/lib/cpp/transport:header",
"@/thrift/lib/cpp:thrift_exception",
],
)

View File

@ -0,0 +1,42 @@
/*
* 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 _THRIFT_PROTOCOL_TBASE64UTILS_H_
#define _THRIFT_PROTOCOL_TBASE64UTILS_H_
#include <stdint.h>
#include <string>
namespace apache { namespace thrift { namespace protocol {
// in must be at least len bytes
// len must be 1, 2, or 3
// buf must be a buffer of at least 4 bytes and may not overlap in
// the data is not padded with '='; the caller can do this if desired
void base64_encode(const uint8_t *in, uint32_t len, uint8_t *buf);
// buf must be a buffer of at least 4 bytes and contain base64 encoded values
// buf will be changed to contain output bytes
// len is number of bytes to consume from input (must be 2, 3, or 4)
// no '=' padding should be included in the input
void base64_decode(uint8_t *buf, uint32_t len);
}}} // apache::thrift::protocol
#endif // #define _THRIFT_PROTOCOL_TBASE64UTILS_H_

View File

@ -0,0 +1,302 @@
/*
* 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 THRIFT_PROTOCOL_TBINARYPROTOCOL_H_
#define THRIFT_PROTOCOL_TBINARYPROTOCOL_H_ 1
#include "thrift/lib/cpp/protocol/TProtocol.h"
#include "thrift/lib/cpp/protocol/TVirtualProtocol.h"
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift { namespace protocol {
/**
* The default binary protocol for thrift. Writes all data in a very basic
* binary format, essentially just spitting out the raw bytes.
*
*/
template <class Transport_>
class TBinaryProtocolT
: public TVirtualProtocol< TBinaryProtocolT<Transport_> > {
public:
static const int32_t VERSION_MASK = 0xffff0000;
static const int32_t VERSION_1 = 0x80010000;
// VERSION_2 (0x80020000) is taken by TDenseProtocol.
TBinaryProtocolT(const boost::shared_ptr<Transport_>& trans) :
TVirtualProtocol< TBinaryProtocolT<Transport_> >(trans),
trans_(trans.get()),
string_limit_(0),
container_limit_(0),
strict_read_(false),
strict_write_(true),
string_buf_(NULL),
string_buf_size_(0) {}
TBinaryProtocolT(const boost::shared_ptr<Transport_>& trans,
int32_t string_limit,
int32_t container_limit,
bool strict_read,
bool strict_write) :
TVirtualProtocol< TBinaryProtocolT<Transport_> >(trans),
trans_(trans.get()),
string_limit_(string_limit),
container_limit_(container_limit),
strict_read_(strict_read),
strict_write_(strict_write),
string_buf_(NULL),
string_buf_size_(0) {}
/**
* Construct a TBinaryProtocolT using a raw pointer to the transport.
*
* The caller is responsible for ensuring that the transport remains valid
* for the lifetime of the protocol.
*/
TBinaryProtocolT(Transport_* trans) :
TVirtualProtocol< TBinaryProtocolT<Transport_> >(trans),
trans_(trans),
string_limit_(0),
container_limit_(0),
strict_read_(false),
strict_write_(true),
string_buf_(NULL),
string_buf_size_(0) {}
~TBinaryProtocolT() {
if (string_buf_ != NULL) {
std::free(string_buf_);
string_buf_size_ = 0;
}
}
void setStringSizeLimit(int32_t string_limit) {
string_limit_ = string_limit;
}
void setContainerSizeLimit(int32_t container_limit) {
container_limit_ = container_limit;
}
void setStrict(bool strict_read, bool strict_write) {
strict_read_ = strict_read;
strict_write_ = strict_write;
}
/**
* Writing functions.
*/
/*ol*/ uint32_t writeMessageBegin(const std::string& name,
const TMessageType messageType,
const int32_t seqid);
/*ol*/ uint32_t writeMessageEnd();
inline uint32_t writeStructBegin(const char* name);
inline uint32_t writeStructEnd();
inline uint32_t writeFieldBegin(const char* name,
const TType fieldType,
const int16_t fieldId);
inline uint32_t writeFieldEnd();
inline uint32_t writeFieldStop();
inline uint32_t writeMapBegin(const TType keyType,
const TType valType,
const uint32_t size);
inline uint32_t writeMapEnd();
inline uint32_t writeListBegin(const TType elemType, const uint32_t size);
inline uint32_t writeListEnd();
inline uint32_t writeSetBegin(const TType elemType, const uint32_t size);
inline uint32_t writeSetEnd();
inline uint32_t writeBool(const bool value);
inline uint32_t writeByte(const int8_t byte);
inline uint32_t writeI16(const int16_t i16);
inline uint32_t writeI32(const int32_t i32);
inline uint32_t writeI64(const int64_t i64);
inline uint32_t writeDouble(const double dub);
template <typename StrType>
inline uint32_t writeString(const StrType& str);
inline uint32_t writeBinary(const std::string& str);
/**
* Reading functions
*/
/*ol*/ uint32_t readMessageBegin(std::string& name,
TMessageType& messageType,
int32_t& seqid);
/*ol*/ uint32_t readMessageEnd();
inline uint32_t readStructBegin(std::string& name);
inline uint32_t readStructEnd();
inline uint32_t readFieldBegin(std::string& name,
TType& fieldType,
int16_t& fieldId);
inline uint32_t readFieldEnd();
inline uint32_t readMapBegin(TType& keyType,
TType& valType,
uint32_t& size);
inline uint32_t readMapEnd();
inline uint32_t readListBegin(TType& elemType, uint32_t& size);
inline uint32_t readListEnd();
inline uint32_t readSetBegin(TType& elemType, uint32_t& size);
inline uint32_t readSetEnd();
inline uint32_t readBool(bool& value);
// Provide the default readBool() implementation for std::vector<bool>
using TVirtualProtocol< TBinaryProtocolT<Transport_> >::readBool;
inline uint32_t readByte(int8_t& byte);
inline uint32_t readI16(int16_t& i16);
inline uint32_t readI32(int32_t& i32);
inline uint32_t readI64(int64_t& i64);
inline uint32_t readDouble(double& dub);
template<typename StrType>
inline uint32_t readString(StrType& str);
inline uint32_t readBinary(std::string& str);
protected:
template<typename StrType>
uint32_t readStringBody(StrType& str, int32_t sz);
Transport_* trans_;
int32_t string_limit_;
int32_t container_limit_;
// Enforce presence of version identifier
bool strict_read_;
bool strict_write_;
// Buffer for reading strings, save for the lifetime of the protocol to
// avoid memory churn allocating memory on every string read
uint8_t* string_buf_;
int32_t string_buf_size_;
};
typedef TBinaryProtocolT<transport::TTransport> TBinaryProtocol;
class TBinaryProtocolFactoryBase : public TProtocolFactory {};
/**
* Constructs binary protocol handlers
*/
template <class Transport_>
class TBinaryProtocolFactoryT : public TBinaryProtocolFactoryBase {
public:
TBinaryProtocolFactoryT() :
string_limit_(0),
container_limit_(0),
strict_read_(false),
strict_write_(true) {}
TBinaryProtocolFactoryT(int32_t string_limit, int32_t container_limit,
bool strict_read, bool strict_write) :
string_limit_(string_limit),
container_limit_(container_limit),
strict_read_(strict_read),
strict_write_(strict_write) {}
virtual ~TBinaryProtocolFactoryT() {}
void setStringSizeLimit(int32_t string_limit) {
string_limit_ = string_limit;
}
void setContainerSizeLimit(int32_t container_limit) {
container_limit_ = container_limit;
}
void setStrict(bool strict_read, bool strict_write) {
strict_read_ = strict_read;
strict_write_ = strict_write;
}
boost::shared_ptr<TProtocol> getProtocol(
boost::shared_ptr<transport::TTransport> trans) {
boost::shared_ptr<Transport_> specific_trans =
boost::dynamic_pointer_cast<Transport_>(trans);
TProtocol* prot;
if (specific_trans) {
prot = new TBinaryProtocolT<Transport_>(specific_trans, string_limit_,
container_limit_, strict_read_,
strict_write_);
} else {
prot = new TBinaryProtocol(trans, string_limit_, container_limit_,
strict_read_, strict_write_);
}
return boost::shared_ptr<TProtocol>(prot);
}
private:
int32_t string_limit_;
int32_t container_limit_;
bool strict_read_;
bool strict_write_;
};
typedef TBinaryProtocolFactoryT<transport::TTransport> TBinaryProtocolFactory;
}}} // apache::thrift::protocol
#include "TBinaryProtocol.tcc"
#endif // #ifndef THRIFT_PROTOCOL_TBINARYPROTOCOL_H_

View File

@ -0,0 +1,441 @@
// Copyright (c) 2006- Facebook
// Distributed under the Thrift Software License
//
// See accompanying file LICENSE or visit the Thrift site at:
// http://developers.facebook.com/thrift/
#ifndef _THRIFT_PROTOCOL_TBINARYPROTOCOL_TCC_
#define _THRIFT_PROTOCOL_TBINARYPROTOCOL_TCC_ 1
#include "thrift/lib/cpp/protocol/TBinaryProtocol.h"
#include <limits>
#include <boost/static_assert.hpp>
namespace apache { namespace thrift { namespace protocol {
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeMessageBegin(const std::string& name,
const TMessageType messageType,
const int32_t seqid) {
if (this->strict_write_) {
int32_t version = (VERSION_1) | ((int32_t)messageType);
uint32_t wsize = 0;
wsize += writeI32(version);
wsize += writeString(name);
wsize += writeI32(seqid);
return wsize;
} else {
uint32_t wsize = 0;
wsize += writeString(name);
wsize += writeByte((int8_t)messageType);
wsize += writeI32(seqid);
return wsize;
}
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeMessageEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeStructBegin(const char* name) {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeStructEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeFieldBegin(const char* name,
const TType fieldType,
const int16_t fieldId) {
uint32_t wsize = 0;
wsize += writeByte((int8_t)fieldType);
wsize += writeI16(fieldId);
return wsize;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeFieldEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeFieldStop() {
return
writeByte((int8_t)T_STOP);
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeMapBegin(const TType keyType,
const TType valType,
const uint32_t size) {
uint32_t wsize = 0;
wsize += writeByte((int8_t)keyType);
wsize += writeByte((int8_t)valType);
wsize += writeI32((int32_t)size);
return wsize;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeMapEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeListBegin(const TType elemType,
const uint32_t size) {
uint32_t wsize = 0;
wsize += writeByte((int8_t) elemType);
wsize += writeI32((int32_t)size);
return wsize;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeListEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeSetBegin(const TType elemType,
const uint32_t size) {
uint32_t wsize = 0;
wsize += writeByte((int8_t)elemType);
wsize += writeI32((int32_t)size);
return wsize;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeSetEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeBool(const bool value) {
uint8_t tmp = value ? 1 : 0;
this->trans_->write(&tmp, 1);
return 1;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeByte(const int8_t byte) {
this->trans_->write((uint8_t*)&byte, 1);
return 1;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeI16(const int16_t i16) {
int16_t net = (int16_t)htons(i16);
this->trans_->write((uint8_t*)&net, 2);
return 2;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeI32(const int32_t i32) {
int32_t net = (int32_t)htonl(i32);
this->trans_->write((uint8_t*)&net, 4);
return 4;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeI64(const int64_t i64) {
int64_t net = (int64_t)htonll(i64);
this->trans_->write((uint8_t*)&net, 8);
return 8;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeDouble(const double dub) {
BOOST_STATIC_ASSERT(sizeof(double) == sizeof(uint64_t));
BOOST_STATIC_ASSERT(std::numeric_limits<double>::is_iec559);
uint64_t bits = bitwise_cast<uint64_t>(dub);
bits = htonll(bits);
this->trans_->write((uint8_t*)&bits, 8);
return 8;
}
template <class Transport_>
template<typename StrType>
uint32_t TBinaryProtocolT<Transport_>::writeString(const StrType& str) {
uint32_t size = str.size();
uint32_t result = writeI32((int32_t)size);
if (size > 0) {
this->trans_->write((uint8_t*)str.data(), size);
}
return result + size;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::writeBinary(const std::string& str) {
return TBinaryProtocolT<Transport_>::writeString(str);
}
/**
* Reading functions
*/
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readMessageBegin(std::string& name,
TMessageType& messageType,
int32_t& seqid) {
uint32_t result = 0;
int32_t sz;
result += readI32(sz);
if (sz < 0) {
// Check for correct version number
int32_t version = sz & VERSION_MASK;
if (version != VERSION_1) {
throw TProtocolException(TProtocolException::BAD_VERSION, "Bad version identifier");
}
messageType = (TMessageType)(sz & 0x000000ff);
result += readString(name);
result += readI32(seqid);
} else {
if (this->strict_read_) {
throw TProtocolException(TProtocolException::BAD_VERSION, "No version identifier... old protocol client in strict mode?");
} else {
// Handle pre-versioned input
int8_t type;
result += readStringBody(name, sz);
result += readByte(type);
messageType = (TMessageType)type;
result += readI32(seqid);
}
}
return result;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readMessageEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readStructBegin(std::string& name) {
name = "";
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readStructEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readFieldBegin(std::string& name,
TType& fieldType,
int16_t& fieldId) {
uint32_t result = 0;
int8_t type;
result += readByte(type);
fieldType = (TType)type;
if (fieldType == T_STOP) {
fieldId = 0;
return result;
}
result += readI16(fieldId);
return result;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readFieldEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readMapBegin(TType& keyType,
TType& valType,
uint32_t& size) {
int8_t k, v;
uint32_t result = 0;
int32_t sizei;
result += readByte(k);
keyType = (TType)k;
result += readByte(v);
valType = (TType)v;
result += readI32(sizei);
if (sizei < 0) {
throw TProtocolException(TProtocolException::NEGATIVE_SIZE);
} else if (this->container_limit_ && sizei > this->container_limit_) {
throw TProtocolException(TProtocolException::SIZE_LIMIT);
}
size = (uint32_t)sizei;
return result;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readMapEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readListBegin(TType& elemType,
uint32_t& size) {
int8_t e;
uint32_t result = 0;
int32_t sizei;
result += readByte(e);
elemType = (TType)e;
result += readI32(sizei);
if (sizei < 0) {
throw TProtocolException(TProtocolException::NEGATIVE_SIZE);
} else if (this->container_limit_ && sizei > this->container_limit_) {
throw TProtocolException(TProtocolException::SIZE_LIMIT);
}
size = (uint32_t)sizei;
return result;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readListEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readSetBegin(TType& elemType,
uint32_t& size) {
int8_t e;
uint32_t result = 0;
int32_t sizei;
result += readByte(e);
elemType = (TType)e;
result += readI32(sizei);
if (sizei < 0) {
throw TProtocolException(TProtocolException::NEGATIVE_SIZE);
} else if (this->container_limit_ && sizei > this->container_limit_) {
throw TProtocolException(TProtocolException::SIZE_LIMIT);
}
size = (uint32_t)sizei;
return result;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readSetEnd() {
return 0;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readBool(bool& value) {
uint8_t b[1];
this->trans_->readAll(b, 1);
value = *(int8_t*)b != 0;
return 1;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readByte(int8_t& byte) {
uint8_t b[1];
this->trans_->readAll(b, 1);
byte = *(int8_t*)b;
return 1;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readI16(int16_t& i16) {
uint8_t b[2];
this->trans_->readAll(b, 2);
i16 = *bitwise_cast<int16_t*>(&b);
i16 = (int16_t)ntohs(i16);
return 2;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readI32(int32_t& i32) {
uint8_t b[4];
this->trans_->readAll(b, 4);
i32 = *bitwise_cast<int32_t*>(&b);
i32 = (int32_t)ntohl(i32);
return 4;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readI64(int64_t& i64) {
uint8_t b[8];
this->trans_->readAll(b, 8);
i64 = *bitwise_cast<int64_t*>(&b);
i64 = (int64_t)ntohll(i64);
return 8;
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readDouble(double& dub) {
BOOST_STATIC_ASSERT(sizeof(double) == sizeof(uint64_t));
BOOST_STATIC_ASSERT(std::numeric_limits<double>::is_iec559);
uint64_t bits;
uint8_t b[8];
this->trans_->readAll(b, 8);
bits = *bitwise_cast<uint64_t*>(&b);
bits = ntohll(bits);
dub = bitwise_cast<double>(bits);
return 8;
}
template <class Transport_>
template<typename StrType>
uint32_t TBinaryProtocolT<Transport_>::readString(StrType& str) {
uint32_t result;
int32_t size;
result = readI32(size);
return result + readStringBody(str, size);
}
template <class Transport_>
uint32_t TBinaryProtocolT<Transport_>::readBinary(std::string& str) {
return TBinaryProtocolT<Transport_>::readString(str);
}
template <class Transport_>
template<typename StrType>
uint32_t TBinaryProtocolT<Transport_>::readStringBody(StrType& str,
int32_t size) {
uint32_t result = 0;
// Catch error cases
if (size < 0) {
throw TProtocolException(TProtocolException::NEGATIVE_SIZE);
}
if (this->string_limit_ > 0 && size > this->string_limit_) {
throw TProtocolException(TProtocolException::SIZE_LIMIT);
}
// Catch empty string case
if (size == 0) {
str.clear();
return result;
}
// Try to borrow first
const uint8_t* borrow_buf;
uint32_t got = size;
if ((borrow_buf = this->trans_->borrow(NULL, &got))) {
str.assign((const char*)borrow_buf, size);
this->trans_->consume(size);
return size;
}
// Use the heap here to prevent stack overflow for v. large strings
if (size > this->string_buf_size_ || this->string_buf_ == NULL) {
void* new_string_buf = std::realloc(this->string_buf_, (uint32_t)size);
if (new_string_buf == NULL) {
throw std::bad_alloc();
}
this->string_buf_ = (uint8_t*)new_string_buf;
this->string_buf_size_ = size;
}
this->trans_->readAll(this->string_buf_, size);
str.assign((char*)this->string_buf_, size);
return (uint32_t)size;
}
}}} // apache::thrift::protocol
#endif // #ifndef _THRIFT_PROTOCOL_TBINARYPROTOCOL_TCC_

View File

@ -0,0 +1,313 @@
/*
* 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 THRIFT_PROTOCOL_TCOMPACTPROTOCOL_H_
#define THRIFT_PROTOCOL_TCOMPACTPROTOCOL_H_ 1
#include "thrift/lib/cpp/protocol/TVirtualProtocol.h"
#include <stack>
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift { namespace protocol {
/**
* C++ Implementation of the Compact Protocol as described in THRIFT-110
*/
template <class Transport_>
class TCompactProtocolT
: public TVirtualProtocol< TCompactProtocolT<Transport_> > {
public:
static const int8_t VERSION_N = 2;
static const int8_t VERSION_LOW = 1;
static const int8_t VERSION_DOUBLE_BE = 2;
protected:
// Normally we can define static const data members of integral
// type here. However there appears to be a gcc issue when the
// high bit is set (http://gcc.gnu.org/bugzilla/show_bug.cgi?id=49896)
// unless we cast to a value that fits in an int8_t (0x82 and 0xE0 are
// uint8_t)
static const int8_t PROTOCOL_ID = static_cast<int8_t>(0x82);
static const int8_t TYPE_MASK = static_cast<int8_t>(0xE0);
static const int8_t VERSION_MASK = 0x1f; // 0001 1111
static const int32_t TYPE_SHIFT_AMOUNT = 5;
Transport_* trans_;
/**
* (Writing) If we encounter a boolean field begin, save the TField here
* so it can have the value incorporated.
*/
struct {
const char* name;
TType fieldType;
int16_t fieldId;
} booleanField_;
/**
* (Reading) If we read a field header, and it's a boolean field, save
* the boolean value here so that readBool can use it.
*/
struct {
bool hasBoolValue;
bool boolValue;
} boolValue_;
/**
* Used to keep track of the last field for the current and previous structs,
* so we can do the delta stuff.
*/
std::stack<int16_t> lastField_;
int16_t lastFieldId_;
int8_t version_;
public:
explicit TCompactProtocolT(boost::shared_ptr<Transport_> trans) :
TVirtualProtocol< TCompactProtocolT<Transport_> >(trans),
trans_(trans.get()),
lastFieldId_(0),
version_(VERSION_N),
string_limit_(0),
string_buf_(NULL),
string_buf_size_(0),
container_limit_(0) {
booleanField_.name = NULL;
boolValue_.hasBoolValue = false;
}
TCompactProtocolT(boost::shared_ptr<Transport_> trans,
int32_t string_limit,
int32_t container_limit) :
TVirtualProtocol< TCompactProtocolT<Transport_> >(trans),
trans_(trans.get()),
lastFieldId_(0),
version_(VERSION_N),
string_limit_(string_limit),
string_buf_(NULL),
string_buf_size_(0),
container_limit_(container_limit) {
booleanField_.name = NULL;
boolValue_.hasBoolValue = false;
}
virtual ~TCompactProtocolT() {
if (string_buf_) {
free(string_buf_);
}
}
/** Set this if you need backwards compatibility with an old version */
void setVersion(const int8_t version) {
version_ = version;
}
/**
* Writing functions
*/
virtual uint32_t writeMessageBegin(const std::string& name,
const TMessageType messageType,
const int32_t seqid);
uint32_t writeStructBegin(const char* name);
uint32_t writeStructEnd();
uint32_t writeFieldBegin(const char* name,
const TType fieldType,
const int16_t fieldId);
uint32_t writeFieldStop();
uint32_t writeListBegin(const TType elemType,
const uint32_t size);
uint32_t writeSetBegin(const TType elemType,
const uint32_t size);
virtual uint32_t writeMapBegin(const TType keyType,
const TType valType,
const uint32_t size);
uint32_t writeBool(const bool value);
uint32_t writeByte(const int8_t byte);
uint32_t writeI16(const int16_t i16);
uint32_t writeI32(const int32_t i32);
uint32_t writeI64(const int64_t i64);
uint32_t writeDouble(const double dub);
uint32_t writeString(const char* str);
template <class String_>
uint32_t writeString(const String_& str);
template <class String_>
uint32_t writeBinary(const String_& str);
/**
* These methods are called by structs, but don't actually have any wired
* output or purpose
*/
virtual uint32_t writeMessageEnd() { return 0; }
uint32_t writeMapEnd() { return 0; }
uint32_t writeListEnd() { return 0; }
uint32_t writeSetEnd() { return 0; }
uint32_t writeFieldEnd() { return 0; }
protected:
int32_t writeFieldBeginInternal(const char* name,
const TType fieldType,
const int16_t fieldId,
int8_t typeOverride);
uint32_t writeCollectionBegin(int8_t elemType, int32_t size);
uint32_t writeVarint32(uint32_t n);
uint32_t writeVarint64(uint64_t n);
uint64_t i64ToZigzag(const int64_t l);
uint32_t i32ToZigzag(const int32_t n);
inline int8_t getCompactType(int8_t ttype);
public:
uint32_t readMessageBegin(std::string& name,
TMessageType& messageType,
int32_t& seqid);
uint32_t readStructBegin(std::string& name);
uint32_t readStructEnd();
uint32_t readFieldBegin(std::string& name,
TType& fieldType,
int16_t& fieldId);
uint32_t readMapBegin(TType& keyType,
TType& valType,
uint32_t& size);
uint32_t readListBegin(TType& elemType,
uint32_t& size);
uint32_t readSetBegin(TType& elemType,
uint32_t& size);
uint32_t readBool(bool& value);
// Provide the default readBool() implementation for std::vector<bool>
using TVirtualProtocol< TCompactProtocolT<Transport_> >::readBool;
uint32_t readByte(int8_t& byte);
uint32_t readI16(int16_t& i16);
uint32_t readI32(int32_t& i32);
uint32_t readI64(int64_t& i64);
uint32_t readDouble(double& dub);
template <class String_>
uint32_t readString(String_& str);
template <class String_>
uint32_t readBinary(String_& str);
/*
*These methods are here for the struct to call, but don't have any wire
* encoding.
*/
uint32_t readMessageEnd() { return 0; }
uint32_t readFieldEnd() { return 0; }
uint32_t readMapEnd() { return 0; }
uint32_t readListEnd() { return 0; }
uint32_t readSetEnd() { return 0; }
protected:
uint32_t readVarint32(int32_t& i32);
uint32_t readVarint64(int64_t& i64);
int32_t zigzagToI32(uint32_t n);
int64_t zigzagToI64(uint64_t n);
TType getTType(int8_t type);
// Buffer for reading strings, save for the lifetime of the protocol to
// avoid memory churn allocating memory on every string read
int32_t string_limit_;
uint8_t* string_buf_;
int32_t string_buf_size_;
int32_t container_limit_;
};
typedef TCompactProtocolT<TTransport> TCompactProtocol;
/**
* Constructs compact protocol handlers
*/
template <class Transport_>
class TCompactProtocolFactoryT : public TProtocolFactory {
public:
TCompactProtocolFactoryT() :
string_limit_(0),
container_limit_(0) {}
TCompactProtocolFactoryT(int32_t string_limit, int32_t container_limit) :
string_limit_(string_limit),
container_limit_(container_limit) {}
virtual ~TCompactProtocolFactoryT() {}
void setStringSizeLimit(int32_t string_limit) {
string_limit_ = string_limit;
}
void setContainerSizeLimit(int32_t container_limit) {
container_limit_ = container_limit;
}
boost::shared_ptr<TProtocol> getProtocol(boost::shared_ptr<TTransport> trans) {
boost::shared_ptr<Transport_> specific_trans =
boost::dynamic_pointer_cast<Transport_>(trans);
TProtocol* prot;
if (specific_trans) {
prot = new TCompactProtocolT<Transport_>(specific_trans, string_limit_,
container_limit_);
} else {
prot = new TCompactProtocol(trans, string_limit_, container_limit_);
}
return boost::shared_ptr<TProtocol>(prot);
}
private:
int32_t string_limit_;
int32_t container_limit_;
};
typedef TCompactProtocolFactoryT<TTransport> TCompactProtocolFactory;
}}} // apache::thrift::protocol
#include "TCompactProtocol.tcc"
#endif

View File

@ -0,0 +1,820 @@
/*
* 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 THRIFT_PROTOCOL_TCOMPACTPROTOCOL_TCC_
#define THRIFT_PROTOCOL_TCOMPACTPROTOCOL_TCC_ 1
#include "thrift/lib/cpp/thrift_config.h"
#include "folly/Likely.h"
#include <limits>
/*
* TCompactProtocol::i*ToZigzag depend on the fact that the right shift
* operator on a signed integer is an arithmetic (sign-extending) shift.
* If this is not the case, the current implementation will not work.
* If anyone encounters this error, we can try to figure out the best
* way to implement an arithmetic right shift on their platform.
*/
#if !defined(SIGNED_RIGHT_SHIFT_IS) || !defined(ARITHMETIC_RIGHT_SHIFT)
# error "Unable to determine the behavior of a signed right shift"
#endif
#if SIGNED_RIGHT_SHIFT_IS != ARITHMETIC_RIGHT_SHIFT
# error "TCompactProtocol currently only works if a signed right shift is arithmetic"
#endif
namespace apache { namespace thrift { namespace protocol {
namespace detail { namespace compact {
enum Types {
CT_STOP = 0x00,
CT_BOOLEAN_TRUE = 0x01,
CT_BOOLEAN_FALSE = 0x02,
CT_BYTE = 0x03,
CT_I16 = 0x04,
CT_I32 = 0x05,
CT_I64 = 0x06,
CT_DOUBLE = 0x07,
CT_BINARY = 0x08,
CT_LIST = 0x09,
CT_SET = 0x0A,
CT_MAP = 0x0B,
CT_STRUCT = 0x0C,
};
const int8_t TTypeToCType[16] = {
CT_STOP, // T_STOP
0, // unused
CT_BOOLEAN_TRUE, // T_BOOL
CT_BYTE, // T_BYTE
CT_DOUBLE, // T_DOUBLE
0, // unused
CT_I16, // T_I16
0, // unused
CT_I32, // T_I32
0, // unused
CT_I64, // T_I64
CT_BINARY, // T_STRING
CT_STRUCT, // T_STRUCT
CT_MAP, // T_MAP
CT_SET, // T_SET
CT_LIST, // T_LIST
};
}} // end detail::compact namespace
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeMessageBegin(
const std::string& name,
const TMessageType messageType,
const int32_t seqid) {
uint32_t wsize = 0;
wsize += writeByte(PROTOCOL_ID);
wsize += writeByte((version_ & VERSION_MASK) | (((int32_t)messageType << TYPE_SHIFT_AMOUNT) & TYPE_MASK));
wsize += writeVarint32(seqid);
wsize += writeString(name);
return wsize;
}
/**
* Write a field header containing the field id and field type. If the
* difference between the current field id and the last one is small (< 15),
* then the field id will be encoded in the 4 MSB as a delta. Otherwise, the
* field id will follow the type header as a zigzag varint.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeFieldBegin(const char* name,
const TType fieldType,
const int16_t fieldId) {
if (fieldType == T_BOOL) {
booleanField_.name = name;
booleanField_.fieldType = fieldType;
booleanField_.fieldId = fieldId;
} else {
return writeFieldBeginInternal(name, fieldType, fieldId, -1);
}
return 0;
}
/**
* Write the STOP symbol so we know there are no more fields in this struct.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeFieldStop() {
return writeByte(T_STOP);
}
/**
* Write a struct begin. This doesn't actually put anything on the wire. We
* use it as an opportunity to put special placeholder markers on the field
* stack so we can get the field id deltas correct.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeStructBegin(const char* name) {
lastField_.push(lastFieldId_);
lastFieldId_ = 0;
return 0;
}
/**
* Write a struct end. This doesn't actually put anything on the wire. We use
* this as an opportunity to pop the last field from the current struct off
* of the field stack.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeStructEnd() {
lastFieldId_ = lastField_.top();
lastField_.pop();
return 0;
}
/**
* Write a List header.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeListBegin(const TType elemType,
const uint32_t size) {
return writeCollectionBegin(elemType, size);
}
/**
* Write a set header.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeSetBegin(const TType elemType,
const uint32_t size) {
return writeCollectionBegin(elemType, size);
}
/**
* Write a map header. If the map is empty, omit the key and value type
* headers, as we don't need any additional information to skip it.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeMapBegin(const TType keyType,
const TType valType,
const uint32_t size) {
uint32_t wsize = 0;
if (size == 0) {
wsize += writeByte(0);
} else {
wsize += writeVarint32(size);
wsize += writeByte(getCompactType(keyType) << 4 | getCompactType(valType));
}
return wsize;
}
/**
* Write a boolean value. Potentially, this could be a boolean field, in
* which case the field header info isn't written yet. If so, decide what the
* right type header is for the value and then write the field header.
* Otherwise, write a single byte.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeBool(const bool value) {
uint32_t wsize = 0;
if (booleanField_.name != NULL) {
// we haven't written the field header yet
wsize += writeFieldBeginInternal(booleanField_.name,
booleanField_.fieldType,
booleanField_.fieldId,
value ? detail::compact::CT_BOOLEAN_TRUE :
detail::compact::CT_BOOLEAN_FALSE);
booleanField_.name = NULL;
} else {
// we're not part of a field, so just write the value
wsize += writeByte(value ? detail::compact::CT_BOOLEAN_TRUE :
detail::compact::CT_BOOLEAN_FALSE);
}
return wsize;
}
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeByte(const int8_t byte) {
trans_->write((uint8_t*)&byte, 1);
return 1;
}
/**
* Write an i16 as a zigzag varint.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeI16(const int16_t i16) {
return writeVarint32(i32ToZigzag(i16));
}
/**
* Write an i32 as a zigzag varint.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeI32(const int32_t i32) {
return writeVarint32(i32ToZigzag(i32));
}
/**
* Write an i64 as a zigzag varint.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeI64(const int64_t i64) {
return writeVarint64(i64ToZigzag(i64));
}
/**
* Write a double to the wire as 8 bytes.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeDouble(const double dub) {
BOOST_STATIC_ASSERT(sizeof(double) == sizeof(uint64_t));
BOOST_STATIC_ASSERT(std::numeric_limits<double>::is_iec559);
uint64_t bits = bitwise_cast<uint64_t>(dub);
if (version_ >= VERSION_DOUBLE_BE) {
bits = htonll(bits);
} else {
bits = htolell(bits);
}
trans_->write((uint8_t*)&bits, 8);
return 8;
}
/**
* Write a string to the wire with a varint size preceding.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeString(const char* str) {
return writeString(std::string(str));
}
template <class Transport_>
template <class String_>
uint32_t TCompactProtocolT<Transport_>::writeString(const String_& str) {
return writeBinary(str);
}
template <class Transport_>
template <class String_>
uint32_t TCompactProtocolT<Transport_>::writeBinary(const String_& str) {
uint32_t ssize = str.size();
uint32_t wsize = writeVarint32(ssize) + ssize;
trans_->write((uint8_t*)str.data(), ssize);
return wsize;
}
//
// Internal Writing methods
//
/**
* The workhorse of writeFieldBegin. It has the option of doing a
* 'type override' of the type header. This is used specifically in the
* boolean field case.
*/
template <class Transport_>
int32_t TCompactProtocolT<Transport_>::writeFieldBeginInternal(
const char* name,
const TType fieldType,
const int16_t fieldId,
int8_t typeOverride) {
uint32_t wsize = 0;
// if there's a type override, use that.
int8_t typeToWrite = (typeOverride == -1 ? getCompactType(fieldType) : typeOverride);
// check if we can use delta encoding for the field id
if (fieldId > lastFieldId_ && fieldId - lastFieldId_ <= 15) {
// write them together
wsize += writeByte((fieldId - lastFieldId_) << 4 | typeToWrite);
} else {
// write them separate
wsize += writeByte(typeToWrite);
wsize += writeI16(fieldId);
}
lastFieldId_ = fieldId;
return wsize;
}
/**
* Abstract method for writing the start of lists and sets. List and sets on
* the wire differ only by the type indicator.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeCollectionBegin(int8_t elemType,
int32_t size) {
uint32_t wsize = 0;
if (size <= 14) {
wsize += writeByte(size << 4 | getCompactType(elemType));
} else {
wsize += writeByte(0xf0 | getCompactType(elemType));
wsize += writeVarint32(size);
}
return wsize;
}
/**
* Write an i32 as a varint. Results in 1-5 bytes on the wire.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeVarint32(uint32_t n) {
uint8_t buf[5];
uint32_t wsize = 0;
while (true) {
if ((n & ~0x7F) == 0) {
buf[wsize++] = (int8_t)n;
break;
} else {
buf[wsize++] = (int8_t)((n & 0x7F) | 0x80);
n >>= 7;
}
}
trans_->write(buf, wsize);
return wsize;
}
/**
* Write an i64 as a varint. Results in 1-10 bytes on the wire.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::writeVarint64(uint64_t n) {
uint8_t buf[10];
uint32_t wsize = 0;
while (true) {
if ((n & ~0x7FL) == 0) {
buf[wsize++] = (int8_t)n;
break;
} else {
buf[wsize++] = (int8_t)((n & 0x7F) | 0x80);
n >>= 7;
}
}
trans_->write(buf, wsize);
return wsize;
}
/**
* Convert l into a zigzag long. This allows negative numbers to be
* represented compactly as a varint.
*/
template <class Transport_>
uint64_t TCompactProtocolT<Transport_>::i64ToZigzag(const int64_t l) {
return (l << 1) ^ (l >> 63);
}
/**
* Convert n into a zigzag int. This allows negative numbers to be
* represented compactly as a varint.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::i32ToZigzag(const int32_t n) {
return (n << 1) ^ (n >> 31);
}
/**
* Given a TType value, find the appropriate detail::compact::Types value
*/
template <class Transport_>
int8_t TCompactProtocolT<Transport_>::getCompactType(int8_t ttype) {
return detail::compact::TTypeToCType[ttype];
}
//
// Reading Methods
//
/**
* Read a message header.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readMessageBegin(
std::string& name,
TMessageType& messageType,
int32_t& seqid) {
uint32_t rsize = 0;
int8_t protocolId;
int8_t versionAndType;
rsize += readByte(protocolId);
if (protocolId != PROTOCOL_ID) {
throw TProtocolException(TProtocolException::BAD_VERSION, "Bad protocol identifier");
}
rsize += readByte(versionAndType);
version_ = (int8_t)(versionAndType & VERSION_MASK);
if (!(version_ <= VERSION_N && version_ >= VERSION_LOW)) {
throw TProtocolException(TProtocolException::BAD_VERSION, "Bad protocol version");
}
messageType = (TMessageType)((versionAndType >> TYPE_SHIFT_AMOUNT) & 0x03);
rsize += readVarint32(seqid);
rsize += readString(name);
return rsize;
}
/**
* Read a struct begin. There's nothing on the wire for this, but it is our
* opportunity to push a new struct begin marker on the field stack.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readStructBegin(std::string& name) {
name = "";
lastField_.push(lastFieldId_);
lastFieldId_ = 0;
return 0;
}
/**
* Doesn't actually consume any wire data, just removes the last field for
* this struct from the field stack.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readStructEnd() {
lastFieldId_ = lastField_.top();
lastField_.pop();
return 0;
}
/**
* Read a field header off the wire.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readFieldBegin(std::string& name,
TType& fieldType,
int16_t& fieldId) {
uint32_t rsize = 0;
int8_t byte;
int8_t type;
rsize += readByte(byte);
type = (byte & 0x0f);
// if it's a stop, then we can return immediately, as the struct is over.
if (type == T_STOP) {
fieldType = T_STOP;
fieldId = 0;
return rsize;
}
// mask off the 4 MSB of the type header. it could contain a field id delta.
int16_t modifier = (int16_t)(((uint8_t)byte & 0xf0) >> 4);
if (modifier == 0) {
// not a delta, look ahead for the zigzag varint field id.
rsize += readI16(fieldId);
} else {
fieldId = (int16_t)(lastFieldId_ + modifier);
}
fieldType = getTType(type);
// if this happens to be a boolean field, the value is encoded in the type
if (type == detail::compact::CT_BOOLEAN_TRUE ||
type == detail::compact::CT_BOOLEAN_FALSE) {
// save the boolean value in a special instance variable.
boolValue_.hasBoolValue = true;
boolValue_.boolValue =
(type == detail::compact::CT_BOOLEAN_TRUE ? true : false);
}
// push the new field onto the field stack so we can keep the deltas going.
lastFieldId_ = fieldId;
return rsize;
}
/**
* Read a map header off the wire. If the size is zero, skip reading the key
* and value type. This means that 0-length maps will yield TMaps without the
* "correct" types.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readMapBegin(TType& keyType,
TType& valType,
uint32_t& size) {
uint32_t rsize = 0;
int8_t kvType = 0;
int32_t msize = 0;
rsize += readVarint32(msize);
if (msize != 0)
rsize += readByte(kvType);
if (msize < 0) {
throw TProtocolException(TProtocolException::NEGATIVE_SIZE);
} else if (container_limit_ && msize > container_limit_) {
throw TProtocolException(TProtocolException::SIZE_LIMIT);
}
keyType = getTType((int8_t)((uint8_t)kvType >> 4));
valType = getTType((int8_t)((uint8_t)kvType & 0xf));
size = (uint32_t)msize;
return rsize;
}
/**
* Read a list header off the wire. If the list size is 0-14, the size will
* be packed into the element type header. If it's a longer list, the 4 MSB
* of the element type header will be 0xF, and a varint will follow with the
* true size.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readListBegin(TType& elemType,
uint32_t& size) {
int8_t size_and_type;
uint32_t rsize = 0;
int32_t lsize;
rsize += readByte(size_and_type);
lsize = ((uint8_t)size_and_type >> 4) & 0x0f;
if (lsize == 15) {
rsize += readVarint32(lsize);
}
if (lsize < 0) {
throw TProtocolException(TProtocolException::NEGATIVE_SIZE);
} else if (container_limit_ && lsize > container_limit_) {
throw TProtocolException(TProtocolException::SIZE_LIMIT);
}
elemType = getTType((int8_t)(size_and_type & 0x0f));
size = (uint32_t)lsize;
return rsize;
}
/**
* Read a set header off the wire. If the set size is 0-14, the size will
* be packed into the element type header. If it's a longer set, the 4 MSB
* of the element type header will be 0xF, and a varint will follow with the
* true size.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readSetBegin(TType& elemType,
uint32_t& size) {
return readListBegin(elemType, size);
}
/**
* Read a boolean off the wire. If this is a boolean field, the value should
* already have been read during readFieldBegin, so we'll just consume the
* pre-stored value. Otherwise, read a byte.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readBool(bool& value) {
if (boolValue_.hasBoolValue == true) {
value = boolValue_.boolValue;
boolValue_.hasBoolValue = false;
return 0;
} else {
int8_t val;
readByte(val);
value = (val == detail::compact::CT_BOOLEAN_TRUE);
return 1;
}
}
/**
* Read a single byte off the wire. Nothing interesting here.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readByte(int8_t& byte) {
uint8_t b[1];
trans_->readAll(b, 1);
byte = *(int8_t*)b;
return 1;
}
/**
* Read an i16 from the wire as a zigzag varint.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readI16(int16_t& i16) {
int32_t value;
uint32_t rsize = readVarint32(value);
i16 = (int16_t)zigzagToI32(value);
return rsize;
}
/**
* Read an i32 from the wire as a zigzag varint.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readI32(int32_t& i32) {
int32_t value;
uint32_t rsize = readVarint32(value);
i32 = zigzagToI32(value);
return rsize;
}
/**
* Read an i64 from the wire as a zigzag varint.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readI64(int64_t& i64) {
int64_t value;
uint32_t rsize = readVarint64(value);
i64 = zigzagToI64(value);
return rsize;
}
/**
* No magic here - just read a double off the wire.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readDouble(double& dub) {
BOOST_STATIC_ASSERT(sizeof(double) == sizeof(uint64_t));
BOOST_STATIC_ASSERT(std::numeric_limits<double>::is_iec559);
union {
uint64_t bits;
uint8_t b[8];
} u;
trans_->readAll(u.b, 8);
if (version_ >= VERSION_DOUBLE_BE) {
u.bits = ntohll(u.bits);
} else {
u.bits = letohll(u.bits);
}
dub = bitwise_cast<double>(u.bits);
return 8;
}
template <class Transport_>
template <class String_>
uint32_t TCompactProtocolT<Transport_>::readString(String_& str) {
return readBinary(str);
}
/**
* Read a byte[] from the wire.
*/
template <class Transport_>
template <class String_>
uint32_t TCompactProtocolT<Transport_>::readBinary(String_& str) {
int32_t rsize = 0;
int32_t size;
rsize += readVarint32(size);
// Catch empty string case
if (size == 0) {
str = "";
return rsize;
}
// Catch error cases
if (size < 0) {
throw TProtocolException(TProtocolException::NEGATIVE_SIZE);
}
if (string_limit_ > 0 && size > string_limit_) {
throw TProtocolException(TProtocolException::SIZE_LIMIT);
}
// Use the heap here to prevent stack overflow for v. large strings
if (size > string_buf_size_ || string_buf_ == NULL) {
void* new_string_buf = std::realloc(string_buf_, (uint32_t)size);
if (new_string_buf == NULL) {
throw std::bad_alloc();
}
string_buf_ = (uint8_t*)new_string_buf;
string_buf_size_ = size;
}
trans_->readAll(string_buf_, size);
str.assign((char*)string_buf_, size);
return rsize + (uint32_t)size;
}
/**
* Read an i32 from the wire as a varint. The MSB of each byte is set
* if there is another byte to follow. This can read up to 5 bytes.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readVarint32(int32_t& i32) {
int64_t val;
uint32_t rsize = readVarint64(val);
i32 = (int32_t)val;
return rsize;
}
/**
* Read an i64 from the wire as a proper varint. The MSB of each byte is set
* if there is another byte to follow. This can read up to 10 bytes.
*/
template <class Transport_>
uint32_t TCompactProtocolT<Transport_>::readVarint64(int64_t& i64) {
uint32_t rsize = 0;
uint64_t val = 0;
int shift = 0;
uint8_t buf[10]; // 64 bits / (7 bits/byte) = 10 bytes.
uint32_t buf_size = sizeof(buf);
const uint8_t* borrowed = trans_->borrow(buf, &buf_size);
// Fast path.
if (borrowed != NULL) {
while (true) {
uint8_t byte = borrowed[rsize];
rsize++;
val |= (uint64_t)(byte & 0x7f) << shift;
shift += 7;
if (!(byte & 0x80)) {
i64 = val;
trans_->consume(rsize);
return rsize;
}
// Have to check for invalid data so we don't crash.
if (UNLIKELY(rsize == sizeof(buf))) {
throw TProtocolException(TProtocolException::INVALID_DATA, "Variable-length int over 10 bytes.");
}
}
}
// Slow path.
else {
while (true) {
uint8_t byte;
rsize += trans_->readAll(&byte, 1);
val |= (uint64_t)(byte & 0x7f) << shift;
shift += 7;
if (!(byte & 0x80)) {
i64 = val;
return rsize;
}
// Might as well check for invalid data on the slow path too.
if (UNLIKELY(rsize >= sizeof(buf))) {
throw TProtocolException(TProtocolException::INVALID_DATA, "Variable-length int over 10 bytes.");
}
}
}
}
/**
* Convert from zigzag int to int.
*/
template <class Transport_>
int32_t TCompactProtocolT<Transport_>::zigzagToI32(uint32_t n) {
return (n >> 1) ^ -(n & 1);
}
/**
* Convert from zigzag long to long.
*/
template <class Transport_>
int64_t TCompactProtocolT<Transport_>::zigzagToI64(uint64_t n) {
return (n >> 1) ^ -(n & 1);
}
template <class Transport_>
TType TCompactProtocolT<Transport_>::getTType(int8_t type) {
switch (type) {
case T_STOP:
return T_STOP;
case detail::compact::CT_BOOLEAN_FALSE:
case detail::compact::CT_BOOLEAN_TRUE:
return T_BOOL;
case detail::compact::CT_BYTE:
return T_BYTE;
case detail::compact::CT_I16:
return T_I16;
case detail::compact::CT_I32:
return T_I32;
case detail::compact::CT_I64:
return T_I64;
case detail::compact::CT_DOUBLE:
return T_DOUBLE;
case detail::compact::CT_BINARY:
return T_STRING;
case detail::compact::CT_LIST:
return T_LIST;
case detail::compact::CT_SET:
return T_SET;
case detail::compact::CT_MAP:
return T_MAP;
case detail::compact::CT_STRUCT:
return T_STRUCT;
default:
throw TLibraryException("don't know what type: " + type);
}
return T_STOP;
}
}}} // apache::thrift::protocol
#endif // THRIFT_PROTOCOL_TCOMPACTPROTOCOL_TCC_

View File

@ -0,0 +1,360 @@
/*
* 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 THRIFT_PROTOCOL_TDEBUGPROTOCOL_H
#define THRIFT_PROTOCOL_TDEBUGPROTOCOL_H
#include "thrift/lib/cpp/protocol/TVirtualProtocol.h"
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift { namespace protocol {
/*
!!! EXPERIMENTAL CODE !!!
This protocol is very much a work in progress.
It doesn't handle many cases properly.
It throws exceptions in many cases.
It probably segfaults in many cases.
Bug reports and feature requests are welcome.
Complaints are not. :R
*/
/**
* Protocol that prints the payload in a nice human-readable format.
* Reading from this protocol is not supported.
*
*/
class TDebugProtocol : public TVirtualProtocol<TDebugProtocol> {
private:
enum write_state_t
{ UNINIT
, STRUCT
, LIST
, SET
, MAP_KEY
, MAP_VALUE
};
public:
TDebugProtocol(boost::shared_ptr<TTransport> trans)
: TVirtualProtocol<TDebugProtocol>(trans)
, trans_(trans.get())
, string_limit_(DEFAULT_STRING_LIMIT)
, string_prefix_size_(DEFAULT_STRING_PREFIX_SIZE)
{
write_state_.push_back(UNINIT);
}
static const int32_t DEFAULT_STRING_LIMIT = 256;
static const int32_t DEFAULT_STRING_PREFIX_SIZE = 16;
void setStringSizeLimit(int32_t string_limit) {
string_limit_ = string_limit;
}
void setStringPrefixSize(int32_t string_prefix_size) {
string_prefix_size_ = string_prefix_size;
}
uint32_t writeMessageBegin(const std::string& name,
const TMessageType messageType,
const int32_t seqid);
uint32_t writeMessageEnd();
uint32_t writeStructBegin(const char* name);
uint32_t writeStructEnd();
uint32_t writeFieldBegin(const char* name,
const TType fieldType,
const int16_t fieldId);
uint32_t writeFieldEnd();
uint32_t writeFieldStop();
uint32_t writeMapBegin(const TType keyType,
const TType valType,
const uint32_t size);
uint32_t writeMapEnd();
uint32_t writeListBegin(const TType elemType,
const uint32_t size);
uint32_t writeListEnd();
uint32_t writeSetBegin(const TType elemType,
const uint32_t size);
uint32_t writeSetEnd();
uint32_t writeBool(const bool value);
uint32_t writeByte(const int8_t byte);
uint32_t writeI16(const int16_t i16);
uint32_t writeI32(const int32_t i32);
uint32_t writeI64(const int64_t i64);
uint32_t writeDouble(const double dub);
uint32_t writeString(const std::string& str);
template <class StrType>
uint32_t writeString(const StrType& str) {
return writeString(std::string(str.data(), str.size()));
}
uint32_t writeBinary(const std::string& str);
private:
void indentUp();
void indentDown();
uint32_t writePlain(const std::string& str);
uint32_t writeIndented(const std::string& str);
uint32_t startItem();
uint32_t endItem();
uint32_t writeItem(const std::string& str);
static std::string fieldTypeName(TType type);
TTransport* trans_;
int32_t string_limit_;
int32_t string_prefix_size_;
std::string indent_str_;
static const int indent_inc = 2;
std::vector<write_state_t> write_state_;
std::vector<int> list_idx_;
};
/**
* Constructs debug protocol handlers
*/
class TDebugProtocolFactory : public TProtocolFactory {
public:
TDebugProtocolFactory() {}
virtual ~TDebugProtocolFactory() {}
boost::shared_ptr<TProtocol> getProtocol(boost::shared_ptr<TTransport> trans) {
return boost::shared_ptr<TProtocol>(new TDebugProtocol(trans));
}
};
}}} // apache::thrift::protocol
// TODO(dreiss): Move (part of) ThriftDebugString into a .cpp file and remove this.
#include "thrift/lib/cpp/transport/TBufferTransports.h"
namespace apache { namespace thrift {
template <typename T>
struct ThriftTypeTraits {
static const apache::thrift::protocol::TType fieldType_ =
apache::thrift::protocol::TType(99);
};
template<>
struct ThriftTypeTraits<bool> {
static const apache::thrift::protocol::TType fieldType_ =
apache::thrift::protocol::T_BOOL;
};
template<>
struct ThriftTypeTraits<int8_t> {
static const apache::thrift::protocol::TType fieldType_ =
apache::thrift::protocol::T_I08;
};
template<>
struct ThriftTypeTraits<int16_t> {
static const apache::thrift::protocol::TType fieldType_ =
apache::thrift::protocol::T_I16;
};
template<>
struct ThriftTypeTraits<int32_t> {
static const apache::thrift::protocol::TType fieldType_ =
apache::thrift::protocol::T_I32;
};
template<>
struct ThriftTypeTraits<int64_t> {
static const apache::thrift::protocol::TType fieldType_ =
apache::thrift::protocol::T_I64;
};
template<>
struct ThriftTypeTraits<double> {
static const apache::thrift::protocol::TType fieldType_ =
apache::thrift::protocol::T_DOUBLE;
};
template<>
struct ThriftTypeTraits<std::string> {
static const apache::thrift::protocol::TType fieldType_ =
apache::thrift::protocol::T_STRING;
};
class TDebugProtocolEx : public apache::thrift::protocol::TDebugProtocol {
public:
TDebugProtocolEx(
boost::shared_ptr<apache::thrift::protocol::TTransport> trans
)
: TDebugProtocol(trans) {
}
template<typename T>
void write(const T& t) {
t.write(this);
}
template<typename T>
void write(const std::vector<T>& c) {
writeListBegin(ThriftTypeTraits<T>::fieldType_, c.size());
typeof(c.begin()) it = c.begin();
for (; it != c.end(); it++) {
write(*it);
}
writeListEnd();
}
template<typename K, typename V>
void write(const std::map<K, V>& c) {
writeMapBegin(
ThriftTypeTraits<K>::fieldType_,
ThriftTypeTraits<V>::fieldType_,
c.size()
);
typeof(c.begin()) it = c.begin();
for (; it != c.end(); it++) {
write(it->first);
write(it->second);
}
writeMapEnd();
}
template<typename K, typename V>
void write(const std::multimap<K, V>& c) {
writeMapBegin(
ThriftTypeTraits<K>::fieldType_,
ThriftTypeTraits<V>::fieldType_,
c.size()
);
typeof(c.begin()) it = c.begin();
for (; it != c.end(); it++) {
write(it->first);
write(it->second);
}
writeMapEnd();
}
template<typename T>
void write(const std::set<T>& c) {
writeSetBegin(
ThriftTypeTraits<T>::fieldType_,
c.size()
);
typeof(c.begin()) it = c.begin();
for (; it != c.end(); it++) {
write(*it);
}
writeSetEnd();
}
void write(const bool value) {
writeBool(value);
}
void write(const int8_t byte) {
writeByte(byte);
}
void write(const int16_t i16) {
writeI16(i16);
}
void write(const int32_t i32) {
writeI32(i32);
}
void write(const int64_t i64) {
writeI64(i64);
}
void write(const double dub) {
writeDouble(dub);
}
void write(const std::string& str) {
writeString(str);
}
template<typename K, typename V>
void write(const std::pair<K, V>& p) {
writeStructBegin("pair");
writeFieldBegin("first", ThriftTypeTraits<K>::fieldType_, 1);
write(p.first);
writeFieldEnd();
writeFieldBegin("second", ThriftTypeTraits<V>::fieldType_, 2);
write(p.second);
writeFieldEnd();
writeStructEnd();
}
};
template<typename T>
std::string ThriftDebugString(const T& ts) {
using namespace apache::thrift::transport;
using namespace apache::thrift::protocol;
TMemoryBuffer* buffer = new TMemoryBuffer;
boost::shared_ptr<TTransport> trans(buffer);
TDebugProtocolEx protocol(trans);
protocol.write(ts);
uint8_t* buf;
uint32_t size;
buffer->getBuffer(&buf, &size);
return std::string((char*)buf, (unsigned int)size);
}
}} // apache::thrift
#endif // #ifndef _THRIFT_PROTOCOL_TDEBUGPROTOCOL_H_

View File

@ -0,0 +1,370 @@
/*
* 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 THRIFT_PROTOCOL_THEADERPROTOCOL_H_
#define THRIFT_PROTOCOL_THEADERPROTOCOL_H_ 1
#include "thrift/lib/cpp/protocol/TProtocol.h"
#include "thrift/lib/cpp/protocol/TProtocolTypes.h"
#include "thrift/lib/cpp/protocol/TVirtualProtocol.h"
#include "thrift/lib/cpp/transport/THeaderTransport.h"
#include "thrift/lib/cpp/util/shared_ptr_util.h"
#include <boost/shared_ptr.hpp>
#include <bitset>
using apache::thrift::transport::THeaderTransport;
using apache::thrift::transport::TTransportPair;
namespace apache { namespace thrift { namespace protocol {
/**
* The header protocol for thrift. Reads unframed, framed, header format,
* and http
*
*/
class THeaderProtocol
: public TVirtualProtocol<THeaderProtocol> {
public:
explicit THeaderProtocol(const boost::shared_ptr<TTransport>& trans,
std::bitset<CLIENT_TYPES_LEN>* clientTypes = NULL,
uint16_t protoId = T_COMPACT_PROTOCOL) :
TVirtualProtocol<THeaderProtocol>(getTransportWrapper(trans,
clientTypes))
, trans_(boost::dynamic_pointer_cast<THeaderTransport, TTransport>(
this->getTransport()))
, protoId_(protoId)
{
trans_->setProtocolId(protoId);
resetProtocol();
}
THeaderProtocol(const boost::shared_ptr<TTransport>& inTrans,
const boost::shared_ptr<TTransport>& outTrans,
std::bitset<CLIENT_TYPES_LEN>* clientTypes = NULL,
uint16_t protoId = T_COMPACT_PROTOCOL) :
TVirtualProtocol<THeaderProtocol>(getInOutTransportWrapper(inTrans,
outTrans,
clientTypes))
, trans_(boost::dynamic_pointer_cast<THeaderTransport, TTransport>(
this->getTransport()))
, protoId_(protoId)
{
trans_->setProtocolId(protoId);
resetProtocol();
}
/**
* Construct a THeaderProtocol using a raw pointer to the transport.
*
* The caller is responsible for ensuring that the transport remains valid
* for the lifetime of the protocol.
*/
THeaderProtocol(TTransport* trans,
std::bitset<CLIENT_TYPES_LEN>* clientTypes,
uint16_t protoId = T_COMPACT_PROTOCOL) :
TVirtualProtocol<THeaderProtocol>(
getTransportWrapper(
boost::shared_ptr<TTransport>(trans,
NoopPtrDestructor<TTransport>()),
clientTypes))
, trans_(boost::dynamic_pointer_cast<THeaderTransport, TTransport>(
this->getTransport()))
, protoId_(protoId)
{
trans_->setProtocolId(protoId);
resetProtocol();
}
~THeaderProtocol() {}
/**
* Functions to work with headers by calling into THeaderTransport
*/
void setProtocolId(uint16_t protoId) {
trans_->setProtocolId(protoId);
resetProtocol();
}
void resetProtocol();
typedef THeaderTransport::StringToStringMap StringToStringMap;
// these work with write headers
void setHeader(const std::string& key, const std::string& value) {
trans_->setHeader(key, value);
}
void setPersistentHeader(const std::string& key, const std::string& value) {
trans_->setPersistentHeader(key, value);
}
void clearHeaders() {
trans_->clearHeaders();
}
void clearPersistentHeaders() {
trans_->clearPersistentHeaders();
}
StringToStringMap& getWriteHeaders() {
return trans_->getWriteHeaders();
}
StringToStringMap& getPersistentWriteHeaders() {
return trans_->getPersistentWriteHeaders();
}
// these work with read headers
const StringToStringMap& getHeaders() const {
return trans_->getHeaders();
}
void setTransform(uint16_t trans) {
trans_->setTransform(trans);
}
std::string getPeerIdentity() const {
return trans_->getPeerIdentity();
}
void setIdentity(const std::string& identity) {
trans_->setIdentity(identity);
}
void setHmac(THeaderTransport::MacCallback macCb,
THeaderTransport::VerifyMacCallback verifyCb) {
trans_->setHmac(macCb, verifyCb);
}
/**
* Writing functions.
*/
/*ol*/ uint32_t writeMessageBegin(const std::string& name,
const TMessageType messageType,
const int32_t seqId);
/*ol*/ uint32_t writeMessageEnd();
uint32_t writeStructBegin(const char* name);
uint32_t writeStructEnd();
uint32_t writeFieldBegin(const char* name,
const TType fieldType,
const int16_t fieldId);
uint32_t writeFieldEnd();
uint32_t writeFieldStop();
uint32_t writeMapBegin(const TType keyType,
const TType valType,
const uint32_t size);
uint32_t writeMapEnd();
uint32_t writeListBegin(const TType elemType, const uint32_t size);
uint32_t writeListEnd();
uint32_t writeSetBegin(const TType elemType, const uint32_t size);
uint32_t writeSetEnd();
uint32_t writeBool(const bool value);
uint32_t writeByte(const int8_t byte);
uint32_t writeI16(const int16_t i16);
uint32_t writeI32(const int32_t i32);
uint32_t writeI64(const int64_t i64);
uint32_t writeDouble(const double dub);
uint32_t writeString(const std::string& str);
uint32_t writeBinary(const std::string& str);
/**
* Reading functions
*/
/*ol*/ uint32_t readMessageBegin(std::string& name,
TMessageType& messageType,
int32_t& seqId);
/*ol*/ uint32_t readMessageEnd();
uint32_t readStructBegin(std::string& name);
uint32_t readStructEnd();
uint32_t readFieldBegin(std::string& name,
TType& fieldType,
int16_t& fieldId);
uint32_t readFieldEnd();
uint32_t readMapBegin(TType& keyType,
TType& valType,
uint32_t& size);
uint32_t readMapEnd();
uint32_t readListBegin(TType& elemType, uint32_t& size);
uint32_t readListEnd();
uint32_t readSetBegin(TType& elemType, uint32_t& size);
uint32_t readSetEnd();
uint32_t readBool(bool& value);
// Provide the default readBool() implementation for std::vector<bool>
using TVirtualProtocol< THeaderProtocol >::readBool;
uint32_t readByte(int8_t& byte);
uint32_t readI16(int16_t& i16);
uint32_t readI32(int32_t& i32);
uint32_t readI64(int64_t& i64);
uint32_t readDouble(double& dub);
uint32_t readString(std::string& str);
uint32_t readBinary(std::string& binary);
protected:
template<typename StrType>
uint32_t readStringBody(StrType& str, int32_t sz);
boost::shared_ptr<TTransport> getTransportWrapper(
const boost::shared_ptr<TTransport>& trans,
std::bitset<CLIENT_TYPES_LEN>* clientTypes) {
if (dynamic_cast<THeaderTransport*>(trans.get()) != NULL) {
return trans;
} else {
return boost::shared_ptr<THeaderTransport>(
new THeaderTransport(trans, clientTypes));
}
}
boost::shared_ptr<TTransport> getInOutTransportWrapper(
const boost::shared_ptr<TTransport>& inTrans,
const boost::shared_ptr<TTransport>& outTrans,
std::bitset<CLIENT_TYPES_LEN>* clientTypes) {
assert(dynamic_cast<THeaderTransport*>(inTrans.get()) == NULL
&& dynamic_cast<THeaderTransport*>(outTrans.get()) == NULL);
return boost::shared_ptr<THeaderTransport>(
new THeaderTransport(inTrans, outTrans, clientTypes)
);
}
boost::shared_ptr<THeaderTransport> trans_;
boost::shared_ptr<TProtocol> proto_;
uint32_t protoId_;
};
/**
* Constructs header protocol handlers
*/
class THeaderProtocolFactory : public TDuplexProtocolFactory {
public:
explicit THeaderProtocolFactory(uint16_t protoId = T_COMPACT_PROTOCOL,
bool disableIdentity = false) {
protoId_ = protoId;
setIdentity_ = disableIdentity;
}
virtual ~THeaderProtocolFactory() {}
void setClientTypes(std::bitset<CLIENT_TYPES_LEN>& clientTypes) {
for (int i = 0; i < CLIENT_TYPES_LEN; i++) {
this->clientTypes[i] = clientTypes[i];
}
}
void setIdentity(const std::string& identity) {
identity_ = identity;
setIdentity_ = true;
}
void setTransform(uint16_t trans) {
trans_.push_back(trans);
}
virtual TProtocolPair getProtocol(
boost::shared_ptr<transport::TTransport> trans) {
THeaderProtocol* prot = new THeaderProtocol(trans, &clientTypes, protoId_);
if(setIdentity_) {
prot->setIdentity(identity_);
}
for (auto& t : trans_) {
prot->setTransform(t);
}
boost::shared_ptr<TProtocol> pprot(prot);
return TProtocolPair(pprot, pprot);
}
virtual TProtocolPair getProtocol(TTransportPair transports) {
THeaderProtocol* prot = new THeaderProtocol(transports.first,
transports.second,
&clientTypes,
protoId_);
if(setIdentity_) {
prot->setIdentity(identity_);
}
for (auto& t : trans_) {
prot->setTransform(t);
}
boost::shared_ptr<TProtocol> pprot(prot);
return TProtocolPair(pprot, pprot);
}
// No implementation of getInputProtocolFactory/getOutputProtocolFactory
// Using base class implementation which return NULL.
private:
std::bitset<CLIENT_TYPES_LEN> clientTypes;
uint16_t protoId_;
bool setIdentity_;
std::vector<uint16_t> trans_;
std::string identity_;
};
}}} // apache::thrift::protocol
#endif // #ifndef THRIFT_PROTOCOL_THEADERPROTOCOL_H_

View File

@ -0,0 +1,348 @@
/*
* 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 THRIFT_PROTOCOL_TJSONPROTOCOL_H_
#define THRIFT_PROTOCOL_TJSONPROTOCOL_H_ 1
#include "TVirtualProtocol.h"
#include <stack>
namespace apache { namespace thrift { namespace protocol {
// Forward declaration
class TJSONContext;
/**
* JSON protocol for Thrift.
*
* Implements a protocol which uses JSON as the wire-format.
*
* Thrift types are represented as described below:
*
* 1. Every Thrift integer type is represented as a JSON number.
*
* 2. Thrift doubles are represented as JSON numbers. Some special values are
* represented as strings:
* a. "NaN" for not-a-number values
* b. "Infinity" for positive infinity
* c. "-Infinity" for negative infinity
*
* 3. Thrift string values are emitted as JSON strings, with appropriate
* escaping.
*
* 4. Thrift binary values are encoded into Base64 and emitted as JSON strings.
* The readBinary() method is written such that it will properly skip if
* called on a Thrift string (although it will decode garbage data).
*
* 5. Thrift structs are represented as JSON objects, with the field ID as the
* key, and the field value represented as a JSON object with a single
* key-value pair. The key is a short string identifier for that type,
* followed by the value. The valid type identifiers are: "tf" for bool,
* "i8" for byte, "i16" for 16-bit integer, "i32" for 32-bit integer, "i64"
* for 64-bit integer, "dbl" for double-precision floating point, "str" for
* string (including binary), "rec" for struct ("records"), "map" for map,
* "lst" for list, "set" for set.
*
* 6. Thrift lists and sets are represented as JSON arrays, with the first
* element of the JSON array being the string identifier for the Thrift
* element type and the second element of the JSON array being the count of
* the Thrift elements. The Thrift elements then follow.
*
* 7. Thrift maps are represented as JSON arrays, with the first two elements
* of the JSON array being the string identifiers for the Thrift key type
* and value type, followed by the count of the Thrift pairs, followed by a
* JSON object containing the key-value pairs. Note that JSON keys can only
* be strings, which means that the key type of the Thrift map should be
* restricted to numeric or string types -- in the case of numerics, they
* are serialized as strings.
*
* 8. Thrift messages are represented as JSON arrays, with the protocol
* version #, the message name, the message type, and the sequence ID as
* the first 4 elements.
*
* More discussion of the double handling is probably warranted. The aim of
* the current implementation is to match as closely as possible the behavior
* of Java's Double.toString(), which has no precision loss. Implementors in
* other languages should strive to achieve that where possible. I have not
* yet verified whether boost:lexical_cast, which is doing that work for me in
* C++, loses any precision, but I am leaving this as a future improvement. I
* may try to provide a C component for this, so that other languages could
* bind to the same underlying implementation for maximum consistency.
*
* Note further that JavaScript itself is not capable of representing
* floating point infinities -- presumably when we have a JavaScript Thrift
* client, this would mean that infinities get converted to not-a-number in
* transmission. I don't know of any work-around for this issue.
*
*/
class TJSONProtocol : public TVirtualProtocol<TJSONProtocol> {
public:
TJSONProtocol(boost::shared_ptr<TTransport> ptrans);
~TJSONProtocol();
private:
void pushContext(boost::shared_ptr<TJSONContext> c);
void popContext();
protected:
uint32_t writeJSONEscapeChar(uint8_t ch);
uint32_t writeJSONChar(uint8_t ch);
uint32_t writeJSONString(const std::string &str);
uint32_t writeJSONBase64(const std::string &str);
template <typename NumberType>
uint32_t writeJSONInteger(NumberType num);
uint32_t writeJSONBool(bool value);
uint32_t writeJSONDouble(double num);
uint32_t writeJSONObjectStart() ;
uint32_t writeJSONObjectEnd();
uint32_t writeJSONArrayStart();
uint32_t writeJSONArrayEnd();
uint32_t readJSONSyntaxChar(uint8_t ch);
uint32_t readJSONEscapeChar(uint8_t *out);
uint32_t readJSONString(std::string &str, bool skipContext = false);
uint32_t readJSONBase64(std::string &str);
uint32_t readJSONNumericChars(std::string &str);
template <typename NumberType>
uint32_t readJSONInteger(NumberType &num);
uint32_t readJSONDouble(double &num);
uint32_t readJSONObjectStart();
uint32_t readJSONObjectEnd();
uint32_t readJSONArrayStart();
uint32_t readJSONArrayEnd();
public:
/**
* Writing functions.
*/
uint32_t writeMessageBegin(const std::string& name,
const TMessageType messageType,
const int32_t seqid);
uint32_t writeMessageEnd();
uint32_t writeStructBegin(const char* name);
uint32_t writeStructEnd();
uint32_t writeFieldBegin(const char* name,
const TType fieldType,
const int16_t fieldId);
uint32_t writeFieldEnd();
uint32_t writeFieldStop();
uint32_t writeMapBegin(const TType keyType,
const TType valType,
const uint32_t size);
uint32_t writeMapEnd();
uint32_t writeListBegin(const TType elemType,
const uint32_t size);
uint32_t writeListEnd();
uint32_t writeSetBegin(const TType elemType,
const uint32_t size);
uint32_t writeSetEnd();
uint32_t writeBool(const bool value);
uint32_t writeByte(const int8_t byte);
uint32_t writeI16(const int16_t i16);
uint32_t writeI32(const int32_t i32);
uint32_t writeI64(const int64_t i64);
uint32_t writeDouble(const double dub);
uint32_t writeString(const std::string& str);
uint32_t writeBinary(const std::string& str);
/**
* Reading functions
*/
uint32_t readMessageBegin(std::string& name,
TMessageType& messageType,
int32_t& seqid);
uint32_t readMessageEnd();
uint32_t readStructBegin(std::string& name);
uint32_t readStructEnd();
uint32_t readFieldBegin(std::string& name,
TType& fieldType,
int16_t& fieldId);
uint32_t readFieldEnd();
uint32_t readMapBegin(TType& keyType,
TType& valType,
uint32_t& size);
uint32_t readMapEnd();
uint32_t readListBegin(TType& elemType,
uint32_t& size);
uint32_t readListEnd();
uint32_t readSetBegin(TType& elemType,
uint32_t& size);
uint32_t readSetEnd();
uint32_t readBool(bool& value);
// Provide the default readBool() implementation for std::vector<bool>
using TVirtualProtocol<TJSONProtocol>::readBool;
uint32_t readByte(int8_t& byte);
uint32_t readI16(int16_t& i16);
uint32_t readI32(int32_t& i32);
uint32_t readI64(int64_t& i64);
uint32_t readDouble(double& dub);
uint32_t readString(std::string& str);
uint32_t readBinary(std::string& str);
class LookaheadReader {
public:
LookaheadReader(TTransport &trans) :
trans_(&trans),
hasData_(false) {
}
uint8_t read() {
if (hasData_) {
hasData_ = false;
}
else {
trans_->readAll(&data_, 1);
}
return data_;
}
uint8_t peek() {
if (!hasData_) {
trans_->readAll(&data_, 1);
}
hasData_ = true;
return data_;
}
private:
TTransport *trans_;
bool hasData_;
uint8_t data_;
};
private:
TTransport* trans_;
std::stack<boost::shared_ptr<TJSONContext> > contexts_;
boost::shared_ptr<TJSONContext> context_;
LookaheadReader reader_;
};
/**
* Constructs input and output protocol objects given transports.
*/
class TJSONProtocolFactory : public TProtocolFactory {
public:
TJSONProtocolFactory() {}
virtual ~TJSONProtocolFactory() {}
boost::shared_ptr<TProtocol> getProtocol(boost::shared_ptr<TTransport> trans) {
return boost::shared_ptr<TProtocol>(new TJSONProtocol(trans));
}
};
}}} // apache::thrift::protocol
// TODO(dreiss): Move part of ThriftJSONString into a .cpp file and remove this.
#include "thrift/lib/cpp/transport/TBufferTransports.h"
/*namespace apache { namespace thrift {
template<typename ThriftStruct>
std::string ThriftJSONString(const ThriftStruct& ts) {
using namespace apache::thrift::transport;
using namespace apache::thrift::protocol;
TMemoryBuffer* buffer = new TMemoryBuffer;
boost::shared_ptr<TTransport> trans(buffer);
TJSONProtocol protocol(trans);
ts.write(&protocol);
uint8_t* buf;
uint32_t size;
buffer->getBuffer(&buf, &size);
return std::string((char*)buf, (unsigned int)size);
}
}} // apache::thrift
*/
#endif // #define THRIFT_PROTOCOL_TJSONPROTOCOL_H_ 1

View File

@ -0,0 +1,287 @@
/**
* Copyright 2012 Facebook
* @author Tudor Bosman (tudorb@fb.com)
*/
#ifndef THRIFT_LIB_CPP_PROTOCOL_TNEUTRONIUMPROTOCOL_H_
#define THRIFT_LIB_CPP_PROTOCOL_TNEUTRONIUMPROTOCOL_H_
#include "thrift/lib/cpp/protocol/TProtocol.h"
#include "thrift/lib/cpp/protocol/TVirtualProtocol.h"
#include "thrift/lib/cpp/protocol/neutronium/Encoder.h"
#include "thrift/lib/cpp/protocol/neutronium/Decoder.h"
namespace apache { namespace thrift { namespace protocol {
class TNeutroniumProtocol
: public TVirtualProtocol<TNeutroniumProtocol> {
public:
TNeutroniumProtocol(const neutronium::Schema* schema,
neutronium::InternTable* internTable,
folly::IOBuf* buf)
: TVirtualProtocol<TNeutroniumProtocol>(nullptr),
enc_(schema, internTable, buf),
dec_(schema, internTable, buf) {
}
void setRootType(int64_t rootType) {
enc_.setRootType(rootType);
dec_.setRootType(rootType);
}
uint32_t writeMessageBegin(const std::string& name,
const TMessageType messageType,
const int32_t seqid) {
LOG(FATAL) << "Message encoding / decoding not implemented";
}
uint32_t writeMessageEnd() {
LOG(FATAL) << "Message encoding / decoding not implemented";
}
uint32_t writeStructBegin(const char* name) {
enc_.writeStructBegin(name);
return 0;
}
uint32_t writeStructEnd() {
enc_.writeStructEnd();
return enc_.bytesWritten();
}
uint32_t writeFieldBegin(const char* name,
const TType fieldType,
const int16_t fieldId) {
enc_.writeFieldBegin(name, fieldType, fieldId);
return 0;
}
uint32_t writeFieldEnd() {
enc_.writeFieldEnd();
return 0;
}
uint32_t writeFieldStop() {
enc_.writeFieldStop();
return 0;
}
uint32_t writeMapBegin(const TType keyType,
const TType valType,
const uint32_t size) {
enc_.writeMapBegin(keyType, valType, size);
return 0;
}
uint32_t writeMapEnd() {
enc_.writeMapEnd();
return 0;
}
uint32_t writeListBegin(const TType elemType, const uint32_t size) {
enc_.writeListBegin(elemType, size);
return 0;
}
uint32_t writeListEnd() {
enc_.writeListEnd();
return 0;
}
uint32_t writeSetBegin(const TType elemType, const uint32_t size) {
enc_.writeSetBegin(elemType, size);
return 0;
}
uint32_t writeSetEnd() {
enc_.writeSetEnd();
return 0;
}
uint32_t writeBool(const bool value) {
enc_.writeBool(value);
return 0;
}
uint32_t writeByte(const int8_t byte) {
enc_.writeByte(byte);
return 0;
}
uint32_t writeI16(const int16_t i16) {
enc_.writeI16(i16);
return 0;
}
uint32_t writeI32(const int32_t i32) {
enc_.writeI32(i32);
return 0;
}
uint32_t writeI64(const int64_t i64) {
enc_.writeI64(i64);
return 0;
}
uint32_t writeDouble(const double dub) {
enc_.writeDouble(dub);
return 0;
}
template <typename StrType>
uint32_t writeString(const StrType& str) {
enc_.writeString(str);
return 0;
}
uint32_t writeBinary(const std::string& str) {
enc_.writeBinary(str);
return 0;
}
/**
* Reading functions
*/
uint32_t readMessageBegin(std::string& name,
TMessageType& messageType,
int32_t& seqid) {
LOG(FATAL) << "Message encoding / decoding not implemented";
}
uint32_t readMessageEnd() {
LOG(FATAL) << "Message encoding / decoding not implemented";
}
uint32_t readStructBegin(std::string& name) {
dec_.readStructBegin();
return 0;
}
uint32_t readStructEnd() {
dec_.readStructEnd();
return dec_.bytesRead();
}
uint32_t readFieldBegin(std::string& name,
TType& fieldType,
int16_t& fieldId) {
dec_.readFieldBegin(fieldType, fieldId);
return 0;
}
uint32_t readFieldEnd() {
dec_.readFieldEnd();
return 0;
}
uint32_t readMapBegin(TType& keyType, TType& valType, uint32_t& size) {
dec_.readMapBegin(keyType, valType, size);
return 0;
}
uint32_t readMapEnd() {
dec_.readMapEnd();
return 0;
}
uint32_t readListBegin(TType& elemType, uint32_t& size) {
dec_.readListBegin(elemType, size);
return 0;
}
uint32_t readListEnd() {
dec_.readListEnd();
return 0;
}
uint32_t readSetBegin(TType& elemType, uint32_t& size) {
dec_.readSetBegin(elemType, size);
return 0;
}
uint32_t readSetEnd() {
dec_.readSetEnd();
return 0;
}
uint32_t readBool(bool& value) {
dec_.readBool(value);
return 0;
}
// Provide the default readBool() implementation for std::vector<bool>
using TVirtualProtocol<TNeutroniumProtocol>::readBool;
uint32_t readByte(int8_t& byte) {
dec_.readByte(byte);
return 0;
}
uint32_t readI16(int16_t& i16) {
dec_.readI16(i16);
return 0;
}
uint32_t readI32(int32_t& i32) {
dec_.readI32(i32);
return 0;
}
uint32_t readI64(int64_t& i64) {
dec_.readI64(i64);
return 0;
}
uint32_t readDouble(double& dub) {
dec_.readDouble(dub);
return 0;
}
template<typename StrType>
uint32_t readString(StrType& str) {
dec_.readString(str);
return 0;
}
uint32_t readBinary(std::string& str) {
dec_.readBinary(str);
return 0;
}
private:
neutronium::Encoder enc_;
neutronium::Decoder dec_;
};
class Neutronium {
public:
explicit Neutronium(const neutronium::Schema* schema,
neutronium::InternTable* internTable,
folly::IOBuf* buf)
: proto_(schema, internTable, buf) {
}
template <class T>
uint32_t serialize(const T& obj) {
proto_.setRootType(T::_reflection_id);
return obj.write(&proto_);
}
template <class T>
uint32_t deserialize(T& obj) {
proto_.setRootType(T::_reflection_id);
return obj.read(&proto_);
}
private:
TNeutroniumProtocol proto_;
};
}}} // apache::thrift::protocol
#endif /* THRIFT_LIB_CPP_PROTOCOL_TNEUTRONIUMPROTOCOL_H_ */

View File

@ -0,0 +1,115 @@
/*
* 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 _THRIFT_PROTOCOL_TPHPSERIALIZEPROTOCOL_H_
#define _THRIFT_PROTOCOL_TPHPSERIALIZEPROTOCOL_H_ 1
#include "thrift/lib/cpp/protocol/TVirtualProtocol.h"
#include "thrift/lib/cpp/transport/TBufferTransports.h"
#include <stack>
#include <boost/shared_ptr.hpp>
namespace apache { namespace thrift { namespace protocol {
/**
* A Thrift protocol for serializing Thrift objects into PHP's
* "serialize" format. Should work properly for objects that
* PHP can properly express. Currently, it can silently corrupt
* data that PHP cannot properly express (lists or bools as map keys,
* very large integers on 32-bit systems, and possibly others).
*/
class TPhpSerializeProtocol : public TVirtualProtocol<TPhpSerializeProtocol> {
public:
TPhpSerializeProtocol(boost::shared_ptr<TTransport> trans)
: TVirtualProtocol<TPhpSerializeProtocol>(trans)
, trans_(trans.get())
{}
uint32_t writeMessageBegin(const std::string& name,
const TMessageType messageType,
const int32_t seqid);
uint32_t writeMessageEnd();
uint32_t writeStructBegin(const char* name);
uint32_t writeStructEnd();
uint32_t writeFieldBegin(const char* name,
const TType fieldType,
const int16_t fieldId);
uint32_t writeFieldEnd();
uint32_t writeFieldStop();
uint32_t writeListBegin(const TType elemType,
const uint32_t size);
uint32_t writeListEnd();
uint32_t writeSetBegin(const TType elemType,
const uint32_t size);
uint32_t writeSetEnd();
uint32_t writeMapBegin(const TType keyType,
const TType valType,
const uint32_t size);
uint32_t writeMapEnd();
uint32_t writeBool(const bool value);
uint32_t writeByte(const int8_t byte);
uint32_t writeI16(const int16_t i16);
uint32_t writeI32(const int32_t i32);
uint32_t writeI64(const int64_t i64);
uint32_t writeDouble(const double dub);
uint32_t writeString(const std::string& str);
uint32_t writeBinary(const std::string& str);
protected:
uint32_t doWriteInt(const int64_t i64);
uint32_t doWriteInt(const std::string& val);
uint32_t doWriteString(const std::string& str, bool is_class);
uint32_t doWriteListBegin(uint32_t size, bool is_map);
uint32_t listKey();
uint32_t write3(const char* v1, const char* v2, const char* v3);
uint32_t write(const char* buf, uint32_t len);
std::stack<int> listPosStack_;
std::stack<int> structSizeStack_;
std::stack< boost::shared_ptr<apache::thrift::transport::TMemoryBuffer> > structBufferStack_;
TTransport* trans_;
};
}}} // apache::thrift::protocol
#endif

Some files were not shown because too many files have changed in this diff Show More