aboutsummaryrefslogtreecommitdiffhomepage
path: root/include/grpc++
diff options
context:
space:
mode:
Diffstat (limited to 'include/grpc++')
-rw-r--r--include/grpc++/client_context.h301
-rw-r--r--include/grpc++/impl/call.h541
-rw-r--r--include/grpc++/impl/codegen/call.h578
-rw-r--r--include/grpc++/impl/codegen/client_context.h353
-rw-r--r--include/grpc++/impl/codegen/serialization_traits.h68
-rw-r--r--include/grpc++/impl/codegen/server_context.h201
-rw-r--r--include/grpc++/impl/codegen/service_type.h161
-rw-r--r--include/grpc++/impl/serialization_traits.h31
-rw-r--r--include/grpc++/impl/service_type.h124
-rw-r--r--include/grpc++/server_context.h164
-rw-r--r--include/grpc++/support/async_stream.h6
11 files changed, 1369 insertions, 1159 deletions
diff --git a/include/grpc++/client_context.h b/include/grpc++/client_context.h
index 02b2ebdff8..3fbe6d1be8 100644
--- a/include/grpc++/client_context.h
+++ b/include/grpc++/client_context.h
@@ -49,305 +49,6 @@
#ifndef GRPCXX_CLIENT_CONTEXT_H
#define GRPCXX_CLIENT_CONTEXT_H
-#include <map>
-#include <memory>
-#include <string>
-
-#include <grpc++/impl/codegen/sync.h>
-#include <grpc++/impl/codegen/security/auth_context.h>
-#include <grpc++/impl/codegen/config.h>
-#include <grpc++/impl/codegen/status.h>
-#include <grpc++/impl/codegen/string_ref.h>
-#include <grpc++/impl/codegen/time.h>
-#include <grpc/impl/codegen/compression_types.h>
-#include <grpc/impl/codegen/propagation_bits.h>
-#include <grpc/impl/codegen/log.h>
-#include <grpc/impl/codegen/time.h>
-
-struct census_context;
-struct grpc_call;
-
-namespace grpc {
-
-class Channel;
-class ChannelInterface;
-class CompletionQueue;
-class CallCredentials;
-class RpcMethod;
-template <class R>
-class ClientReader;
-template <class W>
-class ClientWriter;
-template <class W, class R>
-class ClientReaderWriter;
-template <class R>
-class ClientAsyncReader;
-template <class W>
-class ClientAsyncWriter;
-template <class W, class R>
-class ClientAsyncReaderWriter;
-template <class R>
-class ClientAsyncResponseReader;
-class ServerContext;
-
-/// Options for \a ClientContext::FromServerContext specifying which traits from
-/// the \a ServerContext to propagate (copy) from it into a new \a
-/// ClientContext.
-///
-/// \see ClientContext::FromServerContext
-class PropagationOptions {
- public:
- PropagationOptions() : propagate_(GRPC_PROPAGATE_DEFAULTS) {}
-
- PropagationOptions& enable_deadline_propagation() {
- propagate_ |= GRPC_PROPAGATE_DEADLINE;
- return *this;
- }
-
- PropagationOptions& disable_deadline_propagation() {
- propagate_ &= ~GRPC_PROPAGATE_DEADLINE;
- return *this;
- }
-
- PropagationOptions& enable_census_stats_propagation() {
- propagate_ |= GRPC_PROPAGATE_CENSUS_STATS_CONTEXT;
- return *this;
- }
-
- PropagationOptions& disable_census_stats_propagation() {
- propagate_ &= ~GRPC_PROPAGATE_CENSUS_STATS_CONTEXT;
- return *this;
- }
-
- PropagationOptions& enable_census_tracing_propagation() {
- propagate_ |= GRPC_PROPAGATE_CENSUS_TRACING_CONTEXT;
- return *this;
- }
-
- PropagationOptions& disable_census_tracing_propagation() {
- propagate_ &= ~GRPC_PROPAGATE_CENSUS_TRACING_CONTEXT;
- return *this;
- }
-
- PropagationOptions& enable_cancellation_propagation() {
- propagate_ |= GRPC_PROPAGATE_CANCELLATION;
- return *this;
- }
-
- PropagationOptions& disable_cancellation_propagation() {
- propagate_ &= ~GRPC_PROPAGATE_CANCELLATION;
- return *this;
- }
-
- uint32_t c_bitmask() const { return propagate_; }
-
- private:
- uint32_t propagate_;
-};
-
-namespace testing {
-class InteropClientContextInspector;
-} // namespace testing
-
-class ClientContext {
- public:
- ClientContext();
- ~ClientContext();
-
- /// Create a new \a ClientContext as a child of an incoming server call,
- /// according to \a options (\see PropagationOptions).
- ///
- /// \param server_context The source server context to use as the basis for
- /// constructing the client context.
- /// \param options The options controlling what to copy from the \a
- /// server_context.
- ///
- /// \return A newly constructed \a ClientContext instance based on \a
- /// server_context, with traits propagated (copied) according to \a options.
- static std::unique_ptr<ClientContext> FromServerContext(
- const ServerContext& server_context,
- PropagationOptions options = PropagationOptions());
-
- /// Add the (\a meta_key, \a meta_value) pair to the metadata associated with
- /// a client call. These are made available at the server side by the \a
- /// grpc::ServerContext::client_metadata() method.
- ///
- /// \warning This method should only be called before invoking the rpc.
- ///
- /// \param meta_key The metadata key. If \a meta_value is binary data, it must
- /// end in "-bin".
- /// \param meta_value The metadata value. If its value is binary, it must be
- /// base64-encoding (see https://tools.ietf.org/html/rfc4648#section-4) and \a
- /// meta_key must end in "-bin".
- void AddMetadata(const grpc::string& meta_key,
- const grpc::string& meta_value);
-
- /// Return a collection of initial metadata key-value pairs. Note that keys
- /// may happen more than once (ie, a \a std::multimap is returned).
- ///
- /// \warning This method should only be called after initial metadata has been
- /// received. For streaming calls, see \a
- /// ClientReaderInterface::WaitForInitialMetadata().
- ///
- /// \return A multimap of initial metadata key-value pairs from the server.
- const std::multimap<grpc::string_ref, grpc::string_ref>&
- GetServerInitialMetadata() {
- GPR_ASSERT(initial_metadata_received_);
- return recv_initial_metadata_;
- }
-
- /// Return a collection of trailing metadata key-value pairs. Note that keys
- /// may happen more than once (ie, a \a std::multimap is returned).
- ///
- /// \warning This method is only callable once the stream has finished.
- ///
- /// \return A multimap of metadata trailing key-value pairs from the server.
- const std::multimap<grpc::string_ref, grpc::string_ref>&
- GetServerTrailingMetadata() {
- // TODO(yangg) check finished
- return trailing_metadata_;
- }
-
- /// Set the deadline for the client call.
- ///
- /// \warning This method should only be called before invoking the rpc.
- ///
- /// \param deadline the deadline for the client call. Units are determined by
- /// the type used.
- template <typename T>
- void set_deadline(const T& deadline) {
- TimePoint<T> deadline_tp(deadline);
- deadline_ = deadline_tp.raw_time();
- }
-
-#ifndef GRPC_CXX0X_NO_CHRONO
- /// Return the deadline for the client call.
- std::chrono::system_clock::time_point deadline() {
- return Timespec2Timepoint(deadline_);
- }
-#endif // !GRPC_CXX0X_NO_CHRONO
-
- /// Return a \a gpr_timespec representation of the client call's deadline.
- gpr_timespec raw_deadline() { return deadline_; }
-
- /// Set the per call authority header (see
- /// https://tools.ietf.org/html/rfc7540#section-8.1.2.3).
- void set_authority(const grpc::string& authority) { authority_ = authority; }
-
- /// Return the authentication context for this client call.
- ///
- /// \see grpc::AuthContext.
- std::shared_ptr<const AuthContext> auth_context() const;
-
- /// Set credentials for the client call.
- ///
- /// A credentials object encapsulates all the state needed by a client to
- /// authenticate with a server and make various assertions, e.g., about the
- /// client’s identity, role, or whether it is authorized to make a particular
- /// call.
- ///
- /// \see http://www.grpc.io/docs/guides/auth.html
- void set_credentials(const std::shared_ptr<CallCredentials>& creds) {
- creds_ = creds;
- }
-
- /// Return the compression algorithm to be used by the client call.
- grpc_compression_algorithm compression_algorithm() const {
- return compression_algorithm_;
- }
-
- /// Set \a algorithm to be the compression algorithm used for the client call.
- ///
- /// \param algorith The compression algorithm used for the client call.
- void set_compression_algorithm(grpc_compression_algorithm algorithm);
-
- /// Return the peer uri in a string.
- ///
- /// \warning This value is never authenticated or subject to any security
- /// related code. It must not be used for any authentication related
- /// functionality. Instead, use auth_context.
- ///
- /// \return The call's peer URI.
- grpc::string peer() const;
-
- /// Get and set census context.
- void set_census_context(struct census_context* ccp) { census_context_ = ccp; }
- struct census_context* census_context() const {
- return census_context_;
- }
-
- /// Send a best-effort out-of-band cancel. The call could be in any stage.
- /// e.g. if it is already finished, it may still return success.
- ///
- /// There is no guarantee the call will be cancelled.
- void TryCancel();
-
- /// Global Callbacks
- ///
- /// Can be set exactly once per application to install hooks whenever
- /// a client context is constructed and destructed.
- class GlobalCallbacks {
- public:
- virtual void DefaultConstructor(ClientContext* context) = 0;
- virtual void Destructor(ClientContext* context) = 0;
- };
- static void SetGlobalCallbacks(GlobalCallbacks* callbacks);
-
- private:
- // Disallow copy and assign.
- ClientContext(const ClientContext&);
- ClientContext& operator=(const ClientContext&);
-
- friend class ::grpc::testing::InteropClientContextInspector;
- friend class CallOpClientRecvStatus;
- friend class CallOpRecvInitialMetadata;
- friend class Channel;
- template <class R>
- friend class ::grpc::ClientReader;
- template <class W>
- friend class ::grpc::ClientWriter;
- template <class W, class R>
- friend class ::grpc::ClientReaderWriter;
- template <class R>
- friend class ::grpc::ClientAsyncReader;
- template <class W>
- friend class ::grpc::ClientAsyncWriter;
- template <class W, class R>
- friend class ::grpc::ClientAsyncReaderWriter;
- template <class R>
- friend class ::grpc::ClientAsyncResponseReader;
- template <class InputMessage, class OutputMessage>
- friend Status BlockingUnaryCall(ChannelInterface* channel,
- const RpcMethod& method,
- ClientContext* context,
- const InputMessage& request,
- OutputMessage* result);
-
- grpc_call* call() { return call_; }
- void set_call(grpc_call* call, const std::shared_ptr<Channel>& channel);
-
- grpc::string authority() { return authority_; }
-
- bool initial_metadata_received_;
- std::shared_ptr<Channel> channel_;
- grpc::mutex mu_;
- grpc_call* call_;
- bool call_canceled_;
- gpr_timespec deadline_;
- grpc::string authority_;
- std::shared_ptr<CallCredentials> creds_;
- mutable std::shared_ptr<const AuthContext> auth_context_;
- struct census_context* census_context_;
- std::multimap<grpc::string, grpc::string> send_initial_metadata_;
- std::multimap<grpc::string_ref, grpc::string_ref> recv_initial_metadata_;
- std::multimap<grpc::string_ref, grpc::string_ref> trailing_metadata_;
-
- grpc_call* propagate_from_call_;
- PropagationOptions propagation_options_;
-
- grpc_compression_algorithm compression_algorithm_;
-};
-
-} // namespace grpc
+#include <grpc++/impl/codegen/client_context.h>
#endif // GRPCXX_CLIENT_CONTEXT_H
diff --git a/include/grpc++/impl/call.h b/include/grpc++/impl/call.h
index ec64c9f884..64b6563574 100644
--- a/include/grpc++/impl/call.h
+++ b/include/grpc++/impl/call.h
@@ -34,545 +34,6 @@
#ifndef GRPCXX_IMPL_CALL_H
#define GRPCXX_IMPL_CALL_H
-#include <functional>
-#include <memory>
-#include <map>
-#include <cstring>
-
-#include <grpc/support/alloc.h>
-#include <grpc/impl/codegen/grpc_types.h>
-#include <grpc++/client_context.h>
-#include <grpc++/impl/codegen/call_hook.h>
-#include <grpc++/impl/codegen/completion_queue_tag.h>
-#include <grpc++/impl/serialization_traits.h>
-#include <grpc++/impl/codegen/config.h>
-#include <grpc++/impl/codegen/status.h>
-
-struct grpc_byte_buffer;
-
-namespace grpc {
-
-class ByteBuffer;
-class Call;
-class CallHook;
-class CompletionQueue;
-
-void FillMetadataMap(
- grpc_metadata_array* arr,
- std::multimap<grpc::string_ref, grpc::string_ref>* metadata);
-grpc_metadata* FillMetadataArray(
- const std::multimap<grpc::string, grpc::string>& metadata);
-
-/// Per-message write options.
-class WriteOptions {
- public:
- WriteOptions() : flags_(0) {}
- WriteOptions(const WriteOptions& other) : flags_(other.flags_) {}
-
- /// Clear all flags.
- inline void Clear() { flags_ = 0; }
-
- /// Returns raw flags bitset.
- inline uint32_t flags() const { return flags_; }
-
- /// Sets flag for the disabling of compression for the next message write.
- ///
- /// \sa GRPC_WRITE_NO_COMPRESS
- inline WriteOptions& set_no_compression() {
- SetBit(GRPC_WRITE_NO_COMPRESS);
- return *this;
- }
-
- /// Clears flag for the disabling of compression for the next message write.
- ///
- /// \sa GRPC_WRITE_NO_COMPRESS
- inline WriteOptions& clear_no_compression() {
- ClearBit(GRPC_WRITE_NO_COMPRESS);
- return *this;
- }
-
- /// Get value for the flag indicating whether compression for the next
- /// message write is forcefully disabled.
- ///
- /// \sa GRPC_WRITE_NO_COMPRESS
- inline bool get_no_compression() const {
- return GetBit(GRPC_WRITE_NO_COMPRESS);
- }
-
- /// Sets flag indicating that the write may be buffered and need not go out on
- /// the wire immediately.
- ///
- /// \sa GRPC_WRITE_BUFFER_HINT
- inline WriteOptions& set_buffer_hint() {
- SetBit(GRPC_WRITE_BUFFER_HINT);
- return *this;
- }
-
- /// Clears flag indicating that the write may be buffered and need not go out
- /// on the wire immediately.
- ///
- /// \sa GRPC_WRITE_BUFFER_HINT
- inline WriteOptions& clear_buffer_hint() {
- ClearBit(GRPC_WRITE_BUFFER_HINT);
- return *this;
- }
-
- /// Get value for the flag indicating that the write may be buffered and need
- /// not go out on the wire immediately.
- ///
- /// \sa GRPC_WRITE_BUFFER_HINT
- inline bool get_buffer_hint() const { return GetBit(GRPC_WRITE_BUFFER_HINT); }
-
- WriteOptions& operator=(const WriteOptions& rhs) {
- flags_ = rhs.flags_;
- return *this;
- }
-
- private:
- void SetBit(const uint32_t mask) { flags_ |= mask; }
-
- void ClearBit(const uint32_t mask) { flags_ &= ~mask; }
-
- bool GetBit(const uint32_t mask) const { return (flags_ & mask) != 0; }
-
- uint32_t flags_;
-};
-
-/// Default argument for CallOpSet. I is unused by the class, but can be
-/// used for generating multiple names for the same thing.
-template <int I>
-class CallNoOp {
- protected:
- void AddOp(grpc_op* ops, size_t* nops) {}
- void FinishOp(bool* status, int max_message_size) {}
-};
-
-class CallOpSendInitialMetadata {
- public:
- CallOpSendInitialMetadata() : send_(false) {}
-
- void SendInitialMetadata(
- const std::multimap<grpc::string, grpc::string>& metadata) {
- send_ = true;
- initial_metadata_count_ = metadata.size();
- initial_metadata_ = FillMetadataArray(metadata);
- }
-
- protected:
- void AddOp(grpc_op* ops, size_t* nops) {
- if (!send_) return;
- grpc_op* op = &ops[(*nops)++];
- op->op = GRPC_OP_SEND_INITIAL_METADATA;
- op->flags = 0;
- op->reserved = NULL;
- op->data.send_initial_metadata.count = initial_metadata_count_;
- op->data.send_initial_metadata.metadata = initial_metadata_;
- }
- void FinishOp(bool* status, int max_message_size) {
- if (!send_) return;
- gpr_free(initial_metadata_);
- send_ = false;
- }
-
- bool send_;
- size_t initial_metadata_count_;
- grpc_metadata* initial_metadata_;
-};
-
-class CallOpSendMessage {
- public:
- CallOpSendMessage() : send_buf_(nullptr), own_buf_(false) {}
-
- /// Send \a message using \a options for the write. The \a options are cleared
- /// after use.
- template <class M>
- Status SendMessage(const M& message,
- const WriteOptions& options) GRPC_MUST_USE_RESULT;
-
- template <class M>
- Status SendMessage(const M& message) GRPC_MUST_USE_RESULT;
-
- protected:
- void AddOp(grpc_op* ops, size_t* nops) {
- if (send_buf_ == nullptr) return;
- grpc_op* op = &ops[(*nops)++];
- op->op = GRPC_OP_SEND_MESSAGE;
- op->flags = write_options_.flags();
- op->reserved = NULL;
- op->data.send_message = send_buf_;
- // Flags are per-message: clear them after use.
- write_options_.Clear();
- }
- void FinishOp(bool* status, int max_message_size) {
- if (own_buf_) grpc_byte_buffer_destroy(send_buf_);
- send_buf_ = nullptr;
- }
-
- private:
- grpc_byte_buffer* send_buf_;
- WriteOptions write_options_;
- bool own_buf_;
-};
-
-template <class M>
-Status CallOpSendMessage::SendMessage(const M& message,
- const WriteOptions& options) {
- write_options_ = options;
- return SerializationTraits<M>::Serialize(message, &send_buf_, &own_buf_);
-}
-
-template <class M>
-Status CallOpSendMessage::SendMessage(const M& message) {
- return SendMessage(message, WriteOptions());
-}
-
-template <class R>
-class CallOpRecvMessage {
- public:
- CallOpRecvMessage() : got_message(false), message_(nullptr) {}
-
- void RecvMessage(R* message) { message_ = message; }
-
- bool got_message;
-
- protected:
- void AddOp(grpc_op* ops, size_t* nops) {
- if (message_ == nullptr) return;
- grpc_op* op = &ops[(*nops)++];
- op->op = GRPC_OP_RECV_MESSAGE;
- op->flags = 0;
- op->reserved = NULL;
- op->data.recv_message = &recv_buf_;
- }
-
- void FinishOp(bool* status, int max_message_size) {
- if (message_ == nullptr) return;
- if (recv_buf_) {
- if (*status) {
- got_message = true;
- *status = SerializationTraits<R>::Deserialize(recv_buf_, message_,
- max_message_size).ok();
- } else {
- got_message = false;
- grpc_byte_buffer_destroy(recv_buf_);
- }
- } else {
- got_message = false;
- *status = false;
- }
- message_ = nullptr;
- }
-
- private:
- R* message_;
- grpc_byte_buffer* recv_buf_;
-};
-
-namespace CallOpGenericRecvMessageHelper {
-class DeserializeFunc {
- public:
- virtual Status Deserialize(grpc_byte_buffer* buf, int max_message_size) = 0;
-};
-
-template <class R>
-class DeserializeFuncType GRPC_FINAL : public DeserializeFunc {
- public:
- DeserializeFuncType(R* message) : message_(message) {}
- Status Deserialize(grpc_byte_buffer* buf,
- int max_message_size) GRPC_OVERRIDE {
- return SerializationTraits<R>::Deserialize(buf, message_, max_message_size);
- }
-
- private:
- R* message_; // Not a managed pointer because management is external to this
-};
-} // namespace CallOpGenericRecvMessageHelper
-
-class CallOpGenericRecvMessage {
- public:
- CallOpGenericRecvMessage() : got_message(false) {}
-
- template <class R>
- void RecvMessage(R* message) {
- deserialize_.reset(
- new CallOpGenericRecvMessageHelper::DeserializeFuncType<R>(message));
- }
-
- bool got_message;
-
- protected:
- void AddOp(grpc_op* ops, size_t* nops) {
- if (!deserialize_) return;
- grpc_op* op = &ops[(*nops)++];
- op->op = GRPC_OP_RECV_MESSAGE;
- op->flags = 0;
- op->reserved = NULL;
- op->data.recv_message = &recv_buf_;
- }
-
- void FinishOp(bool* status, int max_message_size) {
- if (!deserialize_) return;
- if (recv_buf_) {
- if (*status) {
- got_message = true;
- *status = deserialize_->Deserialize(recv_buf_, max_message_size).ok();
- } else {
- got_message = false;
- grpc_byte_buffer_destroy(recv_buf_);
- }
- } else {
- got_message = false;
- *status = false;
- }
- deserialize_.reset();
- }
-
- private:
- std::unique_ptr<CallOpGenericRecvMessageHelper::DeserializeFunc> deserialize_;
- grpc_byte_buffer* recv_buf_;
-};
-
-class CallOpClientSendClose {
- public:
- CallOpClientSendClose() : send_(false) {}
-
- void ClientSendClose() { send_ = true; }
-
- protected:
- void AddOp(grpc_op* ops, size_t* nops) {
- if (!send_) return;
- grpc_op* op = &ops[(*nops)++];
- op->op = GRPC_OP_SEND_CLOSE_FROM_CLIENT;
- op->flags = 0;
- op->reserved = NULL;
- }
- void FinishOp(bool* status, int max_message_size) { send_ = false; }
-
- private:
- bool send_;
-};
-
-class CallOpServerSendStatus {
- public:
- CallOpServerSendStatus() : send_status_available_(false) {}
-
- void ServerSendStatus(
- const std::multimap<grpc::string, grpc::string>& trailing_metadata,
- const Status& status) {
- trailing_metadata_count_ = trailing_metadata.size();
- trailing_metadata_ = FillMetadataArray(trailing_metadata);
- send_status_available_ = true;
- send_status_code_ = static_cast<grpc_status_code>(status.error_code());
- send_status_details_ = status.error_message();
- }
-
- protected:
- void AddOp(grpc_op* ops, size_t* nops) {
- if (!send_status_available_) return;
- grpc_op* op = &ops[(*nops)++];
- op->op = GRPC_OP_SEND_STATUS_FROM_SERVER;
- op->data.send_status_from_server.trailing_metadata_count =
- trailing_metadata_count_;
- op->data.send_status_from_server.trailing_metadata = trailing_metadata_;
- op->data.send_status_from_server.status = send_status_code_;
- op->data.send_status_from_server.status_details =
- send_status_details_.empty() ? nullptr : send_status_details_.c_str();
- op->flags = 0;
- op->reserved = NULL;
- }
-
- void FinishOp(bool* status, int max_message_size) {
- if (!send_status_available_) return;
- gpr_free(trailing_metadata_);
- send_status_available_ = false;
- }
-
- private:
- bool send_status_available_;
- grpc_status_code send_status_code_;
- grpc::string send_status_details_;
- size_t trailing_metadata_count_;
- grpc_metadata* trailing_metadata_;
-};
-
-class CallOpRecvInitialMetadata {
- public:
- CallOpRecvInitialMetadata() : recv_initial_metadata_(nullptr) {}
-
- void RecvInitialMetadata(ClientContext* context) {
- context->initial_metadata_received_ = true;
- recv_initial_metadata_ = &context->recv_initial_metadata_;
- }
-
- protected:
- void AddOp(grpc_op* ops, size_t* nops) {
- if (!recv_initial_metadata_) return;
- memset(&recv_initial_metadata_arr_, 0, sizeof(recv_initial_metadata_arr_));
- grpc_op* op = &ops[(*nops)++];
- op->op = GRPC_OP_RECV_INITIAL_METADATA;
- op->data.recv_initial_metadata = &recv_initial_metadata_arr_;
- op->flags = 0;
- op->reserved = NULL;
- }
- void FinishOp(bool* status, int max_message_size) {
- if (recv_initial_metadata_ == nullptr) return;
- FillMetadataMap(&recv_initial_metadata_arr_, recv_initial_metadata_);
- recv_initial_metadata_ = nullptr;
- }
-
- private:
- std::multimap<grpc::string_ref, grpc::string_ref>* recv_initial_metadata_;
- grpc_metadata_array recv_initial_metadata_arr_;
-};
-
-class CallOpClientRecvStatus {
- public:
- CallOpClientRecvStatus() : recv_status_(nullptr) {}
-
- void ClientRecvStatus(ClientContext* context, Status* status) {
- recv_trailing_metadata_ = &context->trailing_metadata_;
- recv_status_ = status;
- }
-
- protected:
- void AddOp(grpc_op* ops, size_t* nops) {
- if (recv_status_ == nullptr) return;
- memset(&recv_trailing_metadata_arr_, 0,
- sizeof(recv_trailing_metadata_arr_));
- status_details_ = nullptr;
- status_details_capacity_ = 0;
- grpc_op* op = &ops[(*nops)++];
- op->op = GRPC_OP_RECV_STATUS_ON_CLIENT;
- op->data.recv_status_on_client.trailing_metadata =
- &recv_trailing_metadata_arr_;
- op->data.recv_status_on_client.status = &status_code_;
- op->data.recv_status_on_client.status_details = &status_details_;
- op->data.recv_status_on_client.status_details_capacity =
- &status_details_capacity_;
- op->flags = 0;
- op->reserved = NULL;
- }
-
- void FinishOp(bool* status, int max_message_size) {
- if (recv_status_ == nullptr) return;
- FillMetadataMap(&recv_trailing_metadata_arr_, recv_trailing_metadata_);
- *recv_status_ = Status(
- static_cast<StatusCode>(status_code_),
- status_details_ ? grpc::string(status_details_) : grpc::string());
- gpr_free(status_details_);
- recv_status_ = nullptr;
- }
-
- private:
- std::multimap<grpc::string_ref, grpc::string_ref>* recv_trailing_metadata_;
- Status* recv_status_;
- grpc_metadata_array recv_trailing_metadata_arr_;
- grpc_status_code status_code_;
- char* status_details_;
- size_t status_details_capacity_;
-};
-
-/// An abstract collection of call ops, used to generate the
-/// grpc_call_op structure to pass down to the lower layers,
-/// and as it is-a CompletionQueueTag, also massages the final
-/// completion into the correct form for consumption in the C++
-/// API.
-class CallOpSetInterface : public CompletionQueueTag {
- public:
- CallOpSetInterface() : max_message_size_(0) {}
- /// Fills in grpc_op, starting from ops[*nops] and moving
- /// upwards.
- virtual void FillOps(grpc_op* ops, size_t* nops) = 0;
-
- void set_max_message_size(int max_message_size) {
- max_message_size_ = max_message_size;
- }
-
- protected:
- int max_message_size_;
-};
-
-/// Primary implementaiton of CallOpSetInterface.
-/// Since we cannot use variadic templates, we declare slots up to
-/// the maximum count of ops we'll need in a set. We leverage the
-/// empty base class optimization to slim this class (especially
-/// when there are many unused slots used). To avoid duplicate base classes,
-/// the template parmeter for CallNoOp is varied by argument position.
-template <class Op1 = CallNoOp<1>, class Op2 = CallNoOp<2>,
- class Op3 = CallNoOp<3>, class Op4 = CallNoOp<4>,
- class Op5 = CallNoOp<5>, class Op6 = CallNoOp<6>>
-class CallOpSet : public CallOpSetInterface,
- public Op1,
- public Op2,
- public Op3,
- public Op4,
- public Op5,
- public Op6 {
- public:
- CallOpSet() : return_tag_(this) {}
- void FillOps(grpc_op* ops, size_t* nops) GRPC_OVERRIDE {
- this->Op1::AddOp(ops, nops);
- this->Op2::AddOp(ops, nops);
- this->Op3::AddOp(ops, nops);
- this->Op4::AddOp(ops, nops);
- this->Op5::AddOp(ops, nops);
- this->Op6::AddOp(ops, nops);
- }
-
- bool FinalizeResult(void** tag, bool* status) GRPC_OVERRIDE {
- this->Op1::FinishOp(status, max_message_size_);
- this->Op2::FinishOp(status, max_message_size_);
- this->Op3::FinishOp(status, max_message_size_);
- this->Op4::FinishOp(status, max_message_size_);
- this->Op5::FinishOp(status, max_message_size_);
- this->Op6::FinishOp(status, max_message_size_);
- *tag = return_tag_;
- return true;
- }
-
- void set_output_tag(void* return_tag) { return_tag_ = return_tag; }
-
- private:
- void* return_tag_;
-};
-
-/// A CallOpSet that does not post completions to the completion queue.
-///
-/// Allows hiding some completions that the C core must generate from
-/// C++ users.
-template <class Op1 = CallNoOp<1>, class Op2 = CallNoOp<2>,
- class Op3 = CallNoOp<3>, class Op4 = CallNoOp<4>,
- class Op5 = CallNoOp<5>, class Op6 = CallNoOp<6>>
-class SneakyCallOpSet : public CallOpSet<Op1, Op2, Op3, Op4, Op5, Op6> {
- public:
- bool FinalizeResult(void** tag, bool* status) GRPC_OVERRIDE {
- typedef CallOpSet<Op1, Op2, Op3, Op4, Op5, Op6> Base;
- return Base::FinalizeResult(tag, status) && false;
- }
-};
-
-// Straightforward wrapping of the C call object
-class Call GRPC_FINAL {
- public:
- /* call is owned by the caller */
- Call(grpc_call* call, CallHook* call_hook_, CompletionQueue* cq);
- Call(grpc_call* call, CallHook* call_hook_, CompletionQueue* cq,
- int max_message_size);
-
- void PerformOps(CallOpSetInterface* ops);
-
- grpc_call* call() { return call_; }
- CompletionQueue* cq() { return cq_; }
-
- int max_message_size() { return max_message_size_; }
-
- private:
- CallHook* call_hook_;
- CompletionQueue* cq_;
- grpc_call* call_;
- int max_message_size_;
-};
-
-} // namespace grpc
+#include <grpc++/impl/codegen/call.h>
#endif // GRPCXX_IMPL_CALL_H
diff --git a/include/grpc++/impl/codegen/call.h b/include/grpc++/impl/codegen/call.h
new file mode 100644
index 0000000000..1e06768ac4
--- /dev/null
+++ b/include/grpc++/impl/codegen/call.h
@@ -0,0 +1,578 @@
+/*
+ *
+ * Copyright 2015-2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ * * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPCXX_IMPL_CODEGEN_CALL_H
+#define GRPCXX_IMPL_CODEGEN_CALL_H
+
+#include <functional>
+#include <memory>
+#include <map>
+#include <cstring>
+
+#include <grpc/impl/codegen/alloc.h>
+#include <grpc/impl/codegen/grpc_types.h>
+#include <grpc++/impl/codegen/client_context.h>
+#include <grpc++/impl/codegen/call_hook.h>
+#include <grpc++/impl/codegen/completion_queue_tag.h>
+#include <grpc++/impl/codegen/serialization_traits.h>
+#include <grpc++/impl/codegen/config.h>
+#include <grpc++/impl/codegen/status.h>
+
+struct grpc_byte_buffer;
+
+namespace grpc {
+
+class ByteBuffer;
+class Call;
+class CallHook;
+class CompletionQueue;
+
+void FillMetadataMap(
+ grpc_metadata_array* arr,
+ std::multimap<grpc::string_ref, grpc::string_ref>* metadata);
+grpc_metadata* FillMetadataArray(
+ const std::multimap<grpc::string, grpc::string>& metadata);
+
+/// Per-message write options.
+class WriteOptions {
+ public:
+ WriteOptions() : flags_(0) {}
+ WriteOptions(const WriteOptions& other) : flags_(other.flags_) {}
+
+ /// Clear all flags.
+ inline void Clear() { flags_ = 0; }
+
+ /// Returns raw flags bitset.
+ inline uint32_t flags() const { return flags_; }
+
+ /// Sets flag for the disabling of compression for the next message write.
+ ///
+ /// \sa GRPC_WRITE_NO_COMPRESS
+ inline WriteOptions& set_no_compression() {
+ SetBit(GRPC_WRITE_NO_COMPRESS);
+ return *this;
+ }
+
+ /// Clears flag for the disabling of compression for the next message write.
+ ///
+ /// \sa GRPC_WRITE_NO_COMPRESS
+ inline WriteOptions& clear_no_compression() {
+ ClearBit(GRPC_WRITE_NO_COMPRESS);
+ return *this;
+ }
+
+ /// Get value for the flag indicating whether compression for the next
+ /// message write is forcefully disabled.
+ ///
+ /// \sa GRPC_WRITE_NO_COMPRESS
+ inline bool get_no_compression() const {
+ return GetBit(GRPC_WRITE_NO_COMPRESS);
+ }
+
+ /// Sets flag indicating that the write may be buffered and need not go out on
+ /// the wire immediately.
+ ///
+ /// \sa GRPC_WRITE_BUFFER_HINT
+ inline WriteOptions& set_buffer_hint() {
+ SetBit(GRPC_WRITE_BUFFER_HINT);
+ return *this;
+ }
+
+ /// Clears flag indicating that the write may be buffered and need not go out
+ /// on the wire immediately.
+ ///
+ /// \sa GRPC_WRITE_BUFFER_HINT
+ inline WriteOptions& clear_buffer_hint() {
+ ClearBit(GRPC_WRITE_BUFFER_HINT);
+ return *this;
+ }
+
+ /// Get value for the flag indicating that the write may be buffered and need
+ /// not go out on the wire immediately.
+ ///
+ /// \sa GRPC_WRITE_BUFFER_HINT
+ inline bool get_buffer_hint() const { return GetBit(GRPC_WRITE_BUFFER_HINT); }
+
+ WriteOptions& operator=(const WriteOptions& rhs) {
+ flags_ = rhs.flags_;
+ return *this;
+ }
+
+ private:
+ void SetBit(const uint32_t mask) { flags_ |= mask; }
+
+ void ClearBit(const uint32_t mask) { flags_ &= ~mask; }
+
+ bool GetBit(const uint32_t mask) const { return (flags_ & mask) != 0; }
+
+ uint32_t flags_;
+};
+
+/// Default argument for CallOpSet. I is unused by the class, but can be
+/// used for generating multiple names for the same thing.
+template <int I>
+class CallNoOp {
+ protected:
+ void AddOp(grpc_op* ops, size_t* nops) {}
+ void FinishOp(bool* status, int max_message_size) {}
+};
+
+class CallOpSendInitialMetadata {
+ public:
+ CallOpSendInitialMetadata() : send_(false) {}
+
+ void SendInitialMetadata(
+ const std::multimap<grpc::string, grpc::string>& metadata) {
+ send_ = true;
+ initial_metadata_count_ = metadata.size();
+ initial_metadata_ = FillMetadataArray(metadata);
+ }
+
+ protected:
+ void AddOp(grpc_op* ops, size_t* nops) {
+ if (!send_) return;
+ grpc_op* op = &ops[(*nops)++];
+ op->op = GRPC_OP_SEND_INITIAL_METADATA;
+ op->flags = 0;
+ op->reserved = NULL;
+ op->data.send_initial_metadata.count = initial_metadata_count_;
+ op->data.send_initial_metadata.metadata = initial_metadata_;
+ }
+ void FinishOp(bool* status, int max_message_size) {
+ if (!send_) return;
+ gpr_free(initial_metadata_);
+ send_ = false;
+ }
+
+ bool send_;
+ size_t initial_metadata_count_;
+ grpc_metadata* initial_metadata_;
+};
+
+class CallOpSendMessage {
+ public:
+ CallOpSendMessage() : send_buf_(nullptr), own_buf_(false) {}
+
+ /// Send \a message using \a options for the write. The \a options are cleared
+ /// after use.
+ template <class M>
+ Status SendMessage(const M& message,
+ const WriteOptions& options) GRPC_MUST_USE_RESULT;
+
+ template <class M>
+ Status SendMessage(const M& message) GRPC_MUST_USE_RESULT;
+
+ protected:
+ void AddOp(grpc_op* ops, size_t* nops) {
+ if (send_buf_ == nullptr) return;
+ grpc_op* op = &ops[(*nops)++];
+ op->op = GRPC_OP_SEND_MESSAGE;
+ op->flags = write_options_.flags();
+ op->reserved = NULL;
+ op->data.send_message = send_buf_;
+ // Flags are per-message: clear them after use.
+ write_options_.Clear();
+ }
+ void FinishOp(bool* status, int max_message_size) {
+ if (own_buf_) grpc_byte_buffer_destroy(send_buf_);
+ send_buf_ = nullptr;
+ }
+
+ private:
+ grpc_byte_buffer* send_buf_;
+ WriteOptions write_options_;
+ bool own_buf_;
+};
+
+template <class M>
+Status CallOpSendMessage::SendMessage(const M& message,
+ const WriteOptions& options) {
+ write_options_ = options;
+ return SerializationTraits<M>::Serialize(message, &send_buf_, &own_buf_);
+}
+
+template <class M>
+Status CallOpSendMessage::SendMessage(const M& message) {
+ return SendMessage(message, WriteOptions());
+}
+
+template <class R>
+class CallOpRecvMessage {
+ public:
+ CallOpRecvMessage() : got_message(false), message_(nullptr) {}
+
+ void RecvMessage(R* message) { message_ = message; }
+
+ bool got_message;
+
+ protected:
+ void AddOp(grpc_op* ops, size_t* nops) {
+ if (message_ == nullptr) return;
+ grpc_op* op = &ops[(*nops)++];
+ op->op = GRPC_OP_RECV_MESSAGE;
+ op->flags = 0;
+ op->reserved = NULL;
+ op->data.recv_message = &recv_buf_;
+ }
+
+ void FinishOp(bool* status, int max_message_size) {
+ if (message_ == nullptr) return;
+ if (recv_buf_) {
+ if (*status) {
+ got_message = true;
+ *status = SerializationTraits<R>::Deserialize(recv_buf_, message_,
+ max_message_size).ok();
+ } else {
+ got_message = false;
+ grpc_byte_buffer_destroy(recv_buf_);
+ }
+ } else {
+ got_message = false;
+ *status = false;
+ }
+ message_ = nullptr;
+ }
+
+ private:
+ R* message_;
+ grpc_byte_buffer* recv_buf_;
+};
+
+namespace CallOpGenericRecvMessageHelper {
+class DeserializeFunc {
+ public:
+ virtual Status Deserialize(grpc_byte_buffer* buf, int max_message_size) = 0;
+};
+
+template <class R>
+class DeserializeFuncType GRPC_FINAL : public DeserializeFunc {
+ public:
+ DeserializeFuncType(R* message) : message_(message) {}
+ Status Deserialize(grpc_byte_buffer* buf,
+ int max_message_size) GRPC_OVERRIDE {
+ return SerializationTraits<R>::Deserialize(buf, message_, max_message_size);
+ }
+
+ private:
+ R* message_; // Not a managed pointer because management is external to this
+};
+} // namespace CallOpGenericRecvMessageHelper
+
+class CallOpGenericRecvMessage {
+ public:
+ CallOpGenericRecvMessage() : got_message(false) {}
+
+ template <class R>
+ void RecvMessage(R* message) {
+ deserialize_.reset(
+ new CallOpGenericRecvMessageHelper::DeserializeFuncType<R>(message));
+ }
+
+ bool got_message;
+
+ protected:
+ void AddOp(grpc_op* ops, size_t* nops) {
+ if (!deserialize_) return;
+ grpc_op* op = &ops[(*nops)++];
+ op->op = GRPC_OP_RECV_MESSAGE;
+ op->flags = 0;
+ op->reserved = NULL;
+ op->data.recv_message = &recv_buf_;
+ }
+
+ void FinishOp(bool* status, int max_message_size) {
+ if (!deserialize_) return;
+ if (recv_buf_) {
+ if (*status) {
+ got_message = true;
+ *status = deserialize_->Deserialize(recv_buf_, max_message_size).ok();
+ } else {
+ got_message = false;
+ grpc_byte_buffer_destroy(recv_buf_);
+ }
+ } else {
+ got_message = false;
+ *status = false;
+ }
+ deserialize_.reset();
+ }
+
+ private:
+ std::unique_ptr<CallOpGenericRecvMessageHelper::DeserializeFunc> deserialize_;
+ grpc_byte_buffer* recv_buf_;
+};
+
+class CallOpClientSendClose {
+ public:
+ CallOpClientSendClose() : send_(false) {}
+
+ void ClientSendClose() { send_ = true; }
+
+ protected:
+ void AddOp(grpc_op* ops, size_t* nops) {
+ if (!send_) return;
+ grpc_op* op = &ops[(*nops)++];
+ op->op = GRPC_OP_SEND_CLOSE_FROM_CLIENT;
+ op->flags = 0;
+ op->reserved = NULL;
+ }
+ void FinishOp(bool* status, int max_message_size) { send_ = false; }
+
+ private:
+ bool send_;
+};
+
+class CallOpServerSendStatus {
+ public:
+ CallOpServerSendStatus() : send_status_available_(false) {}
+
+ void ServerSendStatus(
+ const std::multimap<grpc::string, grpc::string>& trailing_metadata,
+ const Status& status) {
+ trailing_metadata_count_ = trailing_metadata.size();
+ trailing_metadata_ = FillMetadataArray(trailing_metadata);
+ send_status_available_ = true;
+ send_status_code_ = static_cast<grpc_status_code>(status.error_code());
+ send_status_details_ = status.error_message();
+ }
+
+ protected:
+ void AddOp(grpc_op* ops, size_t* nops) {
+ if (!send_status_available_) return;
+ grpc_op* op = &ops[(*nops)++];
+ op->op = GRPC_OP_SEND_STATUS_FROM_SERVER;
+ op->data.send_status_from_server.trailing_metadata_count =
+ trailing_metadata_count_;
+ op->data.send_status_from_server.trailing_metadata = trailing_metadata_;
+ op->data.send_status_from_server.status = send_status_code_;
+ op->data.send_status_from_server.status_details =
+ send_status_details_.empty() ? nullptr : send_status_details_.c_str();
+ op->flags = 0;
+ op->reserved = NULL;
+ }
+
+ void FinishOp(bool* status, int max_message_size) {
+ if (!send_status_available_) return;
+ gpr_free(trailing_metadata_);
+ send_status_available_ = false;
+ }
+
+ private:
+ bool send_status_available_;
+ grpc_status_code send_status_code_;
+ grpc::string send_status_details_;
+ size_t trailing_metadata_count_;
+ grpc_metadata* trailing_metadata_;
+};
+
+class CallOpRecvInitialMetadata {
+ public:
+ CallOpRecvInitialMetadata() : recv_initial_metadata_(nullptr) {}
+
+ void RecvInitialMetadata(ClientContext* context) {
+ context->initial_metadata_received_ = true;
+ recv_initial_metadata_ = &context->recv_initial_metadata_;
+ }
+
+ protected:
+ void AddOp(grpc_op* ops, size_t* nops) {
+ if (!recv_initial_metadata_) return;
+ memset(&recv_initial_metadata_arr_, 0, sizeof(recv_initial_metadata_arr_));
+ grpc_op* op = &ops[(*nops)++];
+ op->op = GRPC_OP_RECV_INITIAL_METADATA;
+ op->data.recv_initial_metadata = &recv_initial_metadata_arr_;
+ op->flags = 0;
+ op->reserved = NULL;
+ }
+ void FinishOp(bool* status, int max_message_size) {
+ if (recv_initial_metadata_ == nullptr) return;
+ FillMetadataMap(&recv_initial_metadata_arr_, recv_initial_metadata_);
+ recv_initial_metadata_ = nullptr;
+ }
+
+ private:
+ std::multimap<grpc::string_ref, grpc::string_ref>* recv_initial_metadata_;
+ grpc_metadata_array recv_initial_metadata_arr_;
+};
+
+class CallOpClientRecvStatus {
+ public:
+ CallOpClientRecvStatus() : recv_status_(nullptr) {}
+
+ void ClientRecvStatus(ClientContext* context, Status* status) {
+ recv_trailing_metadata_ = &context->trailing_metadata_;
+ recv_status_ = status;
+ }
+
+ protected:
+ void AddOp(grpc_op* ops, size_t* nops) {
+ if (recv_status_ == nullptr) return;
+ memset(&recv_trailing_metadata_arr_, 0,
+ sizeof(recv_trailing_metadata_arr_));
+ status_details_ = nullptr;
+ status_details_capacity_ = 0;
+ grpc_op* op = &ops[(*nops)++];
+ op->op = GRPC_OP_RECV_STATUS_ON_CLIENT;
+ op->data.recv_status_on_client.trailing_metadata =
+ &recv_trailing_metadata_arr_;
+ op->data.recv_status_on_client.status = &status_code_;
+ op->data.recv_status_on_client.status_details = &status_details_;
+ op->data.recv_status_on_client.status_details_capacity =
+ &status_details_capacity_;
+ op->flags = 0;
+ op->reserved = NULL;
+ }
+
+ void FinishOp(bool* status, int max_message_size) {
+ if (recv_status_ == nullptr) return;
+ FillMetadataMap(&recv_trailing_metadata_arr_, recv_trailing_metadata_);
+ *recv_status_ = Status(
+ static_cast<StatusCode>(status_code_),
+ status_details_ ? grpc::string(status_details_) : grpc::string());
+ gpr_free(status_details_);
+ recv_status_ = nullptr;
+ }
+
+ private:
+ std::multimap<grpc::string_ref, grpc::string_ref>* recv_trailing_metadata_;
+ Status* recv_status_;
+ grpc_metadata_array recv_trailing_metadata_arr_;
+ grpc_status_code status_code_;
+ char* status_details_;
+ size_t status_details_capacity_;
+};
+
+/// An abstract collection of call ops, used to generate the
+/// grpc_call_op structure to pass down to the lower layers,
+/// and as it is-a CompletionQueueTag, also massages the final
+/// completion into the correct form for consumption in the C++
+/// API.
+class CallOpSetInterface : public CompletionQueueTag {
+ public:
+ CallOpSetInterface() : max_message_size_(0) {}
+ /// Fills in grpc_op, starting from ops[*nops] and moving
+ /// upwards.
+ virtual void FillOps(grpc_op* ops, size_t* nops) = 0;
+
+ void set_max_message_size(int max_message_size) {
+ max_message_size_ = max_message_size;
+ }
+
+ protected:
+ int max_message_size_;
+};
+
+/// Primary implementaiton of CallOpSetInterface.
+/// Since we cannot use variadic templates, we declare slots up to
+/// the maximum count of ops we'll need in a set. We leverage the
+/// empty base class optimization to slim this class (especially
+/// when there are many unused slots used). To avoid duplicate base classes,
+/// the template parmeter for CallNoOp is varied by argument position.
+template <class Op1 = CallNoOp<1>, class Op2 = CallNoOp<2>,
+ class Op3 = CallNoOp<3>, class Op4 = CallNoOp<4>,
+ class Op5 = CallNoOp<5>, class Op6 = CallNoOp<6>>
+class CallOpSet : public CallOpSetInterface,
+ public Op1,
+ public Op2,
+ public Op3,
+ public Op4,
+ public Op5,
+ public Op6 {
+ public:
+ CallOpSet() : return_tag_(this) {}
+ void FillOps(grpc_op* ops, size_t* nops) GRPC_OVERRIDE {
+ this->Op1::AddOp(ops, nops);
+ this->Op2::AddOp(ops, nops);
+ this->Op3::AddOp(ops, nops);
+ this->Op4::AddOp(ops, nops);
+ this->Op5::AddOp(ops, nops);
+ this->Op6::AddOp(ops, nops);
+ }
+
+ bool FinalizeResult(void** tag, bool* status) GRPC_OVERRIDE {
+ this->Op1::FinishOp(status, max_message_size_);
+ this->Op2::FinishOp(status, max_message_size_);
+ this->Op3::FinishOp(status, max_message_size_);
+ this->Op4::FinishOp(status, max_message_size_);
+ this->Op5::FinishOp(status, max_message_size_);
+ this->Op6::FinishOp(status, max_message_size_);
+ *tag = return_tag_;
+ return true;
+ }
+
+ void set_output_tag(void* return_tag) { return_tag_ = return_tag; }
+
+ private:
+ void* return_tag_;
+};
+
+/// A CallOpSet that does not post completions to the completion queue.
+///
+/// Allows hiding some completions that the C core must generate from
+/// C++ users.
+template <class Op1 = CallNoOp<1>, class Op2 = CallNoOp<2>,
+ class Op3 = CallNoOp<3>, class Op4 = CallNoOp<4>,
+ class Op5 = CallNoOp<5>, class Op6 = CallNoOp<6>>
+class SneakyCallOpSet : public CallOpSet<Op1, Op2, Op3, Op4, Op5, Op6> {
+ public:
+ bool FinalizeResult(void** tag, bool* status) GRPC_OVERRIDE {
+ typedef CallOpSet<Op1, Op2, Op3, Op4, Op5, Op6> Base;
+ return Base::FinalizeResult(tag, status) && false;
+ }
+};
+
+// Straightforward wrapping of the C call object
+class Call GRPC_FINAL {
+ public:
+ /* call is owned by the caller */
+ Call(grpc_call* call, CallHook* call_hook_, CompletionQueue* cq);
+ Call(grpc_call* call, CallHook* call_hook_, CompletionQueue* cq,
+ int max_message_size);
+
+ void PerformOps(CallOpSetInterface* ops);
+
+ grpc_call* call() { return call_; }
+ CompletionQueue* cq() { return cq_; }
+
+ int max_message_size() { return max_message_size_; }
+
+ private:
+ CallHook* call_hook_;
+ CompletionQueue* cq_;
+ grpc_call* call_;
+ int max_message_size_;
+};
+
+} // namespace grpc
+
+#endif // GRPCXX_IMPL_CODEGEN_CALL_H
diff --git a/include/grpc++/impl/codegen/client_context.h b/include/grpc++/impl/codegen/client_context.h
new file mode 100644
index 0000000000..9ac3e5bd79
--- /dev/null
+++ b/include/grpc++/impl/codegen/client_context.h
@@ -0,0 +1,353 @@
+/*
+ *
+ * Copyright 2015-2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ * * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+/// A ClientContext allows the person implementing a service client to:
+///
+/// - Add custom metadata key-value pairs that will propagated to the server
+/// side.
+/// - Control call settings such as compression and authentication.
+/// - Initial and trailing metadata coming from the server.
+/// - Get performance metrics (ie, census).
+///
+/// Context settings are only relevant to the call they are invoked with, that
+/// is to say, they aren't sticky. Some of these settings, such as the
+/// compression options, can be made persistant at channel construction time
+/// (see \a grpc::CreateCustomChannel).
+///
+/// \warning ClientContext instances should \em not be reused across rpcs.
+
+#ifndef GRPCXX_IMPL_CODEGEN_CLIENT_CONTEXT_H
+#define GRPCXX_IMPL_CODEGEN_CLIENT_CONTEXT_H
+
+#include <map>
+#include <memory>
+#include <string>
+
+#include <grpc++/impl/codegen/sync.h>
+#include <grpc++/impl/codegen/security/auth_context.h>
+#include <grpc++/impl/codegen/config.h>
+#include <grpc++/impl/codegen/status.h>
+#include <grpc++/impl/codegen/string_ref.h>
+#include <grpc++/impl/codegen/time.h>
+#include <grpc/impl/codegen/compression_types.h>
+#include <grpc/impl/codegen/propagation_bits.h>
+#include <grpc/impl/codegen/log.h>
+#include <grpc/impl/codegen/time.h>
+
+struct census_context;
+struct grpc_call;
+
+namespace grpc {
+
+class Channel;
+class ChannelInterface;
+class CompletionQueue;
+class CallCredentials;
+class RpcMethod;
+template <class R>
+class ClientReader;
+template <class W>
+class ClientWriter;
+template <class W, class R>
+class ClientReaderWriter;
+template <class R>
+class ClientAsyncReader;
+template <class W>
+class ClientAsyncWriter;
+template <class W, class R>
+class ClientAsyncReaderWriter;
+template <class R>
+class ClientAsyncResponseReader;
+class ServerContext;
+
+/// Options for \a ClientContext::FromServerContext specifying which traits from
+/// the \a ServerContext to propagate (copy) from it into a new \a
+/// ClientContext.
+///
+/// \see ClientContext::FromServerContext
+class PropagationOptions {
+ public:
+ PropagationOptions() : propagate_(GRPC_PROPAGATE_DEFAULTS) {}
+
+ PropagationOptions& enable_deadline_propagation() {
+ propagate_ |= GRPC_PROPAGATE_DEADLINE;
+ return *this;
+ }
+
+ PropagationOptions& disable_deadline_propagation() {
+ propagate_ &= ~GRPC_PROPAGATE_DEADLINE;
+ return *this;
+ }
+
+ PropagationOptions& enable_census_stats_propagation() {
+ propagate_ |= GRPC_PROPAGATE_CENSUS_STATS_CONTEXT;
+ return *this;
+ }
+
+ PropagationOptions& disable_census_stats_propagation() {
+ propagate_ &= ~GRPC_PROPAGATE_CENSUS_STATS_CONTEXT;
+ return *this;
+ }
+
+ PropagationOptions& enable_census_tracing_propagation() {
+ propagate_ |= GRPC_PROPAGATE_CENSUS_TRACING_CONTEXT;
+ return *this;
+ }
+
+ PropagationOptions& disable_census_tracing_propagation() {
+ propagate_ &= ~GRPC_PROPAGATE_CENSUS_TRACING_CONTEXT;
+ return *this;
+ }
+
+ PropagationOptions& enable_cancellation_propagation() {
+ propagate_ |= GRPC_PROPAGATE_CANCELLATION;
+ return *this;
+ }
+
+ PropagationOptions& disable_cancellation_propagation() {
+ propagate_ &= ~GRPC_PROPAGATE_CANCELLATION;
+ return *this;
+ }
+
+ uint32_t c_bitmask() const { return propagate_; }
+
+ private:
+ uint32_t propagate_;
+};
+
+namespace testing {
+class InteropClientContextInspector;
+} // namespace testing
+
+class ClientContext {
+ public:
+ ClientContext();
+ ~ClientContext();
+
+ /// Create a new \a ClientContext as a child of an incoming server call,
+ /// according to \a options (\see PropagationOptions).
+ ///
+ /// \param server_context The source server context to use as the basis for
+ /// constructing the client context.
+ /// \param options The options controlling what to copy from the \a
+ /// server_context.
+ ///
+ /// \return A newly constructed \a ClientContext instance based on \a
+ /// server_context, with traits propagated (copied) according to \a options.
+ static std::unique_ptr<ClientContext> FromServerContext(
+ const ServerContext& server_context,
+ PropagationOptions options = PropagationOptions());
+
+ /// Add the (\a meta_key, \a meta_value) pair to the metadata associated with
+ /// a client call. These are made available at the server side by the \a
+ /// grpc::ServerContext::client_metadata() method.
+ ///
+ /// \warning This method should only be called before invoking the rpc.
+ ///
+ /// \param meta_key The metadata key. If \a meta_value is binary data, it must
+ /// end in "-bin".
+ /// \param meta_value The metadata value. If its value is binary, it must be
+ /// base64-encoding (see https://tools.ietf.org/html/rfc4648#section-4) and \a
+ /// meta_key must end in "-bin".
+ void AddMetadata(const grpc::string& meta_key,
+ const grpc::string& meta_value);
+
+ /// Return a collection of initial metadata key-value pairs. Note that keys
+ /// may happen more than once (ie, a \a std::multimap is returned).
+ ///
+ /// \warning This method should only be called after initial metadata has been
+ /// received. For streaming calls, see \a
+ /// ClientReaderInterface::WaitForInitialMetadata().
+ ///
+ /// \return A multimap of initial metadata key-value pairs from the server.
+ const std::multimap<grpc::string_ref, grpc::string_ref>&
+ GetServerInitialMetadata() {
+ GPR_ASSERT(initial_metadata_received_);
+ return recv_initial_metadata_;
+ }
+
+ /// Return a collection of trailing metadata key-value pairs. Note that keys
+ /// may happen more than once (ie, a \a std::multimap is returned).
+ ///
+ /// \warning This method is only callable once the stream has finished.
+ ///
+ /// \return A multimap of metadata trailing key-value pairs from the server.
+ const std::multimap<grpc::string_ref, grpc::string_ref>&
+ GetServerTrailingMetadata() {
+ // TODO(yangg) check finished
+ return trailing_metadata_;
+ }
+
+ /// Set the deadline for the client call.
+ ///
+ /// \warning This method should only be called before invoking the rpc.
+ ///
+ /// \param deadline the deadline for the client call. Units are determined by
+ /// the type used.
+ template <typename T>
+ void set_deadline(const T& deadline) {
+ TimePoint<T> deadline_tp(deadline);
+ deadline_ = deadline_tp.raw_time();
+ }
+
+#ifndef GRPC_CXX0X_NO_CHRONO
+ /// Return the deadline for the client call.
+ std::chrono::system_clock::time_point deadline() {
+ return Timespec2Timepoint(deadline_);
+ }
+#endif // !GRPC_CXX0X_NO_CHRONO
+
+ /// Return a \a gpr_timespec representation of the client call's deadline.
+ gpr_timespec raw_deadline() { return deadline_; }
+
+ /// Set the per call authority header (see
+ /// https://tools.ietf.org/html/rfc7540#section-8.1.2.3).
+ void set_authority(const grpc::string& authority) { authority_ = authority; }
+
+ /// Return the authentication context for this client call.
+ ///
+ /// \see grpc::AuthContext.
+ std::shared_ptr<const AuthContext> auth_context() const;
+
+ /// Set credentials for the client call.
+ ///
+ /// A credentials object encapsulates all the state needed by a client to
+ /// authenticate with a server and make various assertions, e.g., about the
+ /// client’s identity, role, or whether it is authorized to make a particular
+ /// call.
+ ///
+ /// \see http://www.grpc.io/docs/guides/auth.html
+ void set_credentials(const std::shared_ptr<CallCredentials>& creds) {
+ creds_ = creds;
+ }
+
+ /// Return the compression algorithm to be used by the client call.
+ grpc_compression_algorithm compression_algorithm() const {
+ return compression_algorithm_;
+ }
+
+ /// Set \a algorithm to be the compression algorithm used for the client call.
+ ///
+ /// \param algorith The compression algorithm used for the client call.
+ void set_compression_algorithm(grpc_compression_algorithm algorithm);
+
+ /// Return the peer uri in a string.
+ ///
+ /// \warning This value is never authenticated or subject to any security
+ /// related code. It must not be used for any authentication related
+ /// functionality. Instead, use auth_context.
+ ///
+ /// \return The call's peer URI.
+ grpc::string peer() const;
+
+ /// Get and set census context.
+ void set_census_context(struct census_context* ccp) { census_context_ = ccp; }
+ struct census_context* census_context() const {
+ return census_context_;
+ }
+
+ /// Send a best-effort out-of-band cancel. The call could be in any stage.
+ /// e.g. if it is already finished, it may still return success.
+ ///
+ /// There is no guarantee the call will be cancelled.
+ void TryCancel();
+
+ /// Global Callbacks
+ ///
+ /// Can be set exactly once per application to install hooks whenever
+ /// a client context is constructed and destructed.
+ class GlobalCallbacks {
+ public:
+ virtual void DefaultConstructor(ClientContext* context) = 0;
+ virtual void Destructor(ClientContext* context) = 0;
+ };
+ static void SetGlobalCallbacks(GlobalCallbacks* callbacks);
+
+ private:
+ // Disallow copy and assign.
+ ClientContext(const ClientContext&);
+ ClientContext& operator=(const ClientContext&);
+
+ friend class ::grpc::testing::InteropClientContextInspector;
+ friend class CallOpClientRecvStatus;
+ friend class CallOpRecvInitialMetadata;
+ friend class Channel;
+ template <class R>
+ friend class ::grpc::ClientReader;
+ template <class W>
+ friend class ::grpc::ClientWriter;
+ template <class W, class R>
+ friend class ::grpc::ClientReaderWriter;
+ template <class R>
+ friend class ::grpc::ClientAsyncReader;
+ template <class W>
+ friend class ::grpc::ClientAsyncWriter;
+ template <class W, class R>
+ friend class ::grpc::ClientAsyncReaderWriter;
+ template <class R>
+ friend class ::grpc::ClientAsyncResponseReader;
+ template <class InputMessage, class OutputMessage>
+ friend Status BlockingUnaryCall(ChannelInterface* channel,
+ const RpcMethod& method,
+ ClientContext* context,
+ const InputMessage& request,
+ OutputMessage* result);
+
+ grpc_call* call() { return call_; }
+ void set_call(grpc_call* call, const std::shared_ptr<Channel>& channel);
+
+ grpc::string authority() { return authority_; }
+
+ bool initial_metadata_received_;
+ std::shared_ptr<Channel> channel_;
+ grpc::mutex mu_;
+ grpc_call* call_;
+ bool call_canceled_;
+ gpr_timespec deadline_;
+ grpc::string authority_;
+ std::shared_ptr<CallCredentials> creds_;
+ mutable std::shared_ptr<const AuthContext> auth_context_;
+ struct census_context* census_context_;
+ std::multimap<grpc::string, grpc::string> send_initial_metadata_;
+ std::multimap<grpc::string_ref, grpc::string_ref> recv_initial_metadata_;
+ std::multimap<grpc::string_ref, grpc::string_ref> trailing_metadata_;
+
+ grpc_call* propagate_from_call_;
+ PropagationOptions propagation_options_;
+
+ grpc_compression_algorithm compression_algorithm_;
+};
+
+} // namespace grpc
+
+#endif // GRPCXX_IMPL_CODEGEN_CLIENT_CONTEXT_H
diff --git a/include/grpc++/impl/codegen/serialization_traits.h b/include/grpc++/impl/codegen/serialization_traits.h
new file mode 100644
index 0000000000..fa99dbfa9c
--- /dev/null
+++ b/include/grpc++/impl/codegen/serialization_traits.h
@@ -0,0 +1,68 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ * * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPCXX_IMPL_CODEGEN_SERIALIZATION_TRAITS_H
+#define GRPCXX_IMPL_CODEGEN_SERIALIZATION_TRAITS_H
+
+namespace grpc {
+
+/// Defines how to serialize and deserialize some type.
+///
+/// Used for hooking different message serialization API's into GRPC.
+/// Each SerializationTraits implementation must provide the following
+/// functions:
+/// static Status Serialize(const Message& msg,
+/// grpc_byte_buffer** buffer,
+// bool* own_buffer);
+/// static Status Deserialize(grpc_byte_buffer* buffer,
+/// Message* msg,
+/// int max_message_size);
+///
+/// Serialize is required to convert message to a grpc_byte_buffer, and
+/// to store a pointer to that byte buffer at *buffer. *own_buffer should
+/// be set to true if the caller owns said byte buffer, or false if
+/// ownership is retained elsewhere.
+///
+/// Deserialize is required to convert buffer into the message stored at
+/// msg. max_message_size is passed in as a bound on the maximum number of
+/// message bytes Deserialize should accept.
+///
+/// Both functions return a Status, allowing them to explain what went
+/// wrong if required.
+template <class Message,
+ class UnusedButHereForPartialTemplateSpecialization = void>
+class SerializationTraits;
+
+} // namespace grpc
+
+#endif // GRPCXX_IMPL_CODEGEN_SERIALIZATION_TRAITS_H
diff --git a/include/grpc++/impl/codegen/server_context.h b/include/grpc++/impl/codegen/server_context.h
new file mode 100644
index 0000000000..2af9fdaa34
--- /dev/null
+++ b/include/grpc++/impl/codegen/server_context.h
@@ -0,0 +1,201 @@
+/*
+ *
+ * Copyright 2015-2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ * * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPCXX_IMPL_CODEGEN_SERVER_CONTEXT_H
+#define GRPCXX_IMPL_CODEGEN_SERVER_CONTEXT_H
+
+#include <map>
+#include <memory>
+
+#include <grpc/impl/codegen/compression_types.h>
+#include <grpc/impl/codegen/time.h>
+#include <grpc++/impl/codegen/security/auth_context.h>
+#include <grpc++/impl/codegen/config.h>
+#include <grpc++/impl/codegen/string_ref.h>
+#include <grpc++/impl/codegen/time.h>
+
+struct gpr_timespec;
+struct grpc_metadata;
+struct grpc_call;
+struct census_context;
+
+namespace grpc {
+
+class ClientContext;
+template <class W, class R>
+class ServerAsyncReader;
+template <class W>
+class ServerAsyncWriter;
+template <class W>
+class ServerAsyncResponseWriter;
+template <class W, class R>
+class ServerAsyncReaderWriter;
+template <class R>
+class ServerReader;
+template <class W>
+class ServerWriter;
+template <class W, class R>
+class ServerReaderWriter;
+template <class ServiceType, class RequestType, class ResponseType>
+class RpcMethodHandler;
+template <class ServiceType, class RequestType, class ResponseType>
+class ClientStreamingHandler;
+template <class ServiceType, class RequestType, class ResponseType>
+class ServerStreamingHandler;
+template <class ServiceType, class RequestType, class ResponseType>
+class BidiStreamingHandler;
+class UnknownMethodHandler;
+
+class Call;
+class CallOpBuffer;
+class CompletionQueue;
+class Server;
+class ServerInterface;
+
+namespace testing {
+class InteropServerContextInspector;
+} // namespace testing
+
+// Interface of server side rpc context.
+class ServerContext {
+ public:
+ ServerContext(); // for async calls
+ ~ServerContext();
+
+#ifndef GRPC_CXX0X_NO_CHRONO
+ std::chrono::system_clock::time_point deadline() {
+ return Timespec2Timepoint(deadline_);
+ }
+#endif // !GRPC_CXX0X_NO_CHRONO
+
+ gpr_timespec raw_deadline() { return deadline_; }
+
+ void AddInitialMetadata(const grpc::string& key, const grpc::string& value);
+ void AddTrailingMetadata(const grpc::string& key, const grpc::string& value);
+
+ bool IsCancelled() const;
+
+ const std::multimap<grpc::string_ref, grpc::string_ref>& client_metadata() {
+ return client_metadata_;
+ }
+
+ grpc_compression_level compression_level() const {
+ return compression_level_;
+ }
+ void set_compression_level(grpc_compression_level level);
+
+ grpc_compression_algorithm compression_algorithm() const {
+ return compression_algorithm_;
+ }
+ void set_compression_algorithm(grpc_compression_algorithm algorithm);
+
+ std::shared_ptr<const AuthContext> auth_context() const;
+
+ // Return the peer uri in a string.
+ // WARNING: this value is never authenticated or subject to any security
+ // related code. It must not be used for any authentication related
+ // functionality. Instead, use auth_context.
+ grpc::string peer() const;
+
+ const struct census_context* census_context() const;
+
+ // Async only. Has to be called before the rpc starts.
+ // Returns the tag in completion queue when the rpc finishes.
+ // IsCancelled() can then be called to check whether the rpc was cancelled.
+ void AsyncNotifyWhenDone(void* tag) {
+ has_notify_when_done_tag_ = true;
+ async_notify_when_done_tag_ = tag;
+ }
+
+ private:
+ friend class ::grpc::testing::InteropServerContextInspector;
+ friend class ::grpc::ServerInterface;
+ friend class ::grpc::Server;
+ template <class W, class R>
+ friend class ::grpc::ServerAsyncReader;
+ template <class W>
+ friend class ::grpc::ServerAsyncWriter;
+ template <class W>
+ friend class ::grpc::ServerAsyncResponseWriter;
+ template <class W, class R>
+ friend class ::grpc::ServerAsyncReaderWriter;
+ template <class R>
+ friend class ::grpc::ServerReader;
+ template <class W>
+ friend class ::grpc::ServerWriter;
+ template <class W, class R>
+ friend class ::grpc::ServerReaderWriter;
+ template <class ServiceType, class RequestType, class ResponseType>
+ friend class RpcMethodHandler;
+ template <class ServiceType, class RequestType, class ResponseType>
+ friend class ClientStreamingHandler;
+ template <class ServiceType, class RequestType, class ResponseType>
+ friend class ServerStreamingHandler;
+ template <class ServiceType, class RequestType, class ResponseType>
+ friend class BidiStreamingHandler;
+ friend class UnknownMethodHandler;
+ friend class ::grpc::ClientContext;
+
+ // Prevent copying.
+ ServerContext(const ServerContext&);
+ ServerContext& operator=(const ServerContext&);
+
+ class CompletionOp;
+
+ void BeginCompletionOp(Call* call);
+
+ ServerContext(gpr_timespec deadline, grpc_metadata* metadata,
+ size_t metadata_count);
+
+ void set_call(grpc_call* call);
+
+ CompletionOp* completion_op_;
+ bool has_notify_when_done_tag_;
+ void* async_notify_when_done_tag_;
+
+ gpr_timespec deadline_;
+ grpc_call* call_;
+ CompletionQueue* cq_;
+ bool sent_initial_metadata_;
+ mutable std::shared_ptr<const AuthContext> auth_context_;
+ std::multimap<grpc::string_ref, grpc::string_ref> client_metadata_;
+ std::multimap<grpc::string, grpc::string> initial_metadata_;
+ std::multimap<grpc::string, grpc::string> trailing_metadata_;
+
+ grpc_compression_level compression_level_;
+ grpc_compression_algorithm compression_algorithm_;
+};
+
+} // namespace grpc
+
+#endif // GRPCXX_IMPL_CODEGEN_SERVER_CONTEXT_H
diff --git a/include/grpc++/impl/codegen/service_type.h b/include/grpc++/impl/codegen/service_type.h
new file mode 100644
index 0000000000..9e0a6fa2f7
--- /dev/null
+++ b/include/grpc++/impl/codegen/service_type.h
@@ -0,0 +1,161 @@
+/*
+ *
+ * Copyright 2015-2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ * * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPCXX_IMPL_CODEGEN_SERVICE_TYPE_H
+#define GRPCXX_IMPL_CODEGEN_SERVICE_TYPE_H
+
+#include <grpc++/impl/rpc_service_method.h>
+#include <grpc++/impl/serialization_traits.h>
+#include <grpc++/impl/codegen/server_interface.h>
+#include <grpc++/impl/codegen/config.h>
+#include <grpc++/impl/codegen/status.h>
+
+namespace grpc {
+
+class Call;
+class CompletionQueue;
+class Server;
+class ServerInterface;
+class ServerCompletionQueue;
+class ServerContext;
+
+class ServerAsyncStreamingInterface {
+ public:
+ virtual ~ServerAsyncStreamingInterface() {}
+
+ virtual void SendInitialMetadata(void* tag) = 0;
+
+ private:
+ friend class ServerInterface;
+ virtual void BindCall(Call* call) = 0;
+};
+
+class Service {
+ public:
+ Service() : server_(nullptr) {}
+ virtual ~Service() {}
+
+ bool has_async_methods() const {
+ for (auto it = methods_.begin(); it != methods_.end(); ++it) {
+ if (*it && (*it)->handler() == nullptr) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ bool has_synchronous_methods() const {
+ for (auto it = methods_.begin(); it != methods_.end(); ++it) {
+ if (*it && (*it)->handler() != nullptr) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ bool has_generic_methods() const {
+ for (auto it = methods_.begin(); it != methods_.end(); ++it) {
+ if (it->get() == nullptr) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ protected:
+ template <class Message>
+ void RequestAsyncUnary(int index, ServerContext* context, Message* request,
+ ServerAsyncStreamingInterface* stream,
+ CompletionQueue* call_cq,
+ ServerCompletionQueue* notification_cq, void* tag) {
+ server_->RequestAsyncCall(methods_[index].get(), context, stream, call_cq,
+ notification_cq, tag, request);
+ }
+ void RequestAsyncClientStreaming(int index, ServerContext* context,
+ ServerAsyncStreamingInterface* stream,
+ CompletionQueue* call_cq,
+ ServerCompletionQueue* notification_cq,
+ void* tag) {
+ server_->RequestAsyncCall(methods_[index].get(), context, stream, call_cq,
+ notification_cq, tag);
+ }
+ template <class Message>
+ void RequestAsyncServerStreaming(int index, ServerContext* context,
+ Message* request,
+ ServerAsyncStreamingInterface* stream,
+ CompletionQueue* call_cq,
+ ServerCompletionQueue* notification_cq,
+ void* tag) {
+ server_->RequestAsyncCall(methods_[index].get(), context, stream, call_cq,
+ notification_cq, tag, request);
+ }
+ void RequestAsyncBidiStreaming(int index, ServerContext* context,
+ ServerAsyncStreamingInterface* stream,
+ CompletionQueue* call_cq,
+ ServerCompletionQueue* notification_cq,
+ void* tag) {
+ server_->RequestAsyncCall(methods_[index].get(), context, stream, call_cq,
+ notification_cq, tag);
+ }
+
+ void AddMethod(RpcServiceMethod* method) { methods_.emplace_back(method); }
+
+ void MarkMethodAsync(int index) {
+ if (methods_[index].get() == nullptr) {
+ gpr_log(GPR_ERROR,
+ "Cannot mark the method as 'async' because it has already been "
+ "marked as 'generic'.");
+ return;
+ }
+ methods_[index]->ResetHandler();
+ }
+
+ void MarkMethodGeneric(int index) {
+ if (methods_[index]->handler() == nullptr) {
+ gpr_log(GPR_ERROR,
+ "Cannot mark the method as 'generic' because it has already been "
+ "marked as 'async'.");
+ }
+ methods_[index].reset();
+ }
+
+ private:
+ friend class Server;
+ friend class ServerInterface;
+ ServerInterface* server_;
+ std::vector<std::unique_ptr<RpcServiceMethod>> methods_;
+};
+
+} // namespace grpc
+
+#endif // GRPCXX_IMPL_CODEGEN_SERVICE_TYPE_H
diff --git a/include/grpc++/impl/serialization_traits.h b/include/grpc++/impl/serialization_traits.h
index 3ea66a3405..53538299dd 100644
--- a/include/grpc++/impl/serialization_traits.h
+++ b/include/grpc++/impl/serialization_traits.h
@@ -34,35 +34,6 @@
#ifndef GRPCXX_IMPL_SERIALIZATION_TRAITS_H
#define GRPCXX_IMPL_SERIALIZATION_TRAITS_H
-namespace grpc {
-
-/// Defines how to serialize and deserialize some type.
-///
-/// Used for hooking different message serialization API's into GRPC.
-/// Each SerializationTraits implementation must provide the following
-/// functions:
-/// static Status Serialize(const Message& msg,
-/// grpc_byte_buffer** buffer,
-// bool* own_buffer);
-/// static Status Deserialize(grpc_byte_buffer* buffer,
-/// Message* msg,
-/// int max_message_size);
-///
-/// Serialize is required to convert message to a grpc_byte_buffer, and
-/// to store a pointer to that byte buffer at *buffer. *own_buffer should
-/// be set to true if the caller owns said byte buffer, or false if
-/// ownership is retained elsewhere.
-///
-/// Deserialize is required to convert buffer into the message stored at
-/// msg. max_message_size is passed in as a bound on the maximum number of
-/// message bytes Deserialize should accept.
-///
-/// Both functions return a Status, allowing them to explain what went
-/// wrong if required.
-template <class Message,
- class UnusedButHereForPartialTemplateSpecialization = void>
-class SerializationTraits;
-
-} // namespace grpc
+#include <grpc++/impl/codegen/serialization_traits.h>
#endif // GRPCXX_IMPL_SERIALIZATION_TRAITS_H
diff --git a/include/grpc++/impl/service_type.h b/include/grpc++/impl/service_type.h
index b0c106f9cc..b33fbec739 100644
--- a/include/grpc++/impl/service_type.h
+++ b/include/grpc++/impl/service_type.h
@@ -34,128 +34,6 @@
#ifndef GRPCXX_IMPL_SERVICE_TYPE_H
#define GRPCXX_IMPL_SERVICE_TYPE_H
-#include <grpc++/impl/rpc_service_method.h>
-#include <grpc++/impl/serialization_traits.h>
-#include <grpc++/impl/codegen/server_interface.h>
-#include <grpc++/support/config.h>
-#include <grpc++/support/status.h>
-
-namespace grpc {
-
-class Call;
-class CompletionQueue;
-class Server;
-class ServerInterface;
-class ServerCompletionQueue;
-class ServerContext;
-
-class ServerAsyncStreamingInterface {
- public:
- virtual ~ServerAsyncStreamingInterface() {}
-
- virtual void SendInitialMetadata(void* tag) = 0;
-
- private:
- friend class ServerInterface;
- virtual void BindCall(Call* call) = 0;
-};
-
-class Service {
- public:
- Service() : server_(nullptr) {}
- virtual ~Service() {}
-
- bool has_async_methods() const {
- for (auto it = methods_.begin(); it != methods_.end(); ++it) {
- if (*it && (*it)->handler() == nullptr) {
- return true;
- }
- }
- return false;
- }
-
- bool has_synchronous_methods() const {
- for (auto it = methods_.begin(); it != methods_.end(); ++it) {
- if (*it && (*it)->handler() != nullptr) {
- return true;
- }
- }
- return false;
- }
-
- bool has_generic_methods() const {
- for (auto it = methods_.begin(); it != methods_.end(); ++it) {
- if (it->get() == nullptr) {
- return true;
- }
- }
- return false;
- }
-
- protected:
- template <class Message>
- void RequestAsyncUnary(int index, ServerContext* context, Message* request,
- ServerAsyncStreamingInterface* stream,
- CompletionQueue* call_cq,
- ServerCompletionQueue* notification_cq, void* tag) {
- server_->RequestAsyncCall(methods_[index].get(), context, stream, call_cq,
- notification_cq, tag, request);
- }
- void RequestAsyncClientStreaming(int index, ServerContext* context,
- ServerAsyncStreamingInterface* stream,
- CompletionQueue* call_cq,
- ServerCompletionQueue* notification_cq,
- void* tag) {
- server_->RequestAsyncCall(methods_[index].get(), context, stream, call_cq,
- notification_cq, tag);
- }
- template <class Message>
- void RequestAsyncServerStreaming(int index, ServerContext* context,
- Message* request,
- ServerAsyncStreamingInterface* stream,
- CompletionQueue* call_cq,
- ServerCompletionQueue* notification_cq,
- void* tag) {
- server_->RequestAsyncCall(methods_[index].get(), context, stream, call_cq,
- notification_cq, tag, request);
- }
- void RequestAsyncBidiStreaming(int index, ServerContext* context,
- ServerAsyncStreamingInterface* stream,
- CompletionQueue* call_cq,
- ServerCompletionQueue* notification_cq,
- void* tag) {
- server_->RequestAsyncCall(methods_[index].get(), context, stream, call_cq,
- notification_cq, tag);
- }
-
- void AddMethod(RpcServiceMethod* method) { methods_.emplace_back(method); }
-
- void MarkMethodAsync(int index) {
- if (methods_[index].get() == nullptr) {
- gpr_log(GPR_ERROR,
- "Cannot mark the method as 'async' because it has already been "
- "marked as 'generic'.");
- return;
- }
- methods_[index]->ResetHandler();
- }
-
- void MarkMethodGeneric(int index) {
- if (methods_[index]->handler() == nullptr) {
- gpr_log(GPR_ERROR,
- "Cannot mark the method as 'generic' because it has already been "
- "marked as 'async'.");
- }
- methods_[index].reset();
- }
-
- private:
- friend class Server;
- friend class ServerInterface;
- ServerInterface* server_;
- std::vector<std::unique_ptr<RpcServiceMethod>> methods_;
-};
-
-} // namespace grpc
+#include <grpc++/impl/codegen/service_type.h>
#endif // GRPCXX_IMPL_SERVICE_TYPE_H
diff --git a/include/grpc++/server_context.h b/include/grpc++/server_context.h
index 1c3f39e238..214188a03a 100644
--- a/include/grpc++/server_context.h
+++ b/include/grpc++/server_context.h
@@ -34,168 +34,6 @@
#ifndef GRPCXX_SERVER_CONTEXT_H
#define GRPCXX_SERVER_CONTEXT_H
-#include <map>
-#include <memory>
-
-#include <grpc/compression.h>
-#include <grpc/support/time.h>
-#include <grpc++/security/auth_context.h>
-#include <grpc++/support/config.h>
-#include <grpc++/support/string_ref.h>
-#include <grpc++/support/time.h>
-
-struct gpr_timespec;
-struct grpc_metadata;
-struct grpc_call;
-struct census_context;
-
-namespace grpc {
-
-class ClientContext;
-template <class W, class R>
-class ServerAsyncReader;
-template <class W>
-class ServerAsyncWriter;
-template <class W>
-class ServerAsyncResponseWriter;
-template <class W, class R>
-class ServerAsyncReaderWriter;
-template <class R>
-class ServerReader;
-template <class W>
-class ServerWriter;
-template <class W, class R>
-class ServerReaderWriter;
-template <class ServiceType, class RequestType, class ResponseType>
-class RpcMethodHandler;
-template <class ServiceType, class RequestType, class ResponseType>
-class ClientStreamingHandler;
-template <class ServiceType, class RequestType, class ResponseType>
-class ServerStreamingHandler;
-template <class ServiceType, class RequestType, class ResponseType>
-class BidiStreamingHandler;
-class UnknownMethodHandler;
-
-class Call;
-class CallOpBuffer;
-class CompletionQueue;
-class Server;
-class ServerInterface;
-
-namespace testing {
-class InteropServerContextInspector;
-} // namespace testing
-
-// Interface of server side rpc context.
-class ServerContext {
- public:
- ServerContext(); // for async calls
- ~ServerContext();
-
-#ifndef GRPC_CXX0X_NO_CHRONO
- std::chrono::system_clock::time_point deadline() {
- return Timespec2Timepoint(deadline_);
- }
-#endif // !GRPC_CXX0X_NO_CHRONO
-
- gpr_timespec raw_deadline() { return deadline_; }
-
- void AddInitialMetadata(const grpc::string& key, const grpc::string& value);
- void AddTrailingMetadata(const grpc::string& key, const grpc::string& value);
-
- bool IsCancelled() const;
-
- const std::multimap<grpc::string_ref, grpc::string_ref>& client_metadata() {
- return client_metadata_;
- }
-
- grpc_compression_level compression_level() const {
- return compression_level_;
- }
- void set_compression_level(grpc_compression_level level);
-
- grpc_compression_algorithm compression_algorithm() const {
- return compression_algorithm_;
- }
- void set_compression_algorithm(grpc_compression_algorithm algorithm);
-
- std::shared_ptr<const AuthContext> auth_context() const;
-
- // Return the peer uri in a string.
- // WARNING: this value is never authenticated or subject to any security
- // related code. It must not be used for any authentication related
- // functionality. Instead, use auth_context.
- grpc::string peer() const;
-
- const struct census_context* census_context() const;
-
- // Async only. Has to be called before the rpc starts.
- // Returns the tag in completion queue when the rpc finishes.
- // IsCancelled() can then be called to check whether the rpc was cancelled.
- void AsyncNotifyWhenDone(void* tag) {
- has_notify_when_done_tag_ = true;
- async_notify_when_done_tag_ = tag;
- }
-
- private:
- friend class ::grpc::testing::InteropServerContextInspector;
- friend class ::grpc::ServerInterface;
- friend class ::grpc::Server;
- template <class W, class R>
- friend class ::grpc::ServerAsyncReader;
- template <class W>
- friend class ::grpc::ServerAsyncWriter;
- template <class W>
- friend class ::grpc::ServerAsyncResponseWriter;
- template <class W, class R>
- friend class ::grpc::ServerAsyncReaderWriter;
- template <class R>
- friend class ::grpc::ServerReader;
- template <class W>
- friend class ::grpc::ServerWriter;
- template <class W, class R>
- friend class ::grpc::ServerReaderWriter;
- template <class ServiceType, class RequestType, class ResponseType>
- friend class RpcMethodHandler;
- template <class ServiceType, class RequestType, class ResponseType>
- friend class ClientStreamingHandler;
- template <class ServiceType, class RequestType, class ResponseType>
- friend class ServerStreamingHandler;
- template <class ServiceType, class RequestType, class ResponseType>
- friend class BidiStreamingHandler;
- friend class UnknownMethodHandler;
- friend class ::grpc::ClientContext;
-
- // Prevent copying.
- ServerContext(const ServerContext&);
- ServerContext& operator=(const ServerContext&);
-
- class CompletionOp;
-
- void BeginCompletionOp(Call* call);
-
- ServerContext(gpr_timespec deadline, grpc_metadata* metadata,
- size_t metadata_count);
-
- void set_call(grpc_call* call);
-
- CompletionOp* completion_op_;
- bool has_notify_when_done_tag_;
- void* async_notify_when_done_tag_;
-
- gpr_timespec deadline_;
- grpc_call* call_;
- CompletionQueue* cq_;
- bool sent_initial_metadata_;
- mutable std::shared_ptr<const AuthContext> auth_context_;
- std::multimap<grpc::string_ref, grpc::string_ref> client_metadata_;
- std::multimap<grpc::string, grpc::string> initial_metadata_;
- std::multimap<grpc::string, grpc::string> trailing_metadata_;
-
- grpc_compression_level compression_level_;
- grpc_compression_algorithm compression_algorithm_;
-};
-
-} // namespace grpc
+#include <grpc++/impl/codegen/server_context.h>
#endif // GRPCXX_SERVER_CONTEXT_H
diff --git a/include/grpc++/support/async_stream.h b/include/grpc++/support/async_stream.h
index 511ad17852..5a811f9217 100644
--- a/include/grpc++/support/async_stream.h
+++ b/include/grpc++/support/async_stream.h
@@ -35,9 +35,9 @@
#define GRPCXX_SUPPORT_ASYNC_STREAM_H
#include <grpc++/impl/codegen/channel_interface.h>
-#include <grpc++/impl/call.h>
-#include <grpc++/impl/service_type.h>
-#include <grpc++/server_context.h>
+#include <grpc++/impl/codegen/call.h>
+#include <grpc++/impl/codegen/service_type.h>
+#include <grpc++/impl/codegen/server_context.h>
#include <grpc++/impl/codegen/status.h>
namespace grpc {