You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by aarondav <gi...@git.apache.org> on 2014/10/10 09:16:51 UTC

[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

GitHub user aarondav opened a pull request:

    https://github.com/apache/spark/pull/2753

    [SPARK-3453] Netty-based BlockTransferService, extracted from Spark core

    This PR encapsulates #2330, which is itself a continuation of #2240. The first goal of this PR is to provide an alternate, simpler implementation of the ConnectionManager which is based on Netty.
    
    In addition to this goal, however, we want to resolve [SPARK-3796](https://issues.apache.org/jira/browse/SPARK-3796), which calls for a standalone shuffle service which can be integrated into the YARN NodeManager, Standalone Worker, or on its own. This PR makes the first step in this direction by ensuring that the actual Netty service is as small as possible and extracted from Spark core. Given this, we should be able to construct this standalone jar which can be included in other JVMs without incurring significant dependency or runtime issues. The actual work to ensure that such a standalone shuffle service would work in Spark will be left for a future PR, however.
    
    In order to minimize dependencies and allow for the service to be long-running (possibly much longer-running than Spark, and possibly having to support multiple version of Spark simultaneously), the entire service has been ported to Java, where we have full control over the binary compatibility of the components and do not depend on the Scala runtime or version.
    
    These issues: have been addressed by folding in #2330:
    
    SPARK-3453: Refactor Netty module to use BlockTransferService interface
    SPARK-3018: Release all buffers upon task completion/failure
    SPARK-3002: Create a connection pool and reuse clients across different threads
    SPARK-3017: Integration tests and unit tests for connection failures
    SPARK-3049: Make sure client doesn't block when server/connection has error(s)
    SPARK-3502: SO_RCVBUF and SO_SNDBUF should be bootstrap childOption, not option
    SPARK-3503: Disable thread local cache in PooledByteBufAllocator
    
    TODO before mergeable:
    [ ] Implement uploadBlock()
    [ ] Unit tests for RPC side of code
    [ ] Performance testing
    [ ] Turn OFF by default (currently on for unit testing)


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/aarondav/spark netty

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/2753.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2753
    
----
commit 165eab1518f5184ef9609f26d374c5ccefd05472
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-09T07:29:33Z

    [SPARK-3453] Refactor Netty module to use BlockTransferService.
    
    Also includes some partial support for uploading blocks.

commit 1760d3292ecf262e4c77c9e3b28bfd2900d25840
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-09T07:42:37Z

    Use Epoll.isAvailable in BlockServer as well.

commit 2b44cf1b7547919bbe7386e954fe2f56be046790
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-09T21:36:31Z

    Added more documentation.

commit 064747b50a591acb132b2c750957e79f54dfa88f
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-10T06:38:38Z

    Reference count buffers and clean them up properly.

commit b5c8d1fca6d3cf5c2b95395310200c8149a7eb16
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-10T08:09:44Z

    Fixed ShuffleBlockFetcherIteratorSuite.

commit 108c9edaed06c5e046a21c9a8e54c50390da9a0b
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-10T08:10:04Z

    Forgot to add TestSerializer to the commit list.

commit 1be4e8ee7d932821c789cb974310e5d59df4ff84
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-10T08:11:40Z

    Shorten NioManagedBuffer and NettyManagedBuffer class names.

commit cb589ec7b6d3758498249b63b395634efb83d8ba
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-11T02:01:23Z

    Added more test cases covering cleanup when fault happens in ShuffleBlockFetcherIteratorSuite

commit 5cd33d7798ae742e76107bb976d8478ab9476ae7
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-11T02:55:54Z

    Fixed style violation.

commit 9e0cb8736be6d38e3f30766271d28875ceca1ae8
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-11T04:04:56Z

    Fixed BlockClientHandlerSuite

commit d23ed7bfd912770ace7eed7cd0dff2db6ac826e3
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-12T01:28:45Z

    Incorporated feedback from Norman:
    - use same pool for boss and worker
    - remove ioratio
    - disable caching of byte buf allocator
    - childoption sendbuf/receivebuf
    - fire exception through pipeline
    
    In addition:
    - fire failure handler BlockFetchingListener at least once per block.
    - enabled a bunch of ignored tests

commit b2f3281d0de540d38ea5b4c7bf576b775405d56d
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-12T05:12:08Z

    Added connection pooling.

commit 14323a55ebfa7ccc684c2ae78eac299a4426b353
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-12T05:13:02Z

    Removed BlockManager.getLocalShuffleFromDisk.

commit f0a16e9ec7d5c811dff3cd5219548e05077099c8
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-12T07:40:53Z

    Fixed test hanging.

commit 519d64dcb7768b3657438a4cfc85ee8065f56c2a
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-12T21:18:58Z

    Mark private package visibility and MimaExcludes.

commit c066309afbb0e248a8b2b808d997e6b37a2bff1e
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-13T05:42:32Z

    Implement java.io.Closeable interface.

commit 6afc435037a0448d6eb243bd18411ef25e3a2cf7
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-17T05:51:11Z

    Added logging.

commit f63fb4c1976e503238b7d7151f8f45f40ced36e9
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-29T18:13:44Z

    Add more debug message.

commit d68f3286a4a9795dfb61a8a63b8a20b3eafb4821
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-29T18:30:13Z

    Logging close() in case close() fails.

commit 1bdd7eec5d9ddb5a9eb33c9733878aea3ca26ba6
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-29T19:07:53Z

    Fixed tests.

commit bec4ea2b54659cfed6f54e527aa878dfbff829c7
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-29T19:22:01Z

    Removed OIO and added num threads settings.

commit 4b18db29edcdb87577fd033835275fd1c2957dcd
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-29T22:45:05Z

    Copy the buffer in fetchBlockSync.

commit a0518c766f0f4eba24459ffac61dce789fc14092
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-30T02:22:34Z

    Implemented block uploads.

commit 407e59afd3cb7385af9f63dc2263a40c7c21d783
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-30T02:37:28Z

    Fix style violation.

commit f6c220df8406be14fbdb7270682727e1085518a4
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-30T06:30:17Z

    Merge with latest master.

commit 5d98ce3de1deeeb7fbdc26b9303a591c46f1892b
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-30T07:56:32Z

    Flip buffer.

commit f7e7568414692989215d97abce9dda2fe172abb4
Author: Reynold Xin <rx...@apache.org>
Date:   2014-09-30T19:28:21Z

    Fixed spark.shuffle.io.receiveBuffer setting.

commit c0cd242f375e939e1422e30d4b230a8a78b13b88
Author: Aaron Davidson <aa...@databricks.com>
Date:   2014-10-06T00:58:43Z

    [SPARK-3453] Netty-based BlockTransferService, extracted from Spark core
    
    This PR encapsulates #2330, which is itself a continuation of #2240. The first goal of this
    PR is to provide an alternate, simpler implementation of the ConnectionManager which is based on Netty.
    
    In addition to this goal, however, we want to resolve [SPARK-3796](https://issues.apache.org/jira/browse/SPARK-3796), which calls for a
    standalone shuffle service which can be integrated into the YARN NodeManager, Standalone Worker, or
    on its own. This PR makes the first step in this direction by ensuring that the actual Netty service
    is as small as possible and extracted from Spark core. Given this, we should be able to construct
    this standalone jar which can be included in other JVMs without incurring significant dependency or
    runtime issues. The actual work to ensure that such a standalone shuffle service would work in Spark
    will be left for a future PR, however.
    
    In order to minimize dependencies and allow for the service to be long-running (possibly
    much longer-running than Spark, and possibly having to support multiple version of Spark
    simultaneously), the entire service has been ported to Java, where we have full control
    over the binary compatibility of the components and do not depend on the Scala runtime or
    version.
    
    These PRs have been addressed by folding in #2330:
    
    SPARK-3453: Refactor Netty module to use BlockTransferService interface
    SPARK-3018: Release all buffers upon task completion/failure
    SPARK-3002: Create a connection pool and reuse clients across different threads
    SPARK-3017: Integration tests and unit tests for connection failures
    SPARK-3049: Make sure client doesn't block when server/connection has error(s)
    SPARK-3502: SO_RCVBUF and SO_SNDBUF should be bootstrap childOption, not option
    SPARK-3503: Disable thread local cache in PooledByteBufAllocator
    
    TODO before mergeable:
    [ ] Implement uploadBlock()
    [ ] Unit tests for RPC side of code
    [ ] Performance testing
    [ ] Turn OFF by default (currently on for unit testing)

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440311
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java ---
    @@ -0,0 +1,176 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.lang.reflect.Field;
    +import java.net.InetSocketAddress;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.TimeoutException;
    +
    +import io.netty.bootstrap.Bootstrap;
    +import io.netty.buffer.PooledByteBufAllocator;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelInitializer;
    +import io.netty.channel.ChannelOption;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.socket.SocketChannel;
    +import io.netty.util.internal.PlatformDependent;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.TransportContext;
    +import org.apache.spark.network.server.TransportClientHandler;
    +import org.apache.spark.network.util.IOMode;
    +import org.apache.spark.network.util.NettyUtils;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * Factory for creating {@link TransportClient}s by using createClient.
    + *
    + * The factory maintains a connection pool to other hosts and should return the same
    + * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for
    + * all {@link TransportClient}s.
    + */
    +public class TransportClientFactory implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
    +
    +  private final TransportContext context;
    +  private final TransportConf conf;
    +  private final ConcurrentHashMap<SocketAddress, TransportClient> connectionPool;
    +
    +  private final Class<? extends Channel> socketChannelClass;
    +  private final EventLoopGroup workerGroup;
    +
    +  public TransportClientFactory(TransportContext context) {
    +    this.context = context;
    +    this.conf = context.getConf();
    +    this.connectionPool = new ConcurrentHashMap<SocketAddress, TransportClient>();
    +
    +    IOMode ioMode = IOMode.valueOf(conf.ioMode());
    +    this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode);
    +    this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client");
    +  }
    +
    +  /**
    +   * Create a new BlockFetchingClient connecting to the given remote host / port.
    +   *
    +   * This blocks until a connection is successfully established.
    +   *
    +   * Concurrency: This method is safe to call from multiple threads.
    +   */
    +  public TransportClient createClient(String remoteHost, int remotePort) throws TimeoutException {
    +    // Get connection from the connection pool first.
    +    // If it is not found or not active, create a new one.
    +    final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort);
    +    TransportClient cachedClient = connectionPool.get(address);
    +    if (cachedClient != null && cachedClient.isActive()) {
    +      return cachedClient;
    +    } else if (cachedClient != null) {
    +      connectionPool.remove(address, cachedClient); // Remove inactive clients.
    +    }
    +
    +    logger.debug("Creating new connection to " + address);
    +
    +    Bootstrap bootstrap = new Bootstrap();
    +    bootstrap.group(workerGroup)
    +      .channel(socketChannelClass)
    +       // Disable Nagle's Algorithm since we don't want packets to wait
    +      .option(ChannelOption.TCP_NODELAY, true)
    +      .option(ChannelOption.SO_KEEPALIVE, true)
    +      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs());
    +
    +    // Use pooled buffers to reduce temporary buffer allocation
    +    bootstrap.option(ChannelOption.ALLOCATOR, createPooledByteBufAllocator());
    +
    +    bootstrap.handler(new ChannelInitializer<SocketChannel>() {
    +      @Override
    +      public void initChannel(SocketChannel ch) {
    +        TransportClientHandler channelHandler = context.initializePipeline(ch);
    +        TransportClient oldClient = connectionPool.putIfAbsent(address, channelHandler.getClient());
    +        if (oldClient != null) {
    +          logger.debug("Two clients were created concurrently, second one will be disposed.");
    +          ch.close();
    +          // Note: this type of failure is still considered a success by Netty, and thus the
    +          // ChannelFuture will complete successfully.
    +        }
    +      }
    +    });
    +
    +    // Connect to the remote server
    +    ChannelFuture cf = bootstrap.connect(address);
    +    if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) {
    +      throw new TimeoutException(
    +        String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs()));
    +    }
    +
    +    TransportClient client = connectionPool.get(address);
    --- End diff --
    
    we should add another await here to make sure the client is actually established, in case two threads are racing to create clients.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18810179
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/SluiceClient.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.util.UUID;
    +
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +
    +/**
    + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow
    + * efficient transfer of a large amount of data, broken up into chunks with size ranging from
    + * hundreds of KB to a few MB.
    + *
    + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane),
    + * the actual setup of the streams is done outside the scope of Sluice. The convenience method
    + * "sendRPC" is provided to enable control plane communication between the client and server to
    + * perform this setup.
    + *
    + * For example, a typical workflow might be:
    + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100
    + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback)
    + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback)
    + * ...
    + * client.sendRPC(new CloseStream(100))
    + *
    + * Construct an instance of SluiceClient using {@link SluiceClientFactory}. A single SluiceClient
    + * may be used for multiple streams, but any given stream must be restricted to a single client,
    + * in order to avoid out-of-order responses.
    + *
    + * NB: This class is used to make requests to the server, while {@link SluiceClientHandler} is
    + * responsible for handling responses from the server.
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class SluiceClient implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(SluiceClient.class);
    +
    +  private final ChannelFuture cf;
    +  private final SluiceClientHandler handler;
    +
    +  private final String serverAddr;
    +
    +  SluiceClient(ChannelFuture cf, SluiceClientHandler handler) {
    +    this.cf = cf;
    +    this.handler = handler;
    +
    +    if (cf != null && cf.channel() != null && cf.channel().remoteAddress() != null) {
    +      serverAddr = cf.channel().remoteAddress().toString();
    +    } else {
    +      serverAddr = "<unknown address>";
    +    }
    +  }
    +
    +  public boolean isActive() {
    +    return cf.channel().isActive();
    +  }
    +
    +  /**
    +   * Requests a single chunk from the remote side, from the pre-negotiated streamId.
    +   *
    +   * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though
    +   * some streams may not support this.
    +   *
    +   * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed
    +   * to be returned in the same order that they were requested, assuming only a single SluiceClient
    +   * is used to fetch the chunks.
    +   *
    +   * @param streamId Identifier that refers to a stream in the remote StreamManager. This should
    +   *                 be agreed upon by client and server beforehand.
    +   * @param chunkIndex 0-based index of the chunk to fetch
    +   * @param callback Callback invoked upon successful receipt of chunk, or upon any failure.
    +   */
    +  public void fetchChunk(
    +      long streamId,
    +      final int chunkIndex,
    +      final ChunkReceivedCallback callback) {
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr);
    +
    +    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
    +    handler.addFetchRequest(streamChunkId, callback);
    +
    +    cf.channel().writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", streamChunkId, serverAddr,
    +                timeTaken);
    +          } else {
    +            // Fail all blocks.
    +            String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
    +              serverAddr, future.cause().getMessage());
    +            logger.error(errorMsg, future.cause());
    +            future.cause().printStackTrace();
    +            handler.removeFetchRequest(streamChunkId);
    +            callback.onFailure(chunkIndex, new RuntimeException(errorMsg));
    +          }
    +        }
    +      });
    +  }
    +
    +  /**
    +   * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked
    +   * with the server's response or upon any failure.
    +   */
    +  public void sendRpc(byte[] message, final RpcResponseCallback callback) {
    --- End diff --
    
    rather than byte[], how about just an Object, and the server/client takes a configurable serializer that does the serialization?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18755797
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    +
    +  val chunkCallback = new ChunkReceivedCallback {
    +    // On receipt of a chunk, pass it upwards as a block.
    +    def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions {
    +      buffer.retain()
    +      listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer)
    +    }
    +
    +    // On receipt of a failure, fail every block from chunkIndex onwards.
    +    def onFailure(chunkIndex: Int, e: Throwable): Unit = {
    +      blockIds.drop(chunkIndex).foreach { blockId =>
    +        listener.onBlockFetchFailure(blockId, e);
    +      }
    +    }
    +  }
    +
    +  // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle.
    +  client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(),
    --- End diff --
    
    does this even need to be a class on its own? if yes, maybe have a separate init method so we don't get a weird object ctor failure


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59554384
  
    Jenkins, test this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59552944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21857/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60858680
  
      [Test build #22405 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22405/consoleFull) for   PR 2753 at commit [`4a204b8`](https://github.com/apache/spark/commit/4a204b846a8ce2b1cfbab9ed1ec42e8a2f082184).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59557076
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21858/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59570295
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21863/consoleFull) for   PR 2753 at commit [`9da0bc1`](https://github.com/apache/spark/commit/9da0bc11383587c21f6306bb0a2e9fb4b86fbb88).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18811237
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.util;
    +
    +import java.io.Closeable;
    +
    +import com.google.common.io.Closeables;
    +
    +public class JavaUtils {
    +  /** Closes the given object, ignoring IOExceptions. */
    +  @SuppressWarnings("deprecation")
    +  public static void closeQuietly(Closeable closable) {
    +    Closeables.closeQuietly(closable);
    --- End diff --
    
    why bother this indirection instead of using Closeable's directly?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19511498
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java ---
    @@ -0,0 +1,35 @@
    +/*
    + * 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 io.netty.buffer.ByteBuf;
    +
    +/**
    + * Interface for an object which can be encoded into a ByteBuf. Multiple Encodable objects are
    + * stored in a single, pre-allocated ByteBuf, so Encodables must also provide their length.
    --- End diff --
    
    done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19452740
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java ---
    @@ -0,0 +1,32 @@
    +/*
    + * 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.util;
    +
    +import java.util.NoSuchElementException;
    +
    +/** Uses System properties to obtain config values. */
    +public class SystemPropertyConfigProvider extends ConfigProvider {
    --- End diff --
    
    can we move this to test?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19383298
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.{SparkConf, Logging}
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.network.netty.NettyMessages._
    +import org.apache.spark.serializer.{JavaSerializer, Serializer}
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    +
    +  val chunkCallback = new ChunkReceivedCallback {
    +    // On receipt of a chunk, pass it upwards as a block.
    +    def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions {
    +      listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer)
    +    }
    +
    +    // On receipt of a failure, fail every block from chunkIndex onwards.
    +    def onFailure(chunkIndex: Int, e: Throwable): Unit = {
    +      blockIds.drop(chunkIndex).foreach { blockId =>
    +        listener.onBlockFetchFailure(blockId, e);
    +      }
    +    }
    +  }
    +
    +  // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle.
    +  client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(),
    +    new RpcResponseCallback {
    +      override def onSuccess(response: Array[Byte]): Unit = {
    +        try {
    +          streamHandle = ser.deserialize[ShuffleStreamHandle](ByteBuffer.wrap(response))
    +          logTrace(s"Successfully opened block set: $streamHandle! Preparing to fetch chunks.")
    +
    +          // Immediately request all chunks -- we expect that the total size of the request is
    +          // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]].
    +          for (i <- 0 until streamHandle.numChunks) {
    +            client.fetchChunk(streamHandle.streamId, i, chunkCallback)
    +          }
    +        } catch {
    +          case e: Exception =>
    +            logError("Failed while starting block fetches", e)
    +            blockIds.foreach(listener.onBlockFetchFailure(_, e))
    +        }
    +      }
    +
    +      override def onFailure(e: Throwable): Unit = {
    +        logError("Failed while starting block fetches")
    +        blockIds.foreach(listener.onBlockFetchFailure(_, e))
    --- End diff --
    
    here too


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18999337
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.util;
    +
    +import java.io.Closeable;
    +
    +import com.google.common.io.Closeables;
    +
    +public class JavaUtils {
    +  /** Closes the given object, ignoring IOExceptions. */
    +  @SuppressWarnings("deprecation")
    +  public static void closeQuietly(Closeable closable) {
    +    Closeables.closeQuietly(closable);
    --- End diff --
    
    IOException. Checked exceptions are like the worst thing, Java.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19004341
  
    --- Diff: core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala ---
    @@ -31,7 +34,7 @@ trait BlockFetchingListener extends EventListener {
       def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit
    --- End diff --
    
    Done.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59150242
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21758/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60875589
  
      [Test build #22427 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22427/consoleFull) for   PR 2753 at commit [`cadfd28`](https://github.com/apache/spark/commit/cadfd28f116f0dbca11e580a23caf82060bcf922).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19560920
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java ---
    @@ -0,0 +1,182 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.lang.reflect.Field;
    +import java.net.InetSocketAddress;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import io.netty.bootstrap.Bootstrap;
    +import io.netty.buffer.PooledByteBufAllocator;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelInitializer;
    +import io.netty.channel.ChannelOption;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.socket.SocketChannel;
    +import io.netty.util.internal.PlatformDependent;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.TransportContext;
    +import org.apache.spark.network.server.TransportChannelHandler;
    +import org.apache.spark.network.util.IOMode;
    +import org.apache.spark.network.util.NettyUtils;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * Factory for creating {@link TransportClient}s by using createClient.
    + *
    + * The factory maintains a connection pool to other hosts and should return the same
    + * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for
    + * all {@link TransportClient}s.
    + */
    +public class TransportClientFactory implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
    +
    +  private final TransportContext context;
    +  private final TransportConf conf;
    +  private final ConcurrentHashMap<SocketAddress, TransportClient> connectionPool;
    +
    +  private final Class<? extends Channel> socketChannelClass;
    +  private final EventLoopGroup workerGroup;
    +
    +  public TransportClientFactory(TransportContext context) {
    +    this.context = context;
    +    this.conf = context.getConf();
    +    this.connectionPool = new ConcurrentHashMap<SocketAddress, TransportClient>();
    +
    +    IOMode ioMode = IOMode.valueOf(conf.ioMode());
    +    this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode);
    +    // TODO: Make thread pool name configurable.
    +    this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client");
    +  }
    +
    +  /**
    +   * Create a new BlockFetchingClient connecting to the given remote host / port.
    +   *
    +   * This blocks until a connection is successfully established.
    +   *
    +   * Concurrency: This method is safe to call from multiple threads.
    +   */
    +  public TransportClient createClient(String remoteHost, int remotePort) throws TimeoutException {
    +    // Get connection from the connection pool first.
    +    // If it is not found or not active, create a new one.
    +    final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort);
    +    TransportClient cachedClient = connectionPool.get(address);
    +    if (cachedClient != null && cachedClient.isActive()) {
    +      return cachedClient;
    +    } else if (cachedClient != null) {
    +      connectionPool.remove(address, cachedClient); // Remove inactive clients.
    +    }
    +
    +    logger.debug("Creating new connection to " + address);
    +
    +    Bootstrap bootstrap = new Bootstrap();
    +    bootstrap.group(workerGroup)
    +      .channel(socketChannelClass)
    +       // Disable Nagle's Algorithm since we don't want packets to wait
    +      .option(ChannelOption.TCP_NODELAY, true)
    +      .option(ChannelOption.SO_KEEPALIVE, true)
    +      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs());
    +
    +    // Use pooled buffers to reduce temporary buffer allocation
    +    bootstrap.option(ChannelOption.ALLOCATOR, createPooledByteBufAllocator());
    +
    +    final AtomicReference<TransportClient> client = new AtomicReference<TransportClient>();
    +
    +    bootstrap.handler(new ChannelInitializer<SocketChannel>() {
    +      @Override
    +      public void initChannel(SocketChannel ch) {
    +        TransportChannelHandler clientHandler = context.initializePipeline(ch);
    +        client.set(clientHandler.getClient());
    +      }
    +    });
    +
    +    // Connect to the remote server
    +    ChannelFuture cf = bootstrap.connect(address);
    +    if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) {
    +      throw new TimeoutException(
    +        String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs()));
    +    } else if (cf.cause() != null) {
    +      throw new RuntimeException(String.format("Failed to connect to %s", address), cf.cause());
    +    }
    +
    +    // Successful connection
    --- End diff --
    
    in your next PR, please add some comment on the racing, e.g.
    
    "Two threads could race to establish the connections. In the case another thread has already established the connection while this thread is still trying to, we close the connection started by this thread."


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18809666
  
    --- Diff: network/common/pom.xml ---
    @@ -0,0 +1,94 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent</artifactId>
    +    <version>1.2.0-SNAPSHOT</version>
    +    <relativePath>../../pom.xml</relativePath>
    +  </parent>
    +
    +  <groupId>org.apache.spark</groupId>
    +  <artifactId>network</artifactId>
    +  <packaging>jar</packaging>
    +  <name>Shuffle Streaming Service</name>
    +  <url>http://spark.apache.org/</url>
    +  <properties>
    +    <sbt.project.name>network</sbt.project.name>
    +  </properties>
    +
    +  <dependencies>
    +    <!-- Core dependencies -->
    +    <dependency>
    +      <groupId>io.netty</groupId>
    +      <artifactId>netty-all</artifactId>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.slf4j</groupId>
    +      <artifactId>slf4j-api</artifactId>
    +    </dependency>
    +
    +    <!-- Provided dependencies -->
    +    <dependency>
    +      <groupId>com.google.guava</groupId>
    --- End diff --
    
    is the only thing used byte limit stream? if yes maybe we can just have that implemented ...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440017
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java ---
    @@ -0,0 +1,176 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.lang.reflect.Field;
    +import java.net.InetSocketAddress;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.TimeoutException;
    +
    +import io.netty.bootstrap.Bootstrap;
    +import io.netty.buffer.PooledByteBufAllocator;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelInitializer;
    +import io.netty.channel.ChannelOption;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.socket.SocketChannel;
    +import io.netty.util.internal.PlatformDependent;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.TransportContext;
    +import org.apache.spark.network.server.TransportClientHandler;
    +import org.apache.spark.network.util.IOMode;
    +import org.apache.spark.network.util.NettyUtils;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * Factory for creating {@link TransportClient}s by using createClient.
    + *
    + * The factory maintains a connection pool to other hosts and should return the same
    + * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for
    + * all {@link TransportClient}s.
    + */
    +public class TransportClientFactory implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
    +
    +  private final TransportContext context;
    +  private final TransportConf conf;
    +  private final ConcurrentHashMap<SocketAddress, TransportClient> connectionPool;
    +
    +  private final Class<? extends Channel> socketChannelClass;
    +  private final EventLoopGroup workerGroup;
    +
    +  public TransportClientFactory(TransportContext context) {
    +    this.context = context;
    +    this.conf = context.getConf();
    +    this.connectionPool = new ConcurrentHashMap<SocketAddress, TransportClient>();
    +
    +    IOMode ioMode = IOMode.valueOf(conf.ioMode());
    +    this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode);
    +    this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client");
    --- End diff --
    
    add a TODO to make the thread pool name configurable



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19441170
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/response/ChunkFetchSuccess.java ---
    @@ -0,0 +1,82 @@
    +/*
    + * 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.response;
    +
    +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;
    +import org.apache.spark.network.protocol.StreamChunkId;
    +
    +/**
    + * Response to {@link org.apache.spark.network.protocol.request.ChunkFetchRequest} when a chunk
    + * exists and has been successfully fetched.
    + *
    + * Note that the server-side encoding of this messages does NOT include the buffer itself, as this
    + * may be written by Netty in a more efficient manner (i.e., zero-copy write).
    + * Similarly, the client-side decoding will reuse the Netty ByteBuf as the buffer.
    + */
    +public final class ChunkFetchSuccess implements ResponseMessage {
    +  public final StreamChunkId streamChunkId;
    +  public final ManagedBuffer buffer;
    +
    +  public ChunkFetchSuccess(StreamChunkId streamChunkId, ManagedBuffer buffer) {
    +    this.streamChunkId = streamChunkId;
    +    this.buffer = buffer;
    +  }
    +
    +  @Override
    +  public Type type() { return Type.ChunkFetchSuccess; }
    +
    +  @Override
    +  public int encodedLength() {
    +    return streamChunkId.encodedLength();
    +  }
    +
    +  /** Encoding does NOT include buffer itself. See {@link MessageEncoder}. */
    --- End diff --
    
    put quotes around buffer?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60886384
  
      [Test build #22437 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22437/consoleFull) for   PR 2753 at commit [`cadfd28`](https://github.com/apache/spark/commit/cadfd28f116f0dbca11e580a23caf82060bcf922).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58620672
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21580/consoleFull) for   PR 2753 at commit [`c0cd242`](https://github.com/apache/spark/commit/c0cd242f375e939e1422e30d4b230a8a78b13b88).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60868156
  
      [Test build #22421 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22421/consoleFull) for   PR 2753 at commit [`d7be11b`](https://github.com/apache/spark/commit/d7be11b74f6e4ccede5f783742b88ecffeb19add).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60552467
  
      [Test build #22275 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22275/consoleFull) for   PR 2753 at commit [`8dfcceb`](https://github.com/apache/spark/commit/8dfcceb5127b638ece6817e7858c6cbf93461cd6).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19389125
  
    --- Diff: network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java ---
    @@ -0,0 +1,86 @@
    +/*
    + * 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;
    +
    +import io.netty.channel.embedded.EmbeddedChannel;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import org.apache.spark.network.protocol.Message;
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.MessageDecoder;
    +import org.apache.spark.network.protocol.response.MessageEncoder;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +public class ProtocolSuite {
    +  private void testServerToClient(Message msg) {
    +    EmbeddedChannel serverChannel = new EmbeddedChannel(new MessageEncoder());
    +    serverChannel.writeOutbound(msg);
    +
    +    EmbeddedChannel clientChannel = new EmbeddedChannel(
    +        NettyUtils.createFrameDecoder(), new MessageDecoder());
    +
    +    while (!serverChannel.outboundMessages().isEmpty()) {
    +      clientChannel.writeInbound(serverChannel.readOutbound());
    +    }
    +
    +    assertEquals(1, clientChannel.inboundMessages().size());
    +    assertEquals(msg, clientChannel.readInbound());
    +  }
    +
    +  private void testClientToServer(Message msg) {
    +    EmbeddedChannel clientChannel = new EmbeddedChannel(new MessageEncoder());
    +    clientChannel.writeOutbound(msg);
    +
    +    EmbeddedChannel serverChannel = new EmbeddedChannel(
    +        NettyUtils.createFrameDecoder(), new MessageDecoder());
    +
    +    while (!clientChannel.outboundMessages().isEmpty()) {
    +      serverChannel.writeInbound(clientChannel.readOutbound());
    +    }
    +
    +    assertEquals(1, serverChannel.inboundMessages().size());
    +    assertEquals(msg, serverChannel.readInbound());
    +  }
    +
    +  @Test
    +  public void requests() {
    +    testClientToServer(new ChunkFetchRequest(new StreamChunkId(1, 2)));
    +    testClientToServer(new RpcRequest(12345, new byte[0]));
    +    testClientToServer(new RpcRequest(12345, new byte[100]));
    +  }
    +
    +  @Test
    +  public void responses() {
    +    testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(10)));
    +    testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(0)));
    +    testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), "this is an error"));
    +    testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), ""));
    +    testServerToClient(new RpcResponse(12345, new byte[0]));
    +    testServerToClient(new RpcResponse(12345, new byte[1000]));
    +    testServerToClient(new RpcFailure(0, "this is an error"));
    +    testServerToClient(new RpcFailure(0, ""));
    +  }
    +}
    --- End diff --
    
    github didn't notice this, but it's done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59484057
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21852/consoleFull) for   PR 2753 at commit [`d236dfd`](https://github.com/apache/spark/commit/d236dfdd2c6ac21d6eb1ce4e356be0f69aa6eb24).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18999115
  
    --- Diff: network/common/pom.xml ---
    @@ -0,0 +1,94 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent</artifactId>
    +    <version>1.2.0-SNAPSHOT</version>
    +    <relativePath>../../pom.xml</relativePath>
    +  </parent>
    +
    +  <groupId>org.apache.spark</groupId>
    +  <artifactId>network</artifactId>
    +  <packaging>jar</packaging>
    +  <name>Shuffle Streaming Service</name>
    +  <url>http://spark.apache.org/</url>
    +  <properties>
    +    <sbt.project.name>network</sbt.project.name>
    +  </properties>
    +
    +  <dependencies>
    +    <!-- Core dependencies -->
    +    <dependency>
    +      <groupId>io.netty</groupId>
    +      <artifactId>netty-all</artifactId>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.slf4j</groupId>
    +      <artifactId>slf4j-api</artifactId>
    +    </dependency>
    +
    +    <!-- Provided dependencies -->
    +    <dependency>
    +      <groupId>com.google.guava</groupId>
    --- End diff --
    
    We use guava for several relatively minor things, including Charset, Sets/Collections/etc., Preconditions, Objects, and Throwables. I think it's worth the code clarity, though we'll probably have to end up shading 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19499666
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala ---
    @@ -0,0 +1,76 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.network.BlockDataManager
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer}
    +import org.apache.spark.network.client.{TransportClient, RpcResponseCallback}
    +import org.apache.spark.network.server.{DefaultStreamManager, RpcHandler}
    +import org.apache.spark.storage.{StorageLevel, BlockId}
    +
    +import scala.collection.JavaConversions._
    +
    +object NettyMessages {
    +
    +  /** Request to read a set of blocks. Returns [[ShuffleStreamHandle]] to identify the stream. */
    +  case class OpenBlocks(blockIds: Seq[BlockId])
    +
    +  /** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */
    +  case class UploadBlock(blockId: BlockId, blockData: Array[Byte], level: StorageLevel)
    +
    +  /** Identifier for a fixed number of chunks to read from a stream created by [[OpenBlocks]]. */
    +  case class ShuffleStreamHandle(streamId: Long, numChunks: Int)
    +}
    +
    +/**
    + * Serves requests to open blocks by simply registering one chunk per block requested.
    + */
    +class NettyBlockRpcServer(
    +    serializer: Serializer,
    +    streamManager: DefaultStreamManager,
    +    blockManager: BlockDataManager)
    +  extends RpcHandler with Logging {
    +
    +  import NettyMessages._
    +
    +  override def receive(
    +      client: TransportClient,
    +      messageBytes: Array[Byte],
    +      responseContext: RpcResponseCallback): Unit = {
    +    val ser = serializer.newInstance()
    +    val message = ser.deserialize[AnyRef](ByteBuffer.wrap(messageBytes))
    --- End diff --
    
    ah yes. nvm. the first part still stands though :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60863400
  
      [Test build #22415 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22415/consoleFull) for   PR 2753 at commit [`d5d123f`](https://github.com/apache/spark/commit/d5d123fdc3c6f36fa2839ac2f2a4cb50981a09cb).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59624284
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21878/consoleFull) for   PR 2753 at commit [`e5675a4`](https://github.com/apache/spark/commit/e5675a4b919452c1eb68dd3a885f5d7f747e6014).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440798
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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.client;
    +
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import io.netty.channel.Channel;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.response.ResponseMessage;
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.server.MessageHandler;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * Handler that processes server responses, in response to requests issued from a
    + * [[TransportClient]]. It works by tracking the list of outstanding requests (and their callbacks).
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class TransportResponseHandler extends MessageHandler<ResponseMessage> {
    +  private final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class);
    +
    +  private final Channel channel;
    +
    +  private final Map<StreamChunkId, ChunkReceivedCallback> outstandingFetches;
    +
    +  private final Map<Long, RpcResponseCallback> outstandingRpcs;
    +
    +  public TransportResponseHandler(Channel channel) {
    +    this.channel = channel;
    +    this.outstandingFetches = new ConcurrentHashMap<StreamChunkId, ChunkReceivedCallback>();
    +    this.outstandingRpcs = new ConcurrentHashMap<Long, RpcResponseCallback>();
    +  }
    +
    +  public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) {
    +    outstandingFetches.put(streamChunkId, callback);
    +  }
    +
    +  public void removeFetchRequest(StreamChunkId streamChunkId) {
    +    outstandingFetches.remove(streamChunkId);
    +  }
    +
    +  public void addRpcRequest(long tag, RpcResponseCallback callback) {
    +    outstandingRpcs.put(tag, callback);
    +  }
    +
    +  public void removeRpcRequest(long tag) {
    +    outstandingRpcs.remove(tag);
    +  }
    +
    +  /**
    +   * Fire the failure callback for all outstanding requests. This is called when we have an
    +   * uncaught exception or pre-mature connection termination.
    +   */
    +  private void failOutstandingRequests(Throwable cause) {
    +    for (Map.Entry<StreamChunkId, ChunkReceivedCallback> entry : outstandingFetches.entrySet()) {
    +      entry.getValue().onFailure(entry.getKey().chunkIndex, cause);
    +    }
    +    for (Map.Entry<Long, RpcResponseCallback> entry : outstandingRpcs.entrySet()) {
    +      entry.getValue().onFailure(cause);
    +    }
    +
    +    // It's OK if new fetches appear, as they will fail immediately.
    +    outstandingFetches.clear();
    +    outstandingRpcs.clear();
    +  }
    +
    +  @Override
    +  public void channelUnregistered() {
    +    if (numOutstandingRequests() > 0) {
    +      String remoteAddress = NettyUtils.getRemoteAddress(channel);
    +      logger.error("Still have {} requests outstanding when connection from {} is closed",
    +        numOutstandingRequests(), remoteAddress);
    +      failOutstandingRequests(new RuntimeException("Connection from " + remoteAddress + " closed"));
    +    }
    +  }
    +
    +  @Override
    +  public void exceptionCaught(Throwable cause) {
    +    if (numOutstandingRequests() > 0) {
    +      String remoteAddress = NettyUtils.getRemoteAddress(channel);
    +      logger.error("Still have {} requests outstanding when connection from {} is closed",
    +        numOutstandingRequests(), remoteAddress);
    +      failOutstandingRequests(cause);
    +    }
    +  }
    +
    +  @Override
    +  public void handle(ResponseMessage message) {
    +    String remoteAddress = NettyUtils.getRemoteAddress(channel);
    +    if (message instanceof ChunkFetchSuccess) {
    +      ChunkFetchSuccess resp = (ChunkFetchSuccess) message;
    +      ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId);
    +      if (listener == null) {
    +        logger.warn("Got a response for block {} from {} but it is not outstanding",
    --- End diff --
    
    also for the rest of the log msgs in this file


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59550672
  
    @rxin RPC unit tests added, this is good to go on my side (and will turn off by default right before merge).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19442563
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/response/MessageDecoder.java ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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.response;
    +
    +import java.util.List;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.channel.ChannelHandler;
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.handler.codec.MessageToMessageDecoder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.Message;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +
    +/**
    + * Decoder used by the client side to encode server-to-client responses.
    + * This encoder is stateless so it is safe to be shared by multiple threads.
    + */
    +@ChannelHandler.Sharable
    +public final class MessageDecoder extends MessageToMessageDecoder<ByteBuf> {
    +
    +  private final Logger logger = LoggerFactory.getLogger(MessageDecoder.class);
    +  @Override
    +  public void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) {
    +    Message.Type msgType = Message.Type.decode(in);
    +    Message decoded = decode(msgType, in);
    +    assert decoded.type() == msgType;
    +    logger.debug("Received message " + msgType + ": " + decoded);
    --- End diff --
    
    trace


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59474679
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21852/consoleFull) for   PR 2753 at commit [`d236dfd`](https://github.com/apache/spark/commit/d236dfdd2c6ac21d6eb1ce4e356be0f69aa6eb24).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19437966
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java ---
    @@ -0,0 +1,146 @@
    +/*
    + * 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.buffer;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.RandomAccessFile;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +
    +import com.google.common.base.Objects;
    +import com.google.common.io.ByteStreams;
    +import io.netty.channel.DefaultFileRegion;
    +
    +import org.apache.spark.network.util.JavaUtils;
    +
    +/**
    + * A {@link ManagedBuffer} backed by a segment in a file.
    + */
    +public final class FileSegmentManagedBuffer extends ManagedBuffer {
    +
    +  /**
    +   * Memory mapping is expensive and can destabilize the JVM (SPARK-1145, SPARK-3889).
    +   * Avoid unless there's a good reason not to.
    +   */
    +  private static final long MIN_MEMORY_MAP_BYTES = 2 * 1024 * 1024;
    --- End diff --
    
    make this configurable. maybe add a TODO for the next prs


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19438500
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java ---
    @@ -15,18 +15,23 @@
      * limitations under the License.
      */
     
    -package org.apache.spark.network.netty.server
    +package org.apache.spark.network.client;
     
     /**
    - * Header describing a block. This is used only in the server pipeline.
    - *
    - * [[BlockServerHandler]] creates this, and [[BlockHeaderEncoder]] encodes it.
    - *
    - * @param blockSize length of the block content, excluding the length itself.
    - *                 If positive, this is the header for a block (not part of the header).
    - *                 If negative, this is the header and content for an error message.
    - * @param blockId block id
    - * @param error some error message from reading the block
    + * General exception caused by a remote exception while fetching a chunk.
      */
    -private[server]
    -class BlockHeader(val blockSize: Int, val blockId: String, val error: Option[String] = None)
    +public class ChunkFetchFailureException extends RuntimeException {
    --- End diff --
    
    we should probably pass the chunkIndex into the exception message, either override getMessage, or just put it as part of the message.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19364501
  
    --- Diff: network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java ---
    @@ -0,0 +1,86 @@
    +/*
    + * 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;
    +
    +import io.netty.channel.embedded.EmbeddedChannel;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import org.apache.spark.network.protocol.Message;
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.MessageDecoder;
    +import org.apache.spark.network.protocol.response.MessageEncoder;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +public class ProtocolSuite {
    +  private void testServerToClient(Message msg) {
    +    EmbeddedChannel serverChannel = new EmbeddedChannel(new MessageEncoder());
    +    serverChannel.writeOutbound(msg);
    +
    +    EmbeddedChannel clientChannel = new EmbeddedChannel(
    +        NettyUtils.createFrameDecoder(), new MessageDecoder());
    +
    +    while (!serverChannel.outboundMessages().isEmpty()) {
    +      clientChannel.writeInbound(serverChannel.readOutbound());
    +    }
    +
    +    assertEquals(1, clientChannel.inboundMessages().size());
    +    assertEquals(msg, clientChannel.readInbound());
    +  }
    +
    +  private void testClientToServer(Message msg) {
    +    EmbeddedChannel clientChannel = new EmbeddedChannel(new MessageEncoder());
    +    clientChannel.writeOutbound(msg);
    +
    +    EmbeddedChannel serverChannel = new EmbeddedChannel(
    +        NettyUtils.createFrameDecoder(), new MessageDecoder());
    +
    +    while (!clientChannel.outboundMessages().isEmpty()) {
    +      serverChannel.writeInbound(clientChannel.readOutbound());
    +    }
    +
    +    assertEquals(1, serverChannel.inboundMessages().size());
    +    assertEquals(msg, serverChannel.readInbound());
    +  }
    +
    +  @Test
    +  public void requests() {
    +    testClientToServer(new ChunkFetchRequest(new StreamChunkId(1, 2)));
    +    testClientToServer(new RpcRequest(12345, new byte[0]));
    +    testClientToServer(new RpcRequest(12345, new byte[100]));
    +  }
    +
    +  @Test
    +  public void responses() {
    +    testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(10)));
    +    testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(0)));
    +    testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), "this is an error"));
    +    testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), ""));
    +    testServerToClient(new RpcResponse(12345, new byte[0]));
    +    testServerToClient(new RpcResponse(12345, new byte[1000]));
    +    testServerToClient(new RpcFailure(0, "this is an error"));
    +    testServerToClient(new RpcFailure(0, ""));
    +  }
    +}
    --- End diff --
    
    nit  - new line


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19442016
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java ---
    @@ -0,0 +1,99 @@
    +/*
    + * 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.server;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.buffer.ManagedBuffer;
    +
    +/**
    + * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are individually
    + * fetched as chunks by the client.
    + */
    +public class DefaultStreamManager extends StreamManager {
    +  private final Logger logger = LoggerFactory.getLogger(DefaultStreamManager.class);
    +
    +  private final AtomicLong nextStreamId;
    +  private final Map<Long, StreamState> streams;
    +
    +  /** State of a single stream. */
    +  private static class StreamState {
    +    final Iterator<ManagedBuffer> buffers;
    +
    +    int curChunk = 0;
    +
    +    StreamState(Iterator<ManagedBuffer> buffers) {
    +      this.buffers = buffers;
    +    }
    +  }
    +
    +  public DefaultStreamManager() {
    +    // Start with a random stream id to help identifying different streams.
    +    nextStreamId = new AtomicLong((long) new Random().nextInt(Integer.MAX_VALUE) * 1000);
    +    streams = new ConcurrentHashMap<Long, StreamState>();
    +  }
    +
    +  @Override
    +  public ManagedBuffer getChunk(long streamId, int chunkIndex) {
    +    StreamState state = streams.get(streamId);
    +    if (chunkIndex != state.curChunk) {
    +      throw new IllegalStateException(String.format(
    +        "Received out-of-order chunk index %s (expected %s)", chunkIndex, state.curChunk));
    +    } else if (!state.buffers.hasNext()) {
    +      throw new IllegalStateException(String.format(
    +        "Requested chunk index beyond end %s", chunkIndex));
    +    }
    +    state.curChunk += 1;
    +    ManagedBuffer nextChunk = state.buffers.next();
    +
    +    if (!state.buffers.hasNext()) {
    +      logger.trace("Removing stream id {}", streamId);
    +      streams.remove(streamId);
    +    }
    +
    +    return nextChunk;
    +  }
    +
    +  @Override
    +  public void connectionTerminated(long streamId) {
    +    // Release all remaining buffers.
    +    StreamState state = streams.remove(streamId);
    +    if (state != null && state.buffers != null) {
    +      while (state.buffers.hasNext()) {
    +        state.buffers.next().release();
    +      }
    +    }
    +  }
    +
    +  public long registerStream(Iterator<ManagedBuffer> buffers) {
    --- End diff --
    
    can u add javadoc explaining this method and unregisterStream?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58620769
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21580/consoleFull) for   PR 2753 at commit [`c0cd242`](https://github.com/apache/spark/commit/c0cd242f375e939e1422e30d4b230a8a78b13b88).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/2753


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19388840
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/SluiceConfig.java ---
    @@ -0,0 +1,61 @@
    +/*
    + * 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.util;
    +
    +/**
    + * A central location that tracks all the settings we expose to users.
    + */
    +public class SluiceConfig {
    +  private final ConfigProvider conf;
    +
    +  public SluiceConfig(ConfigProvider conf) {
    +    this.conf = conf;
    +  }
    +
    +  /** Port the server listens on. Default to a random port. */
    +  public int serverPort() { return conf.getInt("spark.shuffle.io.port", 0); }
    +
    +  /** IO mode: nio, epoll, or auto (try epoll first and then nio). */
    +  public String ioMode() { return conf.get("spark.shuffle.io.mode", "NIO").toUpperCase(); }
    --- End diff --
    
    nah, let's do nio since it's more stable. Actually let me get rid of auto for this same reason.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58771033
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21651/consoleFull) for   PR 2753 at commit [`9d9b4e1`](https://github.com/apache/spark/commit/9d9b4e1199bdeab7e454878bda61f0b5aecc79ad).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18999267
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.util;
    +
    +import java.util.NoSuchElementException;
    +
    +/**
    + * Provides a mechanism for constructing a {@link SluiceConfig} using some sort of configuration.
    + */
    +public abstract class ConfigProvider {
    +  /** Obtains the value of the given config, throws NoSuchElementException if it doesn't exist. */
    +  public abstract String get(String name);
    +
    +  public String get(String name, String defaultValue) {
    +    try {
    +      return get(name);
    +    } catch (NoSuchElementException e) {
    --- End diff --
    
    Subclasses may override this for a more efficient implementation, but since this is config that probably matters nearly 0. In terms of style, this is suboptimal, but I think offers the simplest (and most SparkConf-compatible) API. :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18809647
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    +
    +  val chunkCallback = new ChunkReceivedCallback {
    +    // On receipt of a chunk, pass it upwards as a block.
    +    def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions {
    +      buffer.retain()
    +      listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer)
    +    }
    +
    +    // On receipt of a failure, fail every block from chunkIndex onwards.
    +    def onFailure(chunkIndex: Int, e: Throwable): Unit = {
    +      blockIds.drop(chunkIndex).foreach { blockId =>
    +        listener.onBlockFetchFailure(blockId, e);
    +      }
    +    }
    +  }
    +
    +  // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle.
    +  client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(),
    --- End diff --
    
    actually - why don't we simply put this in fetchBlocks?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19439010
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClient.java ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow
    + * efficient transfer of a large amount of data, broken up into chunks with size ranging from
    + * hundreds of KB to a few MB.
    + *
    + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane),
    + * the actual setup of the streams is done outside the scope of the transport layer. The convenience
    + * method "sendRPC" is provided to enable control plane communication between the client and server
    + * to perform this setup.
    + *
    + * For example, a typical workflow might be:
    + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100
    + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback)
    + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback)
    + * ...
    + * client.sendRPC(new CloseStream(100))
    + *
    + * Construct an instance of TransportClient using {@link TransportClientFactory}. A single
    + * TransportClient may be used for multiple streams, but any given stream must be restricted to a
    + * single client, in order to avoid out-of-order responses.
    + *
    + * NB: This class is used to make requests to the server, while {@link TransportResponseHandler} is
    + * responsible for handling responses from the server.
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class TransportClient implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClient.class);
    +
    +  private final Channel channel;
    +  private final TransportResponseHandler handler;
    +
    +  public TransportClient(Channel channel, TransportResponseHandler handler) {
    +    this.channel = Preconditions.checkNotNull(channel);
    +    this.handler = Preconditions.checkNotNull(handler);
    +  }
    +
    +  public boolean isActive() {
    +    return channel.isOpen() || channel.isActive();
    +  }
    +
    +  /**
    +   * Requests a single chunk from the remote side, from the pre-negotiated streamId.
    +   *
    +   * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though
    +   * some streams may not support this.
    +   *
    +   * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed
    +   * to be returned in the same order that they were requested, assuming only a single
    +   * TransportClient is used to fetch the chunks.
    +   *
    +   * @param streamId Identifier that refers to a stream in the remote StreamManager. This should
    +   *                 be agreed upon by client and server beforehand.
    +   * @param chunkIndex 0-based index of the chunk to fetch
    +   * @param callback Callback invoked upon successful receipt of chunk, or upon any failure.
    +   */
    +  public void fetchChunk(
    +      long streamId,
    +      final int chunkIndex,
    +      final ChunkReceivedCallback callback) {
    +    final String serverAddr = NettyUtils.getRemoteAddress(channel);
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr);
    +
    +    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
    +    handler.addFetchRequest(streamChunkId, callback);
    +
    +    channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", streamChunkId, serverAddr,
    +              timeTaken);
    +          } else {
    +            String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
    +              serverAddr, future.cause());
    +            logger.error(errorMsg, future.cause());
    +            handler.removeFetchRequest(streamChunkId);
    +            callback.onFailure(chunkIndex, new RuntimeException(errorMsg, future.cause()));
    +          }
    +        }
    +      });
    +  }
    +
    +  /**
    +   * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked
    +   * with the server's response or upon any failure.
    +   */
    +  public void sendRpc(byte[] message, final RpcResponseCallback callback) {
    +    final String serverAddr = NettyUtils.getRemoteAddress(channel);
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending RPC to {}", serverAddr);
    --- End diff --
    
    trace?



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59590626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21868/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59584787
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21868/consoleFull) for   PR 2753 at commit [`ccd4959`](https://github.com/apache/spark/commit/ccd49595e8d0a730489e577b1152ad67027a5687).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19452657
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java ---
    @@ -0,0 +1,162 @@
    +/*
    + * 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.server;
    +
    +import java.util.Set;
    +
    +import com.google.common.base.Throwables;
    +import com.google.common.collect.Sets;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.buffer.ManagedBuffer;
    +import org.apache.spark.network.client.RpcResponseCallback;
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.protocol.Encodable;
    +import org.apache.spark.network.protocol.request.RequestMessage;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * A handler that processes requests from clients and writes chunk data back. Each handler is
    + * attached to a single Netty channel, and keeps track of which streams have been fetched via this
    + * channel, in order to clean them up if the channel is terminated (see #channelUnregistered).
    + *
    + * The messages should have been processed by the pipeline setup by {@link TransportServer}.
    + */
    +public class TransportRequestHandler extends MessageHandler<RequestMessage> {
    +  private final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class);
    +
    +  /** The Netty channel that this handler is associated with. */
    +  private final Channel channel;
    +
    +  /** Client on the same channel allowing us to talk back to the requester. */
    +  private final TransportClient reverseClient;
    +
    +  /** Returns each chunk part of a stream. */
    +  private final StreamManager streamManager;
    +
    +  /** Handles all RPC messages. */
    +  private final RpcHandler rpcHandler;
    +
    +  /** List of all stream ids that have been read on this handler, used for cleanup. */
    +  private final Set<Long> streamIds;
    +
    +  public TransportRequestHandler(
    +    Channel channel,
    --- End diff --
    
    4 space indent here


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59462479
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21837/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18810254
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/SluiceClientHandler.java ---
    @@ -0,0 +1,155 @@
    +/*
    + * 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.client;
    +
    +import java.net.SocketAddress;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.protocol.response.ServerResponse;
    +
    +/**
    + * Handler that processes server responses, in response to requests issued from [[SluiceClient]].
    + * It works by tracking the list of outstanding requests (and their callbacks).
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class SluiceClientHandler extends SimpleChannelInboundHandler<ServerResponse> {
    +  private final Logger logger = LoggerFactory.getLogger(SluiceClientHandler.class);
    +
    +  private final Map<StreamChunkId, ChunkReceivedCallback> outstandingFetches =
    +      new ConcurrentHashMap<StreamChunkId, ChunkReceivedCallback>();
    +
    +  private final Map<Long, RpcResponseCallback> outstandingRpcs =
    +      new ConcurrentHashMap<Long, RpcResponseCallback>();
    +
    +  public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) {
    +    outstandingFetches.put(streamChunkId, callback);
    +  }
    +
    +  public void removeFetchRequest(StreamChunkId streamChunkId) {
    +    outstandingFetches.remove(streamChunkId);
    +  }
    +
    +  public void addRpcRequest(long tag, RpcResponseCallback callback) {
    +    outstandingRpcs.put(tag, callback);
    +  }
    +
    +  public void removeRpcRequest(long tag) {
    +    outstandingRpcs.remove(tag);
    +  }
    +
    +  /**
    +   * Fire the failure callback for all outstanding requests. This is called when we have an
    +   * uncaught exception or pre-mature connection termination.
    +   */
    +  private void failOutstandingRequests(Throwable cause) {
    +    for (Map.Entry<StreamChunkId, ChunkReceivedCallback> entry : outstandingFetches.entrySet()) {
    +      entry.getValue().onFailure(entry.getKey().chunkIndex, cause);
    +    }
    +    // TODO(rxin): Maybe we need to synchronize the access? Otherwise we could clear new requests
    --- End diff --
    
    maybe remove the todo and explain why this is ok instead.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18749775
  
    --- Diff: core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala ---
    @@ -31,7 +34,7 @@ trait BlockFetchingListener extends EventListener {
       def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit
    --- End diff --
    
    i'd add to the doc that if data is going to be passed to a separate thread, call retain / releaes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18999168
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/SluiceClient.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.util.UUID;
    +
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +
    +/**
    + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow
    + * efficient transfer of a large amount of data, broken up into chunks with size ranging from
    + * hundreds of KB to a few MB.
    + *
    + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane),
    + * the actual setup of the streams is done outside the scope of Sluice. The convenience method
    + * "sendRPC" is provided to enable control plane communication between the client and server to
    + * perform this setup.
    + *
    + * For example, a typical workflow might be:
    + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100
    + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback)
    + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback)
    + * ...
    + * client.sendRPC(new CloseStream(100))
    + *
    + * Construct an instance of SluiceClient using {@link SluiceClientFactory}. A single SluiceClient
    + * may be used for multiple streams, but any given stream must be restricted to a single client,
    + * in order to avoid out-of-order responses.
    + *
    + * NB: This class is used to make requests to the server, while {@link SluiceClientHandler} is
    + * responsible for handling responses from the server.
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class SluiceClient implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(SluiceClient.class);
    +
    +  private final ChannelFuture cf;
    +  private final SluiceClientHandler handler;
    +
    +  private final String serverAddr;
    +
    +  SluiceClient(ChannelFuture cf, SluiceClientHandler handler) {
    +    this.cf = cf;
    +    this.handler = handler;
    +
    +    if (cf != null && cf.channel() != null && cf.channel().remoteAddress() != null) {
    +      serverAddr = cf.channel().remoteAddress().toString();
    +    } else {
    +      serverAddr = "<unknown address>";
    +    }
    +  }
    +
    +  public boolean isActive() {
    +    return cf.channel().isActive();
    +  }
    +
    +  /**
    +   * Requests a single chunk from the remote side, from the pre-negotiated streamId.
    +   *
    +   * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though
    +   * some streams may not support this.
    +   *
    +   * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed
    +   * to be returned in the same order that they were requested, assuming only a single SluiceClient
    +   * is used to fetch the chunks.
    +   *
    +   * @param streamId Identifier that refers to a stream in the remote StreamManager. This should
    +   *                 be agreed upon by client and server beforehand.
    +   * @param chunkIndex 0-based index of the chunk to fetch
    +   * @param callback Callback invoked upon successful receipt of chunk, or upon any failure.
    +   */
    +  public void fetchChunk(
    +      long streamId,
    +      final int chunkIndex,
    +      final ChunkReceivedCallback callback) {
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr);
    +
    +    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
    +    handler.addFetchRequest(streamChunkId, callback);
    +
    +    cf.channel().writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", streamChunkId, serverAddr,
    +                timeTaken);
    +          } else {
    +            // Fail all blocks.
    +            String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
    +              serverAddr, future.cause().getMessage());
    +            logger.error(errorMsg, future.cause());
    +            future.cause().printStackTrace();
    +            handler.removeFetchRequest(streamChunkId);
    +            callback.onFailure(chunkIndex, new RuntimeException(errorMsg));
    +          }
    +        }
    +      });
    +  }
    +
    +  /**
    +   * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked
    +   * with the server's response or upon any failure.
    +   */
    +  public void sendRpc(byte[] message, final RpcResponseCallback callback) {
    --- End diff --
    
    Issue here is that we'd have to move Serializer into the network layer. I think the bag-o'-bytes model works fine for a networking layer.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59627310
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21878/consoleFull) for   PR 2753 at commit [`e5675a4`](https://github.com/apache/spark/commit/e5675a4b919452c1eb68dd3a885f5d7f747e6014).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58621442
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21581/consoleFull) for   PR 2753 at commit [`29c6dcf`](https://github.com/apache/spark/commit/29c6dcfaacb2e8b1f0582c6d5e435349c52e29af).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58772869
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21651/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60849461
  
      [Test build #22394 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22394/consoleFull) for   PR 2753 at commit [`2b0d1c0`](https://github.com/apache/spark/commit/2b0d1c064899429eb115d984308eb18eebe7c9e0).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19452698
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/server/TransportServer.java ---
    @@ -0,0 +1,116 @@
    +/*
    + * 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.server;
    +
    +import java.io.Closeable;
    +import java.net.InetSocketAddress;
    +import java.util.concurrent.TimeUnit;
    +
    +import io.netty.bootstrap.ServerBootstrap;
    +import io.netty.buffer.PooledByteBufAllocator;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelInitializer;
    +import io.netty.channel.ChannelOption;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.socket.SocketChannel;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.TransportContext;
    +import org.apache.spark.network.util.IOMode;
    +import org.apache.spark.network.util.NettyUtils;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * Server for the efficient, low-level streaming service.
    + */
    +public class TransportServer implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportServer.class);
    +
    +  private final TransportContext context;
    +  private final TransportConf conf;
    +
    +  private ServerBootstrap bootstrap;
    +  private ChannelFuture channelFuture;
    +  private int port;
    +
    +  public TransportServer(TransportContext context) {
    +    this.context = context;
    +    this.conf = context.getConf();
    +
    +    init();
    +  }
    +
    +  public int getPort() { return port; }
    --- End diff --
    
    better check for whether this is initialized (maybe use negative port number in the beginning), and throw IllegalStateException if it is not.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18810145
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/SluiceClient.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.util.UUID;
    +
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +
    +/**
    + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow
    + * efficient transfer of a large amount of data, broken up into chunks with size ranging from
    + * hundreds of KB to a few MB.
    + *
    + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane),
    + * the actual setup of the streams is done outside the scope of Sluice. The convenience method
    + * "sendRPC" is provided to enable control plane communication between the client and server to
    + * perform this setup.
    + *
    + * For example, a typical workflow might be:
    + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100
    + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback)
    + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback)
    + * ...
    + * client.sendRPC(new CloseStream(100))
    + *
    + * Construct an instance of SluiceClient using {@link SluiceClientFactory}. A single SluiceClient
    + * may be used for multiple streams, but any given stream must be restricted to a single client,
    + * in order to avoid out-of-order responses.
    + *
    + * NB: This class is used to make requests to the server, while {@link SluiceClientHandler} is
    + * responsible for handling responses from the server.
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class SluiceClient implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(SluiceClient.class);
    +
    +  private final ChannelFuture cf;
    +  private final SluiceClientHandler handler;
    +
    +  private final String serverAddr;
    +
    +  SluiceClient(ChannelFuture cf, SluiceClientHandler handler) {
    +    this.cf = cf;
    +    this.handler = handler;
    +
    +    if (cf != null && cf.channel() != null && cf.channel().remoteAddress() != null) {
    +      serverAddr = cf.channel().remoteAddress().toString();
    +    } else {
    +      serverAddr = "<unknown address>";
    +    }
    +  }
    +
    +  public boolean isActive() {
    +    return cf.channel().isActive();
    +  }
    +
    +  /**
    +   * Requests a single chunk from the remote side, from the pre-negotiated streamId.
    +   *
    +   * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though
    +   * some streams may not support this.
    +   *
    +   * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed
    +   * to be returned in the same order that they were requested, assuming only a single SluiceClient
    +   * is used to fetch the chunks.
    +   *
    +   * @param streamId Identifier that refers to a stream in the remote StreamManager. This should
    +   *                 be agreed upon by client and server beforehand.
    +   * @param chunkIndex 0-based index of the chunk to fetch
    +   * @param callback Callback invoked upon successful receipt of chunk, or upon any failure.
    +   */
    +  public void fetchChunk(
    +      long streamId,
    +      final int chunkIndex,
    +      final ChunkReceivedCallback callback) {
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr);
    +
    +    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
    +    handler.addFetchRequest(streamChunkId, callback);
    +
    +    cf.channel().writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", streamChunkId, serverAddr,
    +                timeTaken);
    +          } else {
    +            // Fail all blocks.
    +            String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
    +              serverAddr, future.cause().getMessage());
    +            logger.error(errorMsg, future.cause());
    +            future.cause().printStackTrace();
    --- End diff --
    
    log instead of print?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19442525
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/server/TransportClientHandler.java ---
    @@ -0,0 +1,89 @@
    +/*
    + * 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.server;
    +
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.client.TransportResponseHandler;
    +import org.apache.spark.network.protocol.Message;
    +import org.apache.spark.network.protocol.request.RequestMessage;
    +import org.apache.spark.network.protocol.response.ResponseMessage;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * A handler which is used for delegating requests to the
    + * {@link TransportRequestHandler} and responses to the
    + * {@link org.apache.spark.network.client.TransportResponseHandler}.
    + *
    + * All channels created in the transport layer are bidirectional. When the Client initiates a Netty
    + * Channel with a RequestMessage (which gets handled by the Server's RequestHandler), the Server
    + * will produce a ResponseMessage (handled by the Client's ResponseHandler). However, the Server
    + * also gets a handle on the same Channel, so it may then begin to send RequestMessages to the
    + * Client.
    + * This means that the Client also needs a RequestHandler and the Server needs a ResponseHandler,
    + * for the Client's responses to the Server's requests.
    + */
    +public class TransportClientHandler extends SimpleChannelInboundHandler<Message> {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClientHandler.class);
    +
    +  private final TransportClient client;
    +  private final TransportResponseHandler responseHandler;
    +  private final TransportRequestHandler requestHandler;
    +
    +  public TransportClientHandler(
    +      TransportClient client,
    +      TransportResponseHandler responseHandler,
    +      TransportRequestHandler requestHandler) {
    +    this.client = client;
    +    this.responseHandler = responseHandler;
    +    this.requestHandler = requestHandler;
    +  }
    +
    +  public TransportClient getClient() {
    +    return client;
    +  }
    +
    +  @Override
    +  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +    logger.warn("Exception in connection from " + NettyUtils.getRemoteAddress(ctx.channel()),
    +      cause);
    +    requestHandler.exceptionCaught(cause);
    +    responseHandler.exceptionCaught(cause);
    +    ctx.close();
    +  }
    +
    +  @Override
    +  public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
    +    requestHandler.channelUnregistered();
    --- End diff --
    
    try catch?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19441087
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/request/RequestMessage.java ---
    @@ -15,11 +15,11 @@
      * limitations under the License.
      */
     
    -package org.apache.spark.network.netty
    +package org.apache.spark.network.protocol.request;
    --- End diff --
    
    it is simpler to just remove request / response package since we only have a small number of classes in protocol


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19383250
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.{SparkConf, Logging}
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.network.netty.NettyMessages._
    +import org.apache.spark.serializer.{JavaSerializer, Serializer}
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    --- End diff --
    
    can the fields here be private?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440361
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java ---
    @@ -0,0 +1,176 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.lang.reflect.Field;
    +import java.net.InetSocketAddress;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.TimeoutException;
    +
    +import io.netty.bootstrap.Bootstrap;
    +import io.netty.buffer.PooledByteBufAllocator;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelInitializer;
    +import io.netty.channel.ChannelOption;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.socket.SocketChannel;
    +import io.netty.util.internal.PlatformDependent;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.TransportContext;
    +import org.apache.spark.network.server.TransportClientHandler;
    +import org.apache.spark.network.util.IOMode;
    +import org.apache.spark.network.util.NettyUtils;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * Factory for creating {@link TransportClient}s by using createClient.
    + *
    + * The factory maintains a connection pool to other hosts and should return the same
    + * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for
    + * all {@link TransportClient}s.
    + */
    +public class TransportClientFactory implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
    +
    +  private final TransportContext context;
    +  private final TransportConf conf;
    +  private final ConcurrentHashMap<SocketAddress, TransportClient> connectionPool;
    +
    +  private final Class<? extends Channel> socketChannelClass;
    +  private final EventLoopGroup workerGroup;
    +
    +  public TransportClientFactory(TransportContext context) {
    +    this.context = context;
    +    this.conf = context.getConf();
    +    this.connectionPool = new ConcurrentHashMap<SocketAddress, TransportClient>();
    +
    +    IOMode ioMode = IOMode.valueOf(conf.ioMode());
    +    this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode);
    +    this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client");
    +  }
    +
    +  /**
    +   * Create a new BlockFetchingClient connecting to the given remote host / port.
    +   *
    +   * This blocks until a connection is successfully established.
    +   *
    +   * Concurrency: This method is safe to call from multiple threads.
    +   */
    +  public TransportClient createClient(String remoteHost, int remotePort) throws TimeoutException {
    +    // Get connection from the connection pool first.
    +    // If it is not found or not active, create a new one.
    +    final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort);
    +    TransportClient cachedClient = connectionPool.get(address);
    +    if (cachedClient != null && cachedClient.isActive()) {
    +      return cachedClient;
    +    } else if (cachedClient != null) {
    +      connectionPool.remove(address, cachedClient); // Remove inactive clients.
    +    }
    +
    +    logger.debug("Creating new connection to " + address);
    +
    +    Bootstrap bootstrap = new Bootstrap();
    +    bootstrap.group(workerGroup)
    +      .channel(socketChannelClass)
    +       // Disable Nagle's Algorithm since we don't want packets to wait
    +      .option(ChannelOption.TCP_NODELAY, true)
    +      .option(ChannelOption.SO_KEEPALIVE, true)
    +      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs());
    +
    +    // Use pooled buffers to reduce temporary buffer allocation
    +    bootstrap.option(ChannelOption.ALLOCATOR, createPooledByteBufAllocator());
    +
    +    bootstrap.handler(new ChannelInitializer<SocketChannel>() {
    +      @Override
    +      public void initChannel(SocketChannel ch) {
    +        TransportClientHandler channelHandler = context.initializePipeline(ch);
    +        TransportClient oldClient = connectionPool.putIfAbsent(address, channelHandler.getClient());
    +        if (oldClient != null) {
    +          logger.debug("Two clients were created concurrently, second one will be disposed.");
    +          ch.close();
    +          // Note: this type of failure is still considered a success by Netty, and thus the
    +          // ChannelFuture will complete successfully.
    +        }
    +      }
    +    });
    +
    +    // Connect to the remote server
    +    ChannelFuture cf = bootstrap.connect(address);
    +    if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) {
    +      throw new TimeoutException(
    +        String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs()));
    +    }
    +
    +    TransportClient client = connectionPool.get(address);
    +    if (client == null) {
    +      // The only way we should be able to reach here is if the client we created started out
    +      // in the "inactive" state, and someone else simultaneously tried to create another client to
    +      // the same server. This is an error condition, as the first client failed to connect.
    +      throw new IllegalStateException("Client was unset! Must have been immediately inactive.");
    +    }
    +    return client;
    +  }
    +
    +  /** Close all connections in the connection pool, and shutdown the worker thread pool. */
    +  @Override
    +  public void close() {
    +    for (TransportClient client : connectionPool.values()) {
    +      client.close();
    --- End diff --
    
    better to be more defensive and try catch here


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19438953
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClient.java ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow
    + * efficient transfer of a large amount of data, broken up into chunks with size ranging from
    + * hundreds of KB to a few MB.
    + *
    + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane),
    + * the actual setup of the streams is done outside the scope of the transport layer. The convenience
    + * method "sendRPC" is provided to enable control plane communication between the client and server
    + * to perform this setup.
    + *
    + * For example, a typical workflow might be:
    + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100
    + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback)
    + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback)
    + * ...
    + * client.sendRPC(new CloseStream(100))
    + *
    + * Construct an instance of TransportClient using {@link TransportClientFactory}. A single
    + * TransportClient may be used for multiple streams, but any given stream must be restricted to a
    + * single client, in order to avoid out-of-order responses.
    + *
    + * NB: This class is used to make requests to the server, while {@link TransportResponseHandler} is
    + * responsible for handling responses from the server.
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class TransportClient implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClient.class);
    +
    +  private final Channel channel;
    +  private final TransportResponseHandler handler;
    +
    +  public TransportClient(Channel channel, TransportResponseHandler handler) {
    +    this.channel = Preconditions.checkNotNull(channel);
    +    this.handler = Preconditions.checkNotNull(handler);
    +  }
    +
    +  public boolean isActive() {
    +    return channel.isOpen() || channel.isActive();
    +  }
    +
    +  /**
    +   * Requests a single chunk from the remote side, from the pre-negotiated streamId.
    +   *
    +   * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though
    +   * some streams may not support this.
    +   *
    +   * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed
    +   * to be returned in the same order that they were requested, assuming only a single
    +   * TransportClient is used to fetch the chunks.
    +   *
    +   * @param streamId Identifier that refers to a stream in the remote StreamManager. This should
    +   *                 be agreed upon by client and server beforehand.
    +   * @param chunkIndex 0-based index of the chunk to fetch
    +   * @param callback Callback invoked upon successful receipt of chunk, or upon any failure.
    +   */
    +  public void fetchChunk(
    +      long streamId,
    +      final int chunkIndex,
    +      final ChunkReceivedCallback callback) {
    +    final String serverAddr = NettyUtils.getRemoteAddress(channel);
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr);
    +
    +    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
    +    handler.addFetchRequest(streamChunkId, callback);
    +
    +    channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", streamChunkId, serverAddr,
    --- End diff --
    
    trace?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18810388
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/request/ClientRequestEncoder.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.request;
    +
    +import java.util.List;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.channel.ChannelHandler;
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.handler.codec.MessageToMessageEncoder;
    +
    +/**
    + * Encoder for {@link ClientRequest} used in client side.
    + *
    + * This encoder is stateless so it is safe to be shared by multiple threads.
    + */
    +@ChannelHandler.Sharable
    +public final class ClientRequestEncoder extends MessageToMessageEncoder<ClientRequest> {
    +  @Override
    +  public void encode(ChannelHandlerContext ctx, ClientRequest in, List<Object> out) {
    --- End diff --
    
    maybe comment on the fact that two things will be added to out.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59484064
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21852/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19560302
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java ---
    @@ -0,0 +1,182 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.lang.reflect.Field;
    +import java.net.InetSocketAddress;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import io.netty.bootstrap.Bootstrap;
    +import io.netty.buffer.PooledByteBufAllocator;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelInitializer;
    +import io.netty.channel.ChannelOption;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.socket.SocketChannel;
    +import io.netty.util.internal.PlatformDependent;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.TransportContext;
    +import org.apache.spark.network.server.TransportChannelHandler;
    +import org.apache.spark.network.util.IOMode;
    +import org.apache.spark.network.util.NettyUtils;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * Factory for creating {@link TransportClient}s by using createClient.
    + *
    + * The factory maintains a connection pool to other hosts and should return the same
    + * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for
    + * all {@link TransportClient}s.
    + */
    +public class TransportClientFactory implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
    +
    +  private final TransportContext context;
    +  private final TransportConf conf;
    +  private final ConcurrentHashMap<SocketAddress, TransportClient> connectionPool;
    +
    +  private final Class<? extends Channel> socketChannelClass;
    +  private final EventLoopGroup workerGroup;
    +
    +  public TransportClientFactory(TransportContext context) {
    +    this.context = context;
    +    this.conf = context.getConf();
    +    this.connectionPool = new ConcurrentHashMap<SocketAddress, TransportClient>();
    +
    +    IOMode ioMode = IOMode.valueOf(conf.ioMode());
    +    this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode);
    +    // TODO: Make thread pool name configurable.
    +    this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client");
    +  }
    +
    +  /**
    +   * Create a new BlockFetchingClient connecting to the given remote host / port.
    +   *
    +   * This blocks until a connection is successfully established.
    +   *
    +   * Concurrency: This method is safe to call from multiple threads.
    +   */
    +  public TransportClient createClient(String remoteHost, int remotePort) throws TimeoutException {
    +    // Get connection from the connection pool first.
    +    // If it is not found or not active, create a new one.
    +    final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort);
    +    TransportClient cachedClient = connectionPool.get(address);
    +    if (cachedClient != null && cachedClient.isActive()) {
    --- End diff --
    
    nit - it might be slightly more clear to write it this way
    ```scala
    if (cachedClient != null) {
      if (cachedClient.isActive) {
        return cachedClient;
      } else {
        connectionPool.remove(address, cachedClient); // Remove inactive clients.
      }
    }
    ```
    
    You can do it in the next PR.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440779
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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.client;
    +
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import io.netty.channel.Channel;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.response.ResponseMessage;
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.server.MessageHandler;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * Handler that processes server responses, in response to requests issued from a
    + * [[TransportClient]]. It works by tracking the list of outstanding requests (and their callbacks).
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class TransportResponseHandler extends MessageHandler<ResponseMessage> {
    +  private final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class);
    +
    +  private final Channel channel;
    +
    +  private final Map<StreamChunkId, ChunkReceivedCallback> outstandingFetches;
    +
    +  private final Map<Long, RpcResponseCallback> outstandingRpcs;
    +
    +  public TransportResponseHandler(Channel channel) {
    +    this.channel = channel;
    +    this.outstandingFetches = new ConcurrentHashMap<StreamChunkId, ChunkReceivedCallback>();
    +    this.outstandingRpcs = new ConcurrentHashMap<Long, RpcResponseCallback>();
    +  }
    +
    +  public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) {
    +    outstandingFetches.put(streamChunkId, callback);
    +  }
    +
    +  public void removeFetchRequest(StreamChunkId streamChunkId) {
    +    outstandingFetches.remove(streamChunkId);
    +  }
    +
    +  public void addRpcRequest(long tag, RpcResponseCallback callback) {
    +    outstandingRpcs.put(tag, callback);
    +  }
    +
    +  public void removeRpcRequest(long tag) {
    +    outstandingRpcs.remove(tag);
    +  }
    +
    +  /**
    +   * Fire the failure callback for all outstanding requests. This is called when we have an
    +   * uncaught exception or pre-mature connection termination.
    +   */
    +  private void failOutstandingRequests(Throwable cause) {
    +    for (Map.Entry<StreamChunkId, ChunkReceivedCallback> entry : outstandingFetches.entrySet()) {
    +      entry.getValue().onFailure(entry.getKey().chunkIndex, cause);
    +    }
    +    for (Map.Entry<Long, RpcResponseCallback> entry : outstandingRpcs.entrySet()) {
    +      entry.getValue().onFailure(cause);
    +    }
    +
    +    // It's OK if new fetches appear, as they will fail immediately.
    +    outstandingFetches.clear();
    +    outstandingRpcs.clear();
    +  }
    +
    +  @Override
    +  public void channelUnregistered() {
    +    if (numOutstandingRequests() > 0) {
    +      String remoteAddress = NettyUtils.getRemoteAddress(channel);
    +      logger.error("Still have {} requests outstanding when connection from {} is closed",
    +        numOutstandingRequests(), remoteAddress);
    +      failOutstandingRequests(new RuntimeException("Connection from " + remoteAddress + " closed"));
    +    }
    +  }
    +
    +  @Override
    +  public void exceptionCaught(Throwable cause) {
    +    if (numOutstandingRequests() > 0) {
    +      String remoteAddress = NettyUtils.getRemoteAddress(channel);
    +      logger.error("Still have {} requests outstanding when connection from {} is closed",
    +        numOutstandingRequests(), remoteAddress);
    +      failOutstandingRequests(cause);
    +    }
    +  }
    +
    +  @Override
    +  public void handle(ResponseMessage message) {
    +    String remoteAddress = NettyUtils.getRemoteAddress(channel);
    +    if (message instanceof ChunkFetchSuccess) {
    +      ChunkFetchSuccess resp = (ChunkFetchSuccess) message;
    +      ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId);
    +      if (listener == null) {
    +        logger.warn("Got a response for block {} from {} but it is not outstanding",
    --- End diff --
    
    add to the log "ignoring", .e.g
    
    "Ignoring response for block {} from {} since it is not outstanding"


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440961
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/Message.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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 io.netty.buffer.ByteBuf;
    +
    +/** Messages from the client to the server. */
    --- End diff --
    
    this is just a general msg, not specifically client to server right?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60552449
  
    @rxin All outstanding comments have been addressed and I have re-merged master.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59150237
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21758/consoleFull) for   PR 2753 at commit [`9d9b4e1`](https://github.com/apache/spark/commit/9d9b4e1199bdeab7e454878bda61f0b5aecc79ad).
     * This patch **passes all tests**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18811176
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/IOMode.java ---
    @@ -15,15 +15,13 @@
      * limitations under the License.
      */
     
    -package org.apache.spark.network.netty.client
    -
    -import java.util.EventListener
    -
    -
    -trait BlockClientListener extends EventListener {
    -
    -  def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit
    -
    -  def onFetchFailure(blockId: String, errorMsg: String): Unit
    +package org.apache.spark.network.util;
     
    +/**
    + * Selector for which form of low-level IO we should use.
    + * NIO is always available, while EPOLL is only available on certain machines.
    --- End diff --
    
    certain machines -> linux?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19364870
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/DefaultConfigProvider.java ---
    @@ -0,0 +1,32 @@
    +/*
    + * 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.util;
    +
    +import java.util.NoSuchElementException;
    +
    +/** Uses System properties to obtain config values. */
    +public class DefaultConfigProvider extends ConfigProvider {
    --- End diff --
    
    how about SystemPropertyConfigProvider?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19427924
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,95 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.{SparkConf, Logging}
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.network.netty.NettyMessages._
    +import org.apache.spark.serializer.{JavaSerializer, Serializer}
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, TransportClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: TransportClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  private val ser = serializer.newInstance()
    +
    +  private var streamHandle: ShuffleStreamHandle = _
    +
    +  private val chunkCallback = new ChunkReceivedCallback {
    +    // On receipt of a chunk, pass it upwards as a block.
    +    def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions {
    +      listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer)
    +    }
    +
    +    // On receipt of a failure, fail every block from chunkIndex onwards.
    +    def onFailure(chunkIndex: Int, e: Throwable): Unit = {
    +      blockIds.drop(chunkIndex).foreach { blockId =>
    +        listener.onBlockFetchFailure(blockId, e);
    +      }
    +    }
    +  }
    +
    +  /** Begins the fetching process, calling the listener with every block fetched. */
    +  def start(): Unit = {
    +    // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle.
    +    client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(),
    +      new RpcResponseCallback {
    +        override def onSuccess(response: Array[Byte]): Unit = {
    +          try {
    +            streamHandle = ser.deserialize[ShuffleStreamHandle](ByteBuffer.wrap(response))
    +            logTrace(s"Successfully opened block set: $streamHandle! Preparing to fetch chunks.")
    +
    +            // Immediately request all chunks -- we expect that the total size of the request is
    +            // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]].
    +            for (i <- 0 until streamHandle.numChunks) {
    +              client.fetchChunk(streamHandle.streamId, i, chunkCallback)
    +            }
    +          } catch {
    +            case e: Exception =>
    +              logError("Failed while starting block fetches", e)
    +              blockIds.foreach(blockId => Utils.tryLog(listener.onBlockFetchFailure(blockId, e)))
    +          }
    +        }
    +
    +        override def onFailure(e: Throwable): Unit = {
    +          logError("Failed while starting block fetches")
    --- End diff --
    
    log e here as well?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440233
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java ---
    @@ -0,0 +1,176 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.lang.reflect.Field;
    +import java.net.InetSocketAddress;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.TimeoutException;
    +
    +import io.netty.bootstrap.Bootstrap;
    +import io.netty.buffer.PooledByteBufAllocator;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelInitializer;
    +import io.netty.channel.ChannelOption;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.socket.SocketChannel;
    +import io.netty.util.internal.PlatformDependent;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.TransportContext;
    +import org.apache.spark.network.server.TransportClientHandler;
    +import org.apache.spark.network.util.IOMode;
    +import org.apache.spark.network.util.NettyUtils;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * Factory for creating {@link TransportClient}s by using createClient.
    + *
    + * The factory maintains a connection pool to other hosts and should return the same
    + * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for
    + * all {@link TransportClient}s.
    + */
    +public class TransportClientFactory implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
    +
    +  private final TransportContext context;
    +  private final TransportConf conf;
    +  private final ConcurrentHashMap<SocketAddress, TransportClient> connectionPool;
    +
    +  private final Class<? extends Channel> socketChannelClass;
    +  private final EventLoopGroup workerGroup;
    +
    +  public TransportClientFactory(TransportContext context) {
    +    this.context = context;
    +    this.conf = context.getConf();
    +    this.connectionPool = new ConcurrentHashMap<SocketAddress, TransportClient>();
    +
    +    IOMode ioMode = IOMode.valueOf(conf.ioMode());
    +    this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode);
    +    this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client");
    +  }
    +
    +  /**
    +   * Create a new BlockFetchingClient connecting to the given remote host / port.
    +   *
    +   * This blocks until a connection is successfully established.
    +   *
    +   * Concurrency: This method is safe to call from multiple threads.
    +   */
    +  public TransportClient createClient(String remoteHost, int remotePort) throws TimeoutException {
    +    // Get connection from the connection pool first.
    +    // If it is not found or not active, create a new one.
    +    final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort);
    +    TransportClient cachedClient = connectionPool.get(address);
    +    if (cachedClient != null && cachedClient.isActive()) {
    +      return cachedClient;
    +    } else if (cachedClient != null) {
    +      connectionPool.remove(address, cachedClient); // Remove inactive clients.
    +    }
    +
    +    logger.debug("Creating new connection to " + address);
    +
    +    Bootstrap bootstrap = new Bootstrap();
    +    bootstrap.group(workerGroup)
    +      .channel(socketChannelClass)
    +       // Disable Nagle's Algorithm since we don't want packets to wait
    +      .option(ChannelOption.TCP_NODELAY, true)
    +      .option(ChannelOption.SO_KEEPALIVE, true)
    +      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs());
    +
    +    // Use pooled buffers to reduce temporary buffer allocation
    +    bootstrap.option(ChannelOption.ALLOCATOR, createPooledByteBufAllocator());
    +
    +    bootstrap.handler(new ChannelInitializer<SocketChannel>() {
    +      @Override
    +      public void initChannel(SocketChannel ch) {
    +        TransportClientHandler channelHandler = context.initializePipeline(ch);
    +        TransportClient oldClient = connectionPool.putIfAbsent(address, channelHandler.getClient());
    +        if (oldClient != null) {
    +          logger.debug("Two clients were created concurrently, second one will be disposed.");
    +          ch.close();
    +          // Note: this type of failure is still considered a success by Netty, and thus the
    +          // ChannelFuture will complete successfully.
    +        }
    +      }
    +    });
    +
    +    // Connect to the remote server
    +    ChannelFuture cf = bootstrap.connect(address);
    +    if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) {
    --- End diff --
    
    add a check to make sure the channel is actually being established.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18811632
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.util;
    +
    +import java.util.concurrent.ThreadFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +import io.netty.channel.Channel;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.ServerChannel;
    +import io.netty.channel.epoll.Epoll;
    +import io.netty.channel.epoll.EpollEventLoopGroup;
    +import io.netty.channel.epoll.EpollServerSocketChannel;
    +import io.netty.channel.epoll.EpollSocketChannel;
    +import io.netty.channel.nio.NioEventLoopGroup;
    +import io.netty.channel.socket.nio.NioServerSocketChannel;
    +import io.netty.channel.socket.nio.NioSocketChannel;
    +import io.netty.handler.codec.ByteToMessageDecoder;
    +import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    +
    +/**
    + * Utilities for creating various Netty constructs based on whether we're using EPOLL or NIO.
    + */
    +public class NettyUtils {
    +  /** Creates a Netty EventLoopGroup based on the IOMode. */
    +  public static EventLoopGroup createEventLoop(IOMode mode, int numThreads, String threadPrefix) {
    +    if (mode == IOMode.AUTO) {
    +      mode = autoselectMode();
    +    }
    +
    +    ThreadFactory threadFactory = new ThreadFactoryBuilder()
    +        .setDaemon(true)
    +        .setNameFormat(threadPrefix + "-%d")
    +        .build();
    +
    +    switch(mode) {
    --- End diff --
    
    nitpick: space after switch


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19441849
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java ---
    @@ -0,0 +1,99 @@
    +/*
    + * 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.server;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.buffer.ManagedBuffer;
    +
    +/**
    + * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are individually
    + * fetched as chunks by the client.
    + */
    +public class DefaultStreamManager extends StreamManager {
    +  private final Logger logger = LoggerFactory.getLogger(DefaultStreamManager.class);
    +
    +  private final AtomicLong nextStreamId;
    +  private final Map<Long, StreamState> streams;
    +
    +  /** State of a single stream. */
    +  private static class StreamState {
    +    final Iterator<ManagedBuffer> buffers;
    +
    +    int curChunk = 0;
    +
    +    StreamState(Iterator<ManagedBuffer> buffers) {
    +      this.buffers = buffers;
    +    }
    +  }
    +
    +  public DefaultStreamManager() {
    +    // Start with a random stream id to help identifying different streams.
    --- End diff --
    
    update the comment to explain this is only for debugging purposes and the system actually does not need the guarantee that stream id is unique


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60556917
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22275/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58626122
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21585/Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19383297
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.{SparkConf, Logging}
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.network.netty.NettyMessages._
    +import org.apache.spark.serializer.{JavaSerializer, Serializer}
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    +
    +  val chunkCallback = new ChunkReceivedCallback {
    +    // On receipt of a chunk, pass it upwards as a block.
    +    def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions {
    +      listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer)
    +    }
    +
    +    // On receipt of a failure, fail every block from chunkIndex onwards.
    +    def onFailure(chunkIndex: Int, e: Throwable): Unit = {
    +      blockIds.drop(chunkIndex).foreach { blockId =>
    +        listener.onBlockFetchFailure(blockId, e);
    +      }
    +    }
    +  }
    +
    +  // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle.
    +  client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(),
    +    new RpcResponseCallback {
    +      override def onSuccess(response: Array[Byte]): Unit = {
    +        try {
    +          streamHandle = ser.deserialize[ShuffleStreamHandle](ByteBuffer.wrap(response))
    +          logTrace(s"Successfully opened block set: $streamHandle! Preparing to fetch chunks.")
    +
    +          // Immediately request all chunks -- we expect that the total size of the request is
    +          // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]].
    +          for (i <- 0 until streamHandle.numChunks) {
    +            client.fetchChunk(streamHandle.streamId, i, chunkCallback)
    +          }
    +        } catch {
    +          case e: Exception =>
    +            logError("Failed while starting block fetches", e)
    +            blockIds.foreach(listener.onBlockFetchFailure(_, e))
    --- End diff --
    
    wrap this around a try catch block so we call all block ids even if onBlockFetchFailure throws an exception?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19438223
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java ---
    @@ -0,0 +1,76 @@
    +/*
    + * 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.buffer;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +
    +import com.google.common.base.Objects;
    +import io.netty.buffer.ByteBuf;
    +import io.netty.buffer.ByteBufInputStream;
    +
    +/**
    + * A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}.
    + */
    +public final class NettyManagedBuffer extends ManagedBuffer {
    +  private final ByteBuf buf;
    +
    +  public NettyManagedBuffer(ByteBuf buf) {
    +    this.buf = buf;
    +  }
    +
    +  @Override
    +  public long size() {
    +    return buf.readableBytes();
    +  }
    +
    +  @Override
    +  public ByteBuffer nioByteBuffer() throws IOException {
    +    return buf.nioBuffer();
    +  }
    +
    +  @Override
    +  public InputStream inputStream() throws IOException {
    +    return new ByteBufInputStream(buf);
    +  }
    +
    +  @Override
    +  public ManagedBuffer retain() {
    +    buf.retain();
    +    return this;
    +  }
    +
    +  @Override
    +  public ManagedBuffer release() {
    +    buf.release();
    +    return this;
    +  }
    +
    +  @Override
    +  public Object convertToNetty() throws IOException {
    +    return buf;
    --- End diff --
    
    create a duplicate view of this so caller cannot change the position of the underlying buffer.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60874155
  
      [Test build #22421 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22421/consoleFull) for   PR 2753 at commit [`d7be11b`](https://github.com/apache/spark/commit/d7be11b74f6e4ccede5f783742b88ecffeb19add).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58621380
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440492
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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.client;
    +
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import io.netty.channel.Channel;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.response.ResponseMessage;
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.server.MessageHandler;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * Handler that processes server responses, in response to requests issued from a
    + * [[TransportClient]]. It works by tracking the list of outstanding requests (and their callbacks).
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class TransportResponseHandler extends MessageHandler<ResponseMessage> {
    +  private final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class);
    +
    +  private final Channel channel;
    +
    +  private final Map<StreamChunkId, ChunkReceivedCallback> outstandingFetches;
    +
    +  private final Map<Long, RpcResponseCallback> outstandingRpcs;
    +
    +  public TransportResponseHandler(Channel channel) {
    +    this.channel = channel;
    +    this.outstandingFetches = new ConcurrentHashMap<StreamChunkId, ChunkReceivedCallback>();
    +    this.outstandingRpcs = new ConcurrentHashMap<Long, RpcResponseCallback>();
    +  }
    +
    +  public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) {
    +    outstandingFetches.put(streamChunkId, callback);
    +  }
    +
    +  public void removeFetchRequest(StreamChunkId streamChunkId) {
    +    outstandingFetches.remove(streamChunkId);
    +  }
    +
    +  public void addRpcRequest(long tag, RpcResponseCallback callback) {
    --- End diff --
    
    can u explain what tag is


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19509298
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java ---
    @@ -0,0 +1,162 @@
    +/*
    + * 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.server;
    +
    +import java.util.Set;
    +
    +import com.google.common.base.Throwables;
    +import com.google.common.collect.Sets;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.buffer.ManagedBuffer;
    +import org.apache.spark.network.client.RpcResponseCallback;
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.protocol.Encodable;
    +import org.apache.spark.network.protocol.request.RequestMessage;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * A handler that processes requests from clients and writes chunk data back. Each handler is
    + * attached to a single Netty channel, and keeps track of which streams have been fetched via this
    + * channel, in order to clean them up if the channel is terminated (see #channelUnregistered).
    + *
    + * The messages should have been processed by the pipeline setup by {@link TransportServer}.
    + */
    +public class TransportRequestHandler extends MessageHandler<RequestMessage> {
    +  private final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class);
    +
    +  /** The Netty channel that this handler is associated with. */
    +  private final Channel channel;
    +
    +  /** Client on the same channel allowing us to talk back to the requester. */
    +  private final TransportClient reverseClient;
    +
    +  /** Returns each chunk part of a stream. */
    +  private final StreamManager streamManager;
    +
    +  /** Handles all RPC messages. */
    +  private final RpcHandler rpcHandler;
    +
    +  /** List of all stream ids that have been read on this handler, used for cleanup. */
    +  private final Set<Long> streamIds;
    +
    +  public TransportRequestHandler(
    +    Channel channel,
    +    TransportClient reverseClient,
    +    StreamManager streamManager,
    +    RpcHandler rpcHandler) {
    +    this.channel = channel;
    +    this.reverseClient = reverseClient;
    +    this.streamManager = streamManager;
    +    this.rpcHandler = rpcHandler;
    +    this.streamIds = Sets.newHashSet();
    +  }
    +
    +  @Override
    +  public void exceptionCaught(Throwable cause) {
    --- End diff --
    
    This is done by TransportChannelHandler, which invokes this method.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60708006
  
    @aarondav I'm done with the pass. Let me know when you address the comment and I will merge it (once Jenkins is happy). 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19388855
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.{SparkConf, Logging}
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.network.netty.NettyMessages._
    +import org.apache.spark.serializer.{JavaSerializer, Serializer}
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    --- End diff --
    
    Absolutely!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60874159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22421/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18811627
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/request/ClientRequest.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.request;
    +
    +import io.netty.buffer.ByteBuf;
    +
    +import org.apache.spark.network.protocol.Encodable;
    +
    +/** Messages from the client to the server. */
    +public interface ClientRequest extends Encodable {
    +  /** Used to identify this request type. */
    +  Type type();
    +
    +  /**
    +   * Preceding every serialized ClientRequest is the type, which allows us to deserialize
    +   * the request.
    +   */
    +  public static enum Type implements Encodable {
    +    ChunkFetchRequest(0), RpcRequest(1);
    +
    +    private final byte id;
    +
    +    private Type(int id) {
    +      assert id < 128 : "Cannot have more than 128 request types";
    +      this.id = (byte) id;
    +    }
    +
    +    public byte id() { return id; }
    +
    +    @Override public int encodedLength() { return 1; }
    +
    +    @Override public void encode(ByteBuf buf) { buf.writeByte(id); }
    +
    +    public static Type decode(ByteBuf buf) {
    +      byte id = buf.readByte();
    +      switch(id) {
    --- End diff --
    
    nitpick: space after switch


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19438149
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.buffer;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +
    +/**
    + * This interface provides an immutable view for data in the form of bytes. The implementation
    + * should specify how the data is provided:
    + *
    + * - {@link FileSegmentManagedBuffer}: data backed by part of a file
    + * - {@link NioManagedBuffer}: data backed by a NIO ByteBuffer
    + * - {@link NettyManagedBuffer}: data backed by a Netty ByteBuf
    + *
    + * The concrete buffer implementation might be managed outside the JVM garbage collector.
    + * For example, in the case of {@link NettyManagedBuffer}, the buffers are reference counted.
    + * In that case, if the buffer is going to be passed around to a different thread, retain/release
    + * should be called.
    + */
    +public abstract class ManagedBuffer {
    +
    +  /** Number of bytes of the data. */
    +  public abstract long size();
    +
    +  /**
    +   * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the
    +   * returned ByteBuffer should not affect the content of this buffer.
    +   */
    +  public abstract ByteBuffer nioByteBuffer() throws IOException;
    --- End diff --
    
    maybe comment on in the future, everything in spark should just operate on ManagedBuffer abstraction and we should deprecate nio bytebuffer.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440090
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java ---
    @@ -0,0 +1,176 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.lang.reflect.Field;
    +import java.net.InetSocketAddress;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.TimeoutException;
    +
    +import io.netty.bootstrap.Bootstrap;
    +import io.netty.buffer.PooledByteBufAllocator;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelInitializer;
    +import io.netty.channel.ChannelOption;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.socket.SocketChannel;
    +import io.netty.util.internal.PlatformDependent;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.TransportContext;
    +import org.apache.spark.network.server.TransportClientHandler;
    +import org.apache.spark.network.util.IOMode;
    +import org.apache.spark.network.util.NettyUtils;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * Factory for creating {@link TransportClient}s by using createClient.
    + *
    + * The factory maintains a connection pool to other hosts and should return the same
    + * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for
    + * all {@link TransportClient}s.
    + */
    +public class TransportClientFactory implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
    +
    +  private final TransportContext context;
    +  private final TransportConf conf;
    +  private final ConcurrentHashMap<SocketAddress, TransportClient> connectionPool;
    +
    +  private final Class<? extends Channel> socketChannelClass;
    +  private final EventLoopGroup workerGroup;
    +
    +  public TransportClientFactory(TransportContext context) {
    +    this.context = context;
    +    this.conf = context.getConf();
    +    this.connectionPool = new ConcurrentHashMap<SocketAddress, TransportClient>();
    +
    +    IOMode ioMode = IOMode.valueOf(conf.ioMode());
    +    this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode);
    +    this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client");
    +  }
    +
    +  /**
    +   * Create a new BlockFetchingClient connecting to the given remote host / port.
    +   *
    +   * This blocks until a connection is successfully established.
    +   *
    +   * Concurrency: This method is safe to call from multiple threads.
    +   */
    +  public TransportClient createClient(String remoteHost, int remotePort) throws TimeoutException {
    +    // Get connection from the connection pool first.
    +    // If it is not found or not active, create a new one.
    +    final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort);
    +    TransportClient cachedClient = connectionPool.get(address);
    +    if (cachedClient != null && cachedClient.isActive()) {
    +      return cachedClient;
    +    } else if (cachedClient != null) {
    +      connectionPool.remove(address, cachedClient); // Remove inactive clients.
    +    }
    +
    +    logger.debug("Creating new connection to " + address);
    +
    +    Bootstrap bootstrap = new Bootstrap();
    +    bootstrap.group(workerGroup)
    +      .channel(socketChannelClass)
    +       // Disable Nagle's Algorithm since we don't want packets to wait
    +      .option(ChannelOption.TCP_NODELAY, true)
    +      .option(ChannelOption.SO_KEEPALIVE, true)
    +      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs());
    +
    +    // Use pooled buffers to reduce temporary buffer allocation
    +    bootstrap.option(ChannelOption.ALLOCATOR, createPooledByteBufAllocator());
    +
    +    bootstrap.handler(new ChannelInitializer<SocketChannel>() {
    +      @Override
    +      public void initChannel(SocketChannel ch) {
    +        TransportClientHandler channelHandler = context.initializePipeline(ch);
    +        TransportClient oldClient = connectionPool.putIfAbsent(address, channelHandler.getClient());
    +        if (oldClient != null) {
    +          logger.debug("Two clients were created concurrently, second one will be disposed.");
    +          ch.close();
    --- End diff --
    
    this is neat... kinda scary though


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59475022
  
    @rxin Thanks for the preliminary pass. I've updated the PR to include UploadBlocks and to make communication bidirectionally possible (though this functionality is not used yet, I anticipate its eventual use for "real" upload when efficiency is required). 
    
    I have also completed significant testing on a cluster of r3.2xlarge and r3.8xlarge, including correctness testing as well as raw perf testing. We can max out the 10GigE on the r3.8xlarge using low CPU utilization. In a separate test between containers on the same machine, I was able to see ~7 GB/s with roughly 10% total CPU utilization (mainly split between interrupt servicing and sys). It's not clear what the bottleneck was at that speed, but 7GB/s should be sufficient for most real networks at present and so I did not investigate further.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58622482
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21582/Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59141519
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21758/consoleFull) for   PR 2753 at commit [`9d9b4e1`](https://github.com/apache/spark/commit/9d9b4e1199bdeab7e454878bda61f0b5aecc79ad).
     * This patch **does not merge cleanly**.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60556908
  
      [Test build #22275 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22275/consoleFull) for   PR 2753 at commit [`8dfcceb`](https://github.com/apache/spark/commit/8dfcceb5127b638ece6817e7858c6cbf93461cd6).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19428104
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala ---
    @@ -0,0 +1,76 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.network.BlockDataManager
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer}
    +import org.apache.spark.network.client.{TransportClient, RpcResponseCallback}
    +import org.apache.spark.network.server.{DefaultStreamManager, RpcHandler}
    +import org.apache.spark.storage.{StorageLevel, BlockId}
    +
    +import scala.collection.JavaConversions._
    +
    +object NettyMessages {
    +
    +  /** Request to read a set of blocks. Returns [[ShuffleStreamHandle]] to identify the stream. */
    +  case class OpenBlocks(blockIds: Seq[BlockId])
    +
    +  /** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */
    +  case class UploadBlock(blockId: BlockId, blockData: Array[Byte], level: StorageLevel)
    +
    +  /** Identifier for a fixed number of chunks to read from a stream created by [[OpenBlocks]]. */
    +  case class ShuffleStreamHandle(streamId: Long, numChunks: Int)
    +}
    +
    +/**
    + * Serves requests to open blocks by simply registering one chunk per block requested.
    + */
    +class NettyBlockRpcServer(
    +    serializer: Serializer,
    +    streamManager: DefaultStreamManager,
    +    blockManager: BlockDataManager)
    +  extends RpcHandler with Logging {
    +
    +  import NettyMessages._
    +
    +  override def receive(
    +      client: TransportClient,
    +      messageBytes: Array[Byte],
    +      responseContext: RpcResponseCallback): Unit = {
    +    val ser = serializer.newInstance()
    +    val message = ser.deserialize[AnyRef](ByteBuffer.wrap(messageBytes))
    --- End diff --
    
    can you file a jira ticket to let serializer work on ByteBuf or ManagedBuffer? So we don't need to allocate extra memory in RPC.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19439891
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClient.java ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow
    + * efficient transfer of a large amount of data, broken up into chunks with size ranging from
    + * hundreds of KB to a few MB.
    + *
    + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane),
    + * the actual setup of the streams is done outside the scope of the transport layer. The convenience
    + * method "sendRPC" is provided to enable control plane communication between the client and server
    + * to perform this setup.
    + *
    + * For example, a typical workflow might be:
    + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100
    + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback)
    + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback)
    + * ...
    + * client.sendRPC(new CloseStream(100))
    + *
    + * Construct an instance of TransportClient using {@link TransportClientFactory}. A single
    + * TransportClient may be used for multiple streams, but any given stream must be restricted to a
    + * single client, in order to avoid out-of-order responses.
    + *
    + * NB: This class is used to make requests to the server, while {@link TransportResponseHandler} is
    + * responsible for handling responses from the server.
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class TransportClient implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClient.class);
    +
    +  private final Channel channel;
    +  private final TransportResponseHandler handler;
    +
    +  public TransportClient(Channel channel, TransportResponseHandler handler) {
    +    this.channel = Preconditions.checkNotNull(channel);
    +    this.handler = Preconditions.checkNotNull(handler);
    +  }
    +
    +  public boolean isActive() {
    +    return channel.isOpen() || channel.isActive();
    +  }
    +
    +  /**
    +   * Requests a single chunk from the remote side, from the pre-negotiated streamId.
    +   *
    +   * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though
    +   * some streams may not support this.
    +   *
    +   * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed
    +   * to be returned in the same order that they were requested, assuming only a single
    +   * TransportClient is used to fetch the chunks.
    +   *
    +   * @param streamId Identifier that refers to a stream in the remote StreamManager. This should
    +   *                 be agreed upon by client and server beforehand.
    +   * @param chunkIndex 0-based index of the chunk to fetch
    +   * @param callback Callback invoked upon successful receipt of chunk, or upon any failure.
    +   */
    +  public void fetchChunk(
    +      long streamId,
    +      final int chunkIndex,
    +      final ChunkReceivedCallback callback) {
    +    final String serverAddr = NettyUtils.getRemoteAddress(channel);
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr);
    +
    +    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
    +    handler.addFetchRequest(streamChunkId, callback);
    +
    +    channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", streamChunkId, serverAddr,
    +              timeTaken);
    +          } else {
    +            String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
    --- End diff --
    
    i think we should close the connection here if we fail to flush


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58624889
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60978531
  
    Merging this in master. Thanks!



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19440622
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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.client;
    +
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import io.netty.channel.Channel;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.response.ResponseMessage;
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.server.MessageHandler;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * Handler that processes server responses, in response to requests issued from a
    + * [[TransportClient]]. It works by tracking the list of outstanding requests (and their callbacks).
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class TransportResponseHandler extends MessageHandler<ResponseMessage> {
    +  private final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class);
    +
    +  private final Channel channel;
    +
    +  private final Map<StreamChunkId, ChunkReceivedCallback> outstandingFetches;
    +
    +  private final Map<Long, RpcResponseCallback> outstandingRpcs;
    +
    +  public TransportResponseHandler(Channel channel) {
    +    this.channel = channel;
    +    this.outstandingFetches = new ConcurrentHashMap<StreamChunkId, ChunkReceivedCallback>();
    +    this.outstandingRpcs = new ConcurrentHashMap<Long, RpcResponseCallback>();
    +  }
    +
    +  public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) {
    +    outstandingFetches.put(streamChunkId, callback);
    +  }
    +
    +  public void removeFetchRequest(StreamChunkId streamChunkId) {
    +    outstandingFetches.remove(streamChunkId);
    +  }
    +
    +  public void addRpcRequest(long tag, RpcResponseCallback callback) {
    --- End diff --
    
    actually makes more sense to rename it rpcRequestId or just requestId


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19439584
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/TransportContext.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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;
    +
    +import io.netty.channel.Channel;
    +import io.netty.channel.socket.SocketChannel;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.client.TransportClientFactory;
    +import org.apache.spark.network.client.TransportResponseHandler;
    +import org.apache.spark.network.protocol.response.MessageDecoder;
    +import org.apache.spark.network.protocol.response.MessageEncoder;
    +import org.apache.spark.network.server.RpcHandler;
    +import org.apache.spark.network.server.TransportClientHandler;
    +import org.apache.spark.network.server.TransportRequestHandler;
    +import org.apache.spark.network.server.TransportServer;
    +import org.apache.spark.network.server.StreamManager;
    +import org.apache.spark.network.util.NettyUtils;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * Contains the context to create a {@link TransportServer}, {@link TransportClientFactory}, and to
    + * setup Netty Channel pipelines with a {@link TransportClientHandler}.
    + *
    + * The TransportServer and TransportClientFactory both create a TransportChannelHandler for each
    + * channel. As each TransportChannelHandler contains a TransportClient, this enables server
    + * processes to send messages back to the client on an existing channel.
    + */
    +public class TransportContext {
    --- End diff --
    
    we should also explain here that we have two comm methods: rpc (control) and chunks (data). We should explain briefly the difference of the two, i.e. zero-copy and chunking.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18810206
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/SluiceClient.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.util.UUID;
    +
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +
    +/**
    + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow
    + * efficient transfer of a large amount of data, broken up into chunks with size ranging from
    + * hundreds of KB to a few MB.
    + *
    + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane),
    + * the actual setup of the streams is done outside the scope of Sluice. The convenience method
    + * "sendRPC" is provided to enable control plane communication between the client and server to
    + * perform this setup.
    + *
    + * For example, a typical workflow might be:
    + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100
    + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback)
    + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback)
    + * ...
    + * client.sendRPC(new CloseStream(100))
    + *
    + * Construct an instance of SluiceClient using {@link SluiceClientFactory}. A single SluiceClient
    + * may be used for multiple streams, but any given stream must be restricted to a single client,
    + * in order to avoid out-of-order responses.
    + *
    + * NB: This class is used to make requests to the server, while {@link SluiceClientHandler} is
    + * responsible for handling responses from the server.
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class SluiceClient implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(SluiceClient.class);
    +
    +  private final ChannelFuture cf;
    +  private final SluiceClientHandler handler;
    +
    +  private final String serverAddr;
    +
    +  SluiceClient(ChannelFuture cf, SluiceClientHandler handler) {
    +    this.cf = cf;
    +    this.handler = handler;
    +
    +    if (cf != null && cf.channel() != null && cf.channel().remoteAddress() != null) {
    +      serverAddr = cf.channel().remoteAddress().toString();
    +    } else {
    +      serverAddr = "<unknown address>";
    +    }
    +  }
    +
    +  public boolean isActive() {
    +    return cf.channel().isActive();
    +  }
    +
    +  /**
    +   * Requests a single chunk from the remote side, from the pre-negotiated streamId.
    +   *
    +   * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though
    +   * some streams may not support this.
    +   *
    +   * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed
    +   * to be returned in the same order that they were requested, assuming only a single SluiceClient
    +   * is used to fetch the chunks.
    +   *
    +   * @param streamId Identifier that refers to a stream in the remote StreamManager. This should
    +   *                 be agreed upon by client and server beforehand.
    +   * @param chunkIndex 0-based index of the chunk to fetch
    +   * @param callback Callback invoked upon successful receipt of chunk, or upon any failure.
    +   */
    +  public void fetchChunk(
    +      long streamId,
    +      final int chunkIndex,
    +      final ChunkReceivedCallback callback) {
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr);
    +
    +    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
    +    handler.addFetchRequest(streamChunkId, callback);
    +
    +    cf.channel().writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", streamChunkId, serverAddr,
    +                timeTaken);
    +          } else {
    +            // Fail all blocks.
    +            String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
    +              serverAddr, future.cause().getMessage());
    +            logger.error(errorMsg, future.cause());
    +            future.cause().printStackTrace();
    +            handler.removeFetchRequest(streamChunkId);
    +            callback.onFailure(chunkIndex, new RuntimeException(errorMsg));
    +          }
    +        }
    +      });
    +  }
    +
    +  /**
    +   * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked
    +   * with the server's response or upon any failure.
    +   */
    +  public void sendRpc(byte[] message, final RpcResponseCallback callback) {
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending RPC to {}", serverAddr);
    +
    +    final long tag = UUID.randomUUID().getLeastSignificantBits();
    +    handler.addRpcRequest(tag, callback);
    +
    +    cf.channel().writeAndFlush(new RpcRequest(tag, message)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", tag, serverAddr, timeTaken);
    +          } else {
    +            // Fail all blocks.
    +            String errorMsg = String.format("Failed to send request %s to %s: %s", tag,
    +                serverAddr, future.cause().getMessage());
    +            logger.error(errorMsg, future.cause());
    +            handler.removeRpcRequest(tag);
    +            callback.onFailure(new RuntimeException(errorMsg));
    --- End diff --
    
    pass cause also into the RuntimeException?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19437262
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala ---
    @@ -24,14 +24,14 @@ import java.util.concurrent.atomic.AtomicInteger
     
     import scala.collection.JavaConversions._
     
    -import org.apache.spark.{SparkEnv, SparkConf, Logging}
     import org.apache.spark.executor.ShuffleWriteMetrics
    -import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer}
    +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
     import org.apache.spark.serializer.Serializer
     import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup
     import org.apache.spark.storage._
    -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap}
     import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector}
    +import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap}
    --- End diff --
    
    import out of order here


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60880960
  
      [Test build #22437 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22437/consoleFull) for   PR 2753 at commit [`cadfd28`](https://github.com/apache/spark/commit/cadfd28f116f0dbca11e580a23caf82060bcf922).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19441656
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java ---
    @@ -0,0 +1,35 @@
    +/*
    + * 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 io.netty.buffer.ByteBuf;
    +
    +/**
    + * Interface for an object which can be encoded into a ByteBuf. Multiple Encodable objects are
    + * stored in a single, pre-allocated ByteBuf, so Encodables must also provide their length.
    --- End diff --
    
    we should also add here that Encodable must also have a static decode method, in which it should retain the buffer if it wants to use the buffer and pass it somewhere else


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58628677
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21581/consoleFull) for   PR 2753 at commit [`29c6dcf`](https://github.com/apache/spark/commit/29c6dcfaacb2e8b1f0582c6d5e435349c52e29af).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59561991
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21863/consoleFull) for   PR 2753 at commit [`9da0bc1`](https://github.com/apache/spark/commit/9da0bc11383587c21f6306bb0a2e9fb4b86fbb88).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19452665
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java ---
    @@ -0,0 +1,162 @@
    +/*
    + * 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.server;
    +
    +import java.util.Set;
    +
    +import com.google.common.base.Throwables;
    +import com.google.common.collect.Sets;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.buffer.ManagedBuffer;
    +import org.apache.spark.network.client.RpcResponseCallback;
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.protocol.Encodable;
    +import org.apache.spark.network.protocol.request.RequestMessage;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +import org.apache.spark.network.protocol.response.ChunkFetchFailure;
    +import org.apache.spark.network.protocol.response.ChunkFetchSuccess;
    +import org.apache.spark.network.protocol.response.RpcFailure;
    +import org.apache.spark.network.protocol.response.RpcResponse;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * A handler that processes requests from clients and writes chunk data back. Each handler is
    + * attached to a single Netty channel, and keeps track of which streams have been fetched via this
    + * channel, in order to clean them up if the channel is terminated (see #channelUnregistered).
    + *
    + * The messages should have been processed by the pipeline setup by {@link TransportServer}.
    + */
    +public class TransportRequestHandler extends MessageHandler<RequestMessage> {
    +  private final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class);
    +
    +  /** The Netty channel that this handler is associated with. */
    +  private final Channel channel;
    +
    +  /** Client on the same channel allowing us to talk back to the requester. */
    +  private final TransportClient reverseClient;
    +
    +  /** Returns each chunk part of a stream. */
    +  private final StreamManager streamManager;
    +
    +  /** Handles all RPC messages. */
    +  private final RpcHandler rpcHandler;
    +
    +  /** List of all stream ids that have been read on this handler, used for cleanup. */
    +  private final Set<Long> streamIds;
    +
    +  public TransportRequestHandler(
    +    Channel channel,
    +    TransportClient reverseClient,
    +    StreamManager streamManager,
    +    RpcHandler rpcHandler) {
    +    this.channel = channel;
    +    this.reverseClient = reverseClient;
    +    this.streamManager = streamManager;
    +    this.rpcHandler = rpcHandler;
    +    this.streamIds = Sets.newHashSet();
    +  }
    +
    +  @Override
    +  public void exceptionCaught(Throwable cause) {
    --- End diff --
    
    don't we want to close the connection if there is uncaught exception?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60878626
  
      [Test build #22427 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22427/consoleFull) for   PR 2753 at commit [`cadfd28`](https://github.com/apache/spark/commit/cadfd28f116f0dbca11e580a23caf82060bcf922).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60867860
  
      [Test build #22415 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22415/consoleFull) for   PR 2753 at commit [`d5d123f`](https://github.com/apache/spark/commit/d5d123fdc3c6f36fa2839ac2f2a4cb50981a09cb).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18811635
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.util;
    +
    +import java.util.concurrent.ThreadFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +import io.netty.channel.Channel;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.ServerChannel;
    +import io.netty.channel.epoll.Epoll;
    +import io.netty.channel.epoll.EpollEventLoopGroup;
    +import io.netty.channel.epoll.EpollServerSocketChannel;
    +import io.netty.channel.epoll.EpollSocketChannel;
    +import io.netty.channel.nio.NioEventLoopGroup;
    +import io.netty.channel.socket.nio.NioServerSocketChannel;
    +import io.netty.channel.socket.nio.NioSocketChannel;
    +import io.netty.handler.codec.ByteToMessageDecoder;
    +import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    +
    +/**
    + * Utilities for creating various Netty constructs based on whether we're using EPOLL or NIO.
    + */
    +public class NettyUtils {
    +  /** Creates a Netty EventLoopGroup based on the IOMode. */
    +  public static EventLoopGroup createEventLoop(IOMode mode, int numThreads, String threadPrefix) {
    +    if (mode == IOMode.AUTO) {
    +      mode = autoselectMode();
    +    }
    +
    +    ThreadFactory threadFactory = new ThreadFactoryBuilder()
    +        .setDaemon(true)
    +        .setNameFormat(threadPrefix + "-%d")
    +        .build();
    +
    +    switch(mode) {
    +      case NIO:
    +        return new NioEventLoopGroup(numThreads, threadFactory);
    +      case EPOLL:
    +        return new EpollEventLoopGroup(numThreads, threadFactory);
    +      default:
    +        throw new IllegalArgumentException("Unknown io mode: " + mode);
    +    }
    +  }
    +
    +  /** Returns the correct (client) SocketChannel class based on IOMode. */
    +  public static Class<? extends Channel> getClientChannelClass(IOMode mode) {
    +    if (mode == IOMode.AUTO) {
    +      mode = autoselectMode();
    +    }
    +    switch(mode) {
    --- End diff --
    
    nitpick: space after switch


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59140982
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21757/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18999117
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/SluiceClient.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.util.UUID;
    +
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +
    +/**
    + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow
    + * efficient transfer of a large amount of data, broken up into chunks with size ranging from
    + * hundreds of KB to a few MB.
    + *
    + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane),
    + * the actual setup of the streams is done outside the scope of Sluice. The convenience method
    + * "sendRPC" is provided to enable control plane communication between the client and server to
    + * perform this setup.
    + *
    + * For example, a typical workflow might be:
    + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100
    + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback)
    + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback)
    + * ...
    + * client.sendRPC(new CloseStream(100))
    + *
    + * Construct an instance of SluiceClient using {@link SluiceClientFactory}. A single SluiceClient
    + * may be used for multiple streams, but any given stream must be restricted to a single client,
    + * in order to avoid out-of-order responses.
    + *
    + * NB: This class is used to make requests to the server, while {@link SluiceClientHandler} is
    + * responsible for handling responses from the server.
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class SluiceClient implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(SluiceClient.class);
    +
    +  private final ChannelFuture cf;
    +  private final SluiceClientHandler handler;
    +
    +  private final String serverAddr;
    +
    +  SluiceClient(ChannelFuture cf, SluiceClientHandler handler) {
    +    this.cf = cf;
    +    this.handler = handler;
    +
    +    if (cf != null && cf.channel() != null && cf.channel().remoteAddress() != null) {
    +      serverAddr = cf.channel().remoteAddress().toString();
    +    } else {
    +      serverAddr = "<unknown address>";
    +    }
    +  }
    +
    +  public boolean isActive() {
    +    return cf.channel().isActive();
    +  }
    +
    +  /**
    +   * Requests a single chunk from the remote side, from the pre-negotiated streamId.
    +   *
    +   * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though
    +   * some streams may not support this.
    +   *
    +   * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed
    +   * to be returned in the same order that they were requested, assuming only a single SluiceClient
    +   * is used to fetch the chunks.
    +   *
    +   * @param streamId Identifier that refers to a stream in the remote StreamManager. This should
    +   *                 be agreed upon by client and server beforehand.
    +   * @param chunkIndex 0-based index of the chunk to fetch
    +   * @param callback Callback invoked upon successful receipt of chunk, or upon any failure.
    +   */
    +  public void fetchChunk(
    +      long streamId,
    +      final int chunkIndex,
    +      final ChunkReceivedCallback callback) {
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr);
    +
    +    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
    +    handler.addFetchRequest(streamChunkId, callback);
    +
    +    cf.channel().writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", streamChunkId, serverAddr,
    +                timeTaken);
    +          } else {
    +            // Fail all blocks.
    +            String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
    +              serverAddr, future.cause().getMessage());
    +            logger.error(errorMsg, future.cause());
    +            future.cause().printStackTrace();
    --- End diff --
    
    whoops!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19438048
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.buffer;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +
    +/**
    + * This interface provides an immutable view for data in the form of bytes. The implementation
    + * should specify how the data is provided:
    + *
    + * - {@link FileSegmentManagedBuffer}: data backed by part of a file
    + * - {@link NioManagedBuffer}: data backed by a NIO ByteBuffer
    + * - {@link NettyManagedBuffer}: data backed by a Netty ByteBuf
    + *
    + * The concrete buffer implementation might be managed outside the JVM garbage collector.
    + * For example, in the case of {@link NettyManagedBuffer}, the buffers are reference counted.
    + * In that case, if the buffer is going to be passed around to a different thread, retain/release
    + * should be called.
    + */
    +public abstract class ManagedBuffer {
    +
    +  /** Number of bytes of the data. */
    +  public abstract long size();
    +
    +  /**
    +   * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the
    +   * returned ByteBuffer should not affect the content of this buffer.
    +   */
    +  public abstract ByteBuffer nioByteBuffer() throws IOException;
    +
    +  /**
    +   * Exposes this buffer's data as an InputStream. The underlying implementation does not
    +   * necessarily check for the length of bytes read, so the caller is responsible for making sure
    +   * it does not go over the limit.
    +   */
    +  public abstract InputStream inputStream() throws IOException;
    --- End diff --
    
    createInputStream


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60867867
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22415/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58620770
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21580/Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58628684
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21581/Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59475786
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21851/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60857097
  
      [Test build #22394 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22394/consoleFull) for   PR 2753 at commit [`2b0d1c0`](https://github.com/apache/spark/commit/2b0d1c064899429eb115d984308eb18eebe7c9e0).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19383720
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    +
    +  val chunkCallback = new ChunkReceivedCallback {
    +    // On receipt of a chunk, pass it upwards as a block.
    +    def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions {
    +      buffer.retain()
    +      listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer)
    +    }
    +
    +    // On receipt of a failure, fail every block from chunkIndex onwards.
    +    def onFailure(chunkIndex: Int, e: Throwable): Unit = {
    +      blockIds.drop(chunkIndex).foreach { blockId =>
    +        listener.onBlockFetchFailure(blockId, e);
    +      }
    +    }
    +  }
    +
    +  // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle.
    +  client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(),
    --- End diff --
    
    Ahh, gotcha, totally misunderstood your initial concern. Sure, I can put it in a method. This is not a common pattern in Spark because Scala makes it so convenient to make constructors complicated, but I do agree it's better design.
    
    The reason it's a class onto itself is because there is state related to the fetching of a single set of blocks. Previously, it was more complicated because we were doing chunking over large blocks -- I got rid of that to minimize changes, but I don't want all that logic and state to be buried inside the NettyBlockTransferService code, especially once we add it back in.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19441669
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java ---
    @@ -0,0 +1,35 @@
    +/*
    + * 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 io.netty.buffer.ByteBuf;
    +
    +/**
    + * Interface for an object which can be encoded into a ByteBuf. Multiple Encodable objects are
    + * stored in a single, pre-allocated ByteBuf, so Encodables must also provide their length.
    --- End diff --
    
    also document changes necessary to message type etc


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19437318
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -19,14 +19,13 @@ package org.apache.spark.storage
     
     import java.util.concurrent.LinkedBlockingQueue
     
    -import scala.collection.mutable.ArrayBuffer
    -import scala.collection.mutable.HashSet
    -import scala.collection.mutable.Queue
    +import scala.collection.mutable.{ArrayBuffer, HashSet, Queue}
     
    -import org.apache.spark.{TaskContext, Logging}
    -import org.apache.spark.network.{ManagedBuffer, BlockFetchingListener, BlockTransferService}
    +import org.apache.spark.network.{BlockFetchingListener, BlockTransferService}
     import org.apache.spark.serializer.Serializer
    -import org.apache.spark.util.Utils
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.util.{CompletionIterator, Utils}
    +import org.apache.spark.{Logging, TaskContext}
    --- End diff --
    
    this one is out of order


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59140980
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21757/consoleFull) for   PR 2753 at commit [`ce79e8e`](https://github.com/apache/spark/commit/ce79e8edbc4ada57075c202ff3071af4c96e5ec7).
     * This patch **fails Scala style tests**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59561370
  
    seriously jenkins, test this please without timing out!  :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19487604
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala ---
    @@ -0,0 +1,76 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.network.BlockDataManager
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer}
    +import org.apache.spark.network.client.{TransportClient, RpcResponseCallback}
    +import org.apache.spark.network.server.{DefaultStreamManager, RpcHandler}
    +import org.apache.spark.storage.{StorageLevel, BlockId}
    +
    +import scala.collection.JavaConversions._
    +
    +object NettyMessages {
    +
    +  /** Request to read a set of blocks. Returns [[ShuffleStreamHandle]] to identify the stream. */
    +  case class OpenBlocks(blockIds: Seq[BlockId])
    +
    +  /** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */
    +  case class UploadBlock(blockId: BlockId, blockData: Array[Byte], level: StorageLevel)
    +
    +  /** Identifier for a fixed number of chunks to read from a stream created by [[OpenBlocks]]. */
    +  case class ShuffleStreamHandle(streamId: Long, numChunks: Int)
    +}
    +
    +/**
    + * Serves requests to open blocks by simply registering one chunk per block requested.
    + */
    +class NettyBlockRpcServer(
    +    serializer: Serializer,
    +    streamManager: DefaultStreamManager,
    +    blockManager: BlockDataManager)
    +  extends RpcHandler with Logging {
    +
    +  import NettyMessages._
    +
    +  override def receive(
    +      client: TransportClient,
    +      messageBytes: Array[Byte],
    +      responseContext: RpcResponseCallback): Unit = {
    +    val ser = serializer.newInstance()
    +    val message = ser.deserialize[AnyRef](ByteBuffer.wrap(messageBytes))
    --- End diff --
    
    ByteBuffer.wrap does not allocate extra memory, though, right?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60867311
  
    **[Test build #22405 timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22405/consoleFull)**     for PR 2753 at commit [`4a204b8`](https://github.com/apache/spark/commit/4a204b846a8ce2b1cfbab9ed1ec42e8a2f082184)     after a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19441768
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java ---
    @@ -0,0 +1,99 @@
    +/*
    + * 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.server;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.buffer.ManagedBuffer;
    +
    +/**
    + * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are individually
    + * fetched as chunks by the client.
    + */
    +public class DefaultStreamManager extends StreamManager {
    +  private final Logger logger = LoggerFactory.getLogger(DefaultStreamManager.class);
    +
    +  private final AtomicLong nextStreamId;
    +  private final Map<Long, StreamState> streams;
    +
    +  /** State of a single stream. */
    +  private static class StreamState {
    +    final Iterator<ManagedBuffer> buffers;
    +
    +    int curChunk = 0;
    --- End diff --
    
    can u explain what curChunk is


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18811639
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.util;
    +
    +import java.util.concurrent.ThreadFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +import io.netty.channel.Channel;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.ServerChannel;
    +import io.netty.channel.epoll.Epoll;
    +import io.netty.channel.epoll.EpollEventLoopGroup;
    +import io.netty.channel.epoll.EpollServerSocketChannel;
    +import io.netty.channel.epoll.EpollSocketChannel;
    +import io.netty.channel.nio.NioEventLoopGroup;
    +import io.netty.channel.socket.nio.NioServerSocketChannel;
    +import io.netty.channel.socket.nio.NioSocketChannel;
    +import io.netty.handler.codec.ByteToMessageDecoder;
    +import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    +
    +/**
    + * Utilities for creating various Netty constructs based on whether we're using EPOLL or NIO.
    + */
    +public class NettyUtils {
    +  /** Creates a Netty EventLoopGroup based on the IOMode. */
    +  public static EventLoopGroup createEventLoop(IOMode mode, int numThreads, String threadPrefix) {
    +    if (mode == IOMode.AUTO) {
    +      mode = autoselectMode();
    +    }
    +
    +    ThreadFactory threadFactory = new ThreadFactoryBuilder()
    +        .setDaemon(true)
    +        .setNameFormat(threadPrefix + "-%d")
    +        .build();
    +
    +    switch(mode) {
    +      case NIO:
    +        return new NioEventLoopGroup(numThreads, threadFactory);
    +      case EPOLL:
    +        return new EpollEventLoopGroup(numThreads, threadFactory);
    +      default:
    +        throw new IllegalArgumentException("Unknown io mode: " + mode);
    +    }
    +  }
    +
    +  /** Returns the correct (client) SocketChannel class based on IOMode. */
    +  public static Class<? extends Channel> getClientChannelClass(IOMode mode) {
    +    if (mode == IOMode.AUTO) {
    +      mode = autoselectMode();
    +    }
    +    switch(mode) {
    +      case NIO:
    +        return NioSocketChannel.class;
    +      case EPOLL:
    +        return EpollSocketChannel.class;
    +      default:
    +        throw new IllegalArgumentException("Unknown io mode: " + mode);
    +    }
    +  }
    +
    +  /** Returns the correct ServerSocketChannel class based on IOMode. */
    +  public static Class<? extends ServerChannel> getServerChannelClass(IOMode mode) {
    +    if (mode == IOMode.AUTO) {
    +      mode = autoselectMode();
    +    }
    +    switch(mode) {
    --- End diff --
    
    nitpick: space after switch


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-58772867
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21651/consoleFull)**     for PR 2753 at commit [`9d9b4e1`](https://github.com/apache/spark/commit/9d9b4e1199bdeab7e454878bda61f0b5aecc79ad)     after a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19383243
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    +
    +  val chunkCallback = new ChunkReceivedCallback {
    +    // On receipt of a chunk, pass it upwards as a block.
    +    def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions {
    +      buffer.retain()
    +      listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer)
    +    }
    +
    +    // On receipt of a failure, fail every block from chunkIndex onwards.
    +    def onFailure(chunkIndex: Int, e: Throwable): Unit = {
    +      blockIds.drop(chunkIndex).foreach { blockId =>
    +        listener.onBlockFetchFailure(blockId, e);
    +      }
    +    }
    +  }
    +
    +  // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle.
    +  client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(),
    --- End diff --
    
    as written here it is fine, but in general complicated constructor that can throw exceptions are more error prone in the future. see  http://stackoverflow.com/questions/1371369/can-constructors-throw-exceptions-in-java


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60886387
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22437/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19364810
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/SluiceConfig.java ---
    @@ -0,0 +1,61 @@
    +/*
    + * 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.util;
    +
    +/**
    + * A central location that tracks all the settings we expose to users.
    + */
    +public class SluiceConfig {
    +  private final ConfigProvider conf;
    +
    +  public SluiceConfig(ConfigProvider conf) {
    +    this.conf = conf;
    +  }
    +
    +  /** Port the server listens on. Default to a random port. */
    +  public int serverPort() { return conf.getInt("spark.shuffle.io.port", 0); }
    +
    +  /** IO mode: nio, epoll, or auto (try epoll first and then nio). */
    +  public String ioMode() { return conf.get("spark.shuffle.io.mode", "NIO").toUpperCase(); }
    --- End diff --
    
    you want epoll for default?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19487972
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java ---
    @@ -15,18 +15,23 @@
      * limitations under the License.
      */
     
    -package org.apache.spark.network.netty.server
    +package org.apache.spark.network.client;
     
     /**
    - * Header describing a block. This is used only in the server pipeline.
    - *
    - * [[BlockServerHandler]] creates this, and [[BlockHeaderEncoder]] encodes it.
    - *
    - * @param blockSize length of the block content, excluding the length itself.
    - *                 If positive, this is the header for a block (not part of the header).
    - *                 If negative, this is the header and content for an error message.
    - * @param blockId block id
    - * @param error some error message from reading the block
    + * General exception caused by a remote exception while fetching a chunk.
      */
    -private[server]
    -class BlockHeader(val blockSize: Int, val blockId: String, val error: Option[String] = None)
    +public class ChunkFetchFailureException extends RuntimeException {
    --- End diff --
    
    removed it, it's already part of the message


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60867314
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22405/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60880706
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60867974
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60857101
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22394/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19441890
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java ---
    @@ -0,0 +1,99 @@
    +/*
    + * 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.server;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.buffer.ManagedBuffer;
    +
    +/**
    + * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are individually
    + * fetched as chunks by the client.
    + */
    +public class DefaultStreamManager extends StreamManager {
    +  private final Logger logger = LoggerFactory.getLogger(DefaultStreamManager.class);
    +
    +  private final AtomicLong nextStreamId;
    +  private final Map<Long, StreamState> streams;
    +
    +  /** State of a single stream. */
    +  private static class StreamState {
    +    final Iterator<ManagedBuffer> buffers;
    +
    +    int curChunk = 0;
    --- End diff --
    
    e.g. it is used to track current chunk index and to prevent out of order blah blah


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19004339
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    +
    +  val chunkCallback = new ChunkReceivedCallback {
    +    // On receipt of a chunk, pass it upwards as a block.
    +    def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions {
    +      buffer.retain()
    +      listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer)
    +    }
    +
    +    // On receipt of a failure, fail every block from chunkIndex onwards.
    +    def onFailure(chunkIndex: Int, e: Throwable): Unit = {
    +      blockIds.drop(chunkIndex).foreach { blockId =>
    +        listener.onBlockFetchFailure(blockId, e);
    +      }
    +    }
    +  }
    +
    +  // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle.
    +  client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(),
    --- End diff --
    
    It's not in fetchBlocks because it's simply one of many possible RPCs. For instance, I just added the UploadBlock RPC and anticipate more in the future.
    
    I'm not sure I understand the concern with the constructor.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59627314
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21878/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59559839
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21862/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18810924
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.util;
    +
    +import java.util.NoSuchElementException;
    +
    +/**
    + * Provides a mechanism for constructing a {@link SluiceConfig} using some sort of configuration.
    + */
    +public abstract class ConfigProvider {
    +  /** Obtains the value of the given config, throws NoSuchElementException if it doesn't exist. */
    +  public abstract String get(String name);
    +
    +  public String get(String name, String defaultValue) {
    +    try {
    +      return get(name);
    +    } catch (NoSuchElementException e) {
    --- End diff --
    
    hmm probalby shouldn't be using try catch for expected condition


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19383261
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.netty
    +
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.network.BlockFetchingListener
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.network.buffer.ManagedBuffer
    +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, SluiceClient}
    +import org.apache.spark.storage.BlockId
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Responsible for holding the state for a request for a single set of blocks. This assumes that
    + * the chunks will be returned in the same order as requested, and that there will be exactly
    + * one chunk per block.
    + *
    + * Upon receipt of any block, the listener will be called back. Upon failure part way through,
    + * the listener will receive a failure callback for each outstanding block.
    + */
    +class NettyBlockFetcher(
    +    serializer: Serializer,
    +    client: SluiceClient,
    +    blockIds: Seq[String],
    +    listener: BlockFetchingListener)
    +  extends Logging {
    +
    +  require(blockIds.nonEmpty)
    +
    +  val ser = serializer.newInstance()
    +
    +  var streamHandle: ShuffleStreamHandle = _
    +
    +  val chunkCallback = new ChunkReceivedCallback {
    +    // On receipt of a chunk, pass it upwards as a block.
    +    def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions {
    +      buffer.retain()
    +      listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer)
    +    }
    +
    +    // On receipt of a failure, fail every block from chunkIndex onwards.
    +    def onFailure(chunkIndex: Int, e: Throwable): Unit = {
    +      blockIds.drop(chunkIndex).foreach { blockId =>
    +        listener.onBlockFetchFailure(blockId, e);
    +      }
    +    }
    +  }
    +
    +  // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle.
    +  client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(),
    --- End diff --
    
    note that this is worse in scala because scala doesn't have checked exceptions, i.e. the caller might not know it needs to handle the exceptions or do null check. it is fine when you are the one writing the caller code, but in the future when somebody else changes the code, it becomes more error prone.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59590621
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21868/consoleFull) for   PR 2753 at commit [`ccd4959`](https://github.com/apache/spark/commit/ccd49595e8d0a730489e577b1152ad67027a5687).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60875179
  
    That is a pass with netty turned on. Now I am turning it off for preparation to merge.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r19439894
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/client/TransportClient.java ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.client;
    +
    +import java.io.Closeable;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.protocol.StreamChunkId;
    +import org.apache.spark.network.protocol.request.ChunkFetchRequest;
    +import org.apache.spark.network.protocol.request.RpcRequest;
    +import org.apache.spark.network.util.NettyUtils;
    +
    +/**
    + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow
    + * efficient transfer of a large amount of data, broken up into chunks with size ranging from
    + * hundreds of KB to a few MB.
    + *
    + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane),
    + * the actual setup of the streams is done outside the scope of the transport layer. The convenience
    + * method "sendRPC" is provided to enable control plane communication between the client and server
    + * to perform this setup.
    + *
    + * For example, a typical workflow might be:
    + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100
    + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback)
    + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback)
    + * ...
    + * client.sendRPC(new CloseStream(100))
    + *
    + * Construct an instance of TransportClient using {@link TransportClientFactory}. A single
    + * TransportClient may be used for multiple streams, but any given stream must be restricted to a
    + * single client, in order to avoid out-of-order responses.
    + *
    + * NB: This class is used to make requests to the server, while {@link TransportResponseHandler} is
    + * responsible for handling responses from the server.
    + *
    + * Concurrency: thread safe and can be called from multiple threads.
    + */
    +public class TransportClient implements Closeable {
    +  private final Logger logger = LoggerFactory.getLogger(TransportClient.class);
    +
    +  private final Channel channel;
    +  private final TransportResponseHandler handler;
    +
    +  public TransportClient(Channel channel, TransportResponseHandler handler) {
    +    this.channel = Preconditions.checkNotNull(channel);
    +    this.handler = Preconditions.checkNotNull(handler);
    +  }
    +
    +  public boolean isActive() {
    +    return channel.isOpen() || channel.isActive();
    +  }
    +
    +  /**
    +   * Requests a single chunk from the remote side, from the pre-negotiated streamId.
    +   *
    +   * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though
    +   * some streams may not support this.
    +   *
    +   * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed
    +   * to be returned in the same order that they were requested, assuming only a single
    +   * TransportClient is used to fetch the chunks.
    +   *
    +   * @param streamId Identifier that refers to a stream in the remote StreamManager. This should
    +   *                 be agreed upon by client and server beforehand.
    +   * @param chunkIndex 0-based index of the chunk to fetch
    +   * @param callback Callback invoked upon successful receipt of chunk, or upon any failure.
    +   */
    +  public void fetchChunk(
    +      long streamId,
    +      final int chunkIndex,
    +      final ChunkReceivedCallback callback) {
    +    final String serverAddr = NettyUtils.getRemoteAddress(channel);
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr);
    +
    +    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
    +    handler.addFetchRequest(streamChunkId, callback);
    +
    +    channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", streamChunkId, serverAddr,
    +              timeTaken);
    +          } else {
    +            String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
    +              serverAddr, future.cause());
    +            logger.error(errorMsg, future.cause());
    +            handler.removeFetchRequest(streamChunkId);
    +            callback.onFailure(chunkIndex, new RuntimeException(errorMsg, future.cause()));
    +          }
    +        }
    +      });
    +  }
    +
    +  /**
    +   * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked
    +   * with the server's response or upon any failure.
    +   */
    +  public void sendRpc(byte[] message, final RpcResponseCallback callback) {
    +    final String serverAddr = NettyUtils.getRemoteAddress(channel);
    +    final long startTime = System.currentTimeMillis();
    +    logger.debug("Sending RPC to {}", serverAddr);
    +
    +    final long tag = UUID.randomUUID().getLeastSignificantBits();
    +    handler.addRpcRequest(tag, callback);
    +
    +    channel.writeAndFlush(new RpcRequest(tag, message)).addListener(
    +      new ChannelFutureListener() {
    +        @Override
    +        public void operationComplete(ChannelFuture future) throws Exception {
    +          if (future.isSuccess()) {
    +            long timeTaken = System.currentTimeMillis() - startTime;
    +            logger.debug("Sending request {} to {} took {} ms", tag, serverAddr, timeTaken);
    +          } else {
    +            String errorMsg = String.format("Failed to send RPC %s to %s: %s", tag,
    --- End diff --
    
    i think we should close the connection here if we fail to flush


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59557155
  
    Jenkins, test this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60878629
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22427/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2753#discussion_r18811630
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/protocol/response/ServerResponse.java ---
    @@ -0,0 +1,63 @@
    +/*
    + * 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.response;
    +
    +import io.netty.buffer.ByteBuf;
    +
    +import org.apache.spark.network.protocol.Encodable;
    +
    +/**
    + * Messages from server to client (usually in response to some
    + * {@link org.apache.spark.network.protocol.request.ClientRequest}.
    + */
    +public interface ServerResponse extends Encodable {
    +  /** Used to identify this response type. */
    +  Type type();
    +
    +  /**
    +   * Preceding every serialized ServerResponse is the type, which allows us to deserialize
    +   * the response.
    +   */
    +  public static enum Type implements Encodable {
    +    ChunkFetchSuccess(0), ChunkFetchFailure(1), RpcResponse(2), RpcFailure(3);
    +
    +    private final byte id;
    +
    +    private Type(int id) {
    +      assert id < 128 : "Cannot have more than 128 response types";
    +      this.id = (byte) id;
    +    }
    +
    +    public byte id() { return id; }
    +
    +    @Override public int encodedLength() { return 1; }
    +
    +    @Override public void encode(ByteBuf buf) { buf.writeByte(id); }
    +
    +    public static Type decode(ByteBuf buf) {
    +      byte id = buf.readByte();
    +      switch(id) {
    --- End diff --
    
    nitpick: space after switch


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59140896
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21757/consoleFull) for   PR 2753 at commit [`ce79e8e`](https://github.com/apache/spark/commit/ce79e8edbc4ada57075c202ff3071af4c96e5ec7).
     * This patch **does not merge cleanly**.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-59570305
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21863/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-3453] Netty-based BlockTransferService,...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on the pull request:

    https://github.com/apache/spark/pull/2753#issuecomment-60849695
  
    I have addressed all remaining comments. Please take a look at the revised logic in TransportClientFactory to construct a single client.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org