Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/5377#discussion_r29121094
  
    --- 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();
    --- End diff --
    
    should we advance msg by the readableBytes?


---
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]

Reply via email to