Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why is beforeStart called selectively?

Copy link
Copy Markdown
Contributor Author

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.

} else {
LOG.warn("{} does not implement ClientResponseObserver:beforeStart", mObserver);
}
}

@Override
public DataMessageMarshaller<RespT> getMarshaller() {
return mMarshaller;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,12 @@
import alluxio.grpc.BlockWorkerGrpc;
import alluxio.grpc.CreateLocalBlockRequest;
import alluxio.grpc.CreateLocalBlockResponse;
import alluxio.grpc.DataMessageMarshallerProvider;
import alluxio.grpc.GrpcChannel;
import alluxio.grpc.GrpcChannelBuilder;
import alluxio.grpc.GrpcExceptionUtils;
import alluxio.grpc.GrpcManagedChannelPool;
import alluxio.grpc.DataMessageMarshaller;
import alluxio.grpc.OpenLocalBlockRequest;
import alluxio.grpc.OpenLocalBlockResponse;
import alluxio.grpc.ReadRequest;
Expand All @@ -32,6 +34,7 @@
import alluxio.grpc.RemoveBlockResponse;
import alluxio.grpc.WriteRequest;
import alluxio.grpc.WriteResponse;
import alluxio.grpc.GrpcSerializationUtils;
import alluxio.util.network.NettyUtils;

import com.google.common.io.Closer;
Expand Down Expand Up @@ -79,6 +82,7 @@ public DefaultBlockWorkerClient(Subject subject, SocketAddress address,
// Channel is still reused due to client pooling.
mStreamingChannel = buildChannel(subject, address,
GrpcManagedChannelPool.PoolingStrategy.DISABLED, alluxioConf, workerGroup);
mStreamingChannel.intercept(new StreamSerializationClientInterceptor());
// Uses default pooling strategy for RPC calls for better scalability.
mRpcChannel = buildChannel(subject, address,
GrpcManagedChannelPool.PoolingStrategy.DEFAULT, alluxioConf, workerGroup);
Expand Down Expand Up @@ -117,7 +121,18 @@ public StreamObserver<WriteRequest> writeBlock(StreamObserver<WriteResponse> res

@Override
public StreamObserver<ReadRequest> readBlock(StreamObserver<ReadResponse> responseObserver) {
return mStreamingAsyncStub.readBlock(responseObserver);
if (responseObserver instanceof DataMessageMarshallerProvider) {
DataMessageMarshaller<ReadResponse> marshaller =
((DataMessageMarshallerProvider<ReadResponse>) responseObserver).getMarshaller();
return mStreamingAsyncStub
.withOption(GrpcSerializationUtils.OVERRIDDEN_METHOD_DESCRIPTOR,
BlockWorkerGrpc.getReadBlockMethod().toBuilder()
.setResponseMarshaller(marshaller)
.build())
.readBlock(responseObserver);
} else {
return mStreamingAsyncStub.readBlock(responseObserver);
}
}

@Override
Expand Down
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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,18 +14,20 @@
import alluxio.client.file.FileSystemContext;
import alluxio.conf.AlluxioConfiguration;
import alluxio.conf.PropertyKey;
import alluxio.grpc.DataMessage;
import alluxio.grpc.ReadRequest;
import alluxio.grpc.ReadResponse;
import alluxio.grpc.ReadResponseMarshaller;
import alluxio.network.protocol.databuffer.DataBuffer;
import alluxio.network.protocol.databuffer.NioDataBuffer;
import alluxio.wire.WorkerNetAddress;

import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.nio.ByteBuffer;

import javax.annotation.concurrent.NotThreadSafe;

Expand Down Expand Up @@ -53,6 +55,7 @@ public final class GrpcDataReader implements DataReader {
private final WorkerNetAddress mAddress;

private final GrpcBlockingStream<ReadRequest, ReadResponse> mStream;
private final ReadResponseMarshaller mMarshaller;

/** The next pos to read. */
private long mPosToRead;
Expand All @@ -76,9 +79,15 @@ private GrpcDataReader(FileSystemContext context, WorkerNetAddress address,
mDataTimeoutMs = alluxioConf.getMs(PropertyKey.USER_NETWORK_DATA_TIMEOUT_MS);

mClient = mContext.acquireBlockWorkerClient(address);
mMarshaller = new ReadResponseMarshaller();
try {
mStream = new GrpcBlockingStream<>(mClient::readBlock, mReaderBufferSizeMessages,
address.toString());
if (alluxioConf.getBoolean(PropertyKey.USER_NETWORK_ZEROCOPY_ENABLED)) {
mStream = new GrpcDataMessageBlockingStream<>(mClient::readBlock, mReaderBufferSizeMessages,
address.toString(), mMarshaller);
} else {
mStream = new GrpcBlockingStream<>(mClient::readBlock, mReaderBufferSizeMessages,
address.toString());
}
mStream.send(mReadRequest, mDataTimeoutMs);
} catch (Exception e) {
mContext.releaseBlockWorkerClient(address, mClient);
Expand All @@ -95,16 +104,32 @@ public long pos() {
public DataBuffer readChunk() throws IOException {
Preconditions.checkState(!mClient.isShutdown(),
"Data reader is closed while reading data chunks.");
ByteString buf;
ReadResponse response = mStream.receive(mDataTimeoutMs);
DataBuffer buffer = null;
ReadResponse response = null;
if (mStream instanceof GrpcDataMessageBlockingStream) {
DataMessage<ReadResponse, DataBuffer> message =
((GrpcDataMessageBlockingStream<ReadRequest, ReadResponse>) mStream)
.receiveDataMessage(mDataTimeoutMs);
if (message != null) {
response = message.getMessage();
buffer = message.getBuffer();
Preconditions.checkState(buffer != null, "response should always contain chunk");
}
} else {
response = mStream.receive(mDataTimeoutMs);
if (response != null) {
Preconditions.checkState(response.hasChunk() && response.getChunk().hasData(),
"response should always contain chunk");
ByteBuffer byteBuffer = response.getChunk().getData().asReadOnlyByteBuffer();
buffer = new NioDataBuffer(byteBuffer, byteBuffer.remaining());
}
}
if (response == null) {
return null;
}
Preconditions.checkState(response.hasChunk(), "response should always contain chunk");
buf = response.getChunk().getData();
mPosToRead += buf.size();
mPosToRead += buffer.readableBytes();
Preconditions.checkState(mPosToRead - mReadRequest.getOffset() <= mReadRequest.getLength());
return new NioDataBuffer(buf.asReadOnlyByteBuffer(), buf.size());
return buffer;
}

@Override
Expand All @@ -116,6 +141,7 @@ public void close() throws IOException {
mStream.close();
mStream.waitForComplete(mDataTimeoutMs);
} finally {
mMarshaller.close();
mContext.releaseBlockWorkerClient(mAddress, mClient);
}
}
Expand Down
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);
}
}
Loading