-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-6237][NETWORK] Network-layer changes to allow stream upload. #21346
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
Changes from 7 commits
5683b10
3098b9c
9dddf06
2fef75f
54533c2
32f4f94
331124b
f4d9123
7bd1b43
83c3271
6c086c5
d357885
93a5adf
25e48f5
cf991a9
8a18da5
1a222aa
ea4a1f5
fd62f61
58d52b9
cd11abc
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -32,15 +32,13 @@ | |
| import com.google.common.base.Throwables; | ||
| import com.google.common.util.concurrent.SettableFuture; | ||
| import io.netty.channel.Channel; | ||
| import org.apache.spark.network.buffer.ManagedBuffer; | ||
| import org.apache.spark.network.protocol.*; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import org.apache.spark.network.buffer.NioManagedBuffer; | ||
| import org.apache.spark.network.protocol.ChunkFetchRequest; | ||
| import org.apache.spark.network.protocol.OneWayMessage; | ||
| import org.apache.spark.network.protocol.RpcRequest; | ||
| import org.apache.spark.network.protocol.StreamChunkId; | ||
| import org.apache.spark.network.protocol.StreamRequest; | ||
|
|
||
| import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; | ||
|
|
||
| /** | ||
|
|
@@ -244,6 +242,54 @@ public long sendRpc(ByteBuffer message, RpcResponseCallback callback) { | |
| return requestId; | ||
| } | ||
|
|
||
| /** | ||
| * Send data to the remote end as a stream. This differs from stream() in that this is a request | ||
|
||
| * to *send* data to the remote end, not to receive it from the remote. | ||
| * | ||
| * @param meta meta data associated with the stream, which will be read completely on the | ||
| * receiving end before the stream itself. | ||
| * @param data this will be streamed to the remote end to allow for transferring large amounts | ||
| * of data without reading into memory. | ||
| * @param callback handles the reply -- onSuccess will only be called when both message and data | ||
| * are received successfully. | ||
| */ | ||
| public long uploadStream( | ||
| ManagedBuffer meta, | ||
| ManagedBuffer data, | ||
| RpcResponseCallback callback) { | ||
| long startTime = System.currentTimeMillis(); | ||
|
||
| if (logger.isTraceEnabled()) { | ||
| logger.trace("Sending RPC to {}", getRemoteAddress(channel)); | ||
| } | ||
|
|
||
| long requestId = Math.abs(UUID.randomUUID().getLeastSignificantBits()); | ||
|
||
| handler.addRpcRequest(requestId, callback); | ||
|
|
||
| channel.writeAndFlush(new UploadStream(requestId, meta, data)) | ||
| .addListener(future -> { | ||
| if (future.isSuccess()) { | ||
|
||
| long timeTaken = System.currentTimeMillis() - startTime; | ||
| if (logger.isTraceEnabled()) { | ||
| logger.trace("Sending request {} to {} took {} ms", requestId, | ||
| getRemoteAddress(channel), timeTaken); | ||
| } | ||
| } else { | ||
| String errorMsg = String.format("Failed to send RPC %s to %s: %s", requestId, | ||
| getRemoteAddress(channel), future.cause()); | ||
| logger.error(errorMsg, future.cause()); | ||
| handler.removeRpcRequest(requestId); | ||
| channel.close(); | ||
| try { | ||
| callback.onFailure(new IOException(errorMsg, future.cause())); | ||
| } catch (Exception e) { | ||
| logger.error("Uncaught exception in RPC response callback handler!", e); | ||
| } | ||
| } | ||
| }); | ||
|
|
||
| return requestId; | ||
| } | ||
|
|
||
| /** | ||
| * Synchronously sends an opaque message to the RpcHandler on the server-side, waiting for up to | ||
| * a specified timeout for a response. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,107 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.network.protocol; | ||
|
|
||
| import java.io.IOException; | ||
| import java.nio.ByteBuffer; | ||
|
|
||
| import com.google.common.base.Objects; | ||
| import io.netty.buffer.ByteBuf; | ||
|
|
||
| import org.apache.spark.network.buffer.ManagedBuffer; | ||
| import org.apache.spark.network.buffer.NettyManagedBuffer; | ||
|
|
||
| /** | ||
| * An RPC with data that is sent outside of the frame, so it can be read in a stream. | ||
|
||
| */ | ||
| public final class UploadStream extends AbstractMessage implements RequestMessage { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is it possible to merge UploadStream and RpcRequest into a class?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. perhaps, but do you think that is really that useful? the handling of them is different (both in the network layer and the outer RpcHandler). And other things being equal, I'm biased to fewer changes to existing code paths. |
||
| /** Used to link an RPC request with its response. */ | ||
| public final long requestId; | ||
| public final ManagedBuffer meta; | ||
| public final long bodyByteCount; | ||
|
|
||
| public UploadStream(long requestId, ManagedBuffer meta, ManagedBuffer body) { | ||
| super(body, false); // body is *not* included in the frame | ||
| this.requestId = requestId; | ||
| this.meta = meta; | ||
| bodyByteCount = body.size(); | ||
| } | ||
|
|
||
| // this version is called when decoding the bytes on the receiving end. The body is handled | ||
| // separately. | ||
| private UploadStream(long requestId, ManagedBuffer meta, long bodyByteCount) { | ||
| super(null, false); | ||
| this.requestId = requestId; | ||
| this.meta = meta; | ||
| this.bodyByteCount = bodyByteCount; | ||
| } | ||
|
|
||
| @Override | ||
| public Type type() { return Type.UploadStream; } | ||
|
|
||
| @Override | ||
| public int encodedLength() { | ||
| // the requestId, meta size, meta and bodyByteCount (body is not included) | ||
| return 8 + 4 + ((int) meta.size()) + 8; | ||
| } | ||
|
|
||
| @Override | ||
| public void encode(ByteBuf buf) { | ||
| buf.writeLong(requestId); | ||
| try { | ||
| ByteBuffer metaBuf = meta.nioByteBuffer(); | ||
| buf.writeInt(metaBuf.remaining()); | ||
| buf.writeBytes(metaBuf); | ||
| } catch (IOException io) { | ||
| throw new RuntimeException(io); | ||
| } | ||
| buf.writeLong(bodyByteCount); | ||
| } | ||
|
|
||
| public static UploadStream decode(ByteBuf buf) { | ||
| long requestId = buf.readLong(); | ||
| int metaSize = buf.readInt(); | ||
| ManagedBuffer meta = new NettyManagedBuffer(buf.readRetainedSlice(metaSize)); | ||
| long bodyByteCount = buf.readLong(); | ||
| // This is called by the frame decoder, so the data is still null. We need a StreamInterceptor | ||
| // to read the data. | ||
| return new UploadStream(requestId, meta, bodyByteCount); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return Objects.hashCode(requestId, body()); | ||
|
||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object other) { | ||
| if (other instanceof UploadStream) { | ||
| UploadStream o = (UploadStream) other; | ||
| return requestId == o.requestId && super.equals(o); | ||
| } | ||
| return false; | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return Objects.toStringHelper(this) | ||
| .add("requestId", requestId) | ||
| .add("body", body()) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Similar question here about whether
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. to be honest, this was also just parroted from other classes -- looking now at implementations of ManagedBuffer, if they have a Is that actually useful for debugging? maybe not, don't think I ever actually looked at this.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not actually sure. I wonder if this is a latent problem in the old code waiting to happen in case we turn on trace logging. We can probably investigate that separately, but just wanted to note it since it seemed a little dodgy. |
||
| .toString(); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,15 +38,24 @@ public abstract class RpcHandler { | |
| * | ||
| * This method will not be called in parallel for a single TransportClient (i.e., channel). | ||
| * | ||
| * The rpc *might* included a data stream in <code>streamData</code>(eg. for uploading a large | ||
|
||
| * amount of data which should not be buffered in memory here). Any errors while handling the | ||
| * streamData will lead to failing this entire connection -- all other in-flight rpcs will fail. | ||
|
||
| * If stream data is not null, you *must* call <code>streamData.registerStreamCallback</code> | ||
| * before this method returns. | ||
| * | ||
| * @param client A channel client which enables the handler to make requests back to the sender | ||
| * of this RPC. This will always be the exact same object for a particular channel. | ||
| * @param message The serialized bytes of the RPC. | ||
| * @param streamData StreamData if there is data which is meant to be read via a StreamCallback; | ||
|
||
| * otherwise it is null. | ||
| * @param callback Callback which should be invoked exactly once upon success or failure of the | ||
| * RPC. | ||
| */ | ||
| public abstract void receive( | ||
| TransportClient client, | ||
| ByteBuffer message, | ||
| StreamData streamData, | ||
|
||
| RpcResponseCallback callback); | ||
|
|
||
| /** | ||
|
|
@@ -57,15 +66,15 @@ public abstract void receive( | |
|
|
||
| /** | ||
| * Receives an RPC message that does not expect a reply. The default implementation will | ||
| * call "{@link #receive(TransportClient, ByteBuffer, RpcResponseCallback)}" and log a warning if | ||
| * any of the callback methods are called. | ||
| * call "{@link #receive(TransportClient, ByteBuffer, StreamData, RpcResponseCallback)}" and log a | ||
| * warning if any of the callback methods are called. | ||
| * | ||
| * @param client A channel client which enables the handler to make requests back to the sender | ||
| * of this RPC. This will always be the exact same object for a particular channel. | ||
| * @param message The serialized bytes of the RPC. | ||
| */ | ||
| public void receive(TransportClient client, ByteBuffer message) { | ||
| receive(client, message, ONE_WAY_CALLBACK); | ||
| receive(client, message, null, ONE_WAY_CALLBACK); | ||
| } | ||
|
|
||
| /** | ||
|
|
||
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 don't we need to do this for
TransportRequestHandler?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.
the only purpose of
TransportResponseHandler.deactivateStream()is to include the stream request in the count fornumOutstandingRequests(its not doing any critical cleanup). I will include a comment here explaining that.