aboutsummaryrefslogtreecommitdiffhomepage
path: root/src/csharp/GrpcCore
diff options
context:
space:
mode:
authorGravatar Jan Tattermusch <jtattermusch@google.com>2015-02-13 15:41:41 -0800
committerGravatar Jan Tattermusch <jtattermusch@google.com>2015-02-18 12:51:29 -0800
commit337a2ddba59563e7370b133d63ab8bd9ebeb7232 (patch)
treea637c60ad42fe0b1c1244b83827acc5bd82f04d9 /src/csharp/GrpcCore
parent60ea9130e10fad6a1de0275120f14416712de449 (diff)
migration to new C API
Diffstat (limited to 'src/csharp/GrpcCore')
-rw-r--r--src/csharp/GrpcCore/Calls.cs58
-rw-r--r--src/csharp/GrpcCore/GrpcCore.csproj6
-rw-r--r--src/csharp/GrpcCore/Internal/AsyncCall.cs493
-rw-r--r--src/csharp/GrpcCore/Internal/BatchContextSafeHandle.cs96
-rw-r--r--src/csharp/GrpcCore/Internal/CallSafeHandle.cs138
-rw-r--r--src/csharp/GrpcCore/Internal/ClientStreamingInputObserver.cs (renamed from src/csharp/GrpcCore/Internal/StreamingInputObserver.cs)15
-rw-r--r--src/csharp/GrpcCore/Internal/CompletionQueueSafeHandle.cs16
-rw-r--r--src/csharp/GrpcCore/Internal/Event.cs224
-rw-r--r--src/csharp/GrpcCore/Internal/GrpcThreadPool.cs47
-rw-r--r--src/csharp/GrpcCore/Internal/SafeHandleZeroIsInvalid.cs6
-rw-r--r--src/csharp/GrpcCore/Internal/ServerSafeHandle.cs16
-rw-r--r--src/csharp/GrpcCore/Internal/ServerStreamingOutputObserver.cs (renamed from src/csharp/GrpcCore/Internal/ServerWritingObserver.cs)10
-rw-r--r--src/csharp/GrpcCore/Server.cs31
-rw-r--r--src/csharp/GrpcCore/ServerCallHandler.cs28
14 files changed, 517 insertions, 667 deletions
diff --git a/src/csharp/GrpcCore/Calls.cs b/src/csharp/GrpcCore/Calls.cs
index d89d9a16f9..e5ddd879d6 100644
--- a/src/csharp/GrpcCore/Calls.cs
+++ b/src/csharp/GrpcCore/Calls.cs
@@ -47,50 +47,42 @@ namespace Google.GRPC.Core
{
public static TResponse BlockingUnaryCall<TRequest, TResponse>(Call<TRequest, TResponse> call, TRequest req, CancellationToken token)
{
- //TODO: implement this in real synchronous style once new GRPC C core API is available.
- return AsyncUnaryCall(call, req, token).Result;
+ //TODO: implement this in real synchronous style.
+ try {
+ return AsyncUnaryCall(call, req, token).Result;
+ } catch(AggregateException ae) {
+ foreach (var e in ae.InnerExceptions)
+ {
+ if (e is RpcException)
+ {
+ throw e;
+ }
+ }
+ throw;
+ }
}
public static async Task<TResponse> AsyncUnaryCall<TRequest, TResponse>(Call<TRequest, TResponse> call, TRequest req, CancellationToken token)
{
var asyncCall = new AsyncCall<TRequest, TResponse>(call.RequestSerializer, call.ResponseDeserializer);
- asyncCall.Initialize(call.Channel, call.MethodName);
- asyncCall.Start(false, GetCompletionQueue());
-
- await asyncCall.WriteAsync(req);
- await asyncCall.WritesCompletedAsync();
-
- TResponse response = await asyncCall.ReadAsync();
-
- Status status = await asyncCall.Finished;
-
- if (status.StatusCode != StatusCode.GRPC_STATUS_OK)
- {
- throw new RpcException(status);
- }
- return response;
+ asyncCall.Initialize(call.Channel, GetCompletionQueue(), call.MethodName);
+ return await asyncCall.UnaryCallAsync(req);
}
- public static async Task AsyncServerStreamingCall<TRequest, TResponse>(Call<TRequest, TResponse> call, TRequest req, IObserver<TResponse> outputs, CancellationToken token)
+ public static void AsyncServerStreamingCall<TRequest, TResponse>(Call<TRequest, TResponse> call, TRequest req, IObserver<TResponse> outputs, CancellationToken token)
{
var asyncCall = new AsyncCall<TRequest, TResponse>(call.RequestSerializer, call.ResponseDeserializer);
- asyncCall.Initialize(call.Channel, call.MethodName);
- asyncCall.Start(false, GetCompletionQueue());
- asyncCall.StartReadingToStream(outputs);
-
- await asyncCall.WriteAsync(req);
- await asyncCall.WritesCompletedAsync();
+ asyncCall.Initialize(call.Channel, GetCompletionQueue(), call.MethodName);
+ asyncCall.StartServerStreamingCall(req, outputs);
}
public static ClientStreamingAsyncResult<TRequest, TResponse> AsyncClientStreamingCall<TRequest, TResponse>(Call<TRequest, TResponse> call, CancellationToken token)
{
var asyncCall = new AsyncCall<TRequest, TResponse>(call.RequestSerializer, call.ResponseDeserializer);
- asyncCall.Initialize(call.Channel, call.MethodName);
- asyncCall.Start(false, GetCompletionQueue());
-
- var task = asyncCall.ReadAsync();
- var inputs = new StreamingInputObserver<TRequest, TResponse>(asyncCall);
+ asyncCall.Initialize(call.Channel, GetCompletionQueue(), call.MethodName);
+ var task = asyncCall.ClientStreamingCallAsync();
+ var inputs = new ClientStreamingInputObserver<TRequest, TResponse>(asyncCall);
return new ClientStreamingAsyncResult<TRequest, TResponse>(task, inputs);
}
@@ -102,12 +94,10 @@ namespace Google.GRPC.Core
public static IObserver<TRequest> DuplexStreamingCall<TRequest, TResponse>(Call<TRequest, TResponse> call, IObserver<TResponse> outputs, CancellationToken token)
{
var asyncCall = new AsyncCall<TRequest, TResponse>(call.RequestSerializer, call.ResponseDeserializer);
- asyncCall.Initialize(call.Channel, call.MethodName);
- asyncCall.Start(false, GetCompletionQueue());
+ asyncCall.Initialize(call.Channel, GetCompletionQueue(), call.MethodName);
- asyncCall.StartReadingToStream(outputs);
- var inputs = new StreamingInputObserver<TRequest, TResponse>(asyncCall);
- return inputs;
+ asyncCall.StartDuplexStreamingCall(outputs);
+ return new ClientStreamingInputObserver<TRequest, TResponse>(asyncCall);
}
private static CompletionQueueSafeHandle GetCompletionQueue() {
diff --git a/src/csharp/GrpcCore/GrpcCore.csproj b/src/csharp/GrpcCore/GrpcCore.csproj
index 34b9f6dfb8..a574f181c8 100644
--- a/src/csharp/GrpcCore/GrpcCore.csproj
+++ b/src/csharp/GrpcCore/GrpcCore.csproj
@@ -47,21 +47,21 @@
<Compile Include="Internal\ChannelSafeHandle.cs" />
<Compile Include="Internal\CompletionQueueSafeHandle.cs" />
<Compile Include="Internal\Enums.cs" />
- <Compile Include="Internal\Event.cs" />
<Compile Include="Internal\SafeHandleZeroIsInvalid.cs" />
<Compile Include="Internal\Timespec.cs" />
<Compile Include="Internal\GrpcThreadPool.cs" />
<Compile Include="Internal\AsyncCall.cs" />
<Compile Include="Internal\ServerSafeHandle.cs" />
- <Compile Include="Internal\StreamingInputObserver.cs" />
<Compile Include="Method.cs" />
<Compile Include="ServerCalls.cs" />
<Compile Include="ServerCallHandler.cs" />
- <Compile Include="Internal\ServerWritingObserver.cs" />
<Compile Include="Marshaller.cs" />
<Compile Include="ServerServiceDefinition.cs" />
<Compile Include="Utils\RecordingObserver.cs" />
<Compile Include="Utils\RecordingQueue.cs" />
+ <Compile Include="Internal\BatchContextSafeHandle.cs" />
+ <Compile Include="Internal\ClientStreamingInputObserver.cs" />
+ <Compile Include="Internal\ServerStreamingOutputObserver.cs" />
</ItemGroup>
<Import Project="$(MSBuildBinPath)\Microsoft.CSharp.targets" />
<ItemGroup>
diff --git a/src/csharp/GrpcCore/Internal/AsyncCall.cs b/src/csharp/GrpcCore/Internal/AsyncCall.cs
index d5f3239e1e..ae7428978e 100644
--- a/src/csharp/GrpcCore/Internal/AsyncCall.cs
+++ b/src/csharp/GrpcCore/Internal/AsyncCall.cs
@@ -2,11 +2,11 @@
// 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
@@ -16,7 +16,7 @@
// * 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
@@ -42,38 +42,27 @@ using Google.GRPC.Core.Internal;
namespace Google.GRPC.Core.Internal
{
/// <summary>
- /// Listener for call events that can be delivered from a completion queue.
- /// </summary>
- internal interface ICallEventListener {
-
- void OnClientMetadata();
-
- void OnRead(byte[] payload);
-
- void OnWriteAccepted(GRPCOpError error);
-
- void OnFinishAccepted(GRPCOpError error);
-
- // ignore the status on server
- void OnFinished(Status status);
- }
-
- /// <summary>
/// Handle native call lifecycle and provides convenience methods.
/// </summary>
- internal class AsyncCall<TWrite, TRead>: ICallEventListener, IDisposable
+ internal class AsyncCall<TWrite, TRead> : IDisposable
{
readonly Func<TWrite, byte[]> serializer;
readonly Func<byte[], TRead> deserializer;
- // TODO: make sure the delegate doesn't get garbage collected while
+ // TODO: make sure the delegate doesn't get garbage collected while
// native callbacks are in the completion queue.
- readonly EventCallbackDelegate callbackHandler;
+ readonly CompletionCallbackDelegate unaryResponseHandler;
+ readonly CompletionCallbackDelegate finishedHandler;
+ readonly CompletionCallbackDelegate writeFinishedHandler;
+ readonly CompletionCallbackDelegate readFinishedHandler;
+ readonly CompletionCallbackDelegate halfclosedHandler;
+ readonly CompletionCallbackDelegate finishedServersideHandler;
object myLock = new object();
bool disposed;
CallSafeHandle call;
+ bool server;
bool started;
bool errorOccured;
@@ -85,54 +74,25 @@ namespace Google.GRPC.Core.Internal
TaskCompletionSource<object> writeTcs;
TaskCompletionSource<TRead> readTcs;
+
+ TaskCompletionSource<object> finishedServersideTcs = new TaskCompletionSource<object>();
TaskCompletionSource<object> halfcloseTcs = new TaskCompletionSource<object>();
TaskCompletionSource<Status> finishedTcs = new TaskCompletionSource<Status>();
+ TaskCompletionSource<TRead> unaryResponseTcs;
+
IObserver<TRead> readObserver;
public AsyncCall(Func<TWrite, byte[]> serializer, Func<byte[], TRead> deserializer)
{
this.serializer = serializer;
this.deserializer = deserializer;
- this.callbackHandler = HandleEvent;
- }
-
- public Task WriteAsync(TWrite msg)
- {
- return StartWrite(msg, false).Task;
- }
-
- public Task WritesCompletedAsync()
- {
- WritesDone();
- return halfcloseTcs.Task;
- }
-
- public Task WriteStatusAsync(Status status)
- {
- WriteStatus(status);
- return halfcloseTcs.Task;
- }
-
- public Task<TRead> ReadAsync()
- {
- return StartRead().Task;
- }
-
- public Task Halfclosed
- {
- get
- {
- return halfcloseTcs.Task;
- }
- }
-
- public Task<Status> Finished
- {
- get
- {
- return finishedTcs.Task;
- }
+ this.unaryResponseHandler = HandleUnaryResponseCompletion;
+ this.finishedHandler = HandleFinished;
+ this.writeFinishedHandler = HandleWriteFinished;
+ this.readFinishedHandler = HandleReadFinished;
+ this.halfclosedHandler = HandleHalfclosed;
+ this.finishedServersideHandler = HandleFinishedServerside;
}
/// <summary>
@@ -147,14 +107,14 @@ namespace Google.GRPC.Core.Internal
throw new InvalidOperationException("Already registered an observer.");
}
this.readObserver = readObserver;
- StartRead();
+ ReceiveMessageAsync();
}
}
- public void Initialize(Channel channel, String methodName) {
+ public void Initialize(Channel channel, CompletionQueueSafeHandle cq, String methodName) {
lock (myLock)
{
- this.call = CallSafeHandle.Create(channel.Handle, methodName, channel.Target, Timespec.InfFuture);
+ this.call = CallSafeHandle.Create(channel.Handle, cq, methodName, channel.Target, Timespec.InfFuture);
}
}
@@ -163,44 +123,77 @@ namespace Google.GRPC.Core.Internal
lock(myLock)
{
this.call = call;
+ started = true;
+ server = true;
}
}
- // Client only
- public void Start(bool buffered, CompletionQueueSafeHandle cq)
+
+ public Task<TRead> UnaryCallAsync(TWrite msg)
{
lock (myLock)
{
- if (started)
- {
- throw new InvalidOperationException("Already started.");
- }
-
- call.Invoke(cq, buffered, callbackHandler, callbackHandler);
started = true;
+ halfcloseRequested = true;
+
+ // TODO: handle serialization error...
+ byte[] payload = serializer(msg);
+
+ unaryResponseTcs = new TaskCompletionSource<TRead>();
+ call.StartUnary(payload, unaryResponseHandler);
+
+ return unaryResponseTcs.Task;
}
}
- // Server only
- public void Accept(CompletionQueueSafeHandle cq)
+ public Task<TRead> ClientStreamingCallAsync()
{
lock (myLock)
{
- if (started)
- {
- throw new InvalidOperationException("Already started.");
- }
+ started = true;
+
+ unaryResponseTcs = new TaskCompletionSource<TRead>();
+ call.StartClientStreaming(unaryResponseHandler);
+
+ return unaryResponseTcs.Task;
+ }
+ }
- call.ServerAccept(cq, callbackHandler);
- call.ServerEndInitialMetadata(0);
+ public void StartServerStreamingCall(TWrite msg, IObserver<TRead> readObserver)
+ {
+ lock (myLock)
+ {
started = true;
+ halfcloseRequested = true;
+
+ this.readObserver = readObserver;
+
+ // TODO: handle serialization error...
+ byte[] payload = serializer(msg);
+
+ call.StartServerStreaming(payload, finishedHandler);
+
+ ReceiveMessageAsync();
}
}
- public TaskCompletionSource<object> StartWrite(TWrite msg, bool buffered)
+ public void StartDuplexStreamingCall(IObserver<TRead> readObserver)
{
lock (myLock)
{
+ started = true;
+
+ this.readObserver = readObserver;
+
+ call.StartDuplexStreaming(finishedHandler);
+
+ ReceiveMessageAsync();
+ }
+ }
+
+ public Task SendMessageAsync(TWrite msg) {
+ lock (myLock)
+ {
CheckStarted();
CheckNotFinished();
CheckNoError();
@@ -219,14 +212,13 @@ namespace Google.GRPC.Core.Internal
// TODO: wrap serialization...
byte[] payload = serializer(msg);
- call.StartWrite(payload, buffered, callbackHandler);
+ call.StartSendMessage(payload, writeFinishedHandler);
writeTcs = new TaskCompletionSource<object>();
- return writeTcs;
+ return writeTcs.Task;
}
}
- // client only
- public void WritesDone()
+ public Task SendCloseFromClientAsync()
{
lock (myLock)
{
@@ -240,13 +232,13 @@ namespace Google.GRPC.Core.Internal
throw new InvalidOperationException("Already halfclosed.");
}
- call.WritesDone(callbackHandler);
+ call.StartSendCloseFromClient(halfclosedHandler);
halfcloseRequested = true;
+ return halfcloseTcs.Task;
}
}
- // server only
- public void WriteStatus(Status status)
+ public Task SendStatusFromServerAsync(Status status)
{
lock (myLock)
{
@@ -260,12 +252,13 @@ namespace Google.GRPC.Core.Internal
throw new InvalidOperationException("Already halfclosed.");
}
- call.StartWriteStatus(status, callbackHandler);
+ call.StartSendStatusFromServer(status, halfclosedHandler);
halfcloseRequested = true;
+ return halfcloseTcs.Task;
}
}
- public TaskCompletionSource<TRead> StartRead()
+ public Task<TRead> ReceiveMessageAsync()
{
lock (myLock)
{
@@ -285,10 +278,19 @@ namespace Google.GRPC.Core.Internal
throw new InvalidOperationException("Only one read can be pending at a time");
}
- call.StartRead(callbackHandler);
+ call.StartReceiveMessage(readFinishedHandler);
readTcs = new TaskCompletionSource<TRead>();
- return readTcs;
+ return readTcs.Task;
+ }
+ }
+
+ internal Task StartServerSide()
+ {
+ lock (myLock)
+ {
+ call.StartServerSide(finishedServersideHandler);
+ return finishedServersideTcs.Task;
}
}
@@ -317,107 +319,7 @@ namespace Google.GRPC.Core.Internal
// grpc_call_cancel_with_status is threadsafe
call.CancelWithStatus(status);
}
-
- public void OnClientMetadata()
- {
- // TODO: implement....
- }
-
- public void OnRead(byte[] payload)
- {
- TaskCompletionSource<TRead> oldTcs = null;
- IObserver<TRead> observer = null;
- lock (myLock)
- {
- oldTcs = readTcs;
- readTcs = null;
- if (payload == null)
- {
- doneWithReading = true;
- }
- observer = readObserver;
- }
-
- // TODO: wrap deserialization...
- TRead msg = payload != null ? deserializer(payload) : default(TRead);
-
- oldTcs.SetResult(msg);
-
- // TODO: make sure we deliver reads in the right order.
-
- if (observer != null)
- {
- if (payload != null)
- {
- // TODO: wrap to handle exceptions
- observer.OnNext(msg);
-
- // start a new read
- StartRead();
- }
- else
- {
- // TODO: wrap to handle exceptions;
- observer.OnCompleted();
- }
-
- }
- }
-
- public void OnWriteAccepted(GRPCOpError error)
- {
- TaskCompletionSource<object> oldTcs = null;
- lock (myLock)
- {
- UpdateErrorOccured(error);
- oldTcs = writeTcs;
- writeTcs = null;
- }
-
- if (errorOccured)
- {
- // TODO: use the right type of exception...
- oldTcs.SetException(new Exception("Write failed"));
- }
- else
- {
- // TODO: where does the continuation run?
- oldTcs.SetResult(null);
- }
- }
-
- public void OnFinishAccepted(GRPCOpError error)
- {
- lock (myLock)
- {
- UpdateErrorOccured(error);
- halfclosed = true;
- }
-
- if (errorOccured)
- {
- halfcloseTcs.SetException(new Exception("Halfclose failed"));
-
- }
- else
- {
- halfcloseTcs.SetResult(null);
- }
-
- }
-
- public void OnFinished(Status status)
- {
- lock (myLock)
- {
- finishedStatus = status;
-
- DisposeResourcesIfNeeded();
- }
- finishedTcs.SetResult(status);
-
- }
-
+
public void Dispose()
{
Dispose(true);
@@ -434,7 +336,7 @@ namespace Google.GRPC.Core.Internal
{
call.Dispose();
}
- }
+ }
disposed = true;
}
}
@@ -489,38 +391,195 @@ namespace Google.GRPC.Core.Internal
}
}
- private void HandleEvent(IntPtr eventPtr) {
+ private void CompleteStreamObserver(Status status) {
+ if (status.StatusCode != StatusCode.GRPC_STATUS_OK)
+ {
+ // TODO: wrap to handle exceptions;
+ readObserver.OnError(new RpcException(status));
+ } else {
+ // TODO: wrap to handle exceptions;
+ readObserver.OnCompleted();
+ }
+ }
+
+ private void HandleUnaryResponseCompletion(GRPCOpError error, IntPtr batchContextPtr) {
+ try {
+
+ TaskCompletionSource<TRead> tcs;
+ lock(myLock) {
+ tcs = unaryResponseTcs;
+ }
+
+ // we're done with this call, get rid of the native object.
+ call.Dispose();
+
+ var ctx = new BatchContextSafeHandleNotOwned(batchContextPtr);
+
+ if (error != GRPCOpError.GRPC_OP_OK) {
+ tcs.SetException(new RpcException(
+ new Status(StatusCode.GRPC_STATUS_INTERNAL, "Internal error occured.")
+ ));
+ return;
+ }
+
+ var status = ctx.GetReceivedStatus();
+ if (status.StatusCode != StatusCode.GRPC_STATUS_OK) {
+ tcs.SetException(new RpcException(status));
+ return;
+ }
+
+ // TODO: handle deserialize error...
+ var msg = deserializer(ctx.GetReceivedMessage());
+ tcs.SetResult(msg);
+ } catch(Exception e) {
+ Console.WriteLine("Caught exception in a native handler: " + e);
+ }
+ }
+
+ private void HandleWriteFinished(GRPCOpError error, IntPtr batchContextPtr) {
+ try {
+
+ TaskCompletionSource<object> oldTcs = null;
+ lock (myLock)
+ {
+ UpdateErrorOccured(error);
+ oldTcs = writeTcs;
+ writeTcs = null;
+ }
+
+ if (errorOccured)
+ {
+ // TODO: use the right type of exception...
+ oldTcs.SetException(new Exception("Write failed"));
+ }
+ else
+ {
+ // TODO: where does the continuation run?
+ oldTcs.SetResult(null);
+ }
+
+ } catch(Exception e) {
+ Console.WriteLine("Caught exception in a native handler: " + e);
+ }
+ }
+
+ private void HandleHalfclosed(GRPCOpError error, IntPtr batchContextPtr) {
+ try {
+ lock (myLock)
+ {
+ UpdateErrorOccured(error);
+ halfclosed = true;
+ }
+
+ if (errorOccured)
+ {
+ halfcloseTcs.SetException(new Exception("Halfclose failed"));
+
+ }
+ else
+ {
+ halfcloseTcs.SetResult(null);
+ }
+ } catch(Exception e) {
+ Console.WriteLine("Caught exception in a native handler: " + e);
+ }
+ }
+
+ private void HandleReadFinished(GRPCOpError error, IntPtr batchContextPtr) {
try {
- var ev = new EventSafeHandleNotOwned(eventPtr);
- switch (ev.GetCompletionType())
+
+ var ctx = new BatchContextSafeHandleNotOwned(batchContextPtr);
+ var payload = ctx.GetReceivedMessage();
+
+ TaskCompletionSource<TRead> oldTcs = null;
+ IObserver<TRead> observer = null;
+
+ Nullable<Status> status = null;
+
+ lock (myLock)
{
- case GRPCCompletionType.GRPC_CLIENT_METADATA_READ:
- OnClientMetadata();
- break;
+ oldTcs = readTcs;
+ readTcs = null;
+ if (payload == null)
+ {
+ doneWithReading = true;
+ }
+ observer = readObserver;
+ status = finishedStatus;
+ }
+
+ // TODO: wrap deserialization...
+ TRead msg = payload != null ? deserializer(payload) : default(TRead);
- case GRPCCompletionType.GRPC_READ:
- byte[] payload = ev.GetReadData();
- OnRead(payload);
- break;
+ oldTcs.SetResult(msg);
- case GRPCCompletionType.GRPC_WRITE_ACCEPTED:
- OnWriteAccepted(ev.GetWriteAccepted());
- break;
+ // TODO: make sure we deliver reads in the right order.
- case GRPCCompletionType.GRPC_FINISH_ACCEPTED:
- OnFinishAccepted(ev.GetFinishAccepted());
- break;
+ if (observer != null) {
+ if (payload != null)
+ {
+ // TODO: wrap to handle exceptions
+ observer.OnNext(msg);
+
+ // start a new read
+ ReceiveMessageAsync();
+ }
+ else
+ {
+ if (!server) {
+ if (status.HasValue) {
+ CompleteStreamObserver(status.Value);
+ }
+ } else {
+ // TODO: wrap to handle exceptions..
+ observer.OnCompleted();
+ }
+ // TODO: completeStreamObserver serverside...
+ }
+ }
+ } catch(Exception e) {
+ Console.WriteLine("Caught exception in a native handler: " + e);
+ }
+ }
+
+ private void HandleFinished(GRPCOpError error, IntPtr batchContextPtr) {
+ try {
+ var ctx = new BatchContextSafeHandleNotOwned(batchContextPtr);
+ var status = ctx.GetReceivedStatus();
+
+ bool wasDoneWithReading;
+
+ lock (myLock)
+ {
+ finishedStatus = status;
- case GRPCCompletionType.GRPC_FINISHED:
- OnFinished(ev.GetFinished());
- break;
+ DisposeResourcesIfNeeded();
- default:
- throw new ArgumentException("Unexpected completion type");
+ wasDoneWithReading = doneWithReading;
}
+
+ if (wasDoneWithReading) {
+ CompleteStreamObserver(status);
+ }
+
+ } catch(Exception e) {
+ Console.WriteLine("Caught exception in a native handler: " + e);
+ }
+ }
+
+ private void HandleFinishedServerside(GRPCOpError error, IntPtr batchContextPtr) {
+ try {
+ var ctx = new BatchContextSafeHandleNotOwned(batchContextPtr);
+
+ // TODO: handle error ...
+
+ finishedServersideTcs.SetResult(null);
+
+ call.Dispose();
+
} catch(Exception e) {
Console.WriteLine("Caught exception in a native handler: " + e);
}
}
}
-}
+} \ No newline at end of file
diff --git a/src/csharp/GrpcCore/Internal/BatchContextSafeHandle.cs b/src/csharp/GrpcCore/Internal/BatchContextSafeHandle.cs
new file mode 100644
index 0000000000..ddfd94a3b5
--- /dev/null
+++ b/src/csharp/GrpcCore/Internal/BatchContextSafeHandle.cs
@@ -0,0 +1,96 @@
+#region Copyright notice and license
+
+// 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.
+
+#endregion
+
+using System;
+using System.Runtime.InteropServices;
+using Google.GRPC.Core;
+
+namespace Google.GRPC.Core.Internal
+{
+ /// <summary>
+ /// Not owned version of
+ /// grpcsharp_batch_context
+ /// </summary>
+ internal class BatchContextSafeHandleNotOwned : SafeHandleZeroIsInvalid
+ {
+ [DllImport("grpc_csharp_ext.dll")]
+ static extern IntPtr grpcsharp_batch_context_recv_message_length(BatchContextSafeHandleNotOwned ctx);
+
+ [DllImport("grpc_csharp_ext.dll")]
+ static extern void grpcsharp_batch_context_recv_message_to_buffer(BatchContextSafeHandleNotOwned ctx, byte[] buffer, UIntPtr bufferLen);
+
+ [DllImport("grpc_csharp_ext.dll")]
+ static extern StatusCode grpcsharp_batch_context_recv_status_on_client_status(BatchContextSafeHandleNotOwned ctx);
+
+ [DllImport("grpc_csharp_ext.dll")]
+ static extern IntPtr grpcsharp_batch_context_recv_status_on_client_details(BatchContextSafeHandleNotOwned ctx); // returns const char*
+
+ [DllImport("grpc_csharp_ext.dll")]
+ static extern CallSafeHandle grpcsharp_batch_context_server_rpc_new_call(BatchContextSafeHandleNotOwned ctx);
+
+ [DllImport("grpc_csharp_ext.dll")]
+ static extern IntPtr grpcsharp_batch_context_server_rpc_new_method(BatchContextSafeHandleNotOwned ctx); // returns const char*
+
+ public BatchContextSafeHandleNotOwned(IntPtr handle) : base(false)
+ {
+ SetHandle(handle);
+ }
+
+ public Status GetReceivedStatus()
+ {
+ // TODO: can the native method return string directly?
+ string details = Marshal.PtrToStringAnsi(grpcsharp_batch_context_recv_status_on_client_details(this));
+ return new Status(grpcsharp_batch_context_recv_status_on_client_status(this), details);
+ }
+
+ public byte[] GetReceivedMessage()
+ {
+ IntPtr len = grpcsharp_batch_context_recv_message_length(this);
+ if (len == new IntPtr(-1))
+ {
+ return null;
+ }
+ byte[] data = new byte[(int) len];
+ grpcsharp_batch_context_recv_message_to_buffer(this, data, new UIntPtr((ulong)data.Length));
+ return data;
+ }
+
+ public CallSafeHandle GetServerRpcNewCall() {
+ return grpcsharp_batch_context_server_rpc_new_call(this);
+ }
+
+ public string GetServerRpcNewMethod() {
+ return Marshal.PtrToStringAnsi(grpcsharp_batch_context_server_rpc_new_method(this));
+ }
+ }
+} \ No newline at end of file
diff --git a/src/csharp/GrpcCore/Internal/CallSafeHandle.cs b/src/csharp/GrpcCore/Internal/CallSafeHandle.cs
index e9ccd8d5f9..55d66a62ca 100644
--- a/src/csharp/GrpcCore/Internal/CallSafeHandle.cs
+++ b/src/csharp/GrpcCore/Internal/CallSafeHandle.cs
@@ -2,11 +2,11 @@
// 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
@@ -16,7 +16,7 @@
// * 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
@@ -38,8 +38,8 @@ using Google.GRPC.Core;
namespace Google.GRPC.Core.Internal
{
- // TODO: we need to make sure that the delegates are not collected before invoked.
- internal delegate void EventCallbackDelegate(IntPtr eventPtr);
+ //TODO: rename the delegate
+ internal delegate void CompletionCallbackDelegate(GRPCOpError error, IntPtr batchContextPtr);
/// <summary>
/// grpc_call from <grpc/grpc.h>
@@ -49,142 +49,108 @@ namespace Google.GRPC.Core.Internal
const UInt32 GRPC_WRITE_BUFFER_HINT = 1;
[DllImport("grpc_csharp_ext.dll")]
- static extern CallSafeHandle grpcsharp_channel_create_call_old(ChannelSafeHandle channel, string method, string host, Timespec deadline);
+ static extern CallSafeHandle grpcsharp_channel_create_call(ChannelSafeHandle channel, CompletionQueueSafeHandle cq, string method, string host, Timespec deadline);
[DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCallError grpcsharp_call_add_metadata(CallSafeHandle call, IntPtr metadata, UInt32 flags);
+ static extern GRPCCallError grpcsharp_call_cancel(CallSafeHandle call);
[DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCallError grpcsharp_call_invoke_old(CallSafeHandle call, CompletionQueueSafeHandle cq, IntPtr metadataReadTag, IntPtr finishedTag, UInt32 flags);
-
- [DllImport("grpc_csharp_ext.dll", EntryPoint = "grpcsharp_call_invoke_old")]
- static extern GRPCCallError grpcsharp_call_invoke_old_CALLBACK(CallSafeHandle call, CompletionQueueSafeHandle cq,
- [MarshalAs(UnmanagedType.FunctionPtr)] EventCallbackDelegate metadataReadCallback,
- [MarshalAs(UnmanagedType.FunctionPtr)] EventCallbackDelegate finishedCallback,
- UInt32 flags);
+ static extern GRPCCallError grpcsharp_call_cancel_with_status(CallSafeHandle call, StatusCode status, string description);
[DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCallError grpcsharp_call_server_accept_old(CallSafeHandle call, CompletionQueueSafeHandle completionQueue, IntPtr finishedTag);
-
- [DllImport("grpc_csharp_ext.dll", EntryPoint = "grpcsharp_call_server_accept_old")]
- static extern GRPCCallError grpcsharp_call_server_accept_old_CALLBACK(CallSafeHandle call, CompletionQueueSafeHandle completionQueue, [MarshalAs(UnmanagedType.FunctionPtr)] EventCallbackDelegate finishedCallback);
+ static extern GRPCCallError grpcsharp_call_start_unary(CallSafeHandle call,
+ [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback,
+ byte[] send_buffer, UIntPtr send_buffer_len);
[DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCallError grpcsharp_call_server_end_initial_metadata_old(CallSafeHandle call, UInt32 flags);
+ static extern GRPCCallError grpcsharp_call_start_client_streaming(CallSafeHandle call,
+ [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback);
[DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCallError grpcsharp_call_cancel(CallSafeHandle call);
+ static extern GRPCCallError grpcsharp_call_start_server_streaming(CallSafeHandle call,
+ [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback,
+ byte[] send_buffer, UIntPtr send_buffer_len);
[DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCallError grpcsharp_call_cancel_with_status(CallSafeHandle call, StatusCode status, string description);
+ static extern GRPCCallError grpcsharp_call_start_duplex_streaming(CallSafeHandle call,
+ [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback);
[DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCallError grpcsharp_call_start_write_status_old(CallSafeHandle call, StatusCode statusCode, string statusMessage, IntPtr tag);
-
- [DllImport("grpc_csharp_ext.dll", EntryPoint = "grpcsharp_call_start_write_status_old")]
- static extern GRPCCallError grpcsharp_call_start_write_status_old_CALLBACK(CallSafeHandle call, StatusCode statusCode, string statusMessage, [MarshalAs(UnmanagedType.FunctionPtr)] EventCallbackDelegate callback);
+ static extern GRPCCallError grpcsharp_call_send_message(CallSafeHandle call,
+ [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback,
+ byte[] send_buffer, UIntPtr send_buffer_len);
[DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCallError grpcsharp_call_writes_done_old(CallSafeHandle call, IntPtr tag);
-
- [DllImport("grpc_csharp_ext.dll", EntryPoint = "grpcsharp_call_writes_done_old")]
- static extern GRPCCallError grpcsharp_call_writes_done_old_CALLBACK(CallSafeHandle call, [MarshalAs(UnmanagedType.FunctionPtr)] EventCallbackDelegate callback);
+ static extern GRPCCallError grpcsharp_call_send_close_from_client(CallSafeHandle call,
+ [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback);
[DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCallError grpcsharp_call_start_read_old(CallSafeHandle call, IntPtr tag);
-
- [DllImport("grpc_csharp_ext.dll", EntryPoint = "grpcsharp_call_start_read_old")]
- static extern GRPCCallError grpcsharp_call_start_read_old_CALLBACK(CallSafeHandle call, [MarshalAs(UnmanagedType.FunctionPtr)] EventCallbackDelegate callback);
+ static extern GRPCCallError grpcsharp_call_send_status_from_server(CallSafeHandle call, [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback, StatusCode statusCode, string statusMessage);
[DllImport("grpc_csharp_ext.dll")]
- static extern void grpcsharp_call_start_write_from_copied_buffer(CallSafeHandle call,
- byte[] buffer, UIntPtr length,
- IntPtr tag, UInt32 flags);
+ static extern GRPCCallError grpcsharp_call_recv_message(CallSafeHandle call,
+ [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback);
- [DllImport("grpc_csharp_ext.dll", EntryPoint = "grpcsharp_call_start_write_from_copied_buffer")]
- static extern void grpcsharp_call_start_write_from_copied_buffer_CALLBACK(CallSafeHandle call,
- byte[] buffer, UIntPtr length,
- [MarshalAs(UnmanagedType.FunctionPtr)] EventCallbackDelegate callback,
- UInt32 flags);
+ [DllImport("grpc_csharp_ext.dll")]
+ static extern GRPCCallError grpcsharp_call_start_serverside(CallSafeHandle call,
+ [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback);
- [DllImport("grpc_csharp_ext.dll")]
+ [DllImport("grpc_csharp_ext.dll")]
static extern void grpcsharp_call_destroy(IntPtr call);
- private CallSafeHandle()
- {
- }
-
- /// <summary>
- /// Creates a client call.
- /// </summary>
- public static CallSafeHandle Create(ChannelSafeHandle channel, string method, string host, Timespec deadline)
- {
- return grpcsharp_channel_create_call_old(channel, method, host, deadline);
- }
-
- public void Invoke(CompletionQueueSafeHandle cq, IntPtr metadataReadTag, IntPtr finishedTag, bool buffered)
- {
- AssertCallOk(grpcsharp_call_invoke_old(this, cq, metadataReadTag, finishedTag, GetFlags(buffered)));
- }
-
- public void Invoke(CompletionQueueSafeHandle cq, bool buffered, EventCallbackDelegate metadataReadCallback, EventCallbackDelegate finishedCallback)
- {
- AssertCallOk(grpcsharp_call_invoke_old_CALLBACK(this, cq, metadataReadCallback, finishedCallback, GetFlags(buffered)));
- }
- public void ServerAccept(CompletionQueueSafeHandle cq, IntPtr finishedTag)
+ private CallSafeHandle()
{
- AssertCallOk(grpcsharp_call_server_accept_old(this, cq, finishedTag));
}
- public void ServerAccept(CompletionQueueSafeHandle cq, EventCallbackDelegate callback)
+ public static CallSafeHandle Create(ChannelSafeHandle channel, CompletionQueueSafeHandle cq, string method, string host, Timespec deadline)
{
- AssertCallOk(grpcsharp_call_server_accept_old_CALLBACK(this, cq, callback));
+ return grpcsharp_channel_create_call(channel, cq, method, host, deadline);
}
- public void ServerEndInitialMetadata(UInt32 flags)
+ public void StartUnary(byte[] payload, CompletionCallbackDelegate callback)
{
- AssertCallOk(grpcsharp_call_server_end_initial_metadata_old(this, flags));
+ AssertCallOk(grpcsharp_call_start_unary(this, callback, payload, new UIntPtr((ulong) payload.Length)));
}
- public void StartWrite(byte[] payload, IntPtr tag, bool buffered)
+ public void StartClientStreaming(CompletionCallbackDelegate callback)
{
- grpcsharp_call_start_write_from_copied_buffer(this, payload, new UIntPtr((ulong) payload.Length), tag, GetFlags(buffered));
+ AssertCallOk(grpcsharp_call_start_client_streaming(this, callback));
}
- public void StartWrite(byte[] payload, bool buffered, EventCallbackDelegate callback)
+ public void StartServerStreaming(byte[] payload, CompletionCallbackDelegate callback)
{
- grpcsharp_call_start_write_from_copied_buffer_CALLBACK(this, payload, new UIntPtr((ulong) payload.Length), callback, GetFlags(buffered));
+ AssertCallOk(grpcsharp_call_start_server_streaming(this, callback, payload, new UIntPtr((ulong) payload.Length)));
}
- public void StartWriteStatus(Status status, IntPtr tag)
+ public void StartDuplexStreaming(CompletionCallbackDelegate callback)
{
- AssertCallOk(grpcsharp_call_start_write_status_old(this, status.StatusCode, status.Detail, tag));
+ AssertCallOk(grpcsharp_call_start_duplex_streaming(this, callback));
}
- public void StartWriteStatus(Status status, EventCallbackDelegate callback)
+ public void StartSendMessage(byte[] payload, CompletionCallbackDelegate callback)
{
- AssertCallOk(grpcsharp_call_start_write_status_old_CALLBACK(this, status.StatusCode, status.Detail, callback));
+ AssertCallOk(grpcsharp_call_send_message(this, callback, payload, new UIntPtr((ulong) payload.Length)));
}
- public void WritesDone(IntPtr tag)
+ public void StartSendCloseFromClient(CompletionCallbackDelegate callback)
{
- AssertCallOk(grpcsharp_call_writes_done_old(this, tag));
+ AssertCallOk(grpcsharp_call_send_close_from_client(this, callback));
}
- public void WritesDone(EventCallbackDelegate callback)
+ public void StartSendStatusFromServer(Status status, CompletionCallbackDelegate callback)
{
- AssertCallOk(grpcsharp_call_writes_done_old_CALLBACK(this, callback));
+ AssertCallOk(grpcsharp_call_send_status_from_server(this, callback, status.StatusCode, status.Detail));
}
- public void StartRead(IntPtr tag)
+ public void StartReceiveMessage(CompletionCallbackDelegate callback)
{
- AssertCallOk(grpcsharp_call_start_read_old(this, tag));
+ AssertCallOk(grpcsharp_call_recv_message(this, callback));
}
- public void StartRead(EventCallbackDelegate callback)
+ public void StartServerSide(CompletionCallbackDelegate callback)
{
- AssertCallOk(grpcsharp_call_start_read_old_CALLBACK(this, callback));
+ AssertCallOk(grpcsharp_call_start_serverside(this, callback));
}
public void Cancel()
@@ -212,4 +178,4 @@ namespace Google.GRPC.Core.Internal
return buffered ? 0 : GRPC_WRITE_BUFFER_HINT;
}
}
-}
+} \ No newline at end of file
diff --git a/src/csharp/GrpcCore/Internal/StreamingInputObserver.cs b/src/csharp/GrpcCore/Internal/ClientStreamingInputObserver.cs
index 60837de5e6..4d10a9bdf9 100644
--- a/src/csharp/GrpcCore/Internal/StreamingInputObserver.cs
+++ b/src/csharp/GrpcCore/Internal/ClientStreamingInputObserver.cs
@@ -2,11 +2,11 @@
// 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
@@ -16,7 +16,7 @@
// * 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
@@ -36,19 +36,20 @@ using Google.GRPC.Core.Internal;
namespace Google.GRPC.Core.Internal
{
- internal class StreamingInputObserver<TWrite, TRead> : IObserver<TWrite>
+ internal class ClientStreamingInputObserver<TWrite, TRead> : IObserver<TWrite>
{
readonly AsyncCall<TWrite, TRead> call;
- public StreamingInputObserver(AsyncCall<TWrite, TRead> call)
+ public ClientStreamingInputObserver(AsyncCall<TWrite, TRead> call)
{
this.call = call;
}
public void OnCompleted()
{
+
// TODO: how bad is the Wait here?
- call.WritesCompletedAsync().Wait();
+ call.SendCloseFromClientAsync().Wait();
}
public void OnError(Exception error)
@@ -59,7 +60,7 @@ namespace Google.GRPC.Core.Internal
public void OnNext(TWrite value)
{
// TODO: how bad is the Wait here?
- call.WriteAsync(value).Wait();
+ call.SendMessageAsync(value).Wait();
}
}
}
diff --git a/src/csharp/GrpcCore/Internal/CompletionQueueSafeHandle.cs b/src/csharp/GrpcCore/Internal/CompletionQueueSafeHandle.cs
index 666f220b8c..5ea436df19 100644
--- a/src/csharp/GrpcCore/Internal/CompletionQueueSafeHandle.cs
+++ b/src/csharp/GrpcCore/Internal/CompletionQueueSafeHandle.cs
@@ -46,12 +46,6 @@ namespace Google.GRPC.Core.Internal
static extern CompletionQueueSafeHandle grpcsharp_completion_queue_create();
[DllImport("grpc_csharp_ext.dll")]
- static extern EventSafeHandle grpcsharp_completion_queue_pluck(CompletionQueueSafeHandle cq, IntPtr tag, Timespec deadline);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern EventSafeHandle grpcsharp_completion_queue_next(CompletionQueueSafeHandle cq, Timespec deadline);
-
- [DllImport("grpc_csharp_ext.dll")]
static extern void grpcsharp_completion_queue_shutdown(CompletionQueueSafeHandle cq);
[DllImport("grpc_csharp_ext.dll")]
@@ -69,21 +63,11 @@ namespace Google.GRPC.Core.Internal
return grpcsharp_completion_queue_create();
}
- public EventSafeHandle Next(Timespec deadline)
- {
- return grpcsharp_completion_queue_next(this, deadline);
- }
-
public GRPCCompletionType NextWithCallback()
{
return grpcsharp_completion_queue_next_with_callback(this);
}
- public EventSafeHandle Pluck(IntPtr tag, Timespec deadline)
- {
- return grpcsharp_completion_queue_pluck(this, tag, deadline);
- }
-
public void Shutdown()
{
grpcsharp_completion_queue_shutdown(this);
diff --git a/src/csharp/GrpcCore/Internal/Event.cs b/src/csharp/GrpcCore/Internal/Event.cs
deleted file mode 100644
index 6116e0975a..0000000000
--- a/src/csharp/GrpcCore/Internal/Event.cs
+++ /dev/null
@@ -1,224 +0,0 @@
-#region Copyright notice and license
-
-// 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.
-
-#endregion
-
-using System;
-using System.Runtime.InteropServices;
-using Google.GRPC.Core;
-
-namespace Google.GRPC.Core.Internal
-{
- /// <summary>
- /// grpc_event from grpc/grpc.h
- /// </summary>
- internal class EventSafeHandle : SafeHandleZeroIsInvalid
- {
- [DllImport("grpc_csharp_ext.dll")]
- static extern void grpcsharp_event_finish(IntPtr ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCompletionType grpcsharp_event_type(EventSafeHandle ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern CallSafeHandle grpcsharp_event_call(EventSafeHandle ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern GRPCOpError grpcsharp_event_write_accepted(EventSafeHandle ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern GRPCOpError grpcsharp_event_finish_accepted(EventSafeHandle ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern StatusCode grpcsharp_event_finished_status(EventSafeHandle ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern IntPtr grpcsharp_event_finished_details(EventSafeHandle ev); // returns const char*
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern IntPtr grpcsharp_event_read_length(EventSafeHandle ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern void grpcsharp_event_read_copy_to_buffer(EventSafeHandle ev, byte[] buffer, UIntPtr bufferLen);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern IntPtr grpcsharp_event_server_rpc_new_method(EventSafeHandle ev); // returns const char*
-
- public GRPCCompletionType GetCompletionType()
- {
- return grpcsharp_event_type(this);
- }
-
- public GRPCOpError GetWriteAccepted()
- {
- return grpcsharp_event_write_accepted(this);
- }
-
- public GRPCOpError GetFinishAccepted()
- {
- return grpcsharp_event_finish_accepted(this);
- }
-
- public Status GetFinished()
- {
- // TODO: can the native method return string directly?
- string details = Marshal.PtrToStringAnsi(grpcsharp_event_finished_details(this));
- return new Status(grpcsharp_event_finished_status(this), details);
- }
-
- public byte[] GetReadData()
- {
- IntPtr len = grpcsharp_event_read_length(this);
- if (len == new IntPtr(-1))
- {
- return null;
- }
- byte[] data = new byte[(int) len];
- grpcsharp_event_read_copy_to_buffer(this, data, new UIntPtr((ulong)data.Length));
- return data;
- }
-
- public CallSafeHandle GetCall() {
- return grpcsharp_event_call(this);
- }
-
- public string GetServerRpcNewMethod() {
- // TODO: can the native method return string directly?
- return Marshal.PtrToStringAnsi(grpcsharp_event_server_rpc_new_method(this));
- }
-
- //TODO: client_metadata_read event type
-
- protected override bool ReleaseHandle()
- {
- grpcsharp_event_finish(handle);
- return true;
- }
- }
-
- // TODO: this is basically c&p of EventSafeHandle. Unify!
- /// <summary>
- /// Not owned version of
- /// grpc_event from grpc/grpc.h
- /// </summary>
- internal class EventSafeHandleNotOwned : SafeHandleZeroIsInvalid
- {
- [DllImport("grpc_csharp_ext.dll")]
- static extern void grpcsharp_event_finish(IntPtr ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern GRPCCompletionType grpcsharp_event_type(EventSafeHandleNotOwned ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern CallSafeHandle grpcsharp_event_call(EventSafeHandleNotOwned ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern GRPCOpError grpcsharp_event_write_accepted(EventSafeHandleNotOwned ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern GRPCOpError grpcsharp_event_finish_accepted(EventSafeHandleNotOwned ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern StatusCode grpcsharp_event_finished_status(EventSafeHandleNotOwned ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern IntPtr grpcsharp_event_finished_details(EventSafeHandleNotOwned ev); // returns const char*
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern IntPtr grpcsharp_event_read_length(EventSafeHandleNotOwned ev);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern void grpcsharp_event_read_copy_to_buffer(EventSafeHandleNotOwned ev, byte[] buffer, UIntPtr bufferLen);
-
- [DllImport("grpc_csharp_ext.dll")]
- static extern IntPtr grpcsharp_event_server_rpc_new_method(EventSafeHandleNotOwned ev); // returns const char*
-
- public EventSafeHandleNotOwned() : base(false)
- {
- }
-
- public EventSafeHandleNotOwned(IntPtr handle) : base(false)
- {
- SetHandle(handle);
- }
-
- public GRPCCompletionType GetCompletionType()
- {
- return grpcsharp_event_type(this);
- }
-
- public GRPCOpError GetWriteAccepted()
- {
- return grpcsharp_event_write_accepted(this);
- }
-
- public GRPCOpError GetFinishAccepted()
- {
- return grpcsharp_event_finish_accepted(this);
- }
-
- public Status GetFinished()
- {
- // TODO: can the native method return string directly?
- string details = Marshal.PtrToStringAnsi(grpcsharp_event_finished_details(this));
- return new Status(grpcsharp_event_finished_status(this), details);
- }
-
- public byte[] GetReadData()
- {
- IntPtr len = grpcsharp_event_read_length(this);
- if (len == new IntPtr(-1))
- {
- return null;
- }
- byte[] data = new byte[(int) len];
- grpcsharp_event_read_copy_to_buffer(this, data, new UIntPtr((ulong)data.Length));
- return data;
- }
-
- public CallSafeHandle GetCall() {
- return grpcsharp_event_call(this);
- }
-
- public string GetServerRpcNewMethod() {
- // TODO: can the native method return string directly?
- return Marshal.PtrToStringAnsi(grpcsharp_event_server_rpc_new_method(this));
- }
-
- //TODO: client_metadata_read event type
-
- protected override bool ReleaseHandle()
- {
- grpcsharp_event_finish(handle);
- return true;
- }
- }
-}
diff --git a/src/csharp/GrpcCore/Internal/GrpcThreadPool.cs b/src/csharp/GrpcCore/Internal/GrpcThreadPool.cs
index f8154fa250..634a0b2d72 100644
--- a/src/csharp/GrpcCore/Internal/GrpcThreadPool.cs
+++ b/src/csharp/GrpcCore/Internal/GrpcThreadPool.cs
@@ -48,7 +48,6 @@ namespace Google.GRPC.Core.Internal
readonly object myLock = new object();
readonly List<Thread> threads = new List<Thread>();
readonly int poolSize;
- readonly Action<EventSafeHandle> eventHandler;
CompletionQueueSafeHandle cq;
@@ -56,11 +55,6 @@ namespace Google.GRPC.Core.Internal
this.poolSize = poolSize;
}
- internal GrpcThreadPool(int poolSize, Action<EventSafeHandle> eventHandler) {
- this.poolSize = poolSize;
- this.eventHandler = eventHandler;
- }
-
public void Start() {
lock (myLock)
@@ -104,34 +98,19 @@ namespace Google.GRPC.Core.Internal
}
}
- private Thread CreateAndStartThread(int i) {
- Action body;
- if (eventHandler != null)
- {
- body = ThreadBodyWithHandler;
- }
- else
- {
- body = ThreadBodyNoHandler;
- }
- var thread = new Thread(new ThreadStart(body));
+ private Thread CreateAndStartThread(int i)
+ {
+ var thread = new Thread(new ThreadStart(RunHandlerLoop));
thread.IsBackground = false;
thread.Start();
- if (eventHandler != null)
- {
- thread.Name = "grpc_server_newrpc " + i;
- }
- else
- {
- thread.Name = "grpc " + i;
- }
+ thread.Name = "grpc " + i;
return thread;
}
/// <summary>
/// Body of the polling thread.
/// </summary>
- private void ThreadBodyNoHandler()
+ private void RunHandlerLoop()
{
GRPCCompletionType completionType;
do
@@ -140,22 +119,6 @@ namespace Google.GRPC.Core.Internal
} while(completionType != GRPCCompletionType.GRPC_QUEUE_SHUTDOWN);
Console.WriteLine("Completion queue has shutdown successfully, thread " + Thread.CurrentThread.Name + " exiting.");
}
-
- /// <summary>
- /// Body of the polling thread.
- /// </summary>
- private void ThreadBodyWithHandler()
- {
- GRPCCompletionType completionType;
- do
- {
- using (EventSafeHandle ev = cq.Next(Timespec.InfFuture)) {
- completionType = ev.GetCompletionType();
- eventHandler(ev);
- }
- } while(completionType != GRPCCompletionType.GRPC_QUEUE_SHUTDOWN);
- Console.WriteLine("Completion queue has shutdown successfully, thread " + Thread.CurrentThread.Name + " exiting.");
- }
}
}
diff --git a/src/csharp/GrpcCore/Internal/SafeHandleZeroIsInvalid.cs b/src/csharp/GrpcCore/Internal/SafeHandleZeroIsInvalid.cs
index 74a8ef7b6e..59f08d4ca8 100644
--- a/src/csharp/GrpcCore/Internal/SafeHandleZeroIsInvalid.cs
+++ b/src/csharp/GrpcCore/Internal/SafeHandleZeroIsInvalid.cs
@@ -56,6 +56,12 @@ namespace Google.GRPC.Core.Internal
return handle == IntPtr.Zero;
}
}
+
+ protected override bool ReleaseHandle()
+ {
+ // handle is not owned.
+ return true;
+ }
}
}
diff --git a/src/csharp/GrpcCore/Internal/ServerSafeHandle.cs b/src/csharp/GrpcCore/Internal/ServerSafeHandle.cs
index c91de97ce3..c096602800 100644
--- a/src/csharp/GrpcCore/Internal/ServerSafeHandle.cs
+++ b/src/csharp/GrpcCore/Internal/ServerSafeHandle.cs
@@ -38,13 +38,16 @@ using System.Collections.Concurrent;
namespace Google.GRPC.Core.Internal
{
+ // TODO: we need to make sure that the delegates are not collected before invoked.
+ internal delegate void ServerShutdownCallbackDelegate(IntPtr eventPtr);
+
/// <summary>
/// grpc_server from grpc/grpc.h
/// </summary>
internal sealed class ServerSafeHandle : SafeHandleZeroIsInvalid
{
- [DllImport("grpc_csharp_ext.dll", EntryPoint = "grpcsharp_server_request_call_old")]
- static extern GRPCCallError grpcsharp_server_request_call_old_CALLBACK(ServerSafeHandle server, [MarshalAs(UnmanagedType.FunctionPtr)] EventCallbackDelegate callback);
+ [DllImport("grpc_csharp_ext.dll")]
+ static extern GRPCCallError grpcsharp_server_request_call(ServerSafeHandle server, CompletionQueueSafeHandle cq, [MarshalAs(UnmanagedType.FunctionPtr)] CompletionCallbackDelegate callback);
[DllImport("grpc_csharp_ext.dll")]
static extern ServerSafeHandle grpcsharp_server_create(CompletionQueueSafeHandle cq, IntPtr args);
@@ -63,8 +66,9 @@ namespace Google.GRPC.Core.Internal
[DllImport("grpc_csharp_ext.dll")]
static extern void grpcsharp_server_shutdown(ServerSafeHandle server);
+ // TODO: get rid of the old callback style
[DllImport("grpc_csharp_ext.dll", EntryPoint = "grpcsharp_server_shutdown_and_notify")]
- static extern void grpcsharp_server_shutdown_and_notify_CALLBACK(ServerSafeHandle server, [MarshalAs(UnmanagedType.FunctionPtr)] EventCallbackDelegate callback);
+ static extern void grpcsharp_server_shutdown_and_notify_CALLBACK(ServerSafeHandle server, [MarshalAs(UnmanagedType.FunctionPtr)] ServerShutdownCallbackDelegate callback);
[DllImport("grpc_csharp_ext.dll")]
static extern void grpcsharp_server_destroy(IntPtr server);
@@ -95,14 +99,14 @@ namespace Google.GRPC.Core.Internal
grpcsharp_server_shutdown(this);
}
- public void ShutdownAndNotify(EventCallbackDelegate callback)
+ public void ShutdownAndNotify(ServerShutdownCallbackDelegate callback)
{
grpcsharp_server_shutdown_and_notify_CALLBACK(this, callback);
}
- public GRPCCallError RequestCall(EventCallbackDelegate callback)
+ public GRPCCallError RequestCall(CompletionQueueSafeHandle cq, CompletionCallbackDelegate callback)
{
- return grpcsharp_server_request_call_old_CALLBACK(this, callback);
+ return grpcsharp_server_request_call(this, cq, callback);
}
protected override bool ReleaseHandle()
diff --git a/src/csharp/GrpcCore/Internal/ServerWritingObserver.cs b/src/csharp/GrpcCore/Internal/ServerStreamingOutputObserver.cs
index 1d29864b9f..e9cb65cb3b 100644
--- a/src/csharp/GrpcCore/Internal/ServerWritingObserver.cs
+++ b/src/csharp/GrpcCore/Internal/ServerStreamingOutputObserver.cs
@@ -40,11 +40,11 @@ namespace Google.GRPC.Core.Internal
/// Observer that writes all arriving messages to a call abstraction (in blocking fashion)
/// and then halfcloses the call. Used for server-side call handling.
/// </summary>
- internal class ServerWritingObserver<TWrite, TRead> : IObserver<TWrite>
+ internal class ServerStreamingOutputObserver<TWrite, TRead> : IObserver<TWrite>
{
readonly AsyncCall<TWrite, TRead> call;
- public ServerWritingObserver(AsyncCall<TWrite, TRead> call)
+ public ServerStreamingOutputObserver(AsyncCall<TWrite, TRead> call)
{
this.call = call;
}
@@ -52,19 +52,19 @@ namespace Google.GRPC.Core.Internal
public void OnCompleted()
{
// TODO: how bad is the Wait here?
- call.WriteStatusAsync(new Status(StatusCode.GRPC_STATUS_OK, "")).Wait();
+ call.SendStatusFromServerAsync(new Status(StatusCode.GRPC_STATUS_OK, "")).Wait();
}
public void OnError(Exception error)
{
- // TODO: handle this...
+ // TODO: implement this...
throw new InvalidOperationException("This should never be called.");
}
public void OnNext(TWrite value)
{
// TODO: how bad is the Wait here?
- call.WriteAsync(value).Wait();
+ call.SendMessageAsync(value).Wait();
}
}
}
diff --git a/src/csharp/GrpcCore/Server.cs b/src/csharp/GrpcCore/Server.cs
index 0882a61299..91842d8182 100644
--- a/src/csharp/GrpcCore/Server.cs
+++ b/src/csharp/GrpcCore/Server.cs
@@ -49,8 +49,8 @@ namespace Google.GRPC.Core
{
// TODO: make sure the delegate doesn't get garbage collected while
// native callbacks are in the completion queue.
- readonly EventCallbackDelegate newRpcHandler;
- readonly EventCallbackDelegate serverShutdownHandler;
+ readonly ServerShutdownCallbackDelegate serverShutdownHandler;
+ readonly CompletionCallbackDelegate newServerRpcHandler;
readonly BlockingCollection<NewRpcInfo> newRpcQueue = new BlockingCollection<NewRpcInfo>();
readonly ServerSafeHandle handle;
@@ -61,9 +61,8 @@ namespace Google.GRPC.Core
public Server()
{
- // TODO: what is the tag for server shutdown?
this.handle = ServerSafeHandle.NewServer(GetCompletionQueue(), IntPtr.Zero);
- this.newRpcHandler = HandleNewRpc;
+ this.newServerRpcHandler = HandleNewServerRpc;
this.serverShutdownHandler = HandleServerShutdown;
}
@@ -99,7 +98,7 @@ namespace Google.GRPC.Core
{
var rpcInfo = newRpcQueue.Take();
- Console.WriteLine("Server received RPC " + rpcInfo.Method);
+ //Console.WriteLine("Server received RPC " + rpcInfo.Method);
IServerCallHandler callHandler;
if (!callHandlers.TryGetValue(rpcInfo.Method, out callHandler))
@@ -138,23 +137,25 @@ namespace Google.GRPC.Core
private void AllowOneRpc()
{
- AssertCallOk(handle.RequestCall(newRpcHandler));
+ AssertCallOk(handle.RequestCall(GetCompletionQueue(), newServerRpcHandler));
}
- private void HandleNewRpc(IntPtr eventPtr)
- {
- try
- {
- var ev = new EventSafeHandleNotOwned(eventPtr);
- var rpcInfo = new NewRpcInfo(ev.GetCall(), ev.GetServerRpcNewMethod());
+ private void HandleNewServerRpc(GRPCOpError error, IntPtr batchContextPtr) {
+ try {
+ var ctx = new BatchContextSafeHandleNotOwned(batchContextPtr);
+
+ if (error != GRPCOpError.GRPC_OP_OK) {
+ // TODO: handle error
+ }
+
+ var rpcInfo = new NewRpcInfo(ctx.GetServerRpcNewCall(), ctx.GetServerRpcNewMethod());
// after server shutdown, the callback returns with null call
if (!rpcInfo.Call.IsInvalid) {
newRpcQueue.Add(rpcInfo);
}
- }
- catch (Exception e)
- {
+
+ } catch(Exception e) {
Console.WriteLine("Caught exception in a native handler: " + e);
}
}
diff --git a/src/csharp/GrpcCore/ServerCallHandler.cs b/src/csharp/GrpcCore/ServerCallHandler.cs
index bcce4a091f..3bc3b15396 100644
--- a/src/csharp/GrpcCore/ServerCallHandler.cs
+++ b/src/csharp/GrpcCore/ServerCallHandler.cs
@@ -59,15 +59,16 @@ namespace Google.GRPC.Core
method.RequestMarshaller.Deserializer);
asyncCall.InitializeServer(call);
- asyncCall.Accept(cq);
+
+ var finishedTask = asyncCall.StartServerSide();
- var request = asyncCall.ReadAsync().Result;
+ var request = asyncCall.ReceiveMessageAsync().Result;
- var responseObserver = new ServerWritingObserver<TResponse, TRequest>(asyncCall);
+ var responseObserver = new ServerStreamingOutputObserver<TResponse, TRequest>(asyncCall);
handler(request, responseObserver);
- asyncCall.Halfclosed.Wait();
- asyncCall.Finished.Wait();
+ finishedTask.Wait();
+
}
}
@@ -89,16 +90,16 @@ namespace Google.GRPC.Core
method.RequestMarshaller.Deserializer);
asyncCall.InitializeServer(call);
- asyncCall.Accept(cq);
- var responseObserver = new ServerWritingObserver<TResponse, TRequest>(asyncCall);
+ var finishedTask = asyncCall.StartServerSide();
+
+ var responseObserver = new ServerStreamingOutputObserver<TResponse, TRequest>(asyncCall);
var requestObserver = handler(responseObserver);
// feed the requests
asyncCall.StartReadingToStream(requestObserver);
- asyncCall.Halfclosed.Wait();
- asyncCall.Finished.Wait();
+ finishedTask.Wait();
}
}
@@ -110,11 +111,14 @@ namespace Google.GRPC.Core
AsyncCall<byte[], byte[]> asyncCall = new AsyncCall<byte[], byte[]>(
(payload) => payload, (payload) => payload);
+
asyncCall.InitializeServer(call);
- asyncCall.Accept(cq);
- asyncCall.WriteStatusAsync(new Status(StatusCode.GRPC_STATUS_UNIMPLEMENTED, "No such method.")).Wait();
- asyncCall.Finished.Wait();
+ var finishedTask = asyncCall.StartServerSide();
+
+ asyncCall.SendStatusFromServerAsync(new Status(StatusCode.GRPC_STATUS_UNIMPLEMENTED, "No such method.")).Wait();
+
+ finishedTask.Wait();
}
}
}