You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by squito <gi...@git.apache.org> on 2015/04/07 21:59:42 UTC

[GitHub] spark pull request: [SPARK-6190][core] create LargeByteBuffer for ...

GitHub user squito opened a pull request:

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

    [SPARK-6190][core] create LargeByteBuffer for eliminating 2GB block limit

    

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

    $ git pull https://github.com/squito/spark SPARK-6190_largeBB

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

    https://github.com/apache/spark/pull/5400.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 #5400
    
----
commit 674aea65afb03681c0941cd0c44d1a066b793b74
Author: Imran Rashid <ir...@cloudera.com>
Date:   2015-04-07T18:25:06Z

    ByteArrayChunkOutputStream.slice, for grabbing aribitrary portion of output

commit 9f53203070daa17ddfff5ad1f5b2e94f86614554
Author: Imran Rashid <ir...@cloudera.com>
Date:   2015-04-07T18:19:08Z

    LargeByteBuffer

----


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27924628
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java ---
    @@ -0,0 +1,73 @@
    +/*
    + * 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.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +    public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    +
    +    public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +        return new WrappedLargeByteBuffer(new ByteBuffer[]{buffer});
    +    }
    +
    +    public static LargeByteBuffer asLargeByteBuffer(byte[] bytes) {
    +        return new WrappedLargeByteBuffer(new ByteBuffer[]{ByteBuffer.wrap(bytes)});
    +    }
    +
    +    public static LargeByteBuffer allocate(long size) {
    +        ArrayList<ByteBuffer> chunks = new ArrayList<ByteBuffer>();
    --- End diff --
    
    You could either pre-allocate a `ByteBuffer[]` here, or use `com.google.common.collect.Lists.newArrayListWithExpectedSize`, for efficiency.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153816312
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45029/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107801210
  
     Merged build triggered.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27923630
  
    --- Diff: core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala ---
    @@ -91,4 +95,41 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream {
           ret
         }
       }
    +
    +  /**
    +   * get a copy of the data between the two endpoints
    +   */
    +  def slice(start: Long, until: Long): Array[Byte] = {
    +    require((until - start) < Integer.MAX_VALUE, "max slice length = Integer.MAX_VALUE")
    +    var chunkStart = 0L
    +    var chunkIdx = 0
    +    var foundStart = false
    +    val length = (until - start).toInt
    +    val result = new Array[Byte](length)
    +    while (!foundStart) {
    +      val nextSize = chunkStart + chunks(chunkIdx).size
    +      if (nextSize > start) {
    +        foundStart = true
    +      }
    +      else {
    --- End diff --
    
    nit: move to previous 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28718430
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java ---
    @@ -0,0 +1,113 @@
    +/*
    +* 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.nio.ByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +
    +public interface LargeByteBuffer {
    +  public byte get();
    +
    +  /**
    +   * Bulk copy data from this buffer into the given array.  First checks there is sufficient
    +   * data in this buffer; if not, throws a {@link java.nio.BufferUnderflowException}.
    +   *
    +   * @param dst
    +   * @param offset
    +   * @param length
    +   */
    +  public void get(byte[] dst, int offset, int length);
    +
    +  public LargeByteBuffer rewind();
    +
    +  /**
    +   * Return a deep copy of this buffer.
    +   * The returned buffer will have position == 0.  The position
    +   * of this buffer will not change as a result of copying.
    +   *
    +   * @return a new buffer with a full copy of this buffer's data
    +   */
    +  public LargeByteBuffer deepCopy();
    +
    +  /**
    +   * Advance the position in this buffer by up to <code>n</code> bytes.  <code>n</code> may be
    +   * positive or negative.  It will move the full <code>n</code> unless that moves
    +   * it past the end (or beginning) of the buffer, in which case it will move to the end
    +   * (or beginning).
    +   *
    +   * @return the number of bytes moved forward (can be negative if <code>n</code> is negative)
    +   */
    +  public long skip(long n);
    +
    +  public long position();
    +
    +  /**
    +   * Creates a new byte buffer that shares this buffer's content.
    +   * <p/>
    +   * <p> The content of the new buffer will be that of this buffer.  Changes
    +   * to this buffer's content will be visible in the new buffer, and vice
    +   * versa; the two buffers' positions will be independent.
    +   * <p/>
    +   * <p> The new buffer's position will be identical to those of this buffer
    +   */
    +  public LargeByteBuffer duplicate();
    +
    +
    +  public long remaining();
    +
    +  /**
    +   * Total number of bytes in this buffer
    +   *
    +   * @return
    --- End diff --
    
    It'd be nice to fill in these empty javadoc tags, or remove them.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28723329
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,269 @@
    +/*
    +* 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 com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    --- End diff --
    
    nit: add empty line (and remove the one before)


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31543107
  
    --- Diff: core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.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.buffer
    +
    +import java.io.{File, FileInputStream, FileOutputStream, OutputStream}
    +import java.nio.channels.FileChannel.MapMode
    +
    +import org.junit.Assert._
    +import org.scalatest.Matchers
    +
    +import org.apache.spark.SparkFunSuite
    +
    +class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers {
    +
    +  test("read from large mapped file") {
    +    val testFile = File.createTempFile("large-buffer-input-stream-test", ".bin")
    +
    +    try {
    +      val out: OutputStream = new FileOutputStream(testFile)
    +      val buffer: Array[Byte] = new Array[Byte](1 << 16)
    +      val len: Long = buffer.length.toLong + Integer.MAX_VALUE + 1
    +      (0 until buffer.length).foreach { idx =>
    +        buffer(idx) = idx.toByte
    +      }
    +      (0 until (len / buffer.length).toInt).foreach { idx =>
    +        out.write(buffer)
    +      }
    +      out.close
    +
    +      val channel = new FileInputStream(testFile).getChannel
    +      val buf = LargeByteBufferHelper.mapFile(channel, MapMode.READ_ONLY, 0, len)
    +      val in = new LargeByteBufferInputStream(buf, true)
    +
    +      val read = new Array[Byte](buffer.length)
    +      (0 until (len / buffer.length).toInt).foreach { idx =>
    +        in.disposed should be(false)
    +        in.read(read) should be(read.length)
    +        (0 until buffer.length).foreach { arrIdx =>
    +          assertEquals(buffer(arrIdx), read(arrIdx))
    +        }
    +      }
    +      in.disposed should be(false)
    +      in.read(read) should be(-1)
    +      in.disposed should be(false)
    +      in.close()
    +      in.disposed should be(true)
    +    } finally {
    +      testFile.delete()
    +    }
    +  }
    +
    +  test("dispose on close") {
    +    // don't need to read to the end -- dispose anytime we close
    +    val data = new Array[Byte](10)
    +    val in = new LargeByteBufferInputStream(LargeByteBufferHelper.asLargeByteBuffer(data), true)
    +    in.disposed should be (false)
    +    in.close()
    +    in.disposed should be (true)
    +  }
    +
    +  test("io stream roundtrip") {
    +    val out = new LargeByteBufferOutputStream(128)
    +    (0 until 200).foreach{idx => out.write(idx)}
    --- End diff --
    
    nit: `foreach { idx => ... }`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91379939
  
      [Test build #29972 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29972/consoleFull) for   PR 5400 at commit [`a759242`](https://github.com/apache/spark/commit/a75924234ea3523ae3409a384dfef2dbbfbc9d82).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class LargeByteBufferOutputStream(chunkSize: Int = 65536)`
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `
    
     * This patch does not change any dependencies.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153810648
  
    Merged build started.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28722980
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.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.buffer;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +  public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    +
    +  public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { buffer } );
    +  }
    +
    +  public static LargeByteBuffer asLargeByteBuffer(byte[] bytes) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { ByteBuffer.wrap(bytes) } );
    +  }
    +
    +  public static LargeByteBuffer allocate(long size) {
    +    return allocate(size, MAX_CHUNK);
    +  }
    +
    +  @VisibleForTesting
    +  static LargeByteBuffer allocate(long size, int maxChunk) {
    +    int chunksNeeded = (int) ((size + maxChunk - 1) / maxChunk);
    +    ByteBuffer[] chunks = new ByteBuffer[chunksNeeded];
    +    long remaining = size;
    +    for (int i = 0; i < chunksNeeded; i++) {
    +      int nextSize = (int) Math.min(remaining, maxChunk);
    +      ByteBuffer next = ByteBuffer.allocate(nextSize);
    +      remaining -= nextSize;
    +      chunks[i] = next;
    +    }
    +    if (remaining != 0) throw new IllegalStateException("remaining = " + remaining);
    +    return new WrappedLargeByteBuffer(chunks, maxChunk);
    +  }
    +
    +
    +  public static LargeByteBuffer mapFile(
    +    FileChannel channel,
    +    FileChannel.MapMode mode,
    +    long offset,
    +    long length
    +  ) throws IOException {
    +    int maxChunk = MAX_CHUNK;
    +    ArrayList<Long> offsets = new ArrayList<Long>();
    --- End diff --
    
    `List<Long> offsets = Lists.newArrayList()`
    
    Minor, but it seems like you could use `Lists.newArrayListWithExpectedSize()` which is a little more efficient.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28724997
  
    --- Diff: network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java ---
    @@ -0,0 +1,278 @@
    +/*
    + * 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.*;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.Arrays;
    +import java.util.Random;
    +
    +import org.junit.Test;
    +
    +import static org.junit.Assert.*;
    +
    +public class WrappedLargeByteBufferSuite {
    +
    +  byte[] data = new byte[500];
    --- End diff --
    
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-93086473
  
      [Test build #30280 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30280/consoleFull) for   PR 5400 at commit [`117fb4a`](https://github.com/apache/spark/commit/117fb4a166a722bd352d74de59f26e45100a022c).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28723381
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,269 @@
    +/*
    +* 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 com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    +   * each sub-ByteBuffer (except for the last one) must be exactly this size.  Note that this
    +   * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK.  The only reason it isn't
    +   * is so that we can do tests without creating ginormous buffers.  Public methods force it to
    +   * be LargeByteBufferHelper.MAX_CHUNK
    +   */
    +  private final int subBufferSize;
    +  private long _pos;
    +  @VisibleForTesting
    +  int currentBufferIdx;
    +  @VisibleForTesting
    +  ByteBuffer currentBuffer;
    +
    --- End diff --
    
    nit: delete


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27922902
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * 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.util
    +
    +import java.io.InputStream
    +
    +import org.apache.spark.network.buffer.LargeByteBuffer
    +import org.apache.spark.storage.BlockManager
    +
    +/**
    + * Reads data from a LargeByteBuffer, and optionally cleans it up using buffer.dispose()
    + * at the end of the stream (e.g. to close a memory-mapped file).
    + */
    +private[spark]
    +class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: Boolean = false)
    +  extends InputStream {
    +
    +  override def read(): Int = {
    --- End diff --
    
    It seems from a cursory look at the `InputStream` source code that it could be more efficient to also override `read(byte[], int, int)`.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107670997
  
    Merged build started.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-93052047
  
      [Test build #30272 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30272/consoleFull) for   PR 5400 at commit [`78e4043`](https://github.com/apache/spark/commit/78e40439d1c1515f2197fc465653e4434330278d).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `
    
     * This patch does not change any dependencies.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27923438
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala ---
    @@ -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.util
    +
    +import java.io.OutputStream
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.network.buffer.{LargeByteBufferHelper, WrappedLargeByteBuffer, LargeByteBuffer}
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream
    +
    +private[spark]
    +class LargeByteBufferOutputStream(chunkSize: Int = 65536)
    +  extends OutputStream {
    +
    +  private[util] val output = new ByteArrayChunkOutputStream(chunkSize)
    +
    +  private var _pos = 0
    +
    +  override def write(b: Int): Unit = {
    +    output.write(b)
    +  }
    +
    +  override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = {
    +    output.write(bytes, offs, len)
    +    _pos += len
    +  }
    +
    +  def largeBuffer: LargeByteBuffer = {
    +    largeBuffer(LargeByteBufferHelper.MAX_CHUNK)
    +  }
    +
    +  // exposed for testing
    +  private[util] def largeBuffer(maxChunk: Int): LargeByteBuffer = {
    +    // LargeByteBuffer is supposed to make a "best effort" to get all the data
    +    // in one nio.ByteBuffer, so we want to try to merge the smaller chunks together
    +    // as much as possible.  This is necessary b/c there are a number of parts of spark that
    +    // can only deal w/ one nio.ByteBuffer, and can't use a LargeByteBuffer yet.
    +    val totalSize = output.size
    +    val chunksNeeded = ((totalSize + maxChunk -1) / maxChunk).toInt
    --- End diff --
    
    nit: space before `-`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-92490825
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30190/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27926259
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,252 @@
    +/*
    +* 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 sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  //only public for tests
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  private long _pos;
    +  private int currentBufferIdx;
    +  private ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, findExpectedInitialPosition(underlying));
    +  }
    +
    +  private static long findExpectedInitialPosition(ByteBuffer[] bufs) {
    +    long sum = 0L;
    +    for (ByteBuffer b: bufs) {
    +      if (b.position() > 0) {
    +        // this could still lead to a mix of positions half-way through buffers that
    +        // would be inconsistent -- but we'll discover that in the constructor checks
    +        sum += b.position();
    +      } else {
    +        break;
    +      }
    +    }
    +    return sum;
    +  }
    +
    +  private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) {
    +    this.underlying = underlying;
    +    long sum = 0L;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      long nextSum = sum + b.capacity();
    +      int expectedPosition;
    +      if (nextSum < initialPosition) {
    +        expectedPosition = b.capacity();
    +      } else if (sum > initialPosition) {
    +        expectedPosition = 0;
    +      } else {
    +        expectedPosition = (int) (initialPosition - sum);
    +      }
    +      if (b.position() != expectedPosition) {
    +        throw new IllegalArgumentException("ByteBuffer[" + i + "]:" + b + " was expected to have" +
    +          " position = " + expectedPosition + " to be consistent with the overall " +
    +          "initialPosition = " + initialPosition);
    +      }
    +      sum = nextSum;
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining())
    +      throw new BufferUnderflowException();
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    while (currentBufferIdx > 0) {
    +      if (currentBuffer != null) {
    +        currentBuffer.rewind();
    +      }
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +    }
    +    currentBuffer.rewind();
    +    _pos = 0;
    +    return this;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer deepCopy() {
    +    ByteBuffer[] dataCopy = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      dataCopy[i] = ByteBuffer.allocate(b.capacity());
    +      int originalPosition = b.position();
    +      b.position(0);
    +      dataCopy[i].put(b);
    +      dataCopy[i].position(0);
    +      b.position(originalPosition);
    +    }
    +    return new WrappedLargeByteBuffer(dataCopy);
    +  }
    +
    +  @Override
    +  public byte get() {
    +    byte r = currentBuffer.get();
    +    _pos += 1;
    +    updateCurrentBuffer();
    +    return r;
    +  }
    +
    +  private void updateCurrentBuffer() {
    +    while (currentBuffer != null && !currentBuffer.hasRemaining()) {
    +      currentBufferIdx += 1;
    +      currentBuffer = currentBufferIdx < underlying.length ? underlying[currentBufferIdx] : null;
    +    }
    +  }
    +
    +  @Override
    +  public long position() {
    +    return _pos;
    +  }
    +
    +  @Override
    +  public long skip(long n) {
    +    if (n < 0) {
    +      final long moveTotal = Math.min(-n, _pos);
    +      long toMove = moveTotal;
    +      // move backwards -- set the position to 0 of every buffer's we go back
    +      if (currentBuffer != null) {
    +        currentBufferIdx += 1;
    +      }
    +      while (toMove > 0) {
    +        currentBufferIdx -= 1;
    +        currentBuffer = underlying[currentBufferIdx];
    +        int thisMove = (int) Math.min(toMove, currentBuffer.position());
    +        currentBuffer.position(currentBuffer.position() - thisMove);
    +        toMove -= thisMove;
    +      }
    +      _pos -= moveTotal;
    +      return -moveTotal;
    +    } else if (n > 0) {
    +      final long moveTotal = Math.min(n, remaining());
    +      long toMove = moveTotal;
    +      // move forwards-- set the position to the end of every buffer as we go forwards
    +      currentBufferIdx -= 1;
    +      while (toMove > 0) {
    +        currentBufferIdx += 1;
    +        currentBuffer = underlying[currentBufferIdx];
    +        int thisMove = (int) Math.min(toMove, currentBuffer.remaining());
    +        currentBuffer.position(currentBuffer.position() + thisMove);
    +        toMove -= thisMove;
    +      }
    +      _pos += moveTotal;
    +      return moveTotal;
    +    } else {
    +      return 0;
    +    }
    +  }
    +
    +  @Override
    +  public long remaining() {
    +    return size - _pos;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer duplicate() {
    +    ByteBuffer[] duplicates = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      duplicates[i] = underlying[i].duplicate();
    +    }
    +    return new WrappedLargeByteBuffer(duplicates, _pos);
    +  }
    +
    +  @Override
    +  public long size() {
    +    return size;
    +  }
    +
    +  @Override
    +  public long writeTo(WritableByteChannel channel) throws IOException {
    +    long written = 0l;
    +    for (; currentBufferIdx < underlying.length; currentBufferIdx++) {
    +      currentBuffer = underlying[currentBufferIdx];
    +      written += currentBuffer.remaining();
    +      while (currentBuffer.hasRemaining())
    +        channel.write(currentBuffer);
    +    }
    +    _pos = size();
    +    return written;
    +  }
    +
    +  @Override
    +  public ByteBuffer asByteBuffer() throws BufferTooLargeException {
    +    if (underlying.length > 1) {
    +      throw new BufferTooLargeException(size());
    +    }
    +    return underlying[0];
    +  }
    +
    +  // only needed for tests
    +  public List<ByteBuffer> nioBuffers() {
    +    return Arrays.asList(underlying);
    +  }
    +
    +  /**
    +   * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that
    +   * might cause errors if one attempts to read from the unmapped buffer, but it's better than
    +   * waiting for the GC to find it because that could lead to huge numbers of open files. There's
    +   * unfortunately no standard API to do this.
    +   */
    +  private static void dispose(ByteBuffer buffer) {
    +    if (buffer != null && buffer instanceof MappedByteBuffer) {
    --- End diff --
    
    Isn't it safer to check for `DirectBuffer` 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-93118056
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30280/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-149191804
  
    How is it going? Is it still WIP? I can help 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28717743
  
    --- Diff: core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.{FileInputStream, FileOutputStream, OutputStream, File}
    +import java.nio.ByteBuffer
    +import java.nio.channels.FileChannel.MapMode
    +
    +import org.junit.Assert._
    +import org.scalatest.{FunSuite, Matchers}
    +
    +import org.apache.spark.network.buffer.{LargeByteBuffer, LargeByteBufferHelper, WrappedLargeByteBuffer}
    +
    +class LargeByteBufferInputStreamSuite extends FunSuite with Matchers {
    +
    +  test("read from large mapped file") {
    +    val testFile = File.createTempFile("large-buffer-input-stream-test",".bin")
    +    testFile.deleteOnExit()
    +
    +    val out: OutputStream = new FileOutputStream(testFile)
    +    val buffer: Array[Byte] = new Array[Byte](1 << 16)
    +    val len: Long = 3L << 30
    +    assertTrue(len > Integer.MAX_VALUE)
    --- End diff --
    
    You could just say `len = 42L + Integer.MAX_VALUE`...


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153810626
  
     Merged build triggered.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107670921
  
     Merged build triggered.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-92955919
  
      [Test build #30251 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30251/consoleFull) for   PR 5400 at commit [`d641834`](https://github.com/apache/spark/commit/d641834ea2fb1c55bc418b1a3f61e8a958ae2a51).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28724878
  
    --- Diff: network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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 org.junit.Test;
    +
    +import java.io.*;
    +import java.nio.channels.FileChannel;
    +
    +import static org.junit.Assert.*;
    +
    +public class LargeByteBufferHelperSuite {
    +
    +  @Test
    +  public void testMapFile() throws IOException {
    +    File testFile = File.createTempFile("large-byte-buffer-test", ".bin");
    +    testFile.deleteOnExit();
    +    OutputStream out = new FileOutputStream(testFile);
    +    byte[] buffer = new byte[1 << 16];
    +    long len = 3L << 30;
    +    assertTrue(len > Integer.MAX_VALUE);  // its 1.5x Integer.MAX_VALUE, just a sanity check
    --- End diff --
    
    Similar comments to previous test regarding large temp file and this value.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27926038
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,252 @@
    +/*
    +* 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 sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  //only public for tests
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  private long _pos;
    +  private int currentBufferIdx;
    +  private ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, findExpectedInitialPosition(underlying));
    +  }
    +
    +  private static long findExpectedInitialPosition(ByteBuffer[] bufs) {
    +    long sum = 0L;
    +    for (ByteBuffer b: bufs) {
    +      if (b.position() > 0) {
    +        // this could still lead to a mix of positions half-way through buffers that
    +        // would be inconsistent -- but we'll discover that in the constructor checks
    +        sum += b.position();
    +      } else {
    +        break;
    +      }
    +    }
    +    return sum;
    +  }
    +
    +  private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) {
    +    this.underlying = underlying;
    +    long sum = 0L;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      long nextSum = sum + b.capacity();
    +      int expectedPosition;
    +      if (nextSum < initialPosition) {
    +        expectedPosition = b.capacity();
    +      } else if (sum > initialPosition) {
    +        expectedPosition = 0;
    +      } else {
    +        expectedPosition = (int) (initialPosition - sum);
    +      }
    +      if (b.position() != expectedPosition) {
    +        throw new IllegalArgumentException("ByteBuffer[" + i + "]:" + b + " was expected to have" +
    +          " position = " + expectedPosition + " to be consistent with the overall " +
    +          "initialPosition = " + initialPosition);
    +      }
    +      sum = nextSum;
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining())
    +      throw new BufferUnderflowException();
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    while (currentBufferIdx > 0) {
    +      if (currentBuffer != null) {
    +        currentBuffer.rewind();
    +      }
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +    }
    +    currentBuffer.rewind();
    +    _pos = 0;
    +    return this;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer deepCopy() {
    +    ByteBuffer[] dataCopy = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      dataCopy[i] = ByteBuffer.allocate(b.capacity());
    +      int originalPosition = b.position();
    +      b.position(0);
    +      dataCopy[i].put(b);
    --- End diff --
    
    According to the `ByteBuffer` javadoc, this will only work if the current buffer is in its original state (or after a `rewind`). The interface documentation seems to imply that restriction does not exist.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r43674059
  
    --- Diff: core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala ---
    @@ -106,4 +105,30 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite {
         assert(arrays(1).toSeq === ref.slice(10, 20))
         assert(arrays(2).toSeq === ref.slice(20, 30))
       }
    +
    +  test("slice") {
    +    val ref = new Array[Byte](30)
    +    Random.nextBytes(ref)
    +    val o = new ByteArrayChunkOutputStream(5)
    +    o.write(ref)
    +
    +    for {
    +      start <- (0 until 30)
    +      end <- (start to 30)
    +    } {
    +      withClue(s"start = $start; end = $end") {
    +        try {
    +          assert(o.slice(start, end).toSeq === ref.slice(start, end))
    +        } catch {
    +          case ex => fail(ex)
    +        }
    +      }
    +    }
    +
    +    // errors on bad bounds
    +    intercept[IllegalArgumentException]{o.slice(31, 31)}
    --- End diff --
    
    nit: missing spaces (I think the new style checker will complain about this).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r43673875
  
    --- Diff: core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * 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, FileInputStream, FileOutputStream, OutputStream}
    +import java.nio.channels.FileChannel.MapMode
    +
    +import org.junit.Assert._
    +import org.scalatest.Matchers
    +
    +import org.apache.spark.SparkFunSuite
    +
    +class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers {
    +
    +  test("read from large mapped file") {
    +    val testFile = File.createTempFile("large-buffer-input-stream-test", ".bin")
    +
    +    try {
    +      val out: OutputStream = new FileOutputStream(testFile)
    +      val buffer: Array[Byte] = new Array[Byte](1 << 16)
    +      val len: Long = buffer.length.toLong + Integer.MAX_VALUE + 1
    +      (0 until buffer.length).foreach { idx =>
    +        buffer(idx) = idx.toByte
    +      }
    +      (0 until (len / buffer.length).toInt).foreach { idx =>
    +        out.write(buffer)
    +      }
    +      out.close
    +
    +      val channel = new FileInputStream(testFile).getChannel
    +      val buf = LargeByteBufferHelper.mapFile(channel, MapMode.READ_ONLY, 0, len)
    +      val in = new LargeByteBufferInputStream(buf, true)
    +
    +      val read = new Array[Byte](buffer.length)
    +      (0 until (len / buffer.length).toInt).foreach { idx =>
    +        in.disposed should be(false)
    +        in.read(read) should be(read.length)
    +        (0 until buffer.length).foreach { arrIdx =>
    +          assertEquals(buffer(arrIdx), read(arrIdx))
    +        }
    +      }
    +      in.disposed should be(false)
    +      in.read(read) should be(-1)
    +      in.disposed should be(false)
    +      in.close()
    +      in.disposed should be(true)
    +    } finally {
    +      testFile.delete()
    +    }
    +  }
    +
    +  test("dispose on close") {
    +    // don't need to read to the end -- dispose anytime we close
    +    val data = new Array[Byte](10)
    +    val in = new LargeByteBufferInputStream(LargeByteBufferHelper.asLargeByteBuffer(data), true)
    +    in.disposed should be (false)
    --- End diff --
    
    You could avoid the `disposed` field if you used Mockito here (mock `LargeByteBuffer` and verify `buffer.dispose()` is called).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91367249
  
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-92929866
  
      [Test build #30249 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30249/consoleFull) for   PR 5400 at commit [`e1d8fa8`](https://github.com/apache/spark/commit/e1d8fa80012cc576e636fbcb3ad8470940111b07).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27924156
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.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.buffer;
    +
    +import java.io.IOException;
    +
    +public class BufferTooLargeException extends IOException {
    +  public final long actualSize;
    --- End diff --
    
    Are these fields used anywhere?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-93006636
  
      [Test build #30251 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30251/consoleFull) for   PR 5400 at commit [`d641834`](https://github.com/apache/spark/commit/d641834ea2fb1c55bc418b1a3f61e8a958ae2a51).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class LargeByteBufferOutputStream(chunkSize: Int = 65536)`
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `
    
     * This patch **adds the following new dependencies:**
       * `commons-math3-3.4.1.jar`
    
     * This patch **removes the following dependencies:**
       * `commons-math3-3.1.1.jar`



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27926400
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,252 @@
    +/*
    +* 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 sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  //only public for tests
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  private long _pos;
    +  private int currentBufferIdx;
    +  private ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, findExpectedInitialPosition(underlying));
    +  }
    +
    +  private static long findExpectedInitialPosition(ByteBuffer[] bufs) {
    +    long sum = 0L;
    +    for (ByteBuffer b: bufs) {
    +      if (b.position() > 0) {
    +        // this could still lead to a mix of positions half-way through buffers that
    +        // would be inconsistent -- but we'll discover that in the constructor checks
    +        sum += b.position();
    +      } else {
    +        break;
    +      }
    +    }
    +    return sum;
    +  }
    +
    +  private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) {
    +    this.underlying = underlying;
    +    long sum = 0L;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      long nextSum = sum + b.capacity();
    +      int expectedPosition;
    +      if (nextSum < initialPosition) {
    +        expectedPosition = b.capacity();
    +      } else if (sum > initialPosition) {
    +        expectedPosition = 0;
    +      } else {
    +        expectedPosition = (int) (initialPosition - sum);
    +      }
    +      if (b.position() != expectedPosition) {
    +        throw new IllegalArgumentException("ByteBuffer[" + i + "]:" + b + " was expected to have" +
    +          " position = " + expectedPosition + " to be consistent with the overall " +
    +          "initialPosition = " + initialPosition);
    +      }
    +      sum = nextSum;
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining())
    +      throw new BufferUnderflowException();
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    while (currentBufferIdx > 0) {
    +      if (currentBuffer != null) {
    +        currentBuffer.rewind();
    +      }
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +    }
    +    currentBuffer.rewind();
    +    _pos = 0;
    +    return this;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer deepCopy() {
    +    ByteBuffer[] dataCopy = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      dataCopy[i] = ByteBuffer.allocate(b.capacity());
    +      int originalPosition = b.position();
    +      b.position(0);
    +      dataCopy[i].put(b);
    +      dataCopy[i].position(0);
    +      b.position(originalPosition);
    +    }
    +    return new WrappedLargeByteBuffer(dataCopy);
    +  }
    +
    +  @Override
    +  public byte get() {
    +    byte r = currentBuffer.get();
    +    _pos += 1;
    +    updateCurrentBuffer();
    +    return r;
    +  }
    +
    +  private void updateCurrentBuffer() {
    +    while (currentBuffer != null && !currentBuffer.hasRemaining()) {
    +      currentBufferIdx += 1;
    +      currentBuffer = currentBufferIdx < underlying.length ? underlying[currentBufferIdx] : null;
    +    }
    +  }
    +
    +  @Override
    +  public long position() {
    +    return _pos;
    +  }
    +
    +  @Override
    +  public long skip(long n) {
    +    if (n < 0) {
    +      final long moveTotal = Math.min(-n, _pos);
    +      long toMove = moveTotal;
    +      // move backwards -- set the position to 0 of every buffer's we go back
    +      if (currentBuffer != null) {
    +        currentBufferIdx += 1;
    +      }
    +      while (toMove > 0) {
    +        currentBufferIdx -= 1;
    +        currentBuffer = underlying[currentBufferIdx];
    +        int thisMove = (int) Math.min(toMove, currentBuffer.position());
    +        currentBuffer.position(currentBuffer.position() - thisMove);
    +        toMove -= thisMove;
    +      }
    +      _pos -= moveTotal;
    +      return -moveTotal;
    +    } else if (n > 0) {
    +      final long moveTotal = Math.min(n, remaining());
    +      long toMove = moveTotal;
    +      // move forwards-- set the position to the end of every buffer as we go forwards
    +      currentBufferIdx -= 1;
    +      while (toMove > 0) {
    +        currentBufferIdx += 1;
    +        currentBuffer = underlying[currentBufferIdx];
    +        int thisMove = (int) Math.min(toMove, currentBuffer.remaining());
    +        currentBuffer.position(currentBuffer.position() + thisMove);
    +        toMove -= thisMove;
    +      }
    +      _pos += moveTotal;
    +      return moveTotal;
    +    } else {
    +      return 0;
    +    }
    +  }
    +
    +  @Override
    +  public long remaining() {
    +    return size - _pos;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer duplicate() {
    +    ByteBuffer[] duplicates = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      duplicates[i] = underlying[i].duplicate();
    +    }
    +    return new WrappedLargeByteBuffer(duplicates, _pos);
    +  }
    +
    +  @Override
    +  public long size() {
    +    return size;
    +  }
    +
    +  @Override
    +  public long writeTo(WritableByteChannel channel) throws IOException {
    +    long written = 0l;
    +    for (; currentBufferIdx < underlying.length; currentBufferIdx++) {
    +      currentBuffer = underlying[currentBufferIdx];
    +      written += currentBuffer.remaining();
    +      while (currentBuffer.hasRemaining())
    +        channel.write(currentBuffer);
    +    }
    +    _pos = size();
    +    return written;
    +  }
    +
    +  @Override
    +  public ByteBuffer asByteBuffer() throws BufferTooLargeException {
    +    if (underlying.length > 1) {
    +      throw new BufferTooLargeException(size());
    +    }
    +    return underlying[0];
    --- End diff --
    
    This doesn't really follow the interface's contract.  E.g., it doesn't guarantee the buffer will have position 0; nor does it return as much data as possible (e.g. if the first buffer is less than the maximum size, you could include data from the second buffer, and so on).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107639280
  
    Merged build started.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27924104
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.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.buffer;
    +
    +import java.io.IOException;
    +
    +public class BufferTooLargeException extends IOException {
    +  public final long actualSize;
    +  public final long extra;
    +
    +  public BufferTooLargeException(long actualSize) {
    +    super("LargeByteBuffer is too large to convert.  Size: " + actualSize + "; Size Limit: "
    --- End diff --
    
    You could use `String.format` here; might be easier to read.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153816311
  
    Merged build finished. 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27925729
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,252 @@
    +/*
    +* 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 sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  //only public for tests
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  private long _pos;
    +  private int currentBufferIdx;
    +  private ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, findExpectedInitialPosition(underlying));
    +  }
    +
    +  private static long findExpectedInitialPosition(ByteBuffer[] bufs) {
    +    long sum = 0L;
    +    for (ByteBuffer b: bufs) {
    +      if (b.position() > 0) {
    +        // this could still lead to a mix of positions half-way through buffers that
    +        // would be inconsistent -- but we'll discover that in the constructor checks
    +        sum += b.position();
    +      } else {
    +        break;
    +      }
    +    }
    +    return sum;
    +  }
    +
    +  private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) {
    +    this.underlying = underlying;
    +    long sum = 0L;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      long nextSum = sum + b.capacity();
    +      int expectedPosition;
    +      if (nextSum < initialPosition) {
    +        expectedPosition = b.capacity();
    +      } else if (sum > initialPosition) {
    +        expectedPosition = 0;
    +      } else {
    +        expectedPosition = (int) (initialPosition - sum);
    +      }
    +      if (b.position() != expectedPosition) {
    +        throw new IllegalArgumentException("ByteBuffer[" + i + "]:" + b + " was expected to have" +
    +          " position = " + expectedPosition + " to be consistent with the overall " +
    +          "initialPosition = " + initialPosition);
    +      }
    +      sum = nextSum;
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining())
    +      throw new BufferUnderflowException();
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    while (currentBufferIdx > 0) {
    +      if (currentBuffer != null) {
    +        currentBuffer.rewind();
    +      }
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +    }
    +    currentBuffer.rewind();
    --- End diff --
    
    If you use a `do...while` the loop will take care of this.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-168112705
  
    I'm going to close this pull request. If this is still relevant and you are interested in pushing it forward, please open a new pull request. 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-6190][core] create LargeByteBuffer for ...

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

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


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31541648
  
    --- Diff: core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala ---
    @@ -91,4 +95,45 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream {
           ret
         }
       }
    +
    +  /**
    +   * Get a copy of the data between the two endpoints, start <= idx < until.  Always returns
    +   * an array of size (until - start).  Throws an IllegalArgumentException if
    +   * 0 <= start <= until <= size
    +   */
    +  def slice(start: Long, until: Long): Array[Byte] = {
    +    require((until - start) < Integer.MAX_VALUE, "max slice length = Integer.MAX_VALUE")
    +    require(start >= 0 && start <= until, s"start ($start) must be >= 0 and <= until ($until)")
    +    require(until >= start && until <= size,
    +      s"until ($until) must be >= start ($start) and <= size ($size)")
    +    var chunkStart = 0L
    +    var chunkIdx = 0
    +    val length = (until - start).toInt
    +    var foundStart = false
    +    val result = new Array[Byte](length)
    +    while (!foundStart) {
    +      val nextSize = chunkStart + chunks(chunkIdx).size
    --- End diff --
    
    This feels wrong. You're comparing `start` to the added size of the last two chunks, not to the cumulative size of all previous chunks.
    
    I think L119 should be a `+=` instead of plain `=`, and maybe you need a better named variable than `chunkStart`.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31543872
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,280 @@
    +/*
    +* 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.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +/**
    + * A {@link org.apache.spark.network.buffer.LargeByteBuffer} which may contain multiple
    + * {@link java.nio.ByteBuffer}s.  In order to support <code>asByteBuffer</code>, all
    + * of the underlying ByteBuffers must have size equal to
    + * {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE} (except that last
    + * one).  The underlying ByteBuffers may be on-heap, direct, or memory-mapped.
    + */
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    +   * each sub-ByteBuffer (except for the last one) must be exactly this size.  Note that this
    +   * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK_SIZE.  The only reason it isn't
    +   * is so that we can do tests without creating ginormous buffers.  Public methods force it to
    +   * be LargeByteBufferHelper.MAX_CHUNK_SIZE
    +   */
    +  private final int subBufferSize;
    +  private long _pos;
    +  @VisibleForTesting
    +  int currentBufferIdx;
    +  @VisibleForTesting
    +  ByteBuffer currentBuffer;
    +
    +  /**
    +   * Construct a WrappedLargeByteBuffer from the given ByteBuffers.  Each of the ByteBuffers must
    +   * have size equal to {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE}
    +   * except for the final one.  The buffers are <code>duplicate</code>d, so the position of the
    +   * given buffers and the returned buffer will be independent, though the underlying data will be
    +   * shared.  The constructed buffer will always have position == 0.
    +   */
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, LargeByteBufferHelper.MAX_CHUNK_SIZE);
    +  }
    +
    +  /**
    +   * you do **not** want to call this version.  It leads to a buffer which doesn't properly
    +   * support {@link #asByteBuffer}.  The only reason it exists is to we can have tests which
    +   * don't require 2GB of memory
    +   *
    +   * @param underlying
    +   * @param subBufferSize
    +   */
    +  @VisibleForTesting
    +  WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) {
    +    if (underlying.length == 0) {
    +      throw new IllegalArgumentException("must wrap at least one ByteBuffer");
    +    }
    +    this.underlying = new ByteBuffer[underlying.length];
    +    this.subBufferSize = subBufferSize;
    +    long sum = 0L;
    +
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i].duplicate();
    +      b.position(0);
    +      this.underlying[i] = b;
    +      if (i != underlying.length -1 && b.capacity() != subBufferSize) {
    --- End diff --
    
    nit: `- 1`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91275348
  
      [Test build #29944 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29944/consoleFull) for   PR 5400 at commit [`a759242`](https://github.com/apache/spark/commit/a75924234ea3523ae3409a384dfef2dbbfbc9d82).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28723219
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,269 @@
    +/*
    +* 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 com.google.common.annotations.VisibleForTesting;
    --- End diff --
    
    nit: after `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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28722781
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.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.buffer;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +  public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    +
    +  public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { buffer } );
    +  }
    +
    +  public static LargeByteBuffer asLargeByteBuffer(byte[] bytes) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { ByteBuffer.wrap(bytes) } );
    +  }
    +
    +  public static LargeByteBuffer allocate(long size) {
    +    return allocate(size, MAX_CHUNK);
    +  }
    +
    +  @VisibleForTesting
    +  static LargeByteBuffer allocate(long size, int maxChunk) {
    +    int chunksNeeded = (int) ((size + maxChunk - 1) / maxChunk);
    +    ByteBuffer[] chunks = new ByteBuffer[chunksNeeded];
    +    long remaining = size;
    +    for (int i = 0; i < chunksNeeded; i++) {
    +      int nextSize = (int) Math.min(remaining, maxChunk);
    +      ByteBuffer next = ByteBuffer.allocate(nextSize);
    +      remaining -= nextSize;
    +      chunks[i] = next;
    +    }
    +    if (remaining != 0) throw new IllegalStateException("remaining = " + remaining);
    +    return new WrappedLargeByteBuffer(chunks, maxChunk);
    +  }
    +
    +
    +  public static LargeByteBuffer mapFile(
    +    FileChannel channel,
    --- End diff --
    
    nit: style is to double-indent these


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31543142
  
    --- Diff: core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.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.buffer
    +
    +import java.io.{File, FileInputStream, FileOutputStream, OutputStream}
    +import java.nio.channels.FileChannel.MapMode
    +
    +import org.junit.Assert._
    +import org.scalatest.Matchers
    +
    +import org.apache.spark.SparkFunSuite
    +
    +class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers {
    +
    +  test("read from large mapped file") {
    +    val testFile = File.createTempFile("large-buffer-input-stream-test", ".bin")
    +
    +    try {
    +      val out: OutputStream = new FileOutputStream(testFile)
    +      val buffer: Array[Byte] = new Array[Byte](1 << 16)
    +      val len: Long = buffer.length.toLong + Integer.MAX_VALUE + 1
    +      (0 until buffer.length).foreach { idx =>
    +        buffer(idx) = idx.toByte
    +      }
    +      (0 until (len / buffer.length).toInt).foreach { idx =>
    +        out.write(buffer)
    +      }
    +      out.close
    +
    +      val channel = new FileInputStream(testFile).getChannel
    +      val buf = LargeByteBufferHelper.mapFile(channel, MapMode.READ_ONLY, 0, len)
    +      val in = new LargeByteBufferInputStream(buf, true)
    +
    +      val read = new Array[Byte](buffer.length)
    +      (0 until (len / buffer.length).toInt).foreach { idx =>
    +        in.disposed should be(false)
    +        in.read(read) should be(read.length)
    +        (0 until buffer.length).foreach { arrIdx =>
    +          assertEquals(buffer(arrIdx), read(arrIdx))
    +        }
    +      }
    +      in.disposed should be(false)
    +      in.read(read) should be(-1)
    +      in.disposed should be(false)
    +      in.close()
    +      in.disposed should be(true)
    +    } finally {
    +      testFile.delete()
    +    }
    +  }
    +
    +  test("dispose on close") {
    +    // don't need to read to the end -- dispose anytime we close
    +    val data = new Array[Byte](10)
    +    val in = new LargeByteBufferInputStream(LargeByteBufferHelper.asLargeByteBuffer(data), true)
    +    in.disposed should be (false)
    +    in.close()
    +    in.disposed should be (true)
    +  }
    +
    +  test("io stream roundtrip") {
    +    val out = new LargeByteBufferOutputStream(128)
    +    (0 until 200).foreach{idx => out.write(idx)}
    +    out.close()
    +
    +    val lb = out.largeBuffer(128)
    +    // just make sure that we test reading from multiple chunks
    +    lb.asInstanceOf[WrappedLargeByteBuffer].underlying.size should be > 1
    +
    +    val rawIn = new LargeByteBufferInputStream(lb)
    +    val arr = new Array[Byte](500)
    +    val nRead = rawIn.read(arr, 0, 500)
    +    nRead should be (200)
    +    (0 until 200).foreach{idx =>
    --- End diff --
    
    nit: `foreach { idx =>`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r43674528
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,292 @@
    +/*
    +* 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.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +/**
    + * A {@link org.apache.spark.network.buffer.LargeByteBuffer} which may contain multiple
    + * {@link java.nio.ByteBuffer}s.  In order to support <code>asByteBuffer</code>, all
    + * of the underlying ByteBuffers must have size equal to
    + * {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE} (except that last
    + * one).  The underlying ByteBuffers may be on-heap, direct, or memory-mapped.
    + */
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    +   * each sub-ByteBuffer (except for the last one) must be exactly this size.  Note that this
    +   * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK_SIZE.  The only reason it isn't
    +   * is so that we can do tests without creating ginormous buffers.  Public methods force it to
    +   * be LargeByteBufferHelper.MAX_CHUNK_SIZE
    +   */
    +  private final int subBufferSize;
    +  private long _pos;
    +  @VisibleForTesting
    +  int currentBufferIdx;
    +  @VisibleForTesting
    +  ByteBuffer currentBuffer;
    +
    +  /**
    +   * Construct a WrappedLargeByteBuffer from the given ByteBuffers.  Each of the ByteBuffers must
    +   * have size equal to {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE}
    +   * except for the final one.  The buffers are <code>duplicate</code>d, so the position of the
    +   * given buffers and the returned buffer will be independent, though the underlying data will be
    +   * shared.  The constructed buffer will always have position == 0.
    +   */
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, LargeByteBufferHelper.MAX_CHUNK_SIZE);
    +  }
    +
    +  /**
    +   * you do **not** want to call this version.  It leads to a buffer which doesn't properly
    +   * support {@link #asByteBuffer}.  The only reason it exists is to we can have tests which
    +   * don't require 2GB of memory
    +   *
    +   * @param underlying
    +   * @param subBufferSize
    +   */
    +  @VisibleForTesting
    +  WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) {
    +    if (underlying.length == 0) {
    +      throw new IllegalArgumentException("must wrap at least one ByteBuffer");
    +    }
    +    this.underlying = new ByteBuffer[underlying.length];
    +    this.subBufferSize = subBufferSize;
    +    long sum = 0L;
    +
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i].duplicate();
    +      b.position(0);
    +      this.underlying[i] = b;
    +      if (i != underlying.length - 1 && b.capacity() != subBufferSize) {
    +        // this is to make sure that asByteBuffer() is implemented correctly.  We need the first
    +        // subBuffer to be LargeByteBufferHelper.MAX_CHUNK_SIZE.  We don't *have* to check all the
    +        // subBuffers, but I figure its makes it more consistent this way.  (Also, this check
    +        // really only serves a purpose when using the public constructor -- subBufferSize is a
    +        // a parameter just to allow small tests.)
    +        throw new IllegalArgumentException("All buffers, except for the final one, must have " +
    +          "size = " + subBufferSize);
    +      }
    +      sum += b.capacity();
    +    }
    +    _pos = 0;
    +    currentBufferIdx = 0;
    +    currentBuffer = this.underlying[0];
    +    size = sum;
    +  }
    +
    --- End diff --
    
    nit: too many empty lines


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107642820
  
      [Test build #33906 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33906/consoleFull) for   PR 5400 at commit [`b77bbe2`](https://github.com/apache/spark/commit/b77bbe28e109db72775a592e99572c039b377517).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28723860
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,269 @@
    +/*
    +* 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 com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    +   * each sub-ByteBuffer (except for the last one) must be exactly this size.  Note that this
    +   * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK.  The only reason it isn't
    +   * is so that we can do tests without creating ginormous buffers.  Public methods force it to
    +   * be LargeByteBufferHelper.MAX_CHUNK
    +   */
    +  private final int subBufferSize;
    +  private long _pos;
    +  @VisibleForTesting
    +  int currentBufferIdx;
    +  @VisibleForTesting
    +  ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, LargeByteBufferHelper.MAX_CHUNK);
    +  }
    +
    +  /**
    +   * you do **not** want to call this version.  It leads to a buffer which doesn't properly
    +   * support {@link #asByteBuffer}.  The only reason it exists is to we can have tests which
    +   * don't require 2GB of memory
    +   *
    +   * @param underlying
    +   * @param subBufferSize
    +   */
    +  @VisibleForTesting
    +  WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) {
    +    if (underlying.length == 0) {
    +      throw new IllegalArgumentException("must wrap at least one ByteBuffer");
    +    }
    +    this.underlying = underlying;
    +    this.subBufferSize = subBufferSize;
    +    long sum = 0L;
    +    boolean startFound = false;
    +    long initialPosition = -1;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      if (i != underlying.length -1 && b.capacity() != subBufferSize) {
    +        throw new IllegalArgumentException("All buffers, except for the final one, must have " +
    +          "size = " + subBufferSize);
    +      }
    +      if (startFound) {
    +        if (b.position() != 0) {
    +          throw new IllegalArgumentException("ByteBuffers have inconsistent positions");
    +        }
    +      } else if (b.position() != b.capacity()) {
    +        startFound = true;
    +        initialPosition = sum + b.position();
    +      }
    +      sum += b.capacity();
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining()) {
    +      throw new BufferUnderflowException();
    +    }
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    if (currentBuffer != null) {
    +      currentBuffer.rewind();
    +    }
    +    while (currentBufferIdx > 0) {
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +      currentBuffer.rewind();
    +    }
    +    _pos = 0;
    +    return this;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer deepCopy() {
    +    ByteBuffer[] dataCopy = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      dataCopy[i] = ByteBuffer.allocate(b.capacity());
    +      int originalPosition = b.position();
    +      b.rewind();
    +      dataCopy[i].put(b);
    +      dataCopy[i].position(0);
    +      b.position(originalPosition);
    +    }
    +    return new WrappedLargeByteBuffer(dataCopy, subBufferSize);
    +  }
    +
    +  @Override
    +  public byte get() {
    +    byte r = currentBuffer.get();
    --- End diff --
    
    This will throw an NPE if it is called after you go past the last buffer. Docs say it should throw `BufferUnderflowException` 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-94559761
  
    I left another round of comments. This is looking a lot better, although I need to look at the latter tests in more detail. I'm a little concerned about writing large files during the test, it would be great to have an alternative to that (maybe use mockito?).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107641966
  
      [Test build #33904 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33904/consoleFull) for   PR 5400 at commit [`95588c2`](https://github.com/apache/spark/commit/95588c25bb40a7971ea05d6b261a1a044548feb9).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class LargeByteBufferInputStream extends InputStream `
      * `public class LargeByteBufferOutputStream extends OutputStream `
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-92490225
  
      [Test build #30190 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30190/consoleFull) for   PR 5400 at commit [`c3efa4c`](https://github.com/apache/spark/commit/c3efa4c24986937ffaf18c755323866fa54a073c).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27924419
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java ---
    @@ -0,0 +1,73 @@
    +/*
    + * 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.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +    public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    +
    +    public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +        return new WrappedLargeByteBuffer(new ByteBuffer[]{buffer});
    --- End diff --
    
    nit: use 2-space indentation all over 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107641970
  
    Merged build finished. 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28718397
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java ---
    @@ -0,0 +1,113 @@
    +/*
    +* 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.nio.ByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +
    +public interface LargeByteBuffer {
    +  public byte get();
    +
    +  /**
    +   * Bulk copy data from this buffer into the given array.  First checks there is sufficient
    +   * data in this buffer; if not, throws a {@link java.nio.BufferUnderflowException}.
    +   *
    +   * @param dst
    +   * @param offset
    +   * @param length
    +   */
    +  public void get(byte[] dst, int offset, int length);
    +
    +  public LargeByteBuffer rewind();
    +
    +  /**
    +   * Return a deep copy of this buffer.
    +   * The returned buffer will have position == 0.  The position
    +   * of this buffer will not change as a result of copying.
    +   *
    +   * @return a new buffer with a full copy of this buffer's data
    +   */
    +  public LargeByteBuffer deepCopy();
    +
    +  /**
    +   * Advance the position in this buffer by up to <code>n</code> bytes.  <code>n</code> may be
    +   * positive or negative.  It will move the full <code>n</code> unless that moves
    +   * it past the end (or beginning) of the buffer, in which case it will move to the end
    +   * (or beginning).
    +   *
    +   * @return the number of bytes moved forward (can be negative if <code>n</code> is negative)
    +   */
    +  public long skip(long n);
    +
    +  public long position();
    +
    +  /**
    +   * Creates a new byte buffer that shares this buffer's content.
    +   * <p/>
    +   * <p> The content of the new buffer will be that of this buffer.  Changes
    --- End diff --
    
    super nit: you seem to be mixing styles (`<p/>` vs. `<p></p>`).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27922497
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * 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.util
    +
    +import java.io.InputStream
    +
    +import org.apache.spark.network.buffer.LargeByteBuffer
    +import org.apache.spark.storage.BlockManager
    +
    +/**
    + * Reads data from a LargeByteBuffer, and optionally cleans it up using buffer.dispose()
    + * at the end of the stream (e.g. to close a memory-mapped file).
    + */
    +private[spark]
    +class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: Boolean = false)
    +  extends InputStream {
    +
    +  override def read(): Int = {
    +    if (buffer == null || buffer.remaining() == 0) {
    +      cleanUp()
    +      -1
    +    } else {
    +      val r = buffer.get() & 0xFF
    +      if (buffer.remaining() == 0) {
    +        cleanUp()
    --- End diff --
    
    Isn't this better done as part of `close()`?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28116365
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * 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.util
    +
    +import java.io.InputStream
    +
    +import org.apache.spark.network.buffer.LargeByteBuffer
    +import org.apache.spark.storage.BlockManager
    +
    +/**
    + * Reads data from a LargeByteBuffer, and optionally cleans it up using buffer.dispose()
    + * at the end of the stream (e.g. to close a memory-mapped file).
    + */
    +private[spark]
    +class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: Boolean = false)
    +  extends InputStream {
    +
    +  override def read(): Int = {
    +    if (buffer == null || buffer.remaining() == 0) {
    +      cleanUp()
    +      -1
    +    } else {
    +      val r = buffer.get() & 0xFF
    +      if (buffer.remaining() == 0) {
    +        cleanUp()
    --- End diff --
    
    If anyway is watching on the sidelines -- marcelo and I chatted about this a while and realized there is an issue with the existing use of `ByteBufferInputStream` (where this code was copied from) that prevents it from getting properly disposed in all cases.  I've opened https://issues.apache.org/jira/browse/SPARK-6839


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-92949178
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30249/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28717448
  
    --- Diff: core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.OutputStream
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream
    +
    +private[spark] class LargeByteBufferOutputStream(chunkSize: Int = 65536)
    +  extends OutputStream {
    +
    +  private[buffer] val output = new ByteArrayChunkOutputStream(chunkSize)
    +
    +  override def write(b: Int): Unit = {
    +    output.write(b)
    +  }
    +
    +  override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = {
    +    output.write(bytes, offs, len)
    +  }
    +
    +  def largeBuffer: LargeByteBuffer = {
    +    largeBuffer(LargeByteBufferHelper.MAX_CHUNK)
    +  }
    +
    +  /**
    +   * exposed for testing.  You don't really ever want to call this method -- the returned
    +   * buffer will not implement {{asByteBuffer}} correctly.
    +   */
    +  private[buffer] def largeBuffer(maxChunk: Int): WrappedLargeByteBuffer = {
    +    // LargeByteBuffer is supposed to make a "best effort" to get all the data
    --- End diff --
    
    The scaladoc and this comment look a little confusing. Is this only used for testing, or is it called by non-test Spark code 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-108022625
  
    Hey, one idea for avoiding tests that write huge files: can we use a data generator + streaming checksum function to be able to detect data corruption without having to materialize the entire 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107670171
  
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r43673151
  
    --- Diff: core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java ---
    @@ -0,0 +1,91 @@
    +/*
    + * 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.OutputStream;
    +import java.nio.ByteBuffer;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream;
    +
    +/**
    + * An OutputStream that will write all data to memory.  It supports writing over 2GB
    + * and the resulting data can be retrieved as a
    + * {@link org.apache.spark.network.buffer.LargeByteBuffer}
    + */
    +public class LargeByteBufferOutputStream extends OutputStream {
    +
    +  private final ByteArrayChunkOutputStream output;
    +
    +  /**
    +   * Create a new LargeByteBufferOutputStream which writes to byte arrays of the given size.  Note
    +   * that <code>chunkSize</code> has <b>no effect</b> on the LargeByteBuffer returned by
    +   * {@link #largeBuffer()}.
    +   *
    +   * @param chunkSize size of the byte arrays used by this output stream, in bytes
    +   */
    +  public LargeByteBufferOutputStream(int chunkSize) {
    +    output = new ByteArrayChunkOutputStream(chunkSize);
    +  }
    +
    +  @Override
    +  public void write(int b) {
    +    output.write(b);
    +  }
    +
    +  @Override
    +  public void write(byte[] bytes, int off, int len) {
    +    output.write(bytes, off, len);
    +  }
    +
    +  /**
    +   * Get all of the data written to the stream so far as a LargeByteBuffer.  This method can be
    +   * called multiple times, and each returned buffer will be completely independent (the data
    +   * is copied for each returned buffer).  It does not close the stream.
    +   *
    +   * @return the data written to the stream as a LargeByteBuffer
    +   */
    +  public LargeByteBuffer largeBuffer() {
    +    return largeBuffer(LargeByteBufferHelper.MAX_CHUNK_SIZE);
    +  }
    +
    +  /**
    +   * exposed for testing.  You don't really ever want to call this method -- the returned
    --- End diff --
    
    "exposed for testing." is redundant.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31540997
  
    --- Diff: core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.OutputStream;
    +import java.nio.ByteBuffer;
    +
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream;
    +
    +public class LargeByteBufferOutputStream extends OutputStream {
    +
    +  private final int chunkSize;
    +  final ByteArrayChunkOutputStream output;
    --- End diff --
    
    It feels like all these fields (not just in this class) should be either private or annotated with `@VisibleForTesting`.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-93024398
  
      [Test build #30272 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30272/consoleFull) for   PR 5400 at commit [`78e4043`](https://github.com/apache/spark/commit/78e40439d1c1515f2197fc465653e4434330278d).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153852136
  
    Merged build finished. 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28724093
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,269 @@
    +/*
    +* 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 com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    +   * each sub-ByteBuffer (except for the last one) must be exactly this size.  Note that this
    +   * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK.  The only reason it isn't
    +   * is so that we can do tests without creating ginormous buffers.  Public methods force it to
    +   * be LargeByteBufferHelper.MAX_CHUNK
    +   */
    +  private final int subBufferSize;
    +  private long _pos;
    +  @VisibleForTesting
    +  int currentBufferIdx;
    +  @VisibleForTesting
    +  ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, LargeByteBufferHelper.MAX_CHUNK);
    +  }
    +
    +  /**
    +   * you do **not** want to call this version.  It leads to a buffer which doesn't properly
    +   * support {@link #asByteBuffer}.  The only reason it exists is to we can have tests which
    +   * don't require 2GB of memory
    +   *
    +   * @param underlying
    +   * @param subBufferSize
    +   */
    +  @VisibleForTesting
    +  WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) {
    +    if (underlying.length == 0) {
    +      throw new IllegalArgumentException("must wrap at least one ByteBuffer");
    +    }
    +    this.underlying = underlying;
    +    this.subBufferSize = subBufferSize;
    +    long sum = 0L;
    +    boolean startFound = false;
    +    long initialPosition = -1;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      if (i != underlying.length -1 && b.capacity() != subBufferSize) {
    +        throw new IllegalArgumentException("All buffers, except for the final one, must have " +
    +          "size = " + subBufferSize);
    +      }
    +      if (startFound) {
    +        if (b.position() != 0) {
    +          throw new IllegalArgumentException("ByteBuffers have inconsistent positions");
    +        }
    +      } else if (b.position() != b.capacity()) {
    +        startFound = true;
    +        initialPosition = sum + b.position();
    +      }
    +      sum += b.capacity();
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining()) {
    +      throw new BufferUnderflowException();
    +    }
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    if (currentBuffer != null) {
    +      currentBuffer.rewind();
    +    }
    +    while (currentBufferIdx > 0) {
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +      currentBuffer.rewind();
    +    }
    +    _pos = 0;
    +    return this;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer deepCopy() {
    +    ByteBuffer[] dataCopy = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      dataCopy[i] = ByteBuffer.allocate(b.capacity());
    +      int originalPosition = b.position();
    +      b.rewind();
    +      dataCopy[i].put(b);
    +      dataCopy[i].position(0);
    +      b.position(originalPosition);
    +    }
    +    return new WrappedLargeByteBuffer(dataCopy, subBufferSize);
    +  }
    +
    +  @Override
    +  public byte get() {
    +    byte r = currentBuffer.get();
    +    _pos += 1;
    +    updateCurrentBuffer();
    +    return r;
    +  }
    +
    +  private void updateCurrentBuffer() {
    +    while (currentBuffer != null && !currentBuffer.hasRemaining()) {
    +      currentBufferIdx += 1;
    +      currentBuffer = currentBufferIdx < underlying.length ? underlying[currentBufferIdx] : null;
    +    }
    +  }
    +
    +  @Override
    +  public long position() {
    +    return _pos;
    +  }
    +
    +  @Override
    +  public long skip(long n) {
    +    if (n < 0) {
    +      final long moveTotal = Math.min(-n, _pos);
    +      long toMove = moveTotal;
    +      // move backwards -- set the position to 0 of every buffer's we go back
    +      if (currentBuffer != null) {
    +        currentBufferIdx += 1;
    +      }
    +      while (toMove > 0) {
    +        currentBufferIdx -= 1;
    +        currentBuffer = underlying[currentBufferIdx];
    +        int thisMove = (int) Math.min(toMove, currentBuffer.position());
    +        currentBuffer.position(currentBuffer.position() - thisMove);
    +        toMove -= thisMove;
    +      }
    +      _pos -= moveTotal;
    +      return -moveTotal;
    +    } else if (n > 0) {
    +      final long moveTotal = Math.min(n, remaining());
    +      long toMove = moveTotal;
    +      // move forwards -- set the position to the end of every buffer as we go forwards
    --- End diff --
    
    Similarly inaccurate (not *every* 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31540947
  
    --- Diff: core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.OutputStream;
    +import java.nio.ByteBuffer;
    +
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream;
    +
    +public class LargeByteBufferOutputStream extends OutputStream {
    +
    +  private final int chunkSize;
    +  final ByteArrayChunkOutputStream output;
    +
    +  public LargeByteBufferOutputStream(int chunkSize) {
    +    this.chunkSize = chunkSize;
    +    output = new ByteArrayChunkOutputStream(chunkSize);
    +  }
    +
    +  public void write(int b) {
    +    output.write(b);
    +  }
    +
    +  public void write(byte[] bytes, int off, int len) {
    +    output.write(bytes, off, len);
    +  }
    +
    +  public LargeByteBuffer largeBuffer() {
    +    return largeBuffer(LargeByteBufferHelper.MAX_CHUNK_SIZE);
    +  }
    +
    +/**
    --- End diff --
    
    indentation, @VisibleForTesting


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153818962
  
    **[Test build #45032 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45032/consoleFull)** for PR 5400 at commit [`3447bb9`](https://github.com/apache/spark/commit/3447bb995b53c4d93154328c7c7c06e08a5ec9b9).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107826845
  
      [Test build #33956 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33956/consoleFull) for   PR 5400 at commit [`6c2a115`](https://github.com/apache/spark/commit/6c2a115b970d31400cd77dc8f0236555e382a1af).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class LargeByteBufferInputStream extends InputStream `
      * `public class LargeByteBufferOutputStream extends OutputStream `
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28010305
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.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.buffer;
    +
    +import java.io.IOException;
    +
    +public class BufferTooLargeException extends IOException {
    +  public final long actualSize;
    --- End diff --
    
    they're not used in this PR.  But when we this is actually used in Spark, its nice to catch this exception and give something more useful (eg., an exception which indicates that the error was during block replication, the block was over the limit by XXX, with nice formatting from `Utils.bytesToString`).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27924014
  
    --- Diff: core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * 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.util
    +
    +import scala.util.Random
    +
    +import org.scalatest.{FunSuite, Matchers}
    +
    +import org.apache.spark.network.buffer.WrappedLargeByteBuffer
    +
    +class LargeByteBufferOutputStreamSuite extends FunSuite with Matchers {
    +
    +  test("merged buffers for < 2GB") {
    +    val out = new LargeByteBufferOutputStream(10)
    +    val bytes = new Array[Byte](100)
    +    Random.nextBytes(bytes)
    +    out.write(bytes)
    +
    +    val buffer = out.largeBuffer
    +    buffer.position() should be (0)
    +    buffer.size() should be (100)
    +    val nioBuffer = buffer.asByteBuffer()
    +    nioBuffer.position() should be (0)
    +    nioBuffer.capacity() should be (100)
    +    nioBuffer.limit() should be (100)
    +
    +    val read = new Array[Byte](100)
    +    buffer.get(read, 0, 100)
    +    read should be (bytes)
    +
    +    buffer.rewind()
    +    nioBuffer.get(read)
    +    read should be (bytes)
    +  }
    +
    +  test("chunking") {
    +    val out = new LargeByteBufferOutputStream(10)
    +    val bytes = new Array[Byte](100)
    +    Random.nextBytes(bytes)
    +    out.write(bytes)
    +
    +    (10 to 100 by 10).foreach{chunkSize =>
    --- End diff --
    
    nit: spaces around `{`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28722381
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.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.buffer;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +  public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    +
    +  public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { buffer } );
    +  }
    +
    +  public static LargeByteBuffer asLargeByteBuffer(byte[] bytes) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { ByteBuffer.wrap(bytes) } );
    --- End diff --
    
    `asLargeByteBuffer(ByteBuffer.wrap(bytes))`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153818380
  
     Merged build triggered.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107639210
  
     Merged build triggered.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91367624
  
      [Test build #29972 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29972/consoleFull) for   PR 5400 at commit [`a759242`](https://github.com/apache/spark/commit/a75924234ea3523ae3409a384dfef2dbbfbc9d82).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r43672931
  
    --- Diff: core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java ---
    @@ -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.buffer;
    +
    +import java.io.InputStream;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Reads data from a LargeByteBuffer, and optionally cleans it up using buffer.dispose()
    + * when the stream is closed (e.g. to close a memory-mapped file).
    + */
    +public class LargeByteBufferInputStream extends InputStream {
    +
    +  private LargeByteBuffer buffer;
    +  private final boolean dispose;
    +
    +  public LargeByteBufferInputStream(LargeByteBuffer buffer, boolean dispose) {
    +    this.buffer = buffer;
    +    this.dispose = dispose;
    +  }
    +
    +  public LargeByteBufferInputStream(LargeByteBuffer buffer) {
    +    this(buffer, false);
    +  }
    +
    +  @Override
    +  public int read() {
    +    if (buffer == null || buffer.remaining() == 0) {
    +      return -1;
    +    } else {
    +      return buffer.get() & 0xFF;
    +    }
    +  }
    +
    +  @Override
    +  public int read(byte[] dest) {
    +    return read(dest, 0, dest.length);
    +  }
    +
    +  @Override
    +  public int read(byte[] dest, int offset, int length) {
    +    if (buffer == null || buffer.remaining() == 0) {
    +      return -1;
    +    } else {
    +      int amountToGet = (int) Math.min(buffer.remaining(), length);
    +      buffer.get(dest, offset, amountToGet);
    +      return amountToGet;
    +    }
    +  }
    +
    +  @Override
    +  public long skip(long toSkip) {
    +    if (buffer != null) {
    +      return buffer.skip(toSkip);
    +    } else {
    +      return 0L;
    +    }
    +  }
    +
    +  // only for testing
    --- End diff --
    
    nit: comment is redundant.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27923285
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala ---
    @@ -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.util
    +
    +import java.io.OutputStream
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.network.buffer.{LargeByteBufferHelper, WrappedLargeByteBuffer, LargeByteBuffer}
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream
    +
    +private[spark]
    +class LargeByteBufferOutputStream(chunkSize: Int = 65536)
    +  extends OutputStream {
    +
    +  private[util] val output = new ByteArrayChunkOutputStream(chunkSize)
    +
    +  private var _pos = 0
    +
    +  override def write(b: Int): Unit = {
    +    output.write(b)
    --- End diff --
    
    In fact, it seems like `_pos` is not really used.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107639633
  
      [Test build #33903 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33903/consoleFull) for   PR 5400 at commit [`3c599b2`](https://github.com/apache/spark/commit/3c599b2f01e0e4d1ac2a9d02cf6a8e8924da488d).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91301916
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29944/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-93118047
  
      [Test build #30280 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30280/consoleFull) for   PR 5400 at commit [`117fb4a`](https://github.com/apache/spark/commit/117fb4a166a722bd352d74de59f26e45100a022c).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `
    
     * This patch **adds the following new dependencies:**
       * `snappy-java-1.1.1.7.jar`
    
     * This patch **removes the following dependencies:**
       * `snappy-java-1.1.1.6.jar`



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28724806
  
    --- Diff: network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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 org.junit.Test;
    --- End diff --
    
    nit: after `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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91301898
  
      [Test build #29944 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29944/consoleFull) for   PR 5400 at commit [`a759242`](https://github.com/apache/spark/commit/a75924234ea3523ae3409a384dfef2dbbfbc9d82).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class LargeByteBufferOutputStream(chunkSize: Int = 65536)`
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `
    
     * This patch does not change any dependencies.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-93083321
  
    Thanks for comments @vanzin.  I think I've addressed all of your concerns.  I went back and forth a few times about what the contract of `asByteBuffer` should be, thanks for pointing out the inconsistencies.  I think I've cleaned that up and added more unit tests.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28717072
  
    --- Diff: core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.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.buffer
    +
    +import java.io.InputStream
    +
    +/**
    + * Reads data from a LargeByteBuffer, and optionally cleans it up using buffer.dispose()
    + * when the stream is closed (e.g. to close a memory-mapped file).
    + */
    +private[spark]
    +class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: Boolean = false)
    +  extends InputStream {
    +
    +  override def read(): Int = {
    +    if (buffer == null || buffer.remaining() == 0) {
    +      -1
    +    } else {
    +      buffer.get() & 0xFF
    +    }
    +  }
    +
    +  override def read(dest: Array[Byte]): Int = {
    +    read(dest, 0, dest.length)
    +  }
    +
    +  override def read(dest: Array[Byte], offset: Int, length: Int): Int = {
    +    if (buffer == null || buffer.remaining() == 0) {
    +      -1
    +    } else {
    +      val amountToGet = math.min(buffer.remaining(), length).toInt
    +      buffer.get(dest, offset, amountToGet)
    +      amountToGet
    +    }
    +  }
    +
    +  override def skip(bytes: Long): Long = {
    +    if (buffer != null) {
    +      val skipped = buffer.skip(bytes)
    --- End diff --
    
    skip the `val`?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27926161
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,252 @@
    +/*
    +* 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 sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  //only public for tests
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  private long _pos;
    +  private int currentBufferIdx;
    +  private ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, findExpectedInitialPosition(underlying));
    +  }
    +
    +  private static long findExpectedInitialPosition(ByteBuffer[] bufs) {
    +    long sum = 0L;
    +    for (ByteBuffer b: bufs) {
    +      if (b.position() > 0) {
    +        // this could still lead to a mix of positions half-way through buffers that
    +        // would be inconsistent -- but we'll discover that in the constructor checks
    +        sum += b.position();
    +      } else {
    +        break;
    +      }
    +    }
    +    return sum;
    +  }
    +
    +  private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) {
    +    this.underlying = underlying;
    +    long sum = 0L;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      long nextSum = sum + b.capacity();
    +      int expectedPosition;
    +      if (nextSum < initialPosition) {
    +        expectedPosition = b.capacity();
    +      } else if (sum > initialPosition) {
    +        expectedPosition = 0;
    +      } else {
    +        expectedPosition = (int) (initialPosition - sum);
    +      }
    +      if (b.position() != expectedPosition) {
    +        throw new IllegalArgumentException("ByteBuffer[" + i + "]:" + b + " was expected to have" +
    +          " position = " + expectedPosition + " to be consistent with the overall " +
    +          "initialPosition = " + initialPosition);
    +      }
    +      sum = nextSum;
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining())
    +      throw new BufferUnderflowException();
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    while (currentBufferIdx > 0) {
    +      if (currentBuffer != null) {
    +        currentBuffer.rewind();
    +      }
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +    }
    +    currentBuffer.rewind();
    +    _pos = 0;
    +    return this;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer deepCopy() {
    +    ByteBuffer[] dataCopy = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      dataCopy[i] = ByteBuffer.allocate(b.capacity());
    +      int originalPosition = b.position();
    +      b.position(0);
    +      dataCopy[i].put(b);
    +      dataCopy[i].position(0);
    +      b.position(originalPosition);
    +    }
    +    return new WrappedLargeByteBuffer(dataCopy);
    +  }
    +
    +  @Override
    +  public byte get() {
    +    byte r = currentBuffer.get();
    +    _pos += 1;
    +    updateCurrentBuffer();
    +    return r;
    +  }
    +
    +  private void updateCurrentBuffer() {
    +    while (currentBuffer != null && !currentBuffer.hasRemaining()) {
    +      currentBufferIdx += 1;
    +      currentBuffer = currentBufferIdx < underlying.length ? underlying[currentBufferIdx] : null;
    +    }
    +  }
    +
    +  @Override
    +  public long position() {
    +    return _pos;
    +  }
    +
    +  @Override
    +  public long skip(long n) {
    +    if (n < 0) {
    +      final long moveTotal = Math.min(-n, _pos);
    +      long toMove = moveTotal;
    +      // move backwards -- set the position to 0 of every buffer's we go back
    +      if (currentBuffer != null) {
    +        currentBufferIdx += 1;
    +      }
    +      while (toMove > 0) {
    +        currentBufferIdx -= 1;
    +        currentBuffer = underlying[currentBufferIdx];
    +        int thisMove = (int) Math.min(toMove, currentBuffer.position());
    +        currentBuffer.position(currentBuffer.position() - thisMove);
    +        toMove -= thisMove;
    +      }
    +      _pos -= moveTotal;
    +      return -moveTotal;
    +    } else if (n > 0) {
    +      final long moveTotal = Math.min(n, remaining());
    +      long toMove = moveTotal;
    +      // move forwards-- set the position to the end of every buffer as we go forwards
    +      currentBufferIdx -= 1;
    +      while (toMove > 0) {
    +        currentBufferIdx += 1;
    +        currentBuffer = underlying[currentBufferIdx];
    +        int thisMove = (int) Math.min(toMove, currentBuffer.remaining());
    +        currentBuffer.position(currentBuffer.position() + thisMove);
    +        toMove -= thisMove;
    +      }
    +      _pos += moveTotal;
    +      return moveTotal;
    +    } else {
    +      return 0;
    +    }
    +  }
    +
    +  @Override
    +  public long remaining() {
    +    return size - _pos;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer duplicate() {
    +    ByteBuffer[] duplicates = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      duplicates[i] = underlying[i].duplicate();
    +    }
    +    return new WrappedLargeByteBuffer(duplicates, _pos);
    +  }
    +
    +  @Override
    +  public long size() {
    +    return size;
    +  }
    +
    +  @Override
    +  public long writeTo(WritableByteChannel channel) throws IOException {
    +    long written = 0l;
    +    for (; currentBufferIdx < underlying.length; currentBufferIdx++) {
    +      currentBuffer = underlying[currentBufferIdx];
    +      written += currentBuffer.remaining();
    +      while (currentBuffer.hasRemaining())
    +        channel.write(currentBuffer);
    +    }
    +    _pos = size();
    +    return written;
    +  }
    +
    +  @Override
    +  public ByteBuffer asByteBuffer() throws BufferTooLargeException {
    +    if (underlying.length > 1) {
    +      throw new BufferTooLargeException(size());
    +    }
    +    return underlying[0];
    +  }
    +
    +  // only needed for tests
    --- End diff --
    
    `com.google.common.annotations.VisibleForTesting`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27924329
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.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.buffer;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.List;
    +
    +public interface LargeByteBuffer {
    +    public byte get();
    +
    +    /**
    +     * Bulk copy data from this buffer into the given array.  First checks there is sufficient
    +     * data in this buffer; if not, throws a {@link java.nio.BufferUnderflowException}.
    +     * @param dst
    +     * @param offset
    +     * @param length
    +     */
    +    public void get(byte[] dst,int offset, int length);
    +
    +    public LargeByteBuffer rewind();
    +
    +    /**
    +     * return a deep copy of this buffer.
    +     * The returned buffer will have position == 0.  The position
    +     * of this buffer will not change as a result of copying.
    +     *
    +     * @return a new buffer with a full copy of this buffer's data
    +     */
    +    public LargeByteBuffer deepCopy();
    +
    +    /**
    +     * Advance the position in this buffer by up to <code>n</code> bytes.  <code>n</code> may be
    +     * positive or negative.  It will move the full <code>n</code> unless that moves
    +     * it past the end (or beginning) of the buffer, in which case it will move to the end
    +     * (or beginning).
    +     *
    +     * @return the number of bytes moved forward (can be negative if <code>n</code> is negative)
    +     */
    +    public long skip(long n);
    +
    +    public long position();
    +
    +    /**
    +     * Creates a new byte buffer that shares this buffer's content.
    +     *
    +     * <p> The content of the new buffer will be that of this buffer.  Changes
    +     * to this buffer's content will be visible in the new buffer, and vice
    +     * versa; the two buffers' positions will be independent.
    +     *
    +     * <p> The new buffer's position will be identical to those of this buffer
    +     * */
    +    public LargeByteBuffer duplicate();
    +
    +
    +    public long remaining();
    +
    +    /**
    +     * the total number of bytes in this buffer
    +     * @return
    +     */
    +    public long size();
    +
    +    /**
    +     * writes the data from the current <code>position()</code> to the end of this buffer
    +     * to the given channel.  The <code>position()</code> will be moved to the end of
    +     * the buffer after this.
    +     *
    +     * Note that this method will continually attempt to push data to the given channel.  If the
    +     * channel cannot accept more data, this will continuously retry until the channel accepts
    +     * the data.
    +     *
    +     * @param channel
    +     * @return the number of bytes written to the channel
    +     * @throws IOException
    +     */
    +    public long writeTo(WritableByteChannel channel) throws IOException;
    +
    +    /**
    +     * get the entire contents of this as one ByteBuffer, if possible.  The returned ByteBuffer
    --- End diff --
    
    nit: Get


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28723993
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,269 @@
    +/*
    +* 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 com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    +   * each sub-ByteBuffer (except for the last one) must be exactly this size.  Note that this
    +   * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK.  The only reason it isn't
    +   * is so that we can do tests without creating ginormous buffers.  Public methods force it to
    +   * be LargeByteBufferHelper.MAX_CHUNK
    +   */
    +  private final int subBufferSize;
    +  private long _pos;
    +  @VisibleForTesting
    +  int currentBufferIdx;
    +  @VisibleForTesting
    +  ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, LargeByteBufferHelper.MAX_CHUNK);
    +  }
    +
    +  /**
    +   * you do **not** want to call this version.  It leads to a buffer which doesn't properly
    +   * support {@link #asByteBuffer}.  The only reason it exists is to we can have tests which
    +   * don't require 2GB of memory
    +   *
    +   * @param underlying
    +   * @param subBufferSize
    +   */
    +  @VisibleForTesting
    +  WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) {
    +    if (underlying.length == 0) {
    +      throw new IllegalArgumentException("must wrap at least one ByteBuffer");
    +    }
    +    this.underlying = underlying;
    +    this.subBufferSize = subBufferSize;
    +    long sum = 0L;
    +    boolean startFound = false;
    +    long initialPosition = -1;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      if (i != underlying.length -1 && b.capacity() != subBufferSize) {
    +        throw new IllegalArgumentException("All buffers, except for the final one, must have " +
    +          "size = " + subBufferSize);
    +      }
    +      if (startFound) {
    +        if (b.position() != 0) {
    +          throw new IllegalArgumentException("ByteBuffers have inconsistent positions");
    +        }
    +      } else if (b.position() != b.capacity()) {
    +        startFound = true;
    +        initialPosition = sum + b.position();
    +      }
    +      sum += b.capacity();
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining()) {
    +      throw new BufferUnderflowException();
    +    }
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    if (currentBuffer != null) {
    +      currentBuffer.rewind();
    +    }
    +    while (currentBufferIdx > 0) {
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +      currentBuffer.rewind();
    +    }
    +    _pos = 0;
    +    return this;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer deepCopy() {
    +    ByteBuffer[] dataCopy = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      dataCopy[i] = ByteBuffer.allocate(b.capacity());
    +      int originalPosition = b.position();
    +      b.rewind();
    +      dataCopy[i].put(b);
    +      dataCopy[i].position(0);
    +      b.position(originalPosition);
    +    }
    +    return new WrappedLargeByteBuffer(dataCopy, subBufferSize);
    +  }
    +
    +  @Override
    +  public byte get() {
    +    byte r = currentBuffer.get();
    +    _pos += 1;
    +    updateCurrentBuffer();
    +    return r;
    +  }
    +
    +  private void updateCurrentBuffer() {
    +    while (currentBuffer != null && !currentBuffer.hasRemaining()) {
    +      currentBufferIdx += 1;
    +      currentBuffer = currentBufferIdx < underlying.length ? underlying[currentBufferIdx] : null;
    +    }
    +  }
    +
    +  @Override
    +  public long position() {
    +    return _pos;
    +  }
    +
    +  @Override
    +  public long skip(long n) {
    +    if (n < 0) {
    +      final long moveTotal = Math.min(-n, _pos);
    +      long toMove = moveTotal;
    +      // move backwards -- set the position to 0 of every buffer's we go back
    --- End diff --
    
    Comment is inaccurate.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31545220
  
    --- Diff: core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala ---
    @@ -91,4 +95,45 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream {
           ret
         }
       }
    +
    +  /**
    +   * Get a copy of the data between the two endpoints, start <= idx < until.  Always returns
    +   * an array of size (until - start).  Throws an IllegalArgumentException if
    +   * 0 <= start <= until <= size
    +   */
    +  def slice(start: Long, until: Long): Array[Byte] = {
    +    require((until - start) < Integer.MAX_VALUE, "max slice length = Integer.MAX_VALUE")
    +    require(start >= 0 && start <= until, s"start ($start) must be >= 0 and <= until ($until)")
    +    require(until >= start && until <= size,
    +      s"until ($until) must be >= start ($start) and <= size ($size)")
    +    var chunkStart = 0L
    +    var chunkIdx = 0
    +    val length = (until - start).toInt
    +    var foundStart = false
    +    val result = new Array[Byte](length)
    +    while (!foundStart) {
    +      val nextSize = chunkStart + chunks(chunkIdx).size
    --- End diff --
    
    Can you add a regression test for slice while fixing this?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27923504
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala ---
    @@ -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.util
    +
    +import java.io.OutputStream
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.network.buffer.{LargeByteBufferHelper, WrappedLargeByteBuffer, LargeByteBuffer}
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream
    +
    +private[spark]
    +class LargeByteBufferOutputStream(chunkSize: Int = 65536)
    +  extends OutputStream {
    +
    +  private[util] val output = new ByteArrayChunkOutputStream(chunkSize)
    +
    +  private var _pos = 0
    +
    +  override def write(b: Int): Unit = {
    +    output.write(b)
    +  }
    +
    +  override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = {
    +    output.write(bytes, offs, len)
    +    _pos += len
    +  }
    +
    +  def largeBuffer: LargeByteBuffer = {
    +    largeBuffer(LargeByteBufferHelper.MAX_CHUNK)
    +  }
    +
    +  // exposed for testing
    +  private[util] def largeBuffer(maxChunk: Int): LargeByteBuffer = {
    +    // LargeByteBuffer is supposed to make a "best effort" to get all the data
    +    // in one nio.ByteBuffer, so we want to try to merge the smaller chunks together
    +    // as much as possible.  This is necessary b/c there are a number of parts of spark that
    +    // can only deal w/ one nio.ByteBuffer, and can't use a LargeByteBuffer yet.
    +    val totalSize = output.size
    +    val chunksNeeded = ((totalSize + maxChunk -1) / maxChunk).toInt
    +    val chunks = new Array[Array[Byte]](chunksNeeded)
    +    var remaining = totalSize
    +    var pos = 0
    +    (0 until chunksNeeded).foreach{idx =>
    +      val nextSize = math.min(maxChunk, remaining).toInt
    +      chunks(idx) = output.slice(pos, pos + nextSize)
    +      pos += nextSize
    +      remaining -= nextSize
    +    }
    +    new WrappedLargeByteBuffer(chunks.map{ByteBuffer.wrap})
    --- End diff --
    
    nit: `.map(ByteBuffer.wrap)`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91274550
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29942/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107641357
  
    Merged build started.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107701975
  
      [Test build #33911 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33911/consoleFull) for   PR 5400 at commit [`b77bbe2`](https://github.com/apache/spark/commit/b77bbe28e109db72775a592e99572c039b377517).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class LargeByteBufferInputStream extends InputStream `
      * `public class LargeByteBufferOutputStream extends OutputStream `
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27923226
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala ---
    @@ -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.util
    +
    +import java.io.OutputStream
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.network.buffer.{LargeByteBufferHelper, WrappedLargeByteBuffer, LargeByteBuffer}
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream
    +
    +private[spark]
    +class LargeByteBufferOutputStream(chunkSize: Int = 65536)
    +  extends OutputStream {
    +
    +  private[util] val output = new ByteArrayChunkOutputStream(chunkSize)
    +
    +  private var _pos = 0
    +
    +  override def write(b: Int): Unit = {
    +    output.write(b)
    --- End diff --
    
    Don't you need to update `_pos`?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107642539
  
     Merged build triggered.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153817353
  
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107640380
  
      [Test build #33903 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33903/consoleFull) for   PR 5400 at commit [`3c599b2`](https://github.com/apache/spark/commit/3c599b2f01e0e4d1ac2a9d02cf6a8e8924da488d).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class LargeByteBufferInputStream extends InputStream `
      * `public class LargeByteBufferOutputStream extends OutputStream `
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91379947
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29972/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31545767
  
    --- Diff: network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java ---
    @@ -0,0 +1,84 @@
    +/*
    + * 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.*;
    +import java.nio.channels.FileChannel;
    +
    +import org.junit.Test;
    +
    +import static org.junit.Assert.*;
    +
    +public class LargeByteBufferHelperSuite {
    +
    +  @Test
    +  public void testMapFile() throws IOException {
    +    File testFile = File.createTempFile("large-byte-buffer-test", ".bin");
    +    try {
    +      testFile.deleteOnExit();
    +      OutputStream out = new FileOutputStream(testFile);
    +      byte[] buffer = new byte[1 << 16];
    +      long len = ((long)buffer.length) + Integer.MAX_VALUE + 1;
    +      for (int i = 0; i < buffer.length; i++) {
    +        buffer[i] = (byte) i;
    +      }
    +      for (int i = 0; i < len / buffer.length; i++) {
    +        out.write(buffer);
    --- End diff --
    
    I'm a little concerned that writing the same data over and over might mask some nasty issues like the one I think I found earlier. Perhaps this should be writing random data instead? (That would probably be prohibitively slow with such a large test file, though.)
    
    You could have two `java.util.Random` instances with the same seed to be able to recreate the input data while checking.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31543772
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * 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.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +/**
    + * Utils for creating {@link org.apache.spark.network.buffer.LargeByteBuffer}s, either from
    + * pre-allocated byte arrays, ByteBuffers, or by memory mapping a file.
    + */
    +public class LargeByteBufferHelper {
    +
    +  // netty can't quite send msgs that are a full 2GB -- they need to be slightly smaller
    +  // not sure what the exact limit is, but 200 seems OK.
    +  /**
    +   * The maximum size of any ByteBuffer.
    +   * {@link org.apache.spark.network.buffer.LargeByteBuffer#asByteBuffer} will never return a
    +   * ByteBuffer larger than this.  This is close to the max ByteBuffer size (2GB), minus a small
    +   * amount for message overhead.
    +   */
    +  public static final int MAX_CHUNK_SIZE = Integer.MAX_VALUE - 200;
    +
    +  public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { buffer } );
    +  }
    +
    +  public static LargeByteBuffer asLargeByteBuffer(byte[] bytes) {
    +    return asLargeByteBuffer(ByteBuffer.wrap(bytes));
    +  }
    +
    +  public static LargeByteBuffer allocate(long size) {
    +    return allocate(size, MAX_CHUNK_SIZE);
    +  }
    +
    +  @VisibleForTesting
    +  static LargeByteBuffer allocate(long size, int maxChunk) {
    +    int chunksNeeded = (int) ((size + maxChunk - 1) / maxChunk);
    +    ByteBuffer[] chunks = new ByteBuffer[chunksNeeded];
    +    long remaining = size;
    +    for (int i = 0; i < chunksNeeded; i++) {
    +      int nextSize = (int) Math.min(remaining, maxChunk);
    +      ByteBuffer next = ByteBuffer.allocate(nextSize);
    +      remaining -= nextSize;
    +      chunks[i] = next;
    +    }
    +    if (remaining != 0) {
    +      throw new IllegalStateException("remaining = " + remaining);
    +    }
    +    return new WrappedLargeByteBuffer(chunks, maxChunk);
    +  }
    +
    +
    +  public static LargeByteBuffer mapFile(
    +      FileChannel channel,
    +      FileChannel.MapMode mode,
    +      long offset,
    +      long length
    +  ) throws IOException {
    +    int chunksNeeded = (int) ((length  - 1) / MAX_CHUNK_SIZE) + 1;
    +    ByteBuffer[] chunks = new ByteBuffer[chunksNeeded];
    +    long curPos = offset;
    +    long end = offset + length;
    +    for (int i = 0; i < chunksNeeded; i++) {
    +      long nextPos = Math.min(curPos + MAX_CHUNK_SIZE, end);
    +      chunks[i] = channel.map(mode, curPos, nextPos - curPos);
    +      curPos = nextPos;
    +    }
    +    return new WrappedLargeByteBuffer(chunks);
    +  }
    +
    --- End diff --
    
    super nit: too many empty lines (also in other places).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-93052056
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30272/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107641501
  
      [Test build #33904 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33904/consoleFull) for   PR 5400 at commit [`95588c2`](https://github.com/apache/spark/commit/95588c25bb40a7971ea05d6b261a1a044548feb9).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153818464
  
    Merged build started.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27924456
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java ---
    @@ -0,0 +1,73 @@
    +/*
    + * 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.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +    public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    +
    +    public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +        return new WrappedLargeByteBuffer(new ByteBuffer[]{buffer});
    --- End diff --
    
    another nit: `new ByteBuffer[] { buffer }`, also elsewhere


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28723311
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,269 @@
    +/*
    +* 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 com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  public final ByteBuffer[] underlying;
    --- End diff --
    
    Tests are in the same package, so this could be package-private 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-90715038
  
      [Test build #29808 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29808/consoleFull) for   PR 5400 at commit [`9f53203`](https://github.com/apache/spark/commit/9f53203070daa17ddfff5ad1f5b2e94f86614554).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28722451
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.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.buffer;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +  public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    --- End diff --
    
    Is there a reason for `1000000`? Comment?
    
    Also, should this be `MAX_CHUNK_SIZE`?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107642568
  
    Merged build started.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91269314
  
      [Test build #29942 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29942/consoleFull) for   PR 5400 at commit [`a759242`](https://github.com/apache/spark/commit/a75924234ea3523ae3409a384dfef2dbbfbc9d82).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-169909026
  
    @scwf , 
    
    1. the shuffle output from one mapper to one reducer cannot be more than 2GB.
    2. partitions of an RDD cannot exceed 2GB.



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153810827
  
    Thanks for taking another look @vanzin, fixed those last issues.  Sorry I never responded earlier @tgravescs  -- that one test takes about 10s on my laptop, looks like it took 15s on the last jenkins run.  Personally I feel better w/ the test in there.  But I agree its not adding a ton of value -- happy to scrap it if you prefer.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-169517123
  
    hi @squito, can you explain in which situation users will hit the 2g limit? will a job of processing very large data(such as PB level data) reach this limit?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-169909353
  
    @snnn 2 is not true. Partitions can be as large as possible. The cached size cannot be greater than 2GB.



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107801304
  
    I think I've addressed the comments.  I also cleaned up a couple of bits that were more complicated than needed and added a few more tests. @tgravescs @vanzin @JoshRosen


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-91274692
  
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28723079
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.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.buffer;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +  public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    +
    +  public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { buffer } );
    +  }
    +
    +  public static LargeByteBuffer asLargeByteBuffer(byte[] bytes) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { ByteBuffer.wrap(bytes) } );
    +  }
    +
    +  public static LargeByteBuffer allocate(long size) {
    +    return allocate(size, MAX_CHUNK);
    +  }
    +
    +  @VisibleForTesting
    +  static LargeByteBuffer allocate(long size, int maxChunk) {
    +    int chunksNeeded = (int) ((size + maxChunk - 1) / maxChunk);
    +    ByteBuffer[] chunks = new ByteBuffer[chunksNeeded];
    +    long remaining = size;
    +    for (int i = 0; i < chunksNeeded; i++) {
    +      int nextSize = (int) Math.min(remaining, maxChunk);
    +      ByteBuffer next = ByteBuffer.allocate(nextSize);
    +      remaining -= nextSize;
    +      chunks[i] = next;
    +    }
    +    if (remaining != 0) throw new IllegalStateException("remaining = " + remaining);
    +    return new WrappedLargeByteBuffer(chunks, maxChunk);
    +  }
    +
    +
    +  public static LargeByteBuffer mapFile(
    +    FileChannel channel,
    +    FileChannel.MapMode mode,
    +    long offset,
    +    long length
    +  ) throws IOException {
    +    int maxChunk = MAX_CHUNK;
    +    ArrayList<Long> offsets = new ArrayList<Long>();
    +    long curOffset = offset;
    +    long end = offset + length;
    +    while (curOffset < end) {
    +      offsets.add(curOffset);
    +      int chunkLength = (int) Math.min((end - curOffset), maxChunk);
    +      curOffset += chunkLength;
    +    }
    +    offsets.add(end);
    +    ByteBuffer[] chunks = new ByteBuffer[offsets.size() - 1];
    +    for (int i = 0; i < offsets.size() - 1; i++) {
    --- End diff --
    
    In fact, it seems like it would be possible to do all this in a single loop and avoide the `offsets` list altogether?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153852138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45032/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27922983
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * 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.util
    +
    +import java.io.InputStream
    +
    +import org.apache.spark.network.buffer.LargeByteBuffer
    +import org.apache.spark.storage.BlockManager
    +
    +/**
    + * Reads data from a LargeByteBuffer, and optionally cleans it up using buffer.dispose()
    + * at the end of the stream (e.g. to close a memory-mapped file).
    + */
    +private[spark]
    +class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: Boolean = false)
    +  extends InputStream {
    +
    +  override def read(): Int = {
    +    if (buffer == null || buffer.remaining() == 0) {
    +      cleanUp()
    +      -1
    +    } else {
    +      val r = buffer.get() & 0xFF
    +      if (buffer.remaining() == 0) {
    +        cleanUp()
    --- End diff --
    
    (In fact you do need `close()` in case the stream is closed before EOF is reached.)


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-93006655
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30251/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28723671
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,269 @@
    +/*
    +* 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 com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    +   * each sub-ByteBuffer (except for the last one) must be exactly this size.  Note that this
    +   * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK.  The only reason it isn't
    +   * is so that we can do tests without creating ginormous buffers.  Public methods force it to
    +   * be LargeByteBufferHelper.MAX_CHUNK
    +   */
    +  private final int subBufferSize;
    +  private long _pos;
    +  @VisibleForTesting
    +  int currentBufferIdx;
    +  @VisibleForTesting
    +  ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, LargeByteBufferHelper.MAX_CHUNK);
    +  }
    +
    +  /**
    +   * you do **not** want to call this version.  It leads to a buffer which doesn't properly
    +   * support {@link #asByteBuffer}.  The only reason it exists is to we can have tests which
    +   * don't require 2GB of memory
    +   *
    +   * @param underlying
    +   * @param subBufferSize
    +   */
    +  @VisibleForTesting
    +  WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) {
    +    if (underlying.length == 0) {
    +      throw new IllegalArgumentException("must wrap at least one ByteBuffer");
    +    }
    +    this.underlying = underlying;
    +    this.subBufferSize = subBufferSize;
    +    long sum = 0L;
    +    boolean startFound = false;
    +    long initialPosition = -1;
    --- End diff --
    
    Can you add a comment explaining what all this code is doing?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r43674704
  
    --- Diff: network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java ---
    @@ -0,0 +1,110 @@
    +/*
    + * 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.*;
    +import java.nio.channels.FileChannel;
    +import java.util.Random;
    +
    +import org.junit.Test;
    +
    +import static org.junit.Assert.*;
    +
    +public class LargeByteBufferHelperSuite {
    +
    +  @Test
    +  public void testMapFile() throws IOException {
    +    File testFile = File.createTempFile("large-byte-buffer-test", ".bin");
    +    try {
    +      testFile.deleteOnExit();
    +      OutputStream out = new FileOutputStream(testFile);
    +      byte[] buffer = new byte[1 << 16];
    +      Random rng = new XORShiftRandom(0L);
    +      long len = ((long)buffer.length) + Integer.MAX_VALUE + 1;
    +      for (int i = 0; i < len / buffer.length; i++) {
    +        rng.nextBytes(buffer);
    +        out.write(buffer);
    +      }
    +      out.close();
    +
    +      FileChannel in = new FileInputStream(testFile).getChannel();
    +
    +      //fail quickly on bad bounds
    +      try {
    +        LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, 0, len + 1);
    +        fail("expected exception");
    +      } catch (IOException ioe) {
    +      }
    +      try {
    +        LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, -1, 10);
    +        fail("expected exception");
    +      } catch (IllegalArgumentException iae) {
    +      }
    +
    +      //now try to read from the buffer
    +      LargeByteBuffer buf = LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, 0, len);
    +      assertEquals(len, buf.size());
    +      byte[] read = new byte[buffer.length];
    +      byte[] expected = new byte[buffer.length];
    +      Random rngExpected = new XORShiftRandom(0L);
    +      for (int i = 0; i < len / buffer.length; i++) {
    +        buf.get(read, 0, buffer.length);
    +        // assertArrayEquals() is really slow
    +        rngExpected.nextBytes(expected);
    +        for (int j = 0; j < buffer.length; j++) {
    +          if (read[j] !=  expected[j])
    +            fail("bad byte at (i,j) = (" + i + "," + j + ")");
    +        }
    +      }
    +    } finally {
    +      testFile.delete();
    +    }
    +  }
    +
    +  @Test
    +  public void testAllocate() {
    +    WrappedLargeByteBuffer buf = (WrappedLargeByteBuffer) LargeByteBufferHelper.allocate(95,10);
    --- End diff --
    
    nit: `95, 10`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28012080
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,252 @@
    +/*
    +* 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 sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  //only public for tests
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  private long _pos;
    +  private int currentBufferIdx;
    +  private ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, findExpectedInitialPosition(underlying));
    +  }
    +
    +  private static long findExpectedInitialPosition(ByteBuffer[] bufs) {
    +    long sum = 0L;
    +    for (ByteBuffer b: bufs) {
    +      if (b.position() > 0) {
    +        // this could still lead to a mix of positions half-way through buffers that
    +        // would be inconsistent -- but we'll discover that in the constructor checks
    +        sum += b.position();
    +      } else {
    +        break;
    +      }
    +    }
    +    return sum;
    +  }
    +
    +  private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) {
    +    this.underlying = underlying;
    +    long sum = 0L;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      long nextSum = sum + b.capacity();
    +      int expectedPosition;
    +      if (nextSum < initialPosition) {
    +        expectedPosition = b.capacity();
    +      } else if (sum > initialPosition) {
    +        expectedPosition = 0;
    +      } else {
    +        expectedPosition = (int) (initialPosition - sum);
    +      }
    +      if (b.position() != expectedPosition) {
    +        throw new IllegalArgumentException("ByteBuffer[" + i + "]:" + b + " was expected to have" +
    +          " position = " + expectedPosition + " to be consistent with the overall " +
    +          "initialPosition = " + initialPosition);
    +      }
    +      sum = nextSum;
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining())
    +      throw new BufferUnderflowException();
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    while (currentBufferIdx > 0) {
    +      if (currentBuffer != null) {
    +        currentBuffer.rewind();
    +      }
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +    }
    +    currentBuffer.rewind();
    +    _pos = 0;
    +    return this;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer deepCopy() {
    +    ByteBuffer[] dataCopy = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      dataCopy[i] = ByteBuffer.allocate(b.capacity());
    +      int originalPosition = b.position();
    +      b.position(0);
    +      dataCopy[i].put(b);
    --- End diff --
    
    I set the position of the buffer to 0 just above this: `b.position(0)` -- that's just a rewind without clearing the `mark`, but I can change to `b.rewind()`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27923456
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala ---
    @@ -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.util
    +
    +import java.io.OutputStream
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.network.buffer.{LargeByteBufferHelper, WrappedLargeByteBuffer, LargeByteBuffer}
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream
    +
    +private[spark]
    +class LargeByteBufferOutputStream(chunkSize: Int = 65536)
    +  extends OutputStream {
    +
    +  private[util] val output = new ByteArrayChunkOutputStream(chunkSize)
    +
    +  private var _pos = 0
    +
    +  override def write(b: Int): Unit = {
    +    output.write(b)
    +  }
    +
    +  override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = {
    +    output.write(bytes, offs, len)
    +    _pos += len
    +  }
    +
    +  def largeBuffer: LargeByteBuffer = {
    +    largeBuffer(LargeByteBufferHelper.MAX_CHUNK)
    +  }
    +
    +  // exposed for testing
    +  private[util] def largeBuffer(maxChunk: Int): LargeByteBuffer = {
    +    // LargeByteBuffer is supposed to make a "best effort" to get all the data
    +    // in one nio.ByteBuffer, so we want to try to merge the smaller chunks together
    +    // as much as possible.  This is necessary b/c there are a number of parts of spark that
    +    // can only deal w/ one nio.ByteBuffer, and can't use a LargeByteBuffer yet.
    +    val totalSize = output.size
    +    val chunksNeeded = ((totalSize + maxChunk -1) / maxChunk).toInt
    +    val chunks = new Array[Array[Byte]](chunksNeeded)
    +    var remaining = totalSize
    +    var pos = 0
    +    (0 until chunksNeeded).foreach{idx =>
    --- End diff --
    
    nit: spaces around `{`


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-90737829
  
      [Test build #29808 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29808/consoleFull) for   PR 5400 at commit [`9f53203`](https://github.com/apache/spark/commit/9f53203070daa17ddfff5ad1f5b2e94f86614554).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class LargeByteBufferOutputStream(chunkSize: Int = 65536)`
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `
    
     * This patch does not change any dependencies.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107671379
  
      [Test build #33911 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33911/consoleFull) for   PR 5400 at commit [`b77bbe2`](https://github.com/apache/spark/commit/b77bbe28e109db72775a592e99572c039b377517).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28717954
  
    --- Diff: core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.{FileInputStream, FileOutputStream, OutputStream, File}
    +import java.nio.ByteBuffer
    +import java.nio.channels.FileChannel.MapMode
    +
    +import org.junit.Assert._
    +import org.scalatest.{FunSuite, Matchers}
    +
    +import org.apache.spark.network.buffer.{LargeByteBuffer, LargeByteBufferHelper, WrappedLargeByteBuffer}
    +
    +class LargeByteBufferInputStreamSuite extends FunSuite with Matchers {
    +
    +  test("read from large mapped file") {
    +    val testFile = File.createTempFile("large-buffer-input-stream-test",".bin")
    +    testFile.deleteOnExit()
    --- End diff --
    
    IIUC, this test is creating a humongous test file (> 2GB)... two things:
    
    - how long does it take for the test to run?
    - this file should probably be explicitly deleted in a `finally` block in this test (or using `BeforeAndAfter`), instead of relying on `deleteOnExit`. Just because I'd be wary of leaking 2GB files.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107641341
  
     Merged build triggered.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107664572
  
    Merged build finished. 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31544111
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,280 @@
    +/*
    +* 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.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +/**
    + * A {@link org.apache.spark.network.buffer.LargeByteBuffer} which may contain multiple
    + * {@link java.nio.ByteBuffer}s.  In order to support <code>asByteBuffer</code>, all
    + * of the underlying ByteBuffers must have size equal to
    + * {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE} (except that last
    + * one).  The underlying ByteBuffers may be on-heap, direct, or memory-mapped.
    + */
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    +   * each sub-ByteBuffer (except for the last one) must be exactly this size.  Note that this
    +   * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK_SIZE.  The only reason it isn't
    +   * is so that we can do tests without creating ginormous buffers.  Public methods force it to
    +   * be LargeByteBufferHelper.MAX_CHUNK_SIZE
    +   */
    +  private final int subBufferSize;
    +  private long _pos;
    +  @VisibleForTesting
    +  int currentBufferIdx;
    +  @VisibleForTesting
    +  ByteBuffer currentBuffer;
    +
    +  /**
    +   * Construct a WrappedLargeByteBuffer from the given ByteBuffers.  Each of the ByteBuffers must
    +   * have size equal to {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE}
    +   * except for the final one.  The buffers are <code>duplicate</code>d, so the position of the
    +   * given buffers and the returned buffer will be independent, though the underlying data will be
    +   * shared.  The constructed buffer will always have position == 0.
    +   */
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, LargeByteBufferHelper.MAX_CHUNK_SIZE);
    +  }
    +
    +  /**
    +   * you do **not** want to call this version.  It leads to a buffer which doesn't properly
    +   * support {@link #asByteBuffer}.  The only reason it exists is to we can have tests which
    +   * don't require 2GB of memory
    +   *
    +   * @param underlying
    +   * @param subBufferSize
    +   */
    +  @VisibleForTesting
    +  WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) {
    --- End diff --
    
    It seems like the only use of `subBufferSize` is to throw an exception in this constructor; it doesn't change any logic in the code. In light of that, is it necessary?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r43674126
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java ---
    @@ -0,0 +1,148 @@
    +/*
    +* Licensed to the Apache Software Foundation (ASF) under one or more
    +* contributor license agreements.  See the NOTICE file distributed with
    +* this work for additional information regarding copyright ownership.
    +* The ASF licenses this file to You under the Apache License, Version 2.0
    +* (the "License"); you may not use this file except in compliance with
    +* the License.  You may obtain a copy of the License at
    +*
    +*    http://www.apache.org/licenses/LICENSE-2.0
    +*
    +* Unless required by applicable law or agreed to in writing, software
    +* distributed under the License is distributed on an "AS IS" BASIS,
    +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +* See the License for the specific language governing permissions and
    +* limitations under the License.
    +*/
    +package org.apache.spark.network.buffer;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +
    +/**
    + * A byte buffer which can hold over 2GB.
    + * <p/>
    + * This is roughly similar {@link java.nio.ByteBuffer}, with a limited set of operations relevant
    + * to use in Spark, and without the capacity restrictions of a ByteBuffer.
    + * <p/>
    + * Unlike ByteBuffers, this is read-only, and only supports reading bytes (with both single and bulk
    + * <code>get</code> methods).  It supports random access via <code>skip</code> to move around the
    + * buffer.
    + * <p/>
    + * In general, implementations are expected to support O(1) random access.  Furthermore,
    + * neighboring locations in the buffer are likely to be neighboring in memory, so sequential access
    + * will avoid cache-misses.  However, these are only rough guidelines which may differ in
    + * implementations.
    + * <p/>
    + * Any code which expects a ByteBuffer can obtain one via {@link #asByteBuffer} when possible -- see
    + * that method for a full description of its limitations.
    + * <p/>
    + * Instances of this class can be created with
    + * {@link org.apache.spark.network.buffer.LargeByteBufferHelper},
    + * with a LargeByteBufferOutputStream,
    + * or directly from the implementation
    + * {@link org.apache.spark.network.buffer.WrappedLargeByteBuffer}.
    + */
    +public interface LargeByteBuffer {
    +  public byte get();
    +
    --- End diff --
    
    nit: too many empty lines


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31545270
  
    --- Diff: core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala ---
    @@ -91,4 +95,45 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream {
           ret
         }
       }
    +
    +  /**
    +   * Get a copy of the data between the two endpoints, start <= idx < until.  Always returns
    +   * an array of size (until - start).  Throws an IllegalArgumentException if
    +   * 0 <= start <= until <= size
    +   */
    +  def slice(start: Long, until: Long): Array[Byte] = {
    +    require((until - start) < Integer.MAX_VALUE, "max slice length = Integer.MAX_VALUE")
    +    require(start >= 0 && start <= until, s"start ($start) must be >= 0 and <= until ($until)")
    +    require(until >= start && until <= size,
    +      s"until ($until) must be >= start ($start) and <= size ($size)")
    +    var chunkStart = 0L
    +    var chunkIdx = 0
    +    val length = (until - start).toInt
    +    var foundStart = false
    +    val result = new Array[Byte](length)
    +    while (!foundStart) {
    +      val nextSize = chunkStart + chunks(chunkIdx).size
    --- End diff --
    
    e.g. a test that fails and demonstrates the issue that Marcelo points out 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28724198
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,269 @@
    +/*
    +* 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 com.google.common.annotations.VisibleForTesting;
    +import sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  @VisibleForTesting
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  /**
    +   * each sub-ByteBuffer (except for the last one) must be exactly this size.  Note that this
    +   * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK.  The only reason it isn't
    +   * is so that we can do tests without creating ginormous buffers.  Public methods force it to
    +   * be LargeByteBufferHelper.MAX_CHUNK
    +   */
    +  private final int subBufferSize;
    +  private long _pos;
    +  @VisibleForTesting
    +  int currentBufferIdx;
    +  @VisibleForTesting
    +  ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, LargeByteBufferHelper.MAX_CHUNK);
    +  }
    +
    +  /**
    +   * you do **not** want to call this version.  It leads to a buffer which doesn't properly
    +   * support {@link #asByteBuffer}.  The only reason it exists is to we can have tests which
    +   * don't require 2GB of memory
    +   *
    +   * @param underlying
    +   * @param subBufferSize
    +   */
    +  @VisibleForTesting
    +  WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) {
    +    if (underlying.length == 0) {
    +      throw new IllegalArgumentException("must wrap at least one ByteBuffer");
    +    }
    +    this.underlying = underlying;
    +    this.subBufferSize = subBufferSize;
    +    long sum = 0L;
    +    boolean startFound = false;
    +    long initialPosition = -1;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      if (i != underlying.length -1 && b.capacity() != subBufferSize) {
    +        throw new IllegalArgumentException("All buffers, except for the final one, must have " +
    +          "size = " + subBufferSize);
    +      }
    +      if (startFound) {
    +        if (b.position() != 0) {
    +          throw new IllegalArgumentException("ByteBuffers have inconsistent positions");
    +        }
    +      } else if (b.position() != b.capacity()) {
    +        startFound = true;
    +        initialPosition = sum + b.position();
    +      }
    +      sum += b.capacity();
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining()) {
    +      throw new BufferUnderflowException();
    +    }
    +    int moved = 0;
    +    while (moved < length) {
    +      int toRead = Math.min(length - moved, currentBuffer.remaining());
    +      currentBuffer.get(dest, offset + moved, toRead);
    +      moved += toRead;
    +      updateCurrentBuffer();
    +    }
    +    _pos += moved;
    +  }
    +
    +  @Override
    +  public LargeByteBuffer rewind() {
    +    if (currentBuffer != null) {
    +      currentBuffer.rewind();
    +    }
    +    while (currentBufferIdx > 0) {
    +      currentBufferIdx -= 1;
    +      currentBuffer = underlying[currentBufferIdx];
    +      currentBuffer.rewind();
    +    }
    +    _pos = 0;
    +    return this;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer deepCopy() {
    +    ByteBuffer[] dataCopy = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      dataCopy[i] = ByteBuffer.allocate(b.capacity());
    +      int originalPosition = b.position();
    +      b.rewind();
    +      dataCopy[i].put(b);
    +      dataCopy[i].position(0);
    +      b.position(originalPosition);
    +    }
    +    return new WrappedLargeByteBuffer(dataCopy, subBufferSize);
    +  }
    +
    +  @Override
    +  public byte get() {
    +    byte r = currentBuffer.get();
    +    _pos += 1;
    +    updateCurrentBuffer();
    +    return r;
    +  }
    +
    +  private void updateCurrentBuffer() {
    +    while (currentBuffer != null && !currentBuffer.hasRemaining()) {
    +      currentBufferIdx += 1;
    +      currentBuffer = currentBufferIdx < underlying.length ? underlying[currentBufferIdx] : null;
    +    }
    +  }
    +
    +  @Override
    +  public long position() {
    +    return _pos;
    +  }
    +
    +  @Override
    +  public long skip(long n) {
    +    if (n < 0) {
    +      final long moveTotal = Math.min(-n, _pos);
    +      long toMove = moveTotal;
    +      // move backwards -- set the position to 0 of every buffer's we go back
    +      if (currentBuffer != null) {
    +        currentBufferIdx += 1;
    +      }
    +      while (toMove > 0) {
    +        currentBufferIdx -= 1;
    +        currentBuffer = underlying[currentBufferIdx];
    +        int thisMove = (int) Math.min(toMove, currentBuffer.position());
    +        currentBuffer.position(currentBuffer.position() - thisMove);
    +        toMove -= thisMove;
    +      }
    +      _pos -= moveTotal;
    +      return -moveTotal;
    +    } else if (n > 0) {
    +      final long moveTotal = Math.min(n, remaining());
    +      long toMove = moveTotal;
    +      // move forwards -- set the position to the end of every buffer as we go forwards
    +      currentBufferIdx -= 1;
    +      while (toMove > 0) {
    +        currentBufferIdx += 1;
    +        currentBuffer = underlying[currentBufferIdx];
    +        int thisMove = (int) Math.min(toMove, currentBuffer.remaining());
    +        currentBuffer.position(currentBuffer.position() + thisMove);
    +        toMove -= thisMove;
    +      }
    +      _pos += moveTotal;
    +      return moveTotal;
    +    } else {
    +      return 0;
    +    }
    +  }
    +
    +  @Override
    +  public long remaining() {
    +    return size - _pos;
    +  }
    +
    +  @Override
    +  public WrappedLargeByteBuffer duplicate() {
    +    ByteBuffer[] duplicates = new ByteBuffer[underlying.length];
    +    for (int i = 0; i < underlying.length; i++) {
    +      duplicates[i] = underlying[i].duplicate();
    +    }
    +    return new WrappedLargeByteBuffer(duplicates, subBufferSize);
    +  }
    +
    +  @Override
    +  public long size() {
    +    return size;
    +  }
    +
    +  @Override
    +  public long writeTo(WritableByteChannel channel) throws IOException {
    +    long written = 0l;
    +    for (; currentBufferIdx < underlying.length; currentBufferIdx++) {
    +      currentBuffer = underlying[currentBufferIdx];
    +      written += currentBuffer.remaining();
    +      while (currentBuffer.hasRemaining())
    +        channel.write(currentBuffer);
    +    }
    +    _pos = size();
    +    return written;
    +  }
    +
    +  @Override
    +  public ByteBuffer asByteBuffer() throws BufferTooLargeException {
    +    if (underlying.length == 1) {
    +      ByteBuffer b = underlying[0].duplicate();
    +      b.rewind();
    +      return b;
    +    } else {
    +      // NOTE: if subBufferSize != LargeByteBufferHelper.MAX_CAPACITY, in theory
    +      // we could copy the data into a new buffer.  But we don't want to do any copying.
    +      // The only reason we allow smaller subBufferSize is so that we can have tests which
    +      // don't require 2GB of memory
    +      throw new BufferTooLargeException(size(), underlying[0].capacity());
    +    }
    +  }
    +
    +  @VisibleForTesting
    +  public List<ByteBuffer> nioBuffers() {
    --- End diff --
    
    package-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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31545490
  
    --- Diff: network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java ---
    @@ -0,0 +1,84 @@
    +/*
    + * 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.*;
    +import java.nio.channels.FileChannel;
    +
    +import org.junit.Test;
    +
    +import static org.junit.Assert.*;
    +
    +public class LargeByteBufferHelperSuite {
    +
    +  @Test
    +  public void testMapFile() throws IOException {
    +    File testFile = File.createTempFile("large-byte-buffer-test", ".bin");
    +    try {
    +      testFile.deleteOnExit();
    +      OutputStream out = new FileOutputStream(testFile);
    +      byte[] buffer = new byte[1 << 16];
    +      long len = ((long)buffer.length) + Integer.MAX_VALUE + 1;
    --- End diff --
    
    Ouch, a test that writes a 2GB file... can we use an approach similar to the memory-based one?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-90747474
  
    I didn't look at the tests in detail; I found some discrepancies between the code and the `LargeByteBuffer` interface that should probably be fixed one way or another (either the interface needs updating, or the code needs fixing).


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27923201
  
    --- Diff: core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala ---
    @@ -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.util
    +
    +import java.io.OutputStream
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.network.buffer.{LargeByteBufferHelper, WrappedLargeByteBuffer, LargeByteBuffer}
    +import org.apache.spark.util.io.ByteArrayChunkOutputStream
    +
    +private[spark]
    +class LargeByteBufferOutputStream(chunkSize: Int = 65536)
    +  extends OutputStream {
    +
    +  private[util] val output = new ByteArrayChunkOutputStream(chunkSize)
    +
    +  private var _pos = 0
    +
    +  override def write(b: Int): Unit = {
    +    output.write(b)
    +  }
    +
    +  override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = {
    +    output.write(bytes, offs, len)
    +    _pos += len
    --- End diff --
    
    Shouldn't you use the return value of `output.write` 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-170037893
  
     >>The cached size cannot be greater than 2GB.
    
    @rxin how to understand the `cached size`? the partition size of a cached rdd?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-92949156
  
      [Test build #30249 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30249/consoleFull) for   PR 5400 at commit [`e1d8fa8`](https://github.com/apache/spark/commit/e1d8fa80012cc576e636fbcb3ad8470940111b07).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class LargeByteBufferOutputStream(chunkSize: Int = 65536)`
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `
    
     * This patch **adds the following new dependencies:**
       * `commons-math3-3.4.1.jar`
    
     * This patch **removes the following dependencies:**
       * `commons-math3-3.1.1.jar`



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27924318
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.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.buffer;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.List;
    +
    +public interface LargeByteBuffer {
    +    public byte get();
    +
    +    /**
    +     * Bulk copy data from this buffer into the given array.  First checks there is sufficient
    +     * data in this buffer; if not, throws a {@link java.nio.BufferUnderflowException}.
    +     * @param dst
    +     * @param offset
    +     * @param length
    +     */
    +    public void get(byte[] dst,int offset, int length);
    +
    +    public LargeByteBuffer rewind();
    +
    +    /**
    +     * return a deep copy of this buffer.
    +     * The returned buffer will have position == 0.  The position
    +     * of this buffer will not change as a result of copying.
    +     *
    +     * @return a new buffer with a full copy of this buffer's data
    +     */
    +    public LargeByteBuffer deepCopy();
    +
    +    /**
    +     * Advance the position in this buffer by up to <code>n</code> bytes.  <code>n</code> may be
    +     * positive or negative.  It will move the full <code>n</code> unless that moves
    +     * it past the end (or beginning) of the buffer, in which case it will move to the end
    +     * (or beginning).
    +     *
    +     * @return the number of bytes moved forward (can be negative if <code>n</code> is negative)
    +     */
    +    public long skip(long n);
    +
    +    public long position();
    +
    +    /**
    +     * Creates a new byte buffer that shares this buffer's content.
    +     *
    +     * <p> The content of the new buffer will be that of this buffer.  Changes
    +     * to this buffer's content will be visible in the new buffer, and vice
    +     * versa; the two buffers' positions will be independent.
    +     *
    +     * <p> The new buffer's position will be identical to those of this buffer
    +     * */
    +    public LargeByteBuffer duplicate();
    +
    +
    +    public long remaining();
    +
    +    /**
    +     * the total number of bytes in this buffer
    +     * @return
    +     */
    +    public long size();
    +
    +    /**
    +     * writes the data from the current <code>position()</code> to the end of this buffer
    --- End diff --
    
    nit: Writes


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107702013
  
    Merged build finished. 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107801247
  
    Merged build started.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r43674153
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java ---
    @@ -0,0 +1,148 @@
    +/*
    +* Licensed to the Apache Software Foundation (ASF) under one or more
    +* contributor license agreements.  See the NOTICE file distributed with
    +* this work for additional information regarding copyright ownership.
    +* The ASF licenses this file to You under the Apache License, Version 2.0
    +* (the "License"); you may not use this file except in compliance with
    +* the License.  You may obtain a copy of the License at
    +*
    +*    http://www.apache.org/licenses/LICENSE-2.0
    +*
    +* Unless required by applicable law or agreed to in writing, software
    +* distributed under the License is distributed on an "AS IS" BASIS,
    +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +* See the License for the specific language governing permissions and
    +* limitations under the License.
    +*/
    +package org.apache.spark.network.buffer;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +
    +/**
    + * A byte buffer which can hold over 2GB.
    + * <p/>
    + * This is roughly similar {@link java.nio.ByteBuffer}, with a limited set of operations relevant
    + * to use in Spark, and without the capacity restrictions of a ByteBuffer.
    + * <p/>
    + * Unlike ByteBuffers, this is read-only, and only supports reading bytes (with both single and bulk
    + * <code>get</code> methods).  It supports random access via <code>skip</code> to move around the
    + * buffer.
    + * <p/>
    + * In general, implementations are expected to support O(1) random access.  Furthermore,
    + * neighboring locations in the buffer are likely to be neighboring in memory, so sequential access
    + * will avoid cache-misses.  However, these are only rough guidelines which may differ in
    + * implementations.
    + * <p/>
    + * Any code which expects a ByteBuffer can obtain one via {@link #asByteBuffer} when possible -- see
    + * that method for a full description of its limitations.
    + * <p/>
    + * Instances of this class can be created with
    + * {@link org.apache.spark.network.buffer.LargeByteBufferHelper},
    + * with a LargeByteBufferOutputStream,
    + * or directly from the implementation
    + * {@link org.apache.spark.network.buffer.WrappedLargeByteBuffer}.
    + */
    +public interface LargeByteBuffer {
    +  public byte get();
    +
    +
    +  /**
    +   * Bulk copy data from this buffer into the given array.  First checks there is sufficient
    +   * data in this buffer; if not, throws a {@link java.nio.BufferUnderflowException}.  Behaves
    +   * in the exact same way as <code>get(dst, 0, dst.length)</code>
    +   *
    +   * @param dst the destination array
    +   * @return this buffer
    +   */
    +  public LargeByteBuffer get(byte[] dst);
    +
    +  /**
    +   * Bulk copy data from this buffer into the given array.  First checks there is sufficient
    +   * data in this buffer; if not, throws a {@link java.nio.BufferUnderflowException}.
    +   *
    +   * @param dst the destination array
    +   * @param offset the offset within the destination array to write to
    +   * @param length how many bytes to write
    +   * @return this buffer
    +   */
    +  public LargeByteBuffer get(byte[] dst, int offset, int length);
    +
    --- End diff --
    
    nit: too many empty lines


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28722298
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.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.buffer;
    +
    +import com.google.common.annotations.VisibleForTesting;
    --- End diff --
    
    nit: after `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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-90737842
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29808/
    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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27924849
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,252 @@
    +/*
    +* 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 sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  //only public for tests
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  private long _pos;
    +  private int currentBufferIdx;
    +  private ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, findExpectedInitialPosition(underlying));
    --- End diff --
    
    You don't need to chain the constructor calls like this; you could just call `findExpectedInitialPosition` from the other constructor, or even inline the code.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107664530
  
      [Test build #33906 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33906/consoleFull) for   PR 5400 at commit [`b77bbe2`](https://github.com/apache/spark/commit/b77bbe28e109db72775a592e99572c039b377517).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class LargeByteBufferInputStream extends InputStream `
      * `public class LargeByteBufferOutputStream extends OutputStream `
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `



---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r31546071
  
    --- Diff: network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java ---
    @@ -0,0 +1,84 @@
    +/*
    + * 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.*;
    +import java.nio.channels.FileChannel;
    +
    +import org.junit.Test;
    +
    +import static org.junit.Assert.*;
    +
    +public class LargeByteBufferHelperSuite {
    +
    +  @Test
    +  public void testMapFile() throws IOException {
    +    File testFile = File.createTempFile("large-byte-buffer-test", ".bin");
    +    try {
    +      testFile.deleteOnExit();
    +      OutputStream out = new FileOutputStream(testFile);
    +      byte[] buffer = new byte[1 << 16];
    +      long len = ((long)buffer.length) + Integer.MAX_VALUE + 1;
    +      for (int i = 0; i < buffer.length; i++) {
    +        buffer[i] = (byte) i;
    +      }
    +      for (int i = 0; i < len / buffer.length; i++) {
    +        out.write(buffer);
    --- End diff --
    
    If the only way to get appropriate coverage is with a slow test, then we can isolate that test into a `*SlowTests` suite (or use whatever JUnit's equivalent of ScalaTest's `@Slow` test tags is) and only run it under special circumstances (e.g. pre-release checks or a build that ones once per day).  Before we resort to this, though, let's just see how long it takes with randomized data; no need to over-engineer things if the test turns out to be fast enough as-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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153851952
  
    **[Test build #45032 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45032/consoleFull)** for PR 5400 at commit [`3447bb9`](https://github.com/apache/spark/commit/3447bb995b53c4d93154328c7c7c06e08a5ec9b9).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class LargeByteBufferInputStream extends InputStream `\n  * `public class LargeByteBufferOutputStream extends OutputStream `\n  * `public class JavaAssociationRulesExample `\n  * `public class JavaPrefixSpanExample `\n  * `public class JavaSimpleFPGrowth `\n  * `public class BufferTooLargeException extends IOException `\n  * `public class LargeByteBufferHelper `\n  * `public class WrappedLargeByteBuffer implements LargeByteBuffer `\n  * `class StreamInterceptor implements TransportFrameDecoder.Interceptor `\n  * `public final class ChunkFetchSuccess extends ResponseWithBody `\n  * `public abstract class ResponseWithBody implements ResponseMessage `\n  * `public final class StreamFailure implements ResponseMessage `\n  * `public final class StreamRequest implements RequestMessage `\n  * `public final class StreamResponse extends ResponseWithBody `\n  * `public class TransportFrameDecoder extends Channel
 InboundHandlerAdapter `\n


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-92490808
  
      [Test build #30190 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30190/consoleFull) for   PR 5400 at commit [`c3efa4c`](https://github.com/apache/spark/commit/c3efa4c24986937ffaf18c755323866fa54a073c).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class LargeByteBufferOutputStream(chunkSize: Int = 65536)`
      * `public class BufferTooLargeException extends IOException `
      * `public class LargeByteBufferHelper `
      * `public class WrappedLargeByteBuffer implements LargeByteBuffer `
    
     * This patch does not change any dependencies.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27925119
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,252 @@
    +/*
    +* 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 sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  //only public for tests
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  private long _pos;
    +  private int currentBufferIdx;
    +  private ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, findExpectedInitialPosition(underlying));
    +  }
    +
    +  private static long findExpectedInitialPosition(ByteBuffer[] bufs) {
    --- End diff --
    
    I'm a little confused about what this is supposed to be and what the checks are meant to check... could you write a comment explaining all that?
    
    I have a feeling that this might be over-complicating something that is supposed to be simple.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107801421
  
      [Test build #33956 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/33956/consoleFull) for   PR 5400 at commit [`6c2a115`](https://github.com/apache/spark/commit/6c2a115b970d31400cd77dc8f0236555e382a1af).


---
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 issue #5400: [SPARK-6190][core] create LargeByteBuffer for eliminating...

Posted by vijay1106 <gi...@git.apache.org>.
Github user vijay1106 commented on the issue:

    https://github.com/apache/spark/pull/5400
  
    Hey does this address the issue of spark.sql.autoBroadcastJoinThreshold cannot be more than 2GB?


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28718163
  
    --- Diff: core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.{FileInputStream, FileOutputStream, OutputStream, File}
    +import java.nio.ByteBuffer
    +import java.nio.channels.FileChannel.MapMode
    +
    +import org.junit.Assert._
    +import org.scalatest.{FunSuite, Matchers}
    +
    +import org.apache.spark.network.buffer.{LargeByteBuffer, LargeByteBufferHelper, WrappedLargeByteBuffer}
    +
    +class LargeByteBufferInputStreamSuite extends FunSuite with Matchers {
    +
    +  test("read from large mapped file") {
    +    val testFile = File.createTempFile("large-buffer-input-stream-test",".bin")
    +    testFile.deleteOnExit()
    +
    +    val out: OutputStream = new FileOutputStream(testFile)
    +    val buffer: Array[Byte] = new Array[Byte](1 << 16)
    +    val len: Long = 3L << 30
    +    assertTrue(len > Integer.MAX_VALUE)
    +    (0 until buffer.length).foreach { idx =>
    +      buffer(idx) = idx.toByte
    +    }
    +    (0 until (len / buffer.length).toInt).foreach { idx =>
    +      out.write(buffer)
    +    }
    +    out.close
    +
    +    val channel = new FileInputStream(testFile).getChannel
    +    val buf = LargeByteBufferHelper.mapFile(channel, MapMode.READ_ONLY, 0, len)
    +    val in = new LargeByteBufferInputStream(buf, dispose = true)
    +
    +    val read = new Array[Byte](buffer.length)
    +    (0 until (len / buffer.length).toInt).foreach { idx =>
    +      in.disposed should be (false)
    +      in.read(read) should be (read.length)
    +      (0 until buffer.length).foreach { arrIdx =>
    +        assertEquals(buffer(arrIdx), read(arrIdx))
    +      }
    +    }
    +    in.disposed should be (false)
    +    in.read(read) should be (-1)
    +    in.disposed should be (false)
    +    in.close()
    +    in.disposed should be (true)
    +  }
    +
    +  test("dispose on close") {
    +    // don't need to read to the end -- dispose anytime we close
    +    val data = new Array[Byte](10)
    +    val in = new LargeByteBufferInputStream(LargeByteBufferHelper.asLargeByteBuffer(data),
    +      dispose = true)
    +    in.disposed should be (false)
    +    in.close()
    +    in.disposed should be (true)
    +  }
    +
    +  test("io stream roundtrip") {
    +
    --- End diff --
    
    nit: delete


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28722729
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.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.buffer;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +  public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    +
    +  public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { buffer } );
    +  }
    +
    +  public static LargeByteBuffer asLargeByteBuffer(byte[] bytes) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { ByteBuffer.wrap(bytes) } );
    +  }
    +
    +  public static LargeByteBuffer allocate(long size) {
    +    return allocate(size, MAX_CHUNK);
    +  }
    +
    +  @VisibleForTesting
    +  static LargeByteBuffer allocate(long size, int maxChunk) {
    +    int chunksNeeded = (int) ((size + maxChunk - 1) / maxChunk);
    +    ByteBuffer[] chunks = new ByteBuffer[chunksNeeded];
    +    long remaining = size;
    +    for (int i = 0; i < chunksNeeded; i++) {
    +      int nextSize = (int) Math.min(remaining, maxChunk);
    +      ByteBuffer next = ByteBuffer.allocate(nextSize);
    +      remaining -= nextSize;
    +      chunks[i] = next;
    +    }
    +    if (remaining != 0) throw new IllegalStateException("remaining = " + remaining);
    --- End diff --
    
    nit:
    
        if {
           ...
        }


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107826904
  
    Merged build finished. 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-107640385
  
    Merged build finished. 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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#issuecomment-153142618
  
    I had already reviewed this and I don't see any changes, so my only worry is still the same as Tom's: how long does the large file test takes. I guess it's not horrible if it's a single test taking 10s, but if we could avoid and still be reasonably sure that things work, it would be better.


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r27925549
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java ---
    @@ -0,0 +1,252 @@
    +/*
    +* 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 sun.nio.ch.DirectBuffer;
    +
    +import java.io.IOException;
    +import java.nio.BufferUnderflowException;
    +import java.nio.ByteBuffer;
    +import java.nio.MappedByteBuffer;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class WrappedLargeByteBuffer implements LargeByteBuffer {
    +
    +  //only public for tests
    +  public final ByteBuffer[] underlying;
    +
    +  private final long size;
    +  private long _pos;
    +  private int currentBufferIdx;
    +  private ByteBuffer currentBuffer;
    +
    +
    +  public WrappedLargeByteBuffer(ByteBuffer[] underlying) {
    +    this(underlying, findExpectedInitialPosition(underlying));
    +  }
    +
    +  private static long findExpectedInitialPosition(ByteBuffer[] bufs) {
    +    long sum = 0L;
    +    for (ByteBuffer b: bufs) {
    +      if (b.position() > 0) {
    +        // this could still lead to a mix of positions half-way through buffers that
    +        // would be inconsistent -- but we'll discover that in the constructor checks
    +        sum += b.position();
    +      } else {
    +        break;
    +      }
    +    }
    +    return sum;
    +  }
    +
    +  private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) {
    +    this.underlying = underlying;
    +    long sum = 0L;
    +    for (int i = 0; i < underlying.length; i++) {
    +      ByteBuffer b = underlying[i];
    +      long nextSum = sum + b.capacity();
    +      int expectedPosition;
    +      if (nextSum < initialPosition) {
    +        expectedPosition = b.capacity();
    +      } else if (sum > initialPosition) {
    +        expectedPosition = 0;
    +      } else {
    +        expectedPosition = (int) (initialPosition - sum);
    +      }
    +      if (b.position() != expectedPosition) {
    +        throw new IllegalArgumentException("ByteBuffer[" + i + "]:" + b + " was expected to have" +
    +          " position = " + expectedPosition + " to be consistent with the overall " +
    +          "initialPosition = " + initialPosition);
    +      }
    +      sum = nextSum;
    +    }
    +    _pos = initialPosition;
    +    currentBufferIdx = 0;
    +    currentBuffer = underlying[0];
    +    size = sum;
    +  }
    +
    +  @Override
    +  public void get(byte[] dest, int offset, int length) {
    +    if (length > remaining())
    --- End diff --
    
    nit: add braces


---
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-6190][core] create LargeByteBuffer for ...

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

    https://github.com/apache/spark/pull/5400#discussion_r28722902
  
    --- Diff: network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.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.buffer;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.FileChannel;
    +import java.util.ArrayList;
    +
    +public class LargeByteBufferHelper {
    +
    +  public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000;
    +
    +  public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { buffer } );
    +  }
    +
    +  public static LargeByteBuffer asLargeByteBuffer(byte[] bytes) {
    +    return new WrappedLargeByteBuffer(new ByteBuffer[] { ByteBuffer.wrap(bytes) } );
    +  }
    +
    +  public static LargeByteBuffer allocate(long size) {
    +    return allocate(size, MAX_CHUNK);
    +  }
    +
    +  @VisibleForTesting
    +  static LargeByteBuffer allocate(long size, int maxChunk) {
    +    int chunksNeeded = (int) ((size + maxChunk - 1) / maxChunk);
    +    ByteBuffer[] chunks = new ByteBuffer[chunksNeeded];
    +    long remaining = size;
    +    for (int i = 0; i < chunksNeeded; i++) {
    +      int nextSize = (int) Math.min(remaining, maxChunk);
    +      ByteBuffer next = ByteBuffer.allocate(nextSize);
    +      remaining -= nextSize;
    +      chunks[i] = next;
    +    }
    +    if (remaining != 0) throw new IllegalStateException("remaining = " + remaining);
    +    return new WrappedLargeByteBuffer(chunks, maxChunk);
    +  }
    +
    +
    +  public static LargeByteBuffer mapFile(
    +    FileChannel channel,
    +    FileChannel.MapMode mode,
    +    long offset,
    +    long length
    +  ) throws IOException {
    +    int maxChunk = MAX_CHUNK;
    --- End diff --
    
    not needed


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