-
Notifications
You must be signed in to change notification settings - Fork 2.9k
Implement zero copy for gRPC read path #8353
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
78 changes: 78 additions & 0 deletions
78
...lient/fs/src/main/java/alluxio/client/block/stream/DataMessageClientResponseObserver.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,78 @@ | ||
| /* | ||
| * The Alluxio Open Foundation licenses this work under the Apache License, version 2.0 | ||
| * (the "License"). You may not use this work except in compliance with the License, which is | ||
| * available at www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * This software is distributed on an "AS IS" basis, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, | ||
| * either express or implied, as more fully set forth in the License. | ||
| * | ||
| * See the NOTICE file distributed with this work for information regarding copyright ownership. | ||
| */ | ||
|
|
||
| package alluxio.client.block.stream; | ||
|
|
||
| import alluxio.grpc.DataMessageMarshaller; | ||
| import alluxio.grpc.DataMessageMarshallerProvider; | ||
|
|
||
| import io.grpc.stub.ClientCallStreamObserver; | ||
| import io.grpc.stub.ClientResponseObserver; | ||
| import io.grpc.stub.StreamObserver; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import javax.annotation.concurrent.NotThreadSafe; | ||
|
|
||
| /** | ||
| * A {@link StreamObserver} that handles raw data buffers. | ||
| * | ||
| * @param <RespT> type of the response message | ||
| * @param <ReqT> type of the request message | ||
| */ | ||
| @NotThreadSafe | ||
| public class DataMessageClientResponseObserver<ReqT, RespT> | ||
| implements ClientResponseObserver<ReqT, RespT>, DataMessageMarshallerProvider<RespT> { | ||
| private static final Logger LOG = | ||
| LoggerFactory.getLogger(DataMessageClientResponseObserver.class); | ||
|
|
||
| private final StreamObserver<RespT> mObserver; | ||
| private final DataMessageMarshaller<RespT> mMarshaller; | ||
|
|
||
| /** | ||
| * @param observer the original response observer | ||
| * @param marshaller the marshaller for the response | ||
| */ | ||
| public DataMessageClientResponseObserver(StreamObserver<RespT> observer, | ||
| DataMessageMarshaller<RespT> marshaller) { | ||
| mObserver = observer; | ||
| mMarshaller = marshaller; | ||
| } | ||
|
|
||
| @Override | ||
| public void onNext(RespT value) { | ||
| mObserver.onNext(value); | ||
| } | ||
|
|
||
| @Override | ||
| public void onError(Throwable t) { | ||
| mObserver.onError(t); | ||
| } | ||
|
|
||
| @Override | ||
| public void onCompleted() { | ||
| mObserver.onCompleted(); | ||
| } | ||
|
|
||
| @Override | ||
| public void beforeStart(ClientCallStreamObserver<ReqT> requestStream) { | ||
| if (mObserver instanceof ClientResponseObserver) { | ||
| ((ClientResponseObserver<ReqT, RespT>) mObserver).beforeStart(requestStream); | ||
| } else { | ||
| LOG.warn("{} does not implement ClientResponseObserver:beforeStart", mObserver); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public DataMessageMarshaller<RespT> getMarshaller() { | ||
| return mMarshaller; | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
91 changes: 91 additions & 0 deletions
91
core/client/fs/src/main/java/alluxio/client/block/stream/GrpcDataMessageBlockingStream.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,91 @@ | ||
| /* | ||
| * The Alluxio Open Foundation licenses this work under the Apache License, version 2.0 | ||
| * (the "License"). You may not use this work except in compliance with the License, which is | ||
| * available at www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * This software is distributed on an "AS IS" basis, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, | ||
| * either express or implied, as more fully set forth in the License. | ||
| * | ||
| * See the NOTICE file distributed with this work for information regarding copyright ownership. | ||
| */ | ||
|
|
||
| package alluxio.client.block.stream; | ||
|
|
||
| import alluxio.exception.status.DeadlineExceededException; | ||
| import alluxio.grpc.DataMessage; | ||
| import alluxio.grpc.DataMessageMarshaller; | ||
| import alluxio.network.protocol.databuffer.DataBuffer; | ||
|
|
||
| import io.grpc.stub.StreamObserver; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.function.Function; | ||
|
|
||
| import javax.annotation.concurrent.NotThreadSafe; | ||
|
|
||
| /** | ||
| * A helper class for accessing gRPC bi-directional stream synchronously. | ||
| * | ||
| * @param <ReqT> type of the request | ||
| * @param <ResT> type of the response | ||
| */ | ||
| @NotThreadSafe | ||
| public class GrpcDataMessageBlockingStream<ReqT, ResT> extends GrpcBlockingStream<ReqT, ResT> { | ||
| private final DataMessageMarshaller<ResT> mMarshaller; | ||
|
|
||
| /** | ||
| * @param rpcFunc the gRPC bi-directional stream stub function | ||
| * @param bufferSize maximum number of incoming messages the buffer can hold | ||
| * @param description description of this stream | ||
| * @param deserializer custom deserializer for the response | ||
| */ | ||
| public GrpcDataMessageBlockingStream(Function<StreamObserver<ResT>, StreamObserver<ReqT>> rpcFunc, | ||
| int bufferSize, String description, DataMessageMarshaller<ResT> deserializer) { | ||
| super((resObserver) -> { | ||
| DataMessageClientResponseObserver<ReqT, ResT> newObserver = | ||
| new DataMessageClientResponseObserver<>(resObserver, deserializer); | ||
| StreamObserver<ReqT> requestObserver = rpcFunc.apply(newObserver); | ||
| return requestObserver; | ||
| }, bufferSize, description); | ||
| mMarshaller = deserializer; | ||
| } | ||
|
|
||
| @Override | ||
| public ResT receive(long timeoutMs) throws IOException { | ||
| DataMessage<ResT, DataBuffer> message = receiveDataMessage(timeoutMs); | ||
| if (message == null) { | ||
| return null; | ||
| } | ||
| return mMarshaller.combineData(message); | ||
| } | ||
|
|
||
| /** | ||
| * Receives a response with data buffer from the server. Will wait until a response is received, | ||
| * or throw an exception if times out. Caller of this method must release the buffer after reading | ||
| * the data. | ||
| * | ||
| * @param timeoutMs maximum time to wait before giving up and throwing | ||
| * a {@link DeadlineExceededException} | ||
| * @return the response message with data buffer, or null if the inbound stream is completed | ||
| * @throws IOException if any error occurs | ||
| */ | ||
| public DataMessage<ResT, DataBuffer> receiveDataMessage(long timeoutMs) throws IOException { | ||
| ResT response = super.receive(timeoutMs); | ||
| if (response == null) { | ||
| return null; | ||
| } | ||
| DataBuffer buffer = mMarshaller.pollBuffer(response); | ||
| return new DataMessage<>(response, buffer); | ||
| } | ||
|
|
||
| @Override | ||
| public void waitForComplete(long timeoutMs) throws IOException { | ||
| DataMessage<ResT, DataBuffer> message; | ||
| while (!isCanceled() && (message = receiveDataMessage(timeoutMs)) != null) { | ||
| if (message.getBuffer() != null) { | ||
| message.getBuffer().release(); | ||
| } | ||
| } | ||
| super.waitForComplete(timeoutMs); | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
36 changes: 36 additions & 0 deletions
36
...nt/fs/src/main/java/alluxio/client/block/stream/StreamSerializationClientInterceptor.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,36 @@ | ||
| /* | ||
| * The Alluxio Open Foundation licenses this work under the Apache License, version 2.0 | ||
| * (the "License"). You may not use this work except in compliance with the License, which is | ||
| * available at www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * This software is distributed on an "AS IS" basis, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, | ||
| * either express or implied, as more fully set forth in the License. | ||
| * | ||
| * See the NOTICE file distributed with this work for information regarding copyright ownership. | ||
| */ | ||
|
|
||
| package alluxio.client.block.stream; | ||
|
|
||
| import alluxio.grpc.GrpcSerializationUtils; | ||
|
|
||
| import io.grpc.CallOptions; | ||
| import io.grpc.Channel; | ||
| import io.grpc.ClientCall; | ||
| import io.grpc.ClientInterceptor; | ||
| import io.grpc.MethodDescriptor; | ||
|
|
||
| /** | ||
| * Serialization interceptor for client. | ||
| */ | ||
| public class StreamSerializationClientInterceptor implements ClientInterceptor { | ||
| @Override | ||
| public <ReqT, RespT> ClientCall<ReqT, RespT> interceptCall( | ||
| MethodDescriptor<ReqT, RespT> method, CallOptions callOptions, Channel channel) { | ||
| MethodDescriptor<ReqT, RespT> overriddenMethodDescriptor = | ||
| callOptions.getOption(GrpcSerializationUtils.OVERRIDDEN_METHOD_DESCRIPTOR); | ||
| if (overriddenMethodDescriptor != null) { | ||
| method = overriddenMethodDescriptor; | ||
| } | ||
| return channel.newCall(method, callOptions); | ||
| } | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why is
beforeStartcalled selectively?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is just to make sure the observer provided actually implemented the interface for
beforeStart(). I can put a warning message if it is not implemented.