aboutsummaryrefslogtreecommitdiffhomepage
path: root/include/grpc++/impl/codegen
diff options
context:
space:
mode:
Diffstat (limited to 'include/grpc++/impl/codegen')
-rw-r--r--include/grpc++/impl/codegen/async_stream.h25
-rw-r--r--include/grpc++/impl/codegen/async_unary_call.h9
-rw-r--r--include/grpc++/impl/codegen/call.h74
-rw-r--r--include/grpc++/impl/codegen/client_unary_call.h3
-rw-r--r--include/grpc++/impl/codegen/core_codegen_interface.h82
-rw-r--r--include/grpc++/impl/codegen/impl/async_stream.h462
-rw-r--r--include/grpc++/impl/codegen/impl/status_code_enum.h152
-rw-r--r--include/grpc++/impl/codegen/impl/sync.h (renamed from include/grpc++/impl/codegen/proto_serializer.cc)18
-rw-r--r--include/grpc++/impl/codegen/method_handler_impl.h5
-rw-r--r--include/grpc++/impl/codegen/proto_utils.h52
-rw-r--r--include/grpc++/impl/codegen/string_ref.h74
-rw-r--r--include/grpc++/impl/codegen/sync_stream.h21
12 files changed, 858 insertions, 119 deletions
diff --git a/include/grpc++/impl/codegen/async_stream.h b/include/grpc++/impl/codegen/async_stream.h
index b0410485f8..8b6047a4a3 100644
--- a/include/grpc++/impl/codegen/async_stream.h
+++ b/include/grpc++/impl/codegen/async_stream.h
@@ -35,6 +35,7 @@
#define GRPCXX_IMPL_CODEGEN_ASYNC_STREAM_H
#include <grpc++/impl/codegen/channel_interface.h>
+#include <grpc++/impl/codegen/core_codegen_interface.h>
#include <grpc++/impl/codegen/call.h>
#include <grpc++/impl/codegen/service_type.h>
#include <grpc++/impl/codegen/server_context.h>
@@ -109,13 +110,13 @@ class ClientAsyncReader GRPC_FINAL : public ClientAsyncReaderInterface<R> {
init_ops_.set_output_tag(tag);
init_ops_.SendInitialMetadata(context->send_initial_metadata_);
// TODO(ctiller): don't assert
- GPR_ASSERT(init_ops_.SendMessage(request).ok());
+ GPR_CODEGEN_ASSERT(init_ops_.SendMessage(request).ok());
init_ops_.ClientSendClose();
call_.PerformOps(&init_ops_);
}
void ReadInitialMetadata(void* tag) GRPC_OVERRIDE {
- GPR_ASSERT(!context_->initial_metadata_received_);
+ GPR_CODEGEN_ASSERT(!context_->initial_metadata_received_);
meta_ops_.set_output_tag(tag);
meta_ops_.RecvInitialMetadata(context_);
@@ -177,7 +178,7 @@ class ClientAsyncWriter GRPC_FINAL : public ClientAsyncWriterInterface<W> {
}
void ReadInitialMetadata(void* tag) GRPC_OVERRIDE {
- GPR_ASSERT(!context_->initial_metadata_received_);
+ GPR_CODEGEN_ASSERT(!context_->initial_metadata_received_);
meta_ops_.set_output_tag(tag);
meta_ops_.RecvInitialMetadata(context_);
@@ -187,7 +188,7 @@ class ClientAsyncWriter GRPC_FINAL : public ClientAsyncWriterInterface<W> {
void Write(const W& msg, void* tag) GRPC_OVERRIDE {
write_ops_.set_output_tag(tag);
// TODO(ctiller): don't assert
- GPR_ASSERT(write_ops_.SendMessage(msg).ok());
+ GPR_CODEGEN_ASSERT(write_ops_.SendMessage(msg).ok());
call_.PerformOps(&write_ops_);
}
@@ -243,7 +244,7 @@ class ClientAsyncReaderWriter GRPC_FINAL
}
void ReadInitialMetadata(void* tag) GRPC_OVERRIDE {
- GPR_ASSERT(!context_->initial_metadata_received_);
+ GPR_CODEGEN_ASSERT(!context_->initial_metadata_received_);
meta_ops_.set_output_tag(tag);
meta_ops_.RecvInitialMetadata(context_);
@@ -262,7 +263,7 @@ class ClientAsyncReaderWriter GRPC_FINAL
void Write(const W& msg, void* tag) GRPC_OVERRIDE {
write_ops_.set_output_tag(tag);
// TODO(ctiller): don't assert
- GPR_ASSERT(write_ops_.SendMessage(msg).ok());
+ GPR_CODEGEN_ASSERT(write_ops_.SendMessage(msg).ok());
call_.PerformOps(&write_ops_);
}
@@ -300,7 +301,7 @@ class ServerAsyncReader GRPC_FINAL : public ServerAsyncStreamingInterface,
: call_(nullptr, nullptr, nullptr), ctx_(ctx) {}
void SendInitialMetadata(void* tag) GRPC_OVERRIDE {
- GPR_ASSERT(!ctx_->sent_initial_metadata_);
+ GPR_CODEGEN_ASSERT(!ctx_->sent_initial_metadata_);
meta_ops_.set_output_tag(tag);
meta_ops_.SendInitialMetadata(ctx_->initial_metadata_);
@@ -331,7 +332,7 @@ class ServerAsyncReader GRPC_FINAL : public ServerAsyncStreamingInterface,
}
void FinishWithError(const Status& status, void* tag) {
- GPR_ASSERT(!status.ok());
+ GPR_CODEGEN_ASSERT(!status.ok());
finish_ops_.set_output_tag(tag);
if (!ctx_->sent_initial_metadata_) {
finish_ops_.SendInitialMetadata(ctx_->initial_metadata_);
@@ -360,7 +361,7 @@ class ServerAsyncWriter GRPC_FINAL : public ServerAsyncStreamingInterface,
: call_(nullptr, nullptr, nullptr), ctx_(ctx) {}
void SendInitialMetadata(void* tag) GRPC_OVERRIDE {
- GPR_ASSERT(!ctx_->sent_initial_metadata_);
+ GPR_CODEGEN_ASSERT(!ctx_->sent_initial_metadata_);
meta_ops_.set_output_tag(tag);
meta_ops_.SendInitialMetadata(ctx_->initial_metadata_);
@@ -375,7 +376,7 @@ class ServerAsyncWriter GRPC_FINAL : public ServerAsyncStreamingInterface,
ctx_->sent_initial_metadata_ = true;
}
// TODO(ctiller): don't assert
- GPR_ASSERT(write_ops_.SendMessage(msg).ok());
+ GPR_CODEGEN_ASSERT(write_ops_.SendMessage(msg).ok());
call_.PerformOps(&write_ops_);
}
@@ -409,7 +410,7 @@ class ServerAsyncReaderWriter GRPC_FINAL : public ServerAsyncStreamingInterface,
: call_(nullptr, nullptr, nullptr), ctx_(ctx) {}
void SendInitialMetadata(void* tag) GRPC_OVERRIDE {
- GPR_ASSERT(!ctx_->sent_initial_metadata_);
+ GPR_CODEGEN_ASSERT(!ctx_->sent_initial_metadata_);
meta_ops_.set_output_tag(tag);
meta_ops_.SendInitialMetadata(ctx_->initial_metadata_);
@@ -430,7 +431,7 @@ class ServerAsyncReaderWriter GRPC_FINAL : public ServerAsyncStreamingInterface,
ctx_->sent_initial_metadata_ = true;
}
// TODO(ctiller): don't assert
- GPR_ASSERT(write_ops_.SendMessage(msg).ok());
+ GPR_CODEGEN_ASSERT(write_ops_.SendMessage(msg).ok());
call_.PerformOps(&write_ops_);
}
diff --git a/include/grpc++/impl/codegen/async_unary_call.h b/include/grpc++/impl/codegen/async_unary_call.h
index f3c75dc3b1..9c6dbd5484 100644
--- a/include/grpc++/impl/codegen/async_unary_call.h
+++ b/include/grpc++/impl/codegen/async_unary_call.h
@@ -45,6 +45,7 @@
namespace grpc {
class CompletionQueue;
+extern CoreCodegenInterface* g_core_codegen_interface;
template <class R>
class ClientAsyncResponseReaderInterface {
@@ -68,13 +69,13 @@ class ClientAsyncResponseReader GRPC_FINAL
collection_->init_buf_.SetCollection(collection_);
collection_->init_buf_.SendInitialMetadata(context->send_initial_metadata_);
// TODO(ctiller): don't assert
- GPR_ASSERT(collection_->init_buf_.SendMessage(request).ok());
+ GPR_CODEGEN_ASSERT(collection_->init_buf_.SendMessage(request).ok());
collection_->init_buf_.ClientSendClose();
call_.PerformOps(&collection_->init_buf_);
}
void ReadInitialMetadata(void* tag) {
- GPR_ASSERT(!context_->initial_metadata_received_);
+ GPR_CODEGEN_ASSERT(!context_->initial_metadata_received_);
collection_->meta_buf_.SetCollection(collection_);
collection_->meta_buf_.set_output_tag(tag);
@@ -116,7 +117,7 @@ class ServerAsyncResponseWriter GRPC_FINAL
: call_(nullptr, nullptr, nullptr), ctx_(ctx) {}
void SendInitialMetadata(void* tag) GRPC_OVERRIDE {
- GPR_ASSERT(!ctx_->sent_initial_metadata_);
+ GPR_CODEGEN_ASSERT(!ctx_->sent_initial_metadata_);
meta_buf_.set_output_tag(tag);
meta_buf_.SendInitialMetadata(ctx_->initial_metadata_);
@@ -141,7 +142,7 @@ class ServerAsyncResponseWriter GRPC_FINAL
}
void FinishWithError(const Status& status, void* tag) {
- GPR_ASSERT(!status.ok());
+ GPR_CODEGEN_ASSERT(!status.ok());
finish_buf_.set_output_tag(tag);
if (!ctx_->sent_initial_metadata_) {
finish_buf_.SendInitialMetadata(ctx_->initial_metadata_);
diff --git a/include/grpc++/impl/codegen/call.h b/include/grpc++/impl/codegen/call.h
index 5e450b0d24..ec950b8ea6 100644
--- a/include/grpc++/impl/codegen/call.h
+++ b/include/grpc++/impl/codegen/call.h
@@ -41,12 +41,14 @@
#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/client_context.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/core_codegen_interface.h>
+#include <grpc++/impl/codegen/serialization_traits.h>
#include <grpc++/impl/codegen/status.h>
+#include <grpc++/impl/codegen/string_ref.h>
struct grpc_byte_buffer;
@@ -56,12 +58,39 @@ class ByteBuffer;
class Call;
class CallHook;
class CompletionQueue;
+extern CoreCodegenInterface* g_core_codegen_interface;
-void FillMetadataMap(
+inline 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);
+ std::multimap<grpc::string_ref, grpc::string_ref>* metadata) {
+ for (size_t i = 0; i < arr->count; i++) {
+ // TODO(yangg) handle duplicates?
+ metadata->insert(std::pair<grpc::string_ref, grpc::string_ref>(
+ arr->metadata[i].key, grpc::string_ref(arr->metadata[i].value,
+ arr->metadata[i].value_length)));
+ }
+ g_core_codegen_interface->grpc_metadata_array_destroy(arr);
+ g_core_codegen_interface->grpc_metadata_array_init(arr);
+}
+
+// TODO(yangg) if the map is changed before we send, the pointers will be a
+// mess. Make sure it does not happen.
+inline grpc_metadata* FillMetadataArray(
+ const std::multimap<grpc::string, grpc::string>& metadata) {
+ if (metadata.empty()) {
+ return nullptr;
+ }
+ grpc_metadata* metadata_array =
+ (grpc_metadata*)(g_core_codegen_interface->gpr_malloc(
+ metadata.size() * sizeof(grpc_metadata)));
+ size_t i = 0;
+ for (auto iter = metadata.cbegin(); iter != metadata.cend(); ++iter, ++i) {
+ metadata_array[i].key = iter->first.c_str();
+ metadata_array[i].value = iter->second.c_str();
+ metadata_array[i].value_length = iter->second.size();
+ }
+ return metadata_array;
+}
/// Per-message write options.
class WriteOptions {
@@ -170,7 +199,7 @@ class CallOpSendInitialMetadata {
}
void FinishOp(bool* status, int max_message_size) {
if (!send_) return;
- gpr_free(initial_metadata_);
+ g_core_codegen_interface->gpr_free(initial_metadata_);
send_ = false;
}
@@ -204,7 +233,7 @@ class CallOpSendMessage {
write_options_.Clear();
}
void FinishOp(bool* status, int max_message_size) {
- if (own_buf_) grpc_byte_buffer_destroy(send_buf_);
+ if (own_buf_) g_core_codegen_interface->grpc_byte_buffer_destroy(send_buf_);
send_buf_ = nullptr;
}
@@ -254,7 +283,7 @@ class CallOpRecvMessage {
max_message_size).ok();
} else {
got_message = false;
- grpc_byte_buffer_destroy(recv_buf_);
+ g_core_codegen_interface->grpc_byte_buffer_destroy(recv_buf_);
}
} else {
got_message = false;
@@ -321,7 +350,7 @@ class CallOpGenericRecvMessage {
*status = deserialize_->Deserialize(recv_buf_, max_message_size).ok();
} else {
got_message = false;
- grpc_byte_buffer_destroy(recv_buf_);
+ g_core_codegen_interface->grpc_byte_buffer_destroy(recv_buf_);
}
} else {
got_message = false;
@@ -386,7 +415,7 @@ class CallOpServerSendStatus {
void FinishOp(bool* status, int max_message_size) {
if (!send_status_available_) return;
- gpr_free(trailing_metadata_);
+ g_core_codegen_interface->gpr_free(trailing_metadata_);
send_status_available_ = false;
}
@@ -462,7 +491,7 @@ class CallOpClientRecvStatus {
*recv_status_ = Status(
static_cast<StatusCode>(status_code_),
status_details_ ? grpc::string(status_details_) : grpc::string());
- gpr_free(status_details_);
+ g_core_codegen_interface->gpr_free(status_details_);
recv_status_ = nullptr;
}
@@ -576,11 +605,22 @@ class SneakyCallOpSet : public CallOpSet<Op1, Op2, Op3, Op4, Op5, Op6> {
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);
+ Call(grpc_call* call, CallHook* call_hook, CompletionQueue* cq)
+ : call_hook_(call_hook), cq_(cq), call_(call), max_message_size_(-1) {}
+
+ Call(grpc_call* call, CallHook* call_hook, CompletionQueue* cq,
+ int max_message_size)
+ : call_hook_(call_hook),
+ cq_(cq),
+ call_(call),
+ max_message_size_(max_message_size) {}
+
+ void PerformOps(CallOpSetInterface* ops) {
+ if (max_message_size_ > 0) {
+ ops->set_max_message_size(max_message_size_);
+ }
+ call_hook_->PerformOpsOnCall(ops, this);
+ }
grpc_call* call() { return call_; }
CompletionQueue* cq() { return cq_; }
diff --git a/include/grpc++/impl/codegen/client_unary_call.h b/include/grpc++/impl/codegen/client_unary_call.h
index 0134dec800..0f7c5ec90c 100644
--- a/include/grpc++/impl/codegen/client_unary_call.h
+++ b/include/grpc++/impl/codegen/client_unary_call.h
@@ -36,6 +36,7 @@
#include <grpc++/impl/codegen/call.h>
#include <grpc++/impl/codegen/channel_interface.h>
+#include <grpc++/impl/codegen/core_codegen_interface.h>
#include <grpc++/impl/codegen/config.h>
#include <grpc++/impl/codegen/status.h>
@@ -66,7 +67,7 @@ Status BlockingUnaryCall(ChannelInterface* channel, const RpcMethod& method,
ops.ClientSendClose();
ops.ClientRecvStatus(context, &status);
call.PerformOps(&ops);
- GPR_ASSERT((cq.Pluck(&ops) && ops.got_message) || !status.ok());
+ GPR_CODEGEN_ASSERT((cq.Pluck(&ops) && ops.got_message) || !status.ok());
return status;
}
diff --git a/include/grpc++/impl/codegen/core_codegen_interface.h b/include/grpc++/impl/codegen/core_codegen_interface.h
new file mode 100644
index 0000000000..b1b128cc4a
--- /dev/null
+++ b/include/grpc++/impl/codegen/core_codegen_interface.h
@@ -0,0 +1,82 @@
+/*
+ *
+ * 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.
+ *
+ */
+
+/// XXX
+#ifndef GRPCXX_IMPL_CODEGEN_CORE_CODEGEN_INTERFACE_H
+#define GRPCXX_IMPL_CODEGEN_CORE_CODEGEN_INTERFACE_H
+
+#include <grpc/impl/codegen/grpc_types.h>
+#include <grpc++/impl/codegen/status.h>
+#include <grpc++/impl/codegen/config_protobuf.h>
+
+namespace grpc {
+
+class CoreCodegenInterface {
+ public:
+ virtual grpc_completion_queue* CompletionQueueCreate() = 0;
+ virtual grpc_event CompletionQueuePluck(grpc_completion_queue* cq, void* tag,
+ gpr_timespec deadline) = 0;
+
+ // Serialize the msg into a buffer created inside the function. The caller
+ // should destroy the returned buffer when done with it. If serialization
+ // fails,
+ // false is returned and buffer is left unchanged.
+ virtual Status SerializeProto(const grpc::protobuf::Message& msg,
+ grpc_byte_buffer** buffer) = 0;
+
+ // The caller keeps ownership of buffer and msg.
+ virtual Status DeserializeProto(grpc_byte_buffer* buffer,
+ grpc::protobuf::Message* msg,
+ int max_message_size) = 0;
+
+ virtual void* gpr_malloc(size_t size) = 0;
+ virtual void gpr_free(void* p) = 0;
+
+ virtual void grpc_byte_buffer_destroy(grpc_byte_buffer* bb) = 0;
+ virtual void grpc_metadata_array_init(grpc_metadata_array* array) = 0;
+ virtual void grpc_metadata_array_destroy(grpc_metadata_array* array) = 0;
+
+ virtual void assert_fail(const char* failed_assertion) = 0;
+};
+
+/* XXX */
+#define GPR_CODEGEN_ASSERT(x) \
+ do { \
+ if (!(x)) { \
+ g_core_codegen_interface->assert_fail(#x); \
+ } \
+ } while (0)
+
+} // namespace grpc
+
+#endif // GRPCXX_IMPL_CODEGEN_CORE_CODEGEN_INTERFACE_H
diff --git a/include/grpc++/impl/codegen/impl/async_stream.h b/include/grpc++/impl/codegen/impl/async_stream.h
new file mode 100644
index 0000000000..b0410485f8
--- /dev/null
+++ b/include/grpc++/impl/codegen/impl/async_stream.h
@@ -0,0 +1,462 @@
+/*
+ *
+ * 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_ASYNC_STREAM_H
+#define GRPCXX_IMPL_CODEGEN_ASYNC_STREAM_H
+
+#include <grpc++/impl/codegen/channel_interface.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 {
+
+class CompletionQueue;
+
+/// Common interface for all client side asynchronous streaming.
+class ClientAsyncStreamingInterface {
+ public:
+ virtual ~ClientAsyncStreamingInterface() {}
+
+ /// Request notification of the reading of the initial metadata. Completion
+ /// will be notified by \a tag on the associated completion queue.
+ ///
+ /// \param[in] tag Tag identifying this request.
+ virtual void ReadInitialMetadata(void* tag) = 0;
+
+ /// Request notification completion.
+ ///
+ /// \param[out] status To be updated with the operation status.
+ /// \param[in] tag Tag identifying this request.
+ virtual void Finish(Status* status, void* tag) = 0;
+};
+
+/// An interface that yields a sequence of messages of type \a R.
+template <class R>
+class AsyncReaderInterface {
+ public:
+ virtual ~AsyncReaderInterface() {}
+
+ /// Read a message of type \a R into \a msg. Completion will be notified by \a
+ /// tag on the associated completion queue.
+ ///
+ /// \param[out] msg Where to eventually store the read message.
+ /// \param[in] tag The tag identifying the operation.
+ virtual void Read(R* msg, void* tag) = 0;
+};
+
+/// An interface that can be fed a sequence of messages of type \a W.
+template <class W>
+class AsyncWriterInterface {
+ public:
+ virtual ~AsyncWriterInterface() {}
+
+ /// Request the writing of \a msg with identifying tag \a tag.
+ ///
+ /// Only one write may be outstanding at any given time. This means that
+ /// after calling Write, one must wait to receive \a tag from the completion
+ /// queue BEFORE calling Write again.
+ ///
+ /// \param[in] msg The message to be written.
+ /// \param[in] tag The tag identifying the operation.
+ virtual void Write(const W& msg, void* tag) = 0;
+};
+
+template <class R>
+class ClientAsyncReaderInterface : public ClientAsyncStreamingInterface,
+ public AsyncReaderInterface<R> {};
+
+template <class R>
+class ClientAsyncReader GRPC_FINAL : public ClientAsyncReaderInterface<R> {
+ public:
+ /// Create a stream and write the first request out.
+ template <class W>
+ ClientAsyncReader(ChannelInterface* channel, CompletionQueue* cq,
+ const RpcMethod& method, ClientContext* context,
+ const W& request, void* tag)
+ : context_(context), call_(channel->CreateCall(method, context, cq)) {
+ init_ops_.set_output_tag(tag);
+ init_ops_.SendInitialMetadata(context->send_initial_metadata_);
+ // TODO(ctiller): don't assert
+ GPR_ASSERT(init_ops_.SendMessage(request).ok());
+ init_ops_.ClientSendClose();
+ call_.PerformOps(&init_ops_);
+ }
+
+ void ReadInitialMetadata(void* tag) GRPC_OVERRIDE {
+ GPR_ASSERT(!context_->initial_metadata_received_);
+
+ meta_ops_.set_output_tag(tag);
+ meta_ops_.RecvInitialMetadata(context_);
+ call_.PerformOps(&meta_ops_);
+ }
+
+ void Read(R* msg, void* tag) GRPC_OVERRIDE {
+ read_ops_.set_output_tag(tag);
+ if (!context_->initial_metadata_received_) {
+ read_ops_.RecvInitialMetadata(context_);
+ }
+ read_ops_.RecvMessage(msg);
+ call_.PerformOps(&read_ops_);
+ }
+
+ void Finish(Status* status, void* tag) GRPC_OVERRIDE {
+ finish_ops_.set_output_tag(tag);
+ if (!context_->initial_metadata_received_) {
+ finish_ops_.RecvInitialMetadata(context_);
+ }
+ finish_ops_.ClientRecvStatus(context_, status);
+ call_.PerformOps(&finish_ops_);
+ }
+
+ private:
+ ClientContext* context_;
+ Call call_;
+ CallOpSet<CallOpSendInitialMetadata, CallOpSendMessage, CallOpClientSendClose>
+ init_ops_;
+ CallOpSet<CallOpRecvInitialMetadata> meta_ops_;
+ CallOpSet<CallOpRecvInitialMetadata, CallOpRecvMessage<R>> read_ops_;
+ CallOpSet<CallOpRecvInitialMetadata, CallOpClientRecvStatus> finish_ops_;
+};
+
+/// Common interface for client side asynchronous writing.
+template <class W>
+class ClientAsyncWriterInterface : public ClientAsyncStreamingInterface,
+ public AsyncWriterInterface<W> {
+ public:
+ /// Signal the client is done with the writes.
+ ///
+ /// \param[in] tag The tag identifying the operation.
+ virtual void WritesDone(void* tag) = 0;
+};
+
+template <class W>
+class ClientAsyncWriter GRPC_FINAL : public ClientAsyncWriterInterface<W> {
+ public:
+ template <class R>
+ ClientAsyncWriter(ChannelInterface* channel, CompletionQueue* cq,
+ const RpcMethod& method, ClientContext* context,
+ R* response, void* tag)
+ : context_(context), call_(channel->CreateCall(method, context, cq)) {
+ finish_ops_.RecvMessage(response);
+
+ init_ops_.set_output_tag(tag);
+ init_ops_.SendInitialMetadata(context->send_initial_metadata_);
+ call_.PerformOps(&init_ops_);
+ }
+
+ void ReadInitialMetadata(void* tag) GRPC_OVERRIDE {
+ GPR_ASSERT(!context_->initial_metadata_received_);
+
+ meta_ops_.set_output_tag(tag);
+ meta_ops_.RecvInitialMetadata(context_);
+ call_.PerformOps(&meta_ops_);
+ }
+
+ void Write(const W& msg, void* tag) GRPC_OVERRIDE {
+ write_ops_.set_output_tag(tag);
+ // TODO(ctiller): don't assert
+ GPR_ASSERT(write_ops_.SendMessage(msg).ok());
+ call_.PerformOps(&write_ops_);
+ }
+
+ void WritesDone(void* tag) GRPC_OVERRIDE {
+ writes_done_ops_.set_output_tag(tag);
+ writes_done_ops_.ClientSendClose();
+ call_.PerformOps(&writes_done_ops_);
+ }
+
+ void Finish(Status* status, void* tag) GRPC_OVERRIDE {
+ finish_ops_.set_output_tag(tag);
+ if (!context_->initial_metadata_received_) {
+ finish_ops_.RecvInitialMetadata(context_);
+ }
+ finish_ops_.ClientRecvStatus(context_, status);
+ call_.PerformOps(&finish_ops_);
+ }
+
+ private:
+ ClientContext* context_;
+ Call call_;
+ CallOpSet<CallOpSendInitialMetadata> init_ops_;
+ CallOpSet<CallOpRecvInitialMetadata> meta_ops_;
+ CallOpSet<CallOpSendMessage> write_ops_;
+ CallOpSet<CallOpClientSendClose> writes_done_ops_;
+ CallOpSet<CallOpRecvInitialMetadata, CallOpGenericRecvMessage,
+ CallOpClientRecvStatus> finish_ops_;
+};
+
+/// Client-side interface for asynchronous bi-directional streaming.
+template <class W, class R>
+class ClientAsyncReaderWriterInterface : public ClientAsyncStreamingInterface,
+ public AsyncWriterInterface<W>,
+ public AsyncReaderInterface<R> {
+ public:
+ /// Signal the client is done with the writes.
+ ///
+ /// \param[in] tag The tag identifying the operation.
+ virtual void WritesDone(void* tag) = 0;
+};
+
+template <class W, class R>
+class ClientAsyncReaderWriter GRPC_FINAL
+ : public ClientAsyncReaderWriterInterface<W, R> {
+ public:
+ ClientAsyncReaderWriter(ChannelInterface* channel, CompletionQueue* cq,
+ const RpcMethod& method, ClientContext* context,
+ void* tag)
+ : context_(context), call_(channel->CreateCall(method, context, cq)) {
+ init_ops_.set_output_tag(tag);
+ init_ops_.SendInitialMetadata(context->send_initial_metadata_);
+ call_.PerformOps(&init_ops_);
+ }
+
+ void ReadInitialMetadata(void* tag) GRPC_OVERRIDE {
+ GPR_ASSERT(!context_->initial_metadata_received_);
+
+ meta_ops_.set_output_tag(tag);
+ meta_ops_.RecvInitialMetadata(context_);
+ call_.PerformOps(&meta_ops_);
+ }
+
+ void Read(R* msg, void* tag) GRPC_OVERRIDE {
+ read_ops_.set_output_tag(tag);
+ if (!context_->initial_metadata_received_) {
+ read_ops_.RecvInitialMetadata(context_);
+ }
+ read_ops_.RecvMessage(msg);
+ call_.PerformOps(&read_ops_);
+ }
+
+ void Write(const W& msg, void* tag) GRPC_OVERRIDE {
+ write_ops_.set_output_tag(tag);
+ // TODO(ctiller): don't assert
+ GPR_ASSERT(write_ops_.SendMessage(msg).ok());
+ call_.PerformOps(&write_ops_);
+ }
+
+ void WritesDone(void* tag) GRPC_OVERRIDE {
+ writes_done_ops_.set_output_tag(tag);
+ writes_done_ops_.ClientSendClose();
+ call_.PerformOps(&writes_done_ops_);
+ }
+
+ void Finish(Status* status, void* tag) GRPC_OVERRIDE {
+ finish_ops_.set_output_tag(tag);
+ if (!context_->initial_metadata_received_) {
+ finish_ops_.RecvInitialMetadata(context_);
+ }
+ finish_ops_.ClientRecvStatus(context_, status);
+ call_.PerformOps(&finish_ops_);
+ }
+
+ private:
+ ClientContext* context_;
+ Call call_;
+ CallOpSet<CallOpSendInitialMetadata> init_ops_;
+ CallOpSet<CallOpRecvInitialMetadata> meta_ops_;
+ CallOpSet<CallOpRecvInitialMetadata, CallOpRecvMessage<R>> read_ops_;
+ CallOpSet<CallOpSendMessage> write_ops_;
+ CallOpSet<CallOpClientSendClose> writes_done_ops_;
+ CallOpSet<CallOpRecvInitialMetadata, CallOpClientRecvStatus> finish_ops_;
+};
+
+template <class W, class R>
+class ServerAsyncReader GRPC_FINAL : public ServerAsyncStreamingInterface,
+ public AsyncReaderInterface<R> {
+ public:
+ explicit ServerAsyncReader(ServerContext* ctx)
+ : call_(nullptr, nullptr, nullptr), ctx_(ctx) {}
+
+ void SendInitialMetadata(void* tag) GRPC_OVERRIDE {
+ GPR_ASSERT(!ctx_->sent_initial_metadata_);
+
+ meta_ops_.set_output_tag(tag);
+ meta_ops_.SendInitialMetadata(ctx_->initial_metadata_);
+ ctx_->sent_initial_metadata_ = true;
+ call_.PerformOps(&meta_ops_);
+ }
+
+ void Read(R* msg, void* tag) GRPC_OVERRIDE {
+ read_ops_.set_output_tag(tag);
+ read_ops_.RecvMessage(msg);
+ call_.PerformOps(&read_ops_);
+ }
+
+ void Finish(const W& msg, const Status& status, void* tag) {
+ finish_ops_.set_output_tag(tag);
+ if (!ctx_->sent_initial_metadata_) {
+ finish_ops_.SendInitialMetadata(ctx_->initial_metadata_);
+ ctx_->sent_initial_metadata_ = true;
+ }
+ // The response is dropped if the status is not OK.
+ if (status.ok()) {
+ finish_ops_.ServerSendStatus(ctx_->trailing_metadata_,
+ finish_ops_.SendMessage(msg));
+ } else {
+ finish_ops_.ServerSendStatus(ctx_->trailing_metadata_, status);
+ }
+ call_.PerformOps(&finish_ops_);
+ }
+
+ void FinishWithError(const Status& status, void* tag) {
+ GPR_ASSERT(!status.ok());
+ finish_ops_.set_output_tag(tag);
+ if (!ctx_->sent_initial_metadata_) {
+ finish_ops_.SendInitialMetadata(ctx_->initial_metadata_);
+ ctx_->sent_initial_metadata_ = true;
+ }
+ finish_ops_.ServerSendStatus(ctx_->trailing_metadata_, status);
+ call_.PerformOps(&finish_ops_);
+ }
+
+ private:
+ void BindCall(Call* call) GRPC_OVERRIDE { call_ = *call; }
+
+ Call call_;
+ ServerContext* ctx_;
+ CallOpSet<CallOpSendInitialMetadata> meta_ops_;
+ CallOpSet<CallOpRecvMessage<R>> read_ops_;
+ CallOpSet<CallOpSendInitialMetadata, CallOpSendMessage,
+ CallOpServerSendStatus> finish_ops_;
+};
+
+template <class W>
+class ServerAsyncWriter GRPC_FINAL : public ServerAsyncStreamingInterface,
+ public AsyncWriterInterface<W> {
+ public:
+ explicit ServerAsyncWriter(ServerContext* ctx)
+ : call_(nullptr, nullptr, nullptr), ctx_(ctx) {}
+
+ void SendInitialMetadata(void* tag) GRPC_OVERRIDE {
+ GPR_ASSERT(!ctx_->sent_initial_metadata_);
+
+ meta_ops_.set_output_tag(tag);
+ meta_ops_.SendInitialMetadata(ctx_->initial_metadata_);
+ ctx_->sent_initial_metadata_ = true;
+ call_.PerformOps(&meta_ops_);
+ }
+
+ void Write(const W& msg, void* tag) GRPC_OVERRIDE {
+ write_ops_.set_output_tag(tag);
+ if (!ctx_->sent_initial_metadata_) {
+ write_ops_.SendInitialMetadata(ctx_->initial_metadata_);
+ ctx_->sent_initial_metadata_ = true;
+ }
+ // TODO(ctiller): don't assert
+ GPR_ASSERT(write_ops_.SendMessage(msg).ok());
+ call_.PerformOps(&write_ops_);
+ }
+
+ void Finish(const Status& status, void* tag) {
+ finish_ops_.set_output_tag(tag);
+ if (!ctx_->sent_initial_metadata_) {
+ finish_ops_.SendInitialMetadata(ctx_->initial_metadata_);
+ ctx_->sent_initial_metadata_ = true;
+ }
+ finish_ops_.ServerSendStatus(ctx_->trailing_metadata_, status);
+ call_.PerformOps(&finish_ops_);
+ }
+
+ private:
+ void BindCall(Call* call) GRPC_OVERRIDE { call_ = *call; }
+
+ Call call_;
+ ServerContext* ctx_;
+ CallOpSet<CallOpSendInitialMetadata> meta_ops_;
+ CallOpSet<CallOpSendInitialMetadata, CallOpSendMessage> write_ops_;
+ CallOpSet<CallOpSendInitialMetadata, CallOpServerSendStatus> finish_ops_;
+};
+
+/// Server-side interface for asynchronous bi-directional streaming.
+template <class W, class R>
+class ServerAsyncReaderWriter GRPC_FINAL : public ServerAsyncStreamingInterface,
+ public AsyncWriterInterface<W>,
+ public AsyncReaderInterface<R> {
+ public:
+ explicit ServerAsyncReaderWriter(ServerContext* ctx)
+ : call_(nullptr, nullptr, nullptr), ctx_(ctx) {}
+
+ void SendInitialMetadata(void* tag) GRPC_OVERRIDE {
+ GPR_ASSERT(!ctx_->sent_initial_metadata_);
+
+ meta_ops_.set_output_tag(tag);
+ meta_ops_.SendInitialMetadata(ctx_->initial_metadata_);
+ ctx_->sent_initial_metadata_ = true;
+ call_.PerformOps(&meta_ops_);
+ }
+
+ void Read(R* msg, void* tag) GRPC_OVERRIDE {
+ read_ops_.set_output_tag(tag);
+ read_ops_.RecvMessage(msg);
+ call_.PerformOps(&read_ops_);
+ }
+
+ void Write(const W& msg, void* tag) GRPC_OVERRIDE {
+ write_ops_.set_output_tag(tag);
+ if (!ctx_->sent_initial_metadata_) {
+ write_ops_.SendInitialMetadata(ctx_->initial_metadata_);
+ ctx_->sent_initial_metadata_ = true;
+ }
+ // TODO(ctiller): don't assert
+ GPR_ASSERT(write_ops_.SendMessage(msg).ok());
+ call_.PerformOps(&write_ops_);
+ }
+
+ void Finish(const Status& status, void* tag) {
+ finish_ops_.set_output_tag(tag);
+ if (!ctx_->sent_initial_metadata_) {
+ finish_ops_.SendInitialMetadata(ctx_->initial_metadata_);
+ ctx_->sent_initial_metadata_ = true;
+ }
+ finish_ops_.ServerSendStatus(ctx_->trailing_metadata_, status);
+ call_.PerformOps(&finish_ops_);
+ }
+
+ private:
+ friend class ::grpc::Server;
+
+ void BindCall(Call* call) GRPC_OVERRIDE { call_ = *call; }
+
+ Call call_;
+ ServerContext* ctx_;
+ CallOpSet<CallOpSendInitialMetadata> meta_ops_;
+ CallOpSet<CallOpRecvMessage<R>> read_ops_;
+ CallOpSet<CallOpSendInitialMetadata, CallOpSendMessage> write_ops_;
+ CallOpSet<CallOpSendInitialMetadata, CallOpServerSendStatus> finish_ops_;
+};
+
+} // namespace grpc
+
+#endif // GRPCXX_IMPL_CODEGEN_ASYNC_STREAM_H
diff --git a/include/grpc++/impl/codegen/impl/status_code_enum.h b/include/grpc++/impl/codegen/impl/status_code_enum.h
new file mode 100644
index 0000000000..9a90a18e2a
--- /dev/null
+++ b/include/grpc++/impl/codegen/impl/status_code_enum.h
@@ -0,0 +1,152 @@
+/*
+ *
+ * Copyright 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_STATUS_CODE_ENUM_H
+#define GRPCXX_IMPL_CODEGEN_STATUS_CODE_ENUM_H
+
+namespace grpc {
+
+enum StatusCode {
+ /// Not an error; returned on success.
+ OK = 0,
+
+ /// The operation was cancelled (typically by the caller).
+ CANCELLED = 1,
+
+ /// Unknown error. An example of where this error may be returned is if a
+ /// Status value received from another address space belongs to an error-space
+ /// that is not known in this address space. Also errors raised by APIs that
+ /// do not return enough error information may be converted to this error.
+ UNKNOWN = 2,
+
+ /// Client specified an invalid argument. Note that this differs from
+ /// FAILED_PRECONDITION. INVALID_ARGUMENT indicates arguments that are
+ /// problematic regardless of the state of the system (e.g., a malformed file
+ /// name).
+ INVALID_ARGUMENT = 3,
+
+ /// Deadline expired before operation could complete. For operations that
+ /// change the state of the system, this error may be returned even if the
+ /// operation has completed successfully. For example, a successful response
+ /// from a server could have been delayed long enough for the deadline to
+ /// expire.
+ DEADLINE_EXCEEDED = 4,
+
+ /// Some requested entity (e.g., file or directory) was not found.
+ NOT_FOUND = 5,
+
+ /// Some entity that we attempted to create (e.g., file or directory) already
+ /// exists.
+ ALREADY_EXISTS = 6,
+
+ /// The caller does not have permission to execute the specified operation.
+ /// PERMISSION_DENIED must not be used for rejections caused by exhausting
+ /// some resource (use RESOURCE_EXHAUSTED instead for those errors).
+ /// PERMISSION_DENIED must not be used if the caller can not be identified
+ /// (use UNAUTHENTICATED instead for those errors).
+ PERMISSION_DENIED = 7,
+
+ /// The request does not have valid authentication credentials for the
+ /// operation.
+ UNAUTHENTICATED = 16,
+
+ /// Some resource has been exhausted, perhaps a per-user quota, or perhaps the
+ /// entire file system is out of space.
+ RESOURCE_EXHAUSTED = 8,
+
+ /// Operation was rejected because the system is not in a state required for
+ /// the operation's execution. For example, directory to be deleted may be
+ /// non-empty, an rmdir operation is applied to a non-directory, etc.
+ ///
+ /// A litmus test that may help a service implementor in deciding
+ /// between FAILED_PRECONDITION, ABORTED, and UNAVAILABLE:
+ /// (a) Use UNAVAILABLE if the client can retry just the failing call.
+ /// (b) Use ABORTED if the client should retry at a higher-level
+ /// (e.g., restarting a read-modify-write sequence).
+ /// (c) Use FAILED_PRECONDITION if the client should not retry until
+ /// the system state has been explicitly fixed. E.g., if an "rmdir"
+ /// fails because the directory is non-empty, FAILED_PRECONDITION
+ /// should be returned since the client should not retry unless
+ /// they have first fixed up the directory by deleting files from it.
+ /// (d) Use FAILED_PRECONDITION if the client performs conditional
+ /// REST Get/Update/Delete on a resource and the resource on the
+ /// server does not match the condition. E.g., conflicting
+ /// read-modify-write on the same resource.
+ FAILED_PRECONDITION = 9,
+
+ /// The operation was aborted, typically due to a concurrency issue like
+ /// sequencer check failures, transaction aborts, etc.
+ ///
+ /// See litmus test above for deciding between FAILED_PRECONDITION, ABORTED,
+ /// and UNAVAILABLE.
+ ABORTED = 10,
+
+ /// Operation was attempted past the valid range. E.g., seeking or reading
+ /// past end of file.
+ ///
+ /// Unlike INVALID_ARGUMENT, this error indicates a problem that may be fixed
+ /// if the system state changes. For example, a 32-bit file system will
+ /// generate INVALID_ARGUMENT if asked to read at an offset that is not in the
+ /// range [0,2^32-1], but it will generate OUT_OF_RANGE if asked to read from
+ /// an offset past the current file size.
+ ///
+ /// There is a fair bit of overlap between FAILED_PRECONDITION and
+ /// OUT_OF_RANGE. We recommend using OUT_OF_RANGE (the more specific error)
+ /// when it applies so that callers who are iterating through a space can
+ /// easily look for an OUT_OF_RANGE error to detect when they are done.
+ OUT_OF_RANGE = 11,
+
+ /// Operation is not implemented or not supported/enabled in this service.
+ UNIMPLEMENTED = 12,
+
+ /// Internal errors. Means some invariants expected by underlying System has
+ /// been broken. If you see one of these errors, Something is very broken.
+ INTERNAL = 13,
+
+ /// The service is currently unavailable. This is a most likely a transient
+ /// condition and may be corrected by retrying with a backoff.
+ ///
+ /// See litmus test above for deciding between FAILED_PRECONDITION, ABORTED,
+ /// and UNAVAILABLE.
+ UNAVAILABLE = 14,
+
+ /// Unrecoverable data loss or corruption.
+ DATA_LOSS = 15,
+
+ /// Force users to include a default branch:
+ DO_NOT_USE = -1
+};
+
+} // namespace grpc
+
+#endif // GRPCXX_IMPL_CODEGEN_STATUS_CODE_ENUM_H
diff --git a/include/grpc++/impl/codegen/proto_serializer.cc b/include/grpc++/impl/codegen/impl/sync.h
index 456567e080..375af1543b 100644
--- a/include/grpc++/impl/codegen/proto_serializer.cc
+++ b/include/grpc++/impl/codegen/impl/sync.h
@@ -1,6 +1,6 @@
/*
*
- * Copyright 2016, Google Inc.
+ * Copyright 2015-2016, Google Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
@@ -31,11 +31,15 @@
*
*/
-// TODO(dgq): This file is part of a temporary fix to work around codegen
-// issues.
-//
-// This whole file will be removed in the future.
+#ifndef GRPCXX_IMPL_CODEGEN_SYNC_H
+#define GRPCXX_IMPL_CODEGEN_SYNC_H
-#include <grpc++/impl/codegen/proto_utils.h>
+#include <grpc++/impl/codegen/config.h>
-grpc::ProtoSerializerInterface* grpc::g_proto_serializer = nullptr;
+#ifdef GRPC_CXX0X_NO_THREAD
+#include <grpc++/impl/codegen/sync_no_cxx11.h>
+#else
+#include <grpc++/impl/codegen/sync_cxx11.h>
+#endif
+
+#endif // GRPCXX_IMPL_CODEGEN_SYNC_H
diff --git a/include/grpc++/impl/codegen/method_handler_impl.h b/include/grpc++/impl/codegen/method_handler_impl.h
index 1bf9bf0549..3ecca0a406 100644
--- a/include/grpc++/impl/codegen/method_handler_impl.h
+++ b/include/grpc++/impl/codegen/method_handler_impl.h
@@ -34,6 +34,7 @@
#ifndef GRPCXX_IMPL_CODEGEN_METHOD_HANDLER_IMPL_H
#define GRPCXX_IMPL_CODEGEN_METHOD_HANDLER_IMPL_H
+#include <grpc++/impl/codegen/core_codegen_interface.h>
#include <grpc++/impl/codegen/rpc_service_method.h>
#include <grpc++/impl/codegen/sync_stream.h>
@@ -58,7 +59,7 @@ class RpcMethodHandler : public MethodHandler {
status = func_(service_, param.server_context, &req, &rsp);
}
- GPR_ASSERT(!param.server_context->sent_initial_metadata_);
+ GPR_CODEGEN_ASSERT(!param.server_context->sent_initial_metadata_);
CallOpSet<CallOpSendInitialMetadata, CallOpSendMessage,
CallOpServerSendStatus> ops;
ops.SendInitialMetadata(param.server_context->initial_metadata_);
@@ -93,7 +94,7 @@ class ClientStreamingHandler : public MethodHandler {
ResponseType rsp;
Status status = func_(service_, param.server_context, &reader, &rsp);
- GPR_ASSERT(!param.server_context->sent_initial_metadata_);
+ GPR_CODEGEN_ASSERT(!param.server_context->sent_initial_metadata_);
CallOpSet<CallOpSendInitialMetadata, CallOpSendMessage,
CallOpServerSendStatus> ops;
ops.SendInitialMetadata(param.server_context->initial_metadata_);
diff --git a/include/grpc++/impl/codegen/proto_utils.h b/include/grpc++/impl/codegen/proto_utils.h
index c06fc5d22b..8903a4412e 100644
--- a/include/grpc++/impl/codegen/proto_utils.h
+++ b/include/grpc++/impl/codegen/proto_utils.h
@@ -41,46 +41,11 @@
#include <grpc++/impl/codegen/serialization_traits.h>
#include <grpc++/impl/codegen/config_protobuf.h>
#include <grpc++/impl/codegen/status.h>
+#include <grpc++/impl/codegen/core_codegen_interface.h>
namespace grpc {
-class ProtoSerializerInterface {
- public:
- // Serialize the msg into a buffer created inside the function. The caller
- // should destroy the returned buffer when done with it. If serialization
- // fails,
- // false is returned and buffer is left unchanged.
- virtual Status SerializeProto(const grpc::protobuf::Message& msg,
- grpc_byte_buffer** buffer) = 0;
-
- // The caller keeps ownership of buffer and msg.
- virtual Status DeserializeProto(grpc_byte_buffer* buffer,
- grpc::protobuf::Message* msg,
- int max_message_size) = 0;
-};
-
-// TODO(dgq): This is a temporary fix to work around codegen issues. Its purpose
-// is to hold a polymorphic proto serializer/deserializer instance. It's
-// initialized as part of src/cpp/proto/proto_serializer.cc.
-//
-// This global variable plus all related code (ProtoSerializerInteface,
-// ProtoSerializer) will be removed in the future.
-extern ProtoSerializerInterface* g_proto_serializer;
-
-class ProtoSerializer : public ProtoSerializerInterface {
- public:
- // Serialize the msg into a buffer created inside the function. The caller
- // should destroy the returned buffer when done with it. If serialization
- // fails,
- // false is returned and buffer is left unchanged.
- Status SerializeProto(const grpc::protobuf::Message& msg,
- grpc_byte_buffer** buffer) override;
-
- // The caller keeps ownership of buffer and msg.
- Status DeserializeProto(grpc_byte_buffer* buffer,
- grpc::protobuf::Message* msg,
- int max_message_size) override;
-};
+extern CoreCodegenInterface* g_core_codegen_interface;
template <class T>
class SerializationTraits<T, typename std::enable_if<std::is_base_of<
@@ -89,21 +54,12 @@ class SerializationTraits<T, typename std::enable_if<std::is_base_of<
static Status Serialize(const grpc::protobuf::Message& msg,
grpc_byte_buffer** buffer, bool* own_buffer) {
*own_buffer = true;
- GPR_ASSERT(g_proto_serializer != nullptr &&
- "No ProtoSerializer instance registered. Make sure grpc++ is "
- "being initialized.");
- return g_proto_serializer->SerializeProto(msg, buffer);
+ return g_core_codegen_interface->SerializeProto(msg, buffer);
}
static Status Deserialize(grpc_byte_buffer* buffer,
grpc::protobuf::Message* msg,
int max_message_size) {
- GPR_ASSERT(g_proto_serializer != nullptr &&
- "No ProtoSerializer instance registered. Make sure grpc++ is "
- "being initialized.");
- auto status =
- g_proto_serializer->DeserializeProto(buffer, msg, max_message_size);
- grpc_byte_buffer_destroy(buffer);
- return status;
+ return g_core_codegen_interface->DeserializeProto(buffer, msg, max_message_size);
}
};
diff --git a/include/grpc++/impl/codegen/string_ref.h b/include/grpc++/impl/codegen/string_ref.h
index e3af37e0c2..29d85d92dc 100644
--- a/include/grpc++/impl/codegen/string_ref.h
+++ b/include/grpc++/impl/codegen/string_ref.h
@@ -34,8 +34,12 @@
#ifndef GRPCXX_IMPL_CODEGEN_STRING_REF_H
#define GRPCXX_IMPL_CODEGEN_STRING_REF_H
-#include <iterator>
+#include <string.h>
+
+#include <algorithm>
#include <iosfwd>
+#include <iostream>
+#include <iterator>
#include <grpc++/impl/codegen/config.h>
@@ -56,14 +60,19 @@ class string_ref {
typedef std::reverse_iterator<const_iterator> const_reverse_iterator;
// constants
- const static size_t npos;
+ const static size_t npos = size_t(-1);
// construct/copy.
string_ref() : data_(nullptr), length_(0) {}
string_ref(const string_ref& other)
: data_(other.data_), length_(other.length_) {}
- string_ref& operator=(const string_ref& rhs);
- string_ref(const char* s);
+ string_ref& operator=(const string_ref& rhs) {
+ data_ = rhs.data_;
+ length_ = rhs.length_;
+ return *this;
+ }
+
+ string_ref(const char* s) : data_(s), length_(strlen(s)) {}
string_ref(const char* s, size_t l) : data_(s), length_(l) {}
string_ref(const grpc::string& s) : data_(s.data()), length_(s.length()) {}
@@ -95,13 +104,40 @@ class string_ref {
const char* data() const { return data_; }
// string operations
- int compare(string_ref x) const;
- bool starts_with(string_ref x) const;
- bool ends_with(string_ref x) const;
- size_t find(string_ref s) const;
- size_t find(char c) const;
+ int compare(string_ref x) const {
+ size_t min_size = length_ < x.length_ ? length_ : x.length_;
+ int r = memcmp(data_, x.data_, min_size);
+ if (r < 0) return -1;
+ if (r > 0) return 1;
+ if (length_ < x.length_) return -1;
+ if (length_ > x.length_) return 1;
+ return 0;
+ }
+
+ bool starts_with(string_ref x) const {
+ return length_ >= x.length_ && (memcmp(data_, x.data_, x.length_) == 0);
+ }
- string_ref substr(size_t pos, size_t n = npos) const;
+ bool ends_with(string_ref x) const {
+ return length_ >= x.length_ &&
+ (memcmp(data_ + (length_ - x.length_), x.data_, x.length_) == 0);
+ }
+
+ size_t find(string_ref s) const {
+ auto it = std::search(cbegin(), cend(), s.cbegin(), s.cend());
+ return it == cend() ? npos : std::distance(cbegin(), it);
+ }
+
+ size_t find(char c) const {
+ auto it = std::find(cbegin(), cend(), c);
+ return it == cend() ? npos : std::distance(cbegin(), it);
+ }
+
+ string_ref substr(size_t pos, size_t n = npos) const {
+ if (pos > length_) pos = length_;
+ if (n > (length_ - pos)) n = length_ - pos;
+ return string_ref(data_ + pos, n);
+ }
private:
const char* data_;
@@ -109,14 +145,16 @@ class string_ref {
};
// Comparison operators
-bool operator==(string_ref x, string_ref y);
-bool operator!=(string_ref x, string_ref y);
-bool operator<(string_ref x, string_ref y);
-bool operator>(string_ref x, string_ref y);
-bool operator<=(string_ref x, string_ref y);
-bool operator>=(string_ref x, string_ref y);
-
-std::ostream& operator<<(std::ostream& stream, const string_ref& string);
+inline bool operator==(string_ref x, string_ref y) { return x.compare(y) == 0; }
+inline bool operator!=(string_ref x, string_ref y) { return x.compare(y) != 0; }
+inline bool operator<(string_ref x, string_ref y) { return x.compare(y) < 0; }
+inline bool operator<=(string_ref x, string_ref y) { return x.compare(y) <= 0; }
+inline bool operator>(string_ref x, string_ref y) { return x.compare(y) > 0; }
+inline bool operator>=(string_ref x, string_ref y) { return x.compare(y) >= 0; }
+
+inline std::ostream& operator<<(std::ostream& out, const string_ref& string) {
+ return out << grpc::string(string.begin(), string.end());
+}
} // namespace grpc
diff --git a/include/grpc++/impl/codegen/sync_stream.h b/include/grpc++/impl/codegen/sync_stream.h
index 9ae48bd23d..5f878469ce 100644
--- a/include/grpc++/impl/codegen/sync_stream.h
+++ b/include/grpc++/impl/codegen/sync_stream.h
@@ -38,6 +38,7 @@
#include <grpc++/impl/codegen/channel_interface.h>
#include <grpc++/impl/codegen/client_context.h>
#include <grpc++/impl/codegen/completion_queue.h>
+#include <grpc++/impl/codegen/core_codegen_interface.h>
#include <grpc++/impl/codegen/server_context.h>
#include <grpc++/impl/codegen/service_type.h>
#include <grpc++/impl/codegen/status.h>
@@ -125,14 +126,14 @@ class ClientReader GRPC_FINAL : public ClientReaderInterface<R> {
CallOpClientSendClose> ops;
ops.SendInitialMetadata(context->send_initial_metadata_);
// TODO(ctiller): don't assert
- GPR_ASSERT(ops.SendMessage(request).ok());
+ GPR_CODEGEN_ASSERT(ops.SendMessage(request).ok());
ops.ClientSendClose();
call_.PerformOps(&ops);
cq_.Pluck(&ops);
}
void WaitForInitialMetadata() GRPC_OVERRIDE {
- GPR_ASSERT(!context_->initial_metadata_received_);
+ GPR_CODEGEN_ASSERT(!context_->initial_metadata_received_);
CallOpSet<CallOpRecvInitialMetadata> ops;
ops.RecvInitialMetadata(context_);
@@ -155,7 +156,7 @@ class ClientReader GRPC_FINAL : public ClientReaderInterface<R> {
Status status;
ops.ClientRecvStatus(context_, &status);
call_.PerformOps(&ops);
- GPR_ASSERT(cq_.Pluck(&ops));
+ GPR_CODEGEN_ASSERT(cq_.Pluck(&ops));
return status;
}
@@ -194,7 +195,7 @@ class ClientWriter : public ClientWriterInterface<W> {
}
void WaitForInitialMetadata() {
- GPR_ASSERT(!context_->initial_metadata_received_);
+ GPR_CODEGEN_ASSERT(!context_->initial_metadata_received_);
CallOpSet<CallOpRecvInitialMetadata> ops;
ops.RecvInitialMetadata(context_);
@@ -227,7 +228,7 @@ class ClientWriter : public ClientWriterInterface<W> {
}
finish_ops_.ClientRecvStatus(context_, &status);
call_.PerformOps(&finish_ops_);
- GPR_ASSERT(cq_.Pluck(&finish_ops_));
+ GPR_CODEGEN_ASSERT(cq_.Pluck(&finish_ops_));
return status;
}
@@ -271,7 +272,7 @@ class ClientReaderWriter GRPC_FINAL : public ClientReaderWriterInterface<W, R> {
}
void WaitForInitialMetadata() GRPC_OVERRIDE {
- GPR_ASSERT(!context_->initial_metadata_received_);
+ GPR_CODEGEN_ASSERT(!context_->initial_metadata_received_);
CallOpSet<CallOpRecvInitialMetadata> ops;
ops.RecvInitialMetadata(context_);
@@ -312,7 +313,7 @@ class ClientReaderWriter GRPC_FINAL : public ClientReaderWriterInterface<W, R> {
Status status;
ops.ClientRecvStatus(context_, &status);
call_.PerformOps(&ops);
- GPR_ASSERT(cq_.Pluck(&ops));
+ GPR_CODEGEN_ASSERT(cq_.Pluck(&ops));
return status;
}
@@ -328,7 +329,7 @@ class ServerReader GRPC_FINAL : public ReaderInterface<R> {
ServerReader(Call* call, ServerContext* ctx) : call_(call), ctx_(ctx) {}
void SendInitialMetadata() {
- GPR_ASSERT(!ctx_->sent_initial_metadata_);
+ GPR_CODEGEN_ASSERT(!ctx_->sent_initial_metadata_);
CallOpSet<CallOpSendInitialMetadata> ops;
ops.SendInitialMetadata(ctx_->initial_metadata_);
@@ -355,7 +356,7 @@ class ServerWriter GRPC_FINAL : public WriterInterface<W> {
ServerWriter(Call* call, ServerContext* ctx) : call_(call), ctx_(ctx) {}
void SendInitialMetadata() {
- GPR_ASSERT(!ctx_->sent_initial_metadata_);
+ GPR_CODEGEN_ASSERT(!ctx_->sent_initial_metadata_);
CallOpSet<CallOpSendInitialMetadata> ops;
ops.SendInitialMetadata(ctx_->initial_metadata_);
@@ -391,7 +392,7 @@ class ServerReaderWriter GRPC_FINAL : public WriterInterface<W>,
ServerReaderWriter(Call* call, ServerContext* ctx) : call_(call), ctx_(ctx) {}
void SendInitialMetadata() {
- GPR_ASSERT(!ctx_->sent_initial_metadata_);
+ GPR_CODEGEN_ASSERT(!ctx_->sent_initial_metadata_);
CallOpSet<CallOpSendInitialMetadata> ops;
ops.SendInitialMetadata(ctx_->initial_metadata_);