Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/611#discussion_r86398411
  
    --- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
 ---
    @@ -0,0 +1,460 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.drill.exec.util.filereader;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.buffer.DrillBuf;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.memory.RootAllocatorFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.ByteBufferReadable;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.parquet.hadoop.Footer;
    +import org.apache.parquet.hadoop.ParquetFileReader;
    +import org.apache.parquet.hadoop.metadata.BlockMetaData;
    +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
    +import org.apache.parquet.hadoop.util.CompatibilityUtil;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +import java.util.List;
    +
    +/**
    + * <code>BufferedDirectBufInputStream</code>  reads from the
    + * underlying <code>InputStream</code> in blocks of data, into an
    + * internal buffer. The internal buffer is a direct memory backed
    + * buffer. The implementation is similar to the 
<code>BufferedInputStream</code>
    + * class except that the internal buffer is a Drillbuf and
    + * not a byte array. The mark and reset methods of the underlying
    + * <code>InputStream</code>are not supported.
    + */
    +public class BufferedDirectBufInputStream extends DirectBufInputStream 
implements Closeable {
    +
    +  private static final org.slf4j.Logger logger =
    +      
org.slf4j.LoggerFactory.getLogger(BufferedDirectBufInputStream.class);
    +
    +  private static int defaultBufferSize = 8192 * 1024; // 8 MiB
    +  private static int defaultTempBufferSize = 8192; // 8 KiB
    +
    +  /**
    +   * The internal buffer to keep data read from the underlying inputStream.
    +   * <code>internalBuffer[0]</code>  through <code>internalBuffer[count-1] 
</code>
    +   * contains data read from the underlying  input stream.
    +   */
    +  protected volatile DrillBuf internalBuffer; // the internal buffer
    +
    +  /**
    +   * The number of valid bytes in <code>internalBuffer</code>.
    +   * <code> count </code> is always in the range 
<code>[0,internalBuffer.capacity]</code>
    +   * <code>internalBuffer[count-1]</code> is the last valid byte in the 
buffer.
    +   */
    +  protected int count;
    +
    +  /**
    +   * The current read position in the buffer; the index of the next
    +   * character to be read from the <code>internalBuffer</code> array.
    +   * <p>
    +   * This value is always in the range <code>[0,count]</code>.
    +   * If <code>curPosInBuffer</code> is equal to <code>count></code> then 
we have read
    +   * all the buffered data and the next read (or skip) will require more 
data to be read
    +   * from the underlying input stream.
    +   */
    +  protected int curPosInBuffer;
    +
    +  protected long curPosInStream; // current offset in the input stream
    +
    +  private final int bufSize;
    +
    +  private volatile DrillBuf tempBuffer; // a temp Buffer for use by 
read(byte[] buf, int off, int len)
    +
    +
    +  private DrillBuf getBuf() throws IOException {
    +    checkInputStreamState();
    +    if (internalBuffer == null) {
    +      throw new IOException("Input stream is closed.");
    +    }
    +    return this.internalBuffer;
    +  }
    +
    +  /**
    +   * Creates a <code>BufferedDirectBufInputStream</code>
    +   * with the default (8 MiB) buffer size.
    +   */
    +  public BufferedDirectBufInputStream(InputStream in, BufferAllocator 
allocator, String id,
    +      long startOffset, long totalByteSize, boolean enableHints) {
    +    this(in, allocator, id, startOffset, totalByteSize, defaultBufferSize, 
enableHints);
    +  }
    +
    +  /**
    +   * Creates a <code>BufferedDirectBufInputStream</code>
    +   * with the specified buffer size.
    +   */
    +  public BufferedDirectBufInputStream(InputStream in, BufferAllocator 
allocator, String id,
    +      long startOffset, long totalByteSize, int bufSize, boolean 
enableHints) {
    +    super(in, allocator, id, startOffset, totalByteSize, enableHints);
    +    Preconditions.checkArgument(bufSize >= 0);
    +    // We make the buffer size the smaller of the buffer Size parameter or 
the total Byte Size
    +    // rounded to next highest pwoer of two
    +    int bSize = bufSize < (int) totalByteSize ? bufSize : (int) 
totalByteSize;
    +    // round up to next power of 2
    +    bSize--;
    +    bSize |= bSize >>> 1;
    +    bSize |= bSize >>> 2;
    +    bSize |= bSize >>> 4;
    +    bSize |= bSize >>> 8;
    +    bSize |= bSize >>> 16;
    +    bSize++;
    +    this.bufSize = bSize;
    +
    +  }
    +
    +  @Override
    +  public void init() throws UnsupportedOperationException, IOException {
    +    super.init();
    +    this.internalBuffer = this.allocator.buffer(this.bufSize);
    +    this.tempBuffer = this.allocator.buffer(defaultTempBufferSize);
    +    int bytesRead = getNextBlock();
    +    if (bytesRead <= 0) {
    +      throw new IOException("End of stream reached while initializing 
buffered reader.");
    +    }
    +  }
    +
    +  /**
    +   * Read one more block from the underlying stream.
    +   * Assumes we have reached the end of buffered data
    +   * Assumes it is being called from a synchronized block.
    +   * returns number of bytes read or -1 if EOF
    +   */
    +  private int getNextBlock() throws IOException {
    +    Preconditions.checkState(this.curPosInBuffer >= this.count,
    +        "Internal error: Buffered stream has not been consumed and trying 
to read more from underlying stream");
    +    checkInputStreamState();
    +    DrillBuf buffer = getBuf();
    +    buffer.clear();
    +    this.count = this.curPosInBuffer = 0;
    +
    +    // We *cannot* rely on the totalByteSize being correct because
    +    // metadata for Parquet files is incorrect. So we read as
    +    // much as we can up to the size of the buffer
    +    //int bytesToRead = buffer.capacity() <= (totalByteSize + startOffset 
- curPosInStream ) ?
    +    //    buffer.Capacity() :
    +    //    (int) (totalByteSize + startOffset - curPosInStream );
    +    int bytesToRead = buffer.capacity();
    +
    +    ByteBuffer directBuffer = buffer.nioBuffer(curPosInBuffer, 
bytesToRead);
    --- End diff --
    
    Every read creates heap garbage (the new DirectByteBuffer that holds the 
subrange.) One of the goals of using direct memory is to avoid excess garbage.
    
    Is it worth allowing the reader to read more data than we want (based on 
comment above) to avoid the garbage?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If 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.
---

Reply via email to