-
Notifications
You must be signed in to change notification settings - Fork 28.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge branch 'apache:master' into SPARK-45540
- Loading branch information
Showing
68 changed files
with
4,106 additions
and
283 deletions.
There are no files selected for viewing
This file contains 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
148 changes: 148 additions & 0 deletions
148
...rk-common/src/main/java/org/apache/spark/network/protocol/EncryptedMessageWithHeader.java
This file contains 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,148 @@ | ||
|
||
/* | ||
* 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.EOFException; | ||
import java.io.InputStream; | ||
import javax.annotation.Nullable; | ||
|
||
import com.google.common.base.Preconditions; | ||
import io.netty.buffer.ByteBuf; | ||
import io.netty.buffer.ByteBufAllocator; | ||
import io.netty.channel.ChannelHandlerContext; | ||
import io.netty.handler.stream.ChunkedStream; | ||
import io.netty.handler.stream.ChunkedInput; | ||
|
||
import org.apache.spark.network.buffer.ManagedBuffer; | ||
|
||
/** | ||
* A wrapper message that holds two separate pieces (a header and a body). | ||
* | ||
* The header must be a ByteBuf, while the body can be any InputStream or ChunkedStream | ||
*/ | ||
public class EncryptedMessageWithHeader implements ChunkedInput<ByteBuf> { | ||
|
||
@Nullable private final ManagedBuffer managedBuffer; | ||
private final ByteBuf header; | ||
private final int headerLength; | ||
private final Object body; | ||
private final long bodyLength; | ||
private long totalBytesTransferred; | ||
|
||
/** | ||
* Construct a new EncryptedMessageWithHeader. | ||
* | ||
* @param managedBuffer the {@link ManagedBuffer} that the message body came from. This needs to | ||
* be passed in so that the buffer can be freed when this message is | ||
* deallocated. Ownership of the caller's reference to this buffer is | ||
* transferred to this class, so if the caller wants to continue to use the | ||
* ManagedBuffer in other messages then they will need to call retain() on | ||
* it before passing it to this constructor. | ||
* @param header the message header. | ||
* @param body the message body. | ||
* @param bodyLength the length of the message body, in bytes. | ||
*/ | ||
|
||
public EncryptedMessageWithHeader( | ||
@Nullable ManagedBuffer managedBuffer, ByteBuf header, Object body, long bodyLength) { | ||
Preconditions.checkArgument(body instanceof InputStream || body instanceof ChunkedStream, | ||
"Body must be an InputStream or a ChunkedStream."); | ||
this.managedBuffer = managedBuffer; | ||
this.header = header; | ||
this.headerLength = header.readableBytes(); | ||
this.body = body; | ||
this.bodyLength = bodyLength; | ||
this.totalBytesTransferred = 0; | ||
} | ||
|
||
@Override | ||
public ByteBuf readChunk(ChannelHandlerContext ctx) throws Exception { | ||
return readChunk(ctx.alloc()); | ||
} | ||
|
||
@Override | ||
public ByteBuf readChunk(ByteBufAllocator allocator) throws Exception { | ||
if (isEndOfInput()) { | ||
return null; | ||
} | ||
|
||
if (totalBytesTransferred < headerLength) { | ||
totalBytesTransferred += headerLength; | ||
return header.retain(); | ||
} else if (body instanceof InputStream) { | ||
InputStream stream = (InputStream) body; | ||
int available = stream.available(); | ||
if (available <= 0) { | ||
available = (int) (length() - totalBytesTransferred); | ||
} else { | ||
available = (int) Math.min(available, length() - totalBytesTransferred); | ||
} | ||
ByteBuf buffer = allocator.buffer(available); | ||
int toRead = Math.min(available, buffer.writableBytes()); | ||
int read = buffer.writeBytes(stream, toRead); | ||
if (read >= 0) { | ||
totalBytesTransferred += read; | ||
return buffer; | ||
} else { | ||
throw new EOFException("Unable to read bytes from InputStream"); | ||
} | ||
} else if (body instanceof ChunkedStream) { | ||
ChunkedStream stream = (ChunkedStream) body; | ||
long old = stream.transferredBytes(); | ||
ByteBuf buffer = stream.readChunk(allocator); | ||
long read = stream.transferredBytes() - old; | ||
if (read >= 0) { | ||
totalBytesTransferred += read; | ||
assert(totalBytesTransferred <= length()); | ||
return buffer; | ||
} else { | ||
throw new EOFException("Unable to read bytes from ChunkedStream"); | ||
} | ||
} else { | ||
return null; | ||
} | ||
} | ||
|
||
@Override | ||
public long length() { | ||
return headerLength + bodyLength; | ||
} | ||
|
||
@Override | ||
public long progress() { | ||
return totalBytesTransferred; | ||
} | ||
|
||
@Override | ||
public boolean isEndOfInput() throws Exception { | ||
return (headerLength + bodyLength) == totalBytesTransferred; | ||
} | ||
|
||
@Override | ||
public void close() throws Exception { | ||
header.release(); | ||
if (managedBuffer != null) { | ||
managedBuffer.release(); | ||
} | ||
if (body instanceof InputStream) { | ||
((InputStream) body).close(); | ||
} else if (body instanceof ChunkedStream) { | ||
((ChunkedStream) body).close(); | ||
} | ||
} | ||
} |
108 changes: 108 additions & 0 deletions
108
common/network-common/src/main/java/org/apache/spark/network/protocol/SslMessageEncoder.java
This file contains 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,108 @@ | ||
/* | ||
* 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.InputStream; | ||
import java.util.List; | ||
|
||
import io.netty.buffer.ByteBuf; | ||
import io.netty.buffer.Unpooled; | ||
import io.netty.channel.ChannelHandler; | ||
import io.netty.channel.ChannelHandlerContext; | ||
import io.netty.handler.codec.MessageToMessageEncoder; | ||
import io.netty.handler.stream.ChunkedStream; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* Encoder used by the server side to encode secure (SSL) server-to-client responses. | ||
* This encoder is stateless so it is safe to be shared by multiple threads. | ||
*/ | ||
@ChannelHandler.Sharable | ||
public final class SslMessageEncoder extends MessageToMessageEncoder<Message> { | ||
|
||
private final Logger logger = LoggerFactory.getLogger(SslMessageEncoder.class); | ||
|
||
private SslMessageEncoder() {} | ||
|
||
public static final SslMessageEncoder INSTANCE = new SslMessageEncoder(); | ||
|
||
/** | ||
* Encodes a Message by invoking its encode() method. For non-data messages, we will add one | ||
* ByteBuf to 'out' containing the total frame length, the message type, and the message itself. | ||
* In the case of a ChunkFetchSuccess, we will also add the ManagedBuffer corresponding to the | ||
* data to 'out'. | ||
*/ | ||
@Override | ||
public void encode(ChannelHandlerContext ctx, Message in, List<Object> out) throws Exception { | ||
Object body = null; | ||
long bodyLength = 0; | ||
boolean isBodyInFrame = false; | ||
|
||
// If the message has a body, take it out... | ||
// For SSL, zero-copy transfer will not work, so we will check if | ||
// the body is an InputStream, and if so, use an EncryptedMessageWithHeader | ||
// to wrap the header+body appropriately (for thread safety). | ||
if (in.body() != null) { | ||
try { | ||
bodyLength = in.body().size(); | ||
body = in.body().convertToNettyForSsl(); | ||
isBodyInFrame = in.isBodyInFrame(); | ||
} catch (Exception e) { | ||
in.body().release(); | ||
if (in instanceof AbstractResponseMessage) { | ||
AbstractResponseMessage resp = (AbstractResponseMessage) in; | ||
// Re-encode this message as a failure response. | ||
String error = e.getMessage() != null ? e.getMessage() : "null"; | ||
logger.error(String.format("Error processing %s for client %s", | ||
in, ctx.channel().remoteAddress()), e); | ||
encode(ctx, resp.createFailureResponse(error), out); | ||
} else { | ||
throw e; | ||
} | ||
return; | ||
} | ||
} | ||
|
||
Message.Type msgType = in.type(); | ||
// All messages have the frame length, message type, and message itself. The frame length | ||
// may optionally include the length of the body data, depending on what message is being | ||
// sent. | ||
int headerLength = 8 + msgType.encodedLength() + in.encodedLength(); | ||
long frameLength = headerLength + (isBodyInFrame ? bodyLength : 0); | ||
ByteBuf header = ctx.alloc().buffer(headerLength); | ||
header.writeLong(frameLength); | ||
msgType.encode(header); | ||
in.encode(header); | ||
assert header.writableBytes() == 0; | ||
|
||
if (body != null && bodyLength > 0) { | ||
if (body instanceof ByteBuf) { | ||
out.add(Unpooled.wrappedBuffer(header, (ByteBuf) body)); | ||
} else if (body instanceof InputStream || body instanceof ChunkedStream) { | ||
// For now, assume the InputStream is doing proper chunking. | ||
out.add(new EncryptedMessageWithHeader(in.body(), header, body, bodyLength)); | ||
} else { | ||
throw new IllegalArgumentException( | ||
"Body must be a ByteBuf, ChunkedStream or an InputStream"); | ||
} | ||
} else { | ||
out.add(header); | ||
} | ||
} | ||
} |
Oops, something went wrong.