[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-18 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-12 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138502867
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,313 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0, but the value is 
" + bufferSizeInBytes);
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes, but the" +
+"value is " + readAheadThresholdInBytes );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+return (!activeBuffer.hasRemaining() && 
!readAheadBuffer.hasRemaining() && 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-12 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138502374
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,313 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0, but the value is 
" + bufferSizeInBytes);
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes, but the" +
+"value is " + readAheadThresholdInBytes );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+return (!activeBuffer.hasRemaining() && 
!readAheadBuffer.hasRemaining() && endOfStream);

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-12 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138473231
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-12 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138430322
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138230590
  
--- Diff: 
core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
 ---
@@ -72,10 +72,15 @@ public UnsafeSorterSpillReader(
   bufferSizeBytes = DEFAULT_BUFFER_SIZE_BYTES;
 }
 
+final Double readAheadFraction =
+SparkEnv.get() == null ? 0.5 :
+ 
SparkEnv.get().conf().getDouble("spark.unsafe.sorter.spill.read.ahead.fraction",
 0.5);
+
 final InputStream bs =
 new NioBufferedFileInputStream(file, (int) bufferSizeBytes);
 try {
-  this.in = serializerManager.wrapStream(blockId, bs);
+  this.in = new 
ReadAheadInputStream(serializerManager.wrapStream(blockId, bs),
--- End diff --

sure, done.


---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138231043
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138230901
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138230596
  
--- Diff: 
core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
 ---
@@ -72,10 +72,15 @@ public UnsafeSorterSpillReader(
   bufferSizeBytes = DEFAULT_BUFFER_SIZE_BYTES;
 }
 
+final Double readAheadFraction =
--- End diff --

done


---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138225592
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138201871
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138207519
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138193555
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138225438
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138198506
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
--- End diff --

nit: double spaces



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138207456
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138195259
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
--- End diff --

nit: could you add the `bufferSizeInBytes` value to the error message?


---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138194732
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138204793
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138200321
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,317 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean hasRemaining() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138194101
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138189593
  
--- Diff: 
core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
 ---
@@ -72,10 +72,15 @@ public UnsafeSorterSpillReader(
   bufferSizeBytes = DEFAULT_BUFFER_SIZE_BYTES;
 }
 
+final Double readAheadFraction =
+SparkEnv.get() == null ? 0.5 :
+ 
SparkEnv.get().conf().getDouble("spark.unsafe.sorter.spill.read.ahead.fraction",
 0.5);
+
 final InputStream bs =
 new NioBufferedFileInputStream(file, (int) bufferSizeBytes);
 try {
-  this.in = serializerManager.wrapStream(blockId, bs);
+  this.in = new 
ReadAheadInputStream(serializerManager.wrapStream(blockId, bs),
--- End diff --

Could you add an internal conf to disable it? It will allow the user to 
disable it when the new feature causes a regression.


---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138189733
  
--- Diff: 
core/src/test/java/org/apache/spark/io/GenericFileInputStreamSuite.java ---
@@ -50,17 +52,16 @@ public void tearDown() {
 inputFile.delete();
   }
 
+
--- End diff --

nit: extra empty line


---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138201264
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138188910
  
--- Diff: 
core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
 ---
@@ -72,10 +72,15 @@ public UnsafeSorterSpillReader(
   bufferSizeBytes = DEFAULT_BUFFER_SIZE_BYTES;
 }
 
+final Double readAheadFraction =
--- End diff --

nit: Double -> double


---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138195292
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
--- End diff --

ditto


---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138200201
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138194341
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138208239
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-09-11 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r138198579
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,315 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private static final ThreadLocal oneByte = 
ThreadLocal.withInitial(() -> new byte[1]);
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && 
endOfStream) {
+  return true;
+}
+return  false;
--- End diff --

why not just `return 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-29 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135975209
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-29 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135950441
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-29 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135895413
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-29 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135889304
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-29 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135886270
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-29 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135884679
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-29 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135882122
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,317 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean hasRemaining() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-29 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135882063
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,317 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean hasRemaining() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
--- End diff --

I see. My bad, changed accordingly


---
If your project is set 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-29 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135881985
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,317 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean hasRemaining() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135694280
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,317 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean hasRemaining() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135694119
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,317 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean hasRemaining() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135695472
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135695625
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135693340
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,317 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+import org.apache.spark.util.ThreadUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean readInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean readAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
ThreadUtils.newDaemonSingleThreadExecutor("read-ahead");
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean hasRemaining() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
--- End diff --

I meant `hasRemaining` on the buffer's instead of `remaining() == 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135692852
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135692546
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135689398
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135689367
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135689239
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
--- End diff --

done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135689225
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
--- End diff --

done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135689194
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135689169
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135689113
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135689067
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135688838
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-28 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r135688946
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134868135
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
--- End diff --

skip this, will just add to noise ... I added it because an NPE after 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134853414
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134849181
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134835600
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
--- End diff --

tiny nit: `hasRemaining` 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 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134852288
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134848805
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134839956
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134829463
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
--- End diff --

nit: `isReadInProgress` -> `readInProgress`
Similarly for `isReadAborted` ... java naming convention of fields vs 
methods.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134840925
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134850427
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134848419
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134840651
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134835831
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
--- End diff --

`assert null != byteBuffer`


---
If your project is set up for it, 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134841279
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134849981
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,292 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0,
+"bufferSizeInBytes should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 &&
+readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be greater than 0 and less 
than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread sitalkedia
Github user sitalkedia closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread sitalkedia
GitHub user sitalkedia reopened a pull request:

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

[SPARK-21113][CORE] Read ahead input stream to amortize disk IO cost …

Profiling some of our big jobs, we see that around 30% of the time is being 
spent in reading the spill files from disk. In order to amortize the disk IO 
cost, the idea is to implement a read ahead input stream which asynchronously 
reads ahead from the underlying input stream when specified amount of data has 
been read from the current buffer. It does it by maintaining two buffer - 
active buffer and read ahead buffer. The active buffer contains data which 
should be returned when a read() call is issued. The read-ahead buffer is used 
to asynchronously read from the underlying input stream and once the active 
buffer is exhausted, we flip the two buffers so that we can start reading from 
the read ahead buffer without being blocked in disk I/O.

## How was this patch tested?

Tested by running a job on the cluster and could see up to 8% CPU 
improvement. 


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

$ git pull https://github.com/sitalkedia/spark read_ahead_buffer

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

https://github.com/apache/spark/pull/18317.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 #18317






---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-23 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r134821455
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,288 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-13 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r132846768
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,288 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
   

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-13 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r132844586
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,288 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
--- End diff --

nit: Do we want to ensure that the line length is below 100? @cloud-fan 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-06 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131535514
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,288 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
   

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-06 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131535496
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-05 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131516266
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-04 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131514782
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-04 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131502049
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,289 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private ReentrantLock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  /**
+   * Creates a ReadAheadInputStream with the specified buffer 
size and read-ahead
+   * threshold
+   *
+   * @param   inputStream The underlying input stream.
+   * @param   bufferSizeInBytes   The buffer size.
+   * @param   readAheadThresholdInBytes   If the active buffer has less 
data than the read-ahead
+   *  threshold, an async read is 
triggered.
+   */
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-04 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131494491
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
--- End diff --

done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-04 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131494505
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-04 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131494464
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-04 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131493686
  
--- Diff: 
core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
 ---
@@ -73,7 +73,9 @@ public UnsafeSorterSpillReader(
 }
 
 final InputStream bs =
-new NioBufferedFileInputStream(file, (int) bufferSizeBytes);
+new ReadAheadInputStream(
+new NioBufferedFileInputStream(file, (int) bufferSizeBytes),
+(int)bufferSizeBytes, (int)bufferSizeBytes / 2);
--- End diff --

Our test shows best result was obtained when threshold = 0.5* buffer size. 
I made it configurable anyways in case someone wants to play with it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-04 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131493511
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-04 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131493491
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-04 Thread sitalkedia
Github user sitalkedia commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r131493469
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-02 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r130933824
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-02 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r130929979
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-02 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r130929922
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-02 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r130927420
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-02 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r130921551
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
--- End diff --

Can we add @param for these parameters?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-08-02 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r130916927
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead buffer

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-07-07 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r126158891
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-07-07 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r126160543
  
--- Diff: 
core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
 ---
@@ -73,7 +73,9 @@ public UnsafeSorterSpillReader(
 }
 
 final InputStream bs =
-new NioBufferedFileInputStream(file, (int) bufferSizeBytes);
+new ReadAheadInputStream(
+new NioBufferedFileInputStream(file, (int) bufferSizeBytes),
+(int)bufferSizeBytes, (int)bufferSizeBytes / 2);
--- End diff --

Do we have a test on what would be the best threshold? Should we consider 
make this configurable?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-07-07 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r126069366
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-07-07 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18317#discussion_r126157396
  
--- Diff: core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java 
---
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.spark.storage.StorageUtils;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * {@link InputStream} implementation which asynchronously reads ahead 
from the underlying input
+ * stream when specified amount of data has been read from the current 
buffer. It does it by maintaining
+ * two buffer - active buffer and read ahead buffer. Active buffer 
contains data which should be returned
+ * when a read() call is issued. The read ahead buffer is used to 
asynchronously read from the underlying
+ * input stream and once the current active buffer is exhausted, we flip 
the two buffers so that we can
+ * start reading from the read ahead buffer without being blocked in disk 
I/O.
+ */
+public class ReadAheadInputStream extends InputStream {
+
+  private Lock stateChangeLock = new ReentrantLock();
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer activeBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private ByteBuffer readAheadBuffer;
+
+  @GuardedBy("stateChangeLock")
+  private boolean endOfStream;
+
+  @GuardedBy("stateChangeLock")
+  // true if async read is in progress
+  private boolean isReadInProgress;
+
+  @GuardedBy("stateChangeLock")
+  // true if read is aborted due to an exception in reading from 
underlying input stream.
+  private boolean isReadAborted;
+
+  @GuardedBy("stateChangeLock")
+  private Exception readException;
+
+  // If the remaining data size in the current buffer is below this 
threshold,
+  // we issue an async read from the underlying input stream.
+  private final int readAheadThresholdInBytes;
+
+  private final InputStream underlyingInputStream;
+
+  private final ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+
+  private final Condition asyncReadComplete = 
stateChangeLock.newCondition();
+
+  private final byte[] oneByte = new byte[1];
+
+  public ReadAheadInputStream(InputStream inputStream, int 
bufferSizeInBytes, int readAheadThresholdInBytes) {
+Preconditions.checkArgument(bufferSizeInBytes > 0, "bufferSizeInBytes 
should be greater than 0");
+Preconditions.checkArgument(readAheadThresholdInBytes > 0 && 
readAheadThresholdInBytes < bufferSizeInBytes,
+"readAheadThresholdInBytes should be 
greater than 0 and less than bufferSizeInBytes" );
+activeBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes);
+this.readAheadThresholdInBytes = readAheadThresholdInBytes;
+this.underlyingInputStream = inputStream;
+activeBuffer.flip();
+readAheadBuffer.flip();
+  }
+
+  private boolean isEndOfStream() {
+if(activeBuffer.remaining() == 0 && readAheadBuffer.remaining() == 0 
&& endOfStream) {
+  return true;
+}
+return  false;
+  }
+
+
+  private void readAsync(final ByteBuffer byteBuffer) throws IOException {
+stateChangeLock.lock();
+if (endOfStream || isReadInProgress) {
+  stateChangeLock.unlock();
+  return;
+}
+byteBuffer.position(0);
+byteBuffer.flip();
+isReadInProgress = true;
+stateChangeLock.unlock();
+executorService.execute(() -> {
+  byte[] arr;
+  stateChangeLock.lock();
+  arr = byteBuffer.array();
+  stateChangeLock.unlock();
+  // Please note that it is safe to release the lock and read into the 
read ahead 

[GitHub] spark pull request #18317: [SPARK-21113][CORE] Read ahead input stream to am...

2017-06-15 Thread sitalkedia
GitHub user sitalkedia opened a pull request:

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

[SPARK-21113][CORE] Read ahead input stream to amortize disk IO cost …

Profiling some of our big jobs, we see that around 30% of the time is being 
spent in reading the spill files from disk. In order to amortize the disk IO 
cost, the idea is to implement a read ahead input stream which asynchronously 
reads ahead from the underlying input stream when specified amount of data has 
been read from the current buffer. It does it by maintaining two buffer - 
active buffer and read ahead buffer. The active buffer contains data which 
should be returned when a read() call is issued. The read-ahead buffer is used 
to asynchronously read from the underlying input stream and once the active 
buffer is exhausted, we flip the two buffers so that we can start reading from 
the read ahead buffer without being blocked in disk I/O.

## How was this patch tested?

Tested by running a job on the cluster and could see up to 8% CPU 
improvement. 


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

$ git pull https://github.com/sitalkedia/spark read_ahead_buffer

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

https://github.com/apache/spark/pull/18317.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 #18317


commit d42000fe288a22bbfc56b4fef8bf32273aea6218
Author: Sital Kedia 
Date:   2017-06-09T18:06:21Z

[SPARK-21113][CORE] Read ahead input stream to amortize disk IO cost in the 
Spill reader




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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