Github user aarondav commented on a diff in the pull request:
https://github.com/apache/spark/pull/5377#discussion_r29196457
--- Diff:
network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java
---
@@ -0,0 +1,260 @@
+/*
+ * 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.sasl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.FileRegion;
+import io.netty.handler.codec.MessageToMessageDecoder;
+import io.netty.util.AbstractReferenceCounted;
+import io.netty.util.ReferenceCountUtil;
+
+import org.apache.spark.network.util.ByteArrayWritableChannel;
+import org.apache.spark.network.util.NettyUtils;
+
+class SaslEncryption {
+
+ @VisibleForTesting
+ static final String ENCRYPTION_HANDLER_NAME = "saslEncryption";
+
+ /**
+ * Adds channel handlers that perform encryption / decryption of data
using SASL.
+ *
+ * @param channel The channel.
+ * @param backend The SASL backend.
+ * @param maxOutboundBlockSize Max size in bytes of outgoing encrypted
blocks, to control
+ * memory usage.
+ */
+ static void addToChannel(
+ Channel channel,
+ SaslEncryptionBackend backend,
+ int maxOutboundBlockSize) {
+ channel.pipeline()
+ .addFirst(ENCRYPTION_HANDLER_NAME, new EncryptionHandler(backend,
maxOutboundBlockSize))
+ .addFirst("saslDecryption", new DecryptionHandler(backend))
+ .addFirst("saslFrameDecoder", NettyUtils.createFrameDecoder());
+ }
+
+ private static class EncryptionHandler extends
ChannelOutboundHandlerAdapter {
+
+ private final int maxOutboundBlockSize;
+ private final SaslEncryptionBackend backend;
+
+ EncryptionHandler(SaslEncryptionBackend backend, int
maxOutboundBlockSize) {
+ this.backend = backend;
+ this.maxOutboundBlockSize = maxOutboundBlockSize;
+ }
+
+ /**
+ * Wrap the incoming message in an implementation that will perform
encryption lazily. This is
+ * needed to guarantee ordering of the outgoing encrypted packets -
they need to be decrypted in
+ * the same order, and netty doesn't have an atomic
ChannelHandlerContext.write() API, so it
+ * does not guarantee any ordering.
+ */
+ @Override
+ public void write(ChannelHandlerContext ctx, Object msg,
ChannelPromise promise)
+ throws Exception {
+
+ ctx.write(new EncryptedMessage(backend, msg, maxOutboundBlockSize),
promise);
+ }
+
+ @Override
+ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception
{
+ try {
+ backend.dispose();
+ } finally {
+ super.handlerRemoved(ctx);
+ }
+ }
+
+ }
+
+ private static class DecryptionHandler extends
MessageToMessageDecoder<ByteBuf> {
+
+ private final SaslEncryptionBackend backend;
+
+ DecryptionHandler(SaslEncryptionBackend backend) {
+ this.backend = backend;
+ }
+
+ @Override
+ protected void decode(ChannelHandlerContext ctx, ByteBuf msg,
List<Object> out)
+ throws Exception {
+
+ byte[] data;
+ int offset;
+ int length = msg.readableBytes();
+ if (msg.hasArray()) {
+ data = msg.array();
+ offset = msg.arrayOffset();
+ } else {
+ data = new byte[length];
+ msg.readBytes(data);
+ offset = 0;
+ }
+
+ out.add(Unpooled.wrappedBuffer(backend.unwrap(data, offset,
length)));
+ }
+
+ }
+
+ private static class EncryptedMessage extends AbstractReferenceCounted
implements FileRegion {
+
+ private final SaslEncryptionBackend backend;
+ private final boolean isByteBuf;
+ private final ByteBuf buf;
+ private final FileRegion region;
+ private final ByteArrayWritableChannel byteChannel;
+
+ private ByteBuf currentHeader;
+ private ByteBuffer currentChunk;
+ private long currentChunkSize;
+ private long unencryptedChunkSize;
+ private long transferred;
+
+ EncryptedMessage(SaslEncryptionBackend backend, Object msg, int
maxOutboundBlockSize) {
+ Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof
FileRegion,
+ "Unrecognized message type: %s", msg.getClass().getName());
+ this.backend = backend;
+ this.isByteBuf = msg instanceof ByteBuf;
+ this.buf = isByteBuf ? (ByteBuf) msg : null;
+ this.region = isByteBuf ? null : (FileRegion) msg;
+ this.byteChannel = new
ByteArrayWritableChannel(maxOutboundBlockSize);
+ }
+
+ /**
+ * Returns the size of the original (unencrypted) message.
+ *
+ * This makes assumptions about how netty treats FileRegion instances,
because there's no way
+ * to know beforehand what will be the size of the encrypted message.
Namely, it assumes
+ * that netty will try to transfer data from this message while
+ * <code>transfered() < count()</code>. So these two methods return,
technically, wrong data,
+ * but netty doesn't know better.
+ */
+ @Override
+ public long count() {
+ return isByteBuf ? buf.readableBytes() : region.count();
+ }
+
+ @Override
+ public long position() {
+ return 0;
+ }
+
+ /**
+ * Returns an approximation of the amount of data transferred. See
{@link #count()}.
+ */
+ @Override
+ public long transfered() {
+ return transferred;
+ }
+
+ /**
+ * Transfers data from the original message to the channel, encrypting
it in the process.
+ *
+ * This method also breaks down the original message into smaller
chunks when needed. This
+ * is done to keep memory usage under control. This avoids having to
copy the whole message
+ * data into memory at once, and can avoid ballooning memory usage
when transferring large
+ * messages such as shuffle blocks.
+ *
+ * The {@link #transfered()} counter also behaves a little funny, in
that it won't go forward
+ * until a whole chunk has been written. This is done because the code
can't use the actual
+ * number of bytes written to the channel as the transferred count
(see {@link #count()}).
+ * Instead, once an encrypted chunk is written to the output
(including its header), the
+ * size of the original block will be added to the {@link
#transfered()} amount.
+ */
+ @Override
+ public long transferTo(final WritableByteChannel target, final long
position)
+ throws IOException {
+
+ Preconditions.checkArgument(position == transfered(), "Invalid
position.");
+
+ long written = 0;
+ do {
+ if (currentChunk == null) {
+ nextChunk();
+ }
+
+ if (currentHeader.readableBytes() > 0) {
+ int bytesWritten = target.write(currentHeader.nioBuffer());
+ currentHeader.skipBytes(bytesWritten);
+ if (currentHeader.readableBytes() > 0) {
+ // Break out of loop if there are still header bytes left to
write.
+ break;
+ }
+ }
+
+ target.write(currentChunk);
+ if (!currentChunk.hasRemaining()) {
+ // Only update the count of written bytes once a full chunk has
been written.
+ // See method javadoc.
+ written += unencryptedChunkSize;
+ currentHeader.release();
+ currentHeader = null;
+ currentChunk = null;
+ currentChunkSize = 0;
+ }
+ } while (currentChunk == null && transfered() + written < count());
+
+ transferred += written;
+ return written;
+ }
+
+ private void nextChunk() throws IOException {
+ byteChannel.reset();
+ if (isByteBuf) {
+ int copied = byteChannel.write(buf.nioBuffer());
--- End diff --
I think `Channels.newChannel(outputStream)` might work.
(The reason I'd especially like this solution is that we could keep
ByteArrayWritableChannel in src/test if we could just resolve this use-case
without it.)
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]