You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2016/11/05 00:11:28 UTC

[01/10] drill git commit: DRILL-4800: Use a buffering input stream in the Parquet reader

Repository: drill
Updated Branches:
  refs/heads/master 190d5d46d -> 9411b26ec


DRILL-4800: Use a buffering input stream in the Parquet reader


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/fe2334ee
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/fe2334ee
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/fe2334ee

Branch: refs/heads/master
Commit: fe2334ee0234aa37637f17d7833e734333b73892
Parents: 190d5d4
Author: Parth Chandra <pa...@apache.org>
Authored: Fri Jun 10 14:56:41 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Wed Nov 2 17:23:27 2016 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/ExecConstants.java    |   4 +
 .../apache/drill/exec/ops/OperatorContext.java  |   6 +-
 .../drill/exec/ops/OperatorContextImpl.java     |   5 +
 .../server/options/SystemOptionManager.java     |   1 +
 .../store/parquet/columnreaders/PageReader.java | 122 +++--
 .../columnreaders/ParquetRecordReader.java      |  16 +-
 .../BufferedDirectBufInputStream.java           | 460 +++++++++++++++++++
 .../util/filereader/DirectBufInputStream.java   | 166 +++++++
 8 files changed, 727 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/fe2334ee/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 21015bb..ba6b084 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -147,6 +147,10 @@ public interface ExecConstants {
   String PARQUET_READER_INT96_AS_TIMESTAMP = "store.parquet.reader.int96_as_timestamp";
   OptionValidator PARQUET_READER_INT96_AS_TIMESTAMP_VALIDATOR = new BooleanValidator(PARQUET_READER_INT96_AS_TIMESTAMP, false);
 
+  // Use a buffering reader for parquet page reader
+  String PARQUET_PAGEREADER_USE_BUFFERED_READ = "store.parquet.reader.pagereader.bufferedread";
+  OptionValidator PARQUET_PAGEREADER_USE_BUFFERED_READ_VALIDATOR = new  BooleanValidator(PARQUET_PAGEREADER_USE_BUFFERED_READ, true);
+
   OptionValidator COMPILE_SCALAR_REPLACEMENT = new BooleanValidator("exec.compile.scalar_replacement", false);
 
   String JSON_ALL_TEXT_MODE = "store.json.all_text_mode";

http://git-wip-us.apache.org/repos/asf/drill/blob/fe2334ee/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
index 2c169a4..33fa288 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
@@ -23,6 +23,8 @@ import io.netty.buffer.DrillBuf;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.concurrent.Callable;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
@@ -43,6 +45,8 @@ public abstract class OperatorContext {
 
   public abstract OperatorStats getStats();
 
+  public abstract ExecutorService getExecutor();
+
   public abstract ExecutionControls getExecutionControls();
 
   public abstract DrillFileSystem newFileSystem(Configuration conf) throws IOException;
@@ -72,4 +76,4 @@ public abstract class OperatorContext {
     return i;
   }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/fe2334ee/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
index 8217afd..85f0ccb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
@@ -95,6 +95,11 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
     return manager.getManagedBuffer(size);
   }
 
+  // Allow and operator to use the thread pool
+  public ExecutorService getExecutor() {
+    return executor;
+  }
+
   public ExecutionControls getExecutionControls() {
     return executionControls;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/fe2334ee/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index f272c9d..8b67fdb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -99,6 +99,7 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING_VALIDATOR,
       ExecConstants.PARQUET_VECTOR_FILL_THRESHOLD_VALIDATOR,
       ExecConstants.PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR,
+      ExecConstants.PARQUET_PAGEREADER_USE_BUFFERED_READ_VALIDATOR,
       ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR,
       ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP_VALIDATOR,
       ExecConstants.JSON_READER_ALL_TEXT_MODE_VALIDATOR,

http://git-wip-us.apache.org/repos/asf/drill/blob/fe2334ee/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
index e7b4b6e..078e4ce 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
@@ -17,18 +17,17 @@
  */
 package org.apache.drill.exec.store.parquet.columnreaders;
 
-import static org.apache.parquet.column.Encoding.valueOf;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.util.filereader.BufferedDirectBufInputStream;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.store.parquet.ColumnDataReader;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.store.parquet.ParquetFormatPlugin;
 import org.apache.drill.exec.store.parquet.ParquetReaderStats;
+import org.apache.drill.exec.util.filereader.DirectBufInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -37,6 +36,7 @@ import org.apache.parquet.column.Dictionary;
 import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.ValuesType;
 import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.statistics.Statistics;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.dictionary.DictionaryValuesReader;
 import org.apache.parquet.format.PageHeader;
@@ -48,22 +48,26 @@ import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 import org.apache.parquet.schema.PrimitiveType;
 
-import com.google.common.base.Preconditions;
-import com.google.common.base.Stopwatch;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
+import static org.apache.parquet.column.Encoding.valueOf;
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.fromParquetStatistics;
 
 // class to keep track of the read position of variable length columns
 final class PageReader {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PageReader.class);
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(
+      org.apache.drill.exec.store.parquet.columnreaders.PageReader.class);
 
   public static final ParquetMetadataConverter METADATA_CONVERTER = ParquetFormatPlugin.parquetMetadataConverter;
 
-  private final ColumnReader<?> parentColumnReader;
-  private final ColumnDataReader dataReader;
-
-  // buffer to store bytes of current page
+  private final org.apache.drill.exec.store.parquet.columnreaders.ColumnReader<?> parentColumnReader;
+  //private final ColumnDataReader dataReader;
+  private final DirectBufInputStream dataReader;
+  //der; buffer to store bytes of current page
   DrillBuf pageData;
 
   // for variable length data we need to keep track of our current position in the page data
@@ -108,8 +112,8 @@ final class PageReader {
 
   private final ParquetReaderStats stats;
 
-  PageReader(ColumnReader<?> parentStatus, FileSystem fs, Path path, ColumnChunkMetaData columnChunkMetaData)
-    throws ExecutionSetupException{
+  PageReader(org.apache.drill.exec.store.parquet.columnreaders.ColumnReader<?> parentStatus, FileSystem fs, Path path, ColumnChunkMetaData columnChunkMetaData)
+    throws ExecutionSetupException {
     this.parentColumnReader = parentStatus;
     allocatedDictionaryBuffers = new ArrayList<ByteBuf>();
     codecFactory = parentColumnReader.parentReader.getCodecFactory();
@@ -117,8 +121,22 @@ final class PageReader {
     long start = columnChunkMetaData.getFirstDataPageOffset();
     try {
       inputStream  = fs.open(path);
-      this.dataReader = new ColumnDataReader(inputStream, start, columnChunkMetaData.getTotalSize());
-      loadDictionaryIfExists(parentStatus, columnChunkMetaData, inputStream);
+      BufferAllocator allocator =  parentColumnReader.parentReader.getOperatorContext().getAllocator();
+      //TODO: make read batch size configurable
+      columnChunkMetaData.getTotalUncompressedSize();
+      boolean useBufferedReader  = parentColumnReader.parentReader.getFragmentContext().getOptions()
+          .getOption(ExecConstants.PARQUET_PAGEREADER_USE_BUFFERED_READ).bool_val;
+      if (useBufferedReader) {
+        this.dataReader = new BufferedDirectBufInputStream(inputStream, allocator, path.getName(),
+            columnChunkMetaData.getStartingPos(), columnChunkMetaData.getTotalSize(), 8 * 1024 * 1024,
+            true);
+      } else {
+        this.dataReader = new DirectBufInputStream(inputStream, allocator, path.getName(),
+            columnChunkMetaData.getStartingPos(), columnChunkMetaData.getTotalSize(), true);
+      }
+      dataReader.init();
+
+      loadDictionaryIfExists(parentStatus, columnChunkMetaData, dataReader);
 
     } catch (IOException e) {
       throw new ExecutionSetupException("Error opening or reading metadata for parquet file at location: "
@@ -127,16 +145,16 @@ final class PageReader {
 
   }
 
-  private void loadDictionaryIfExists(final ColumnReader<?> parentStatus,
-      final ColumnChunkMetaData columnChunkMetaData, final FSDataInputStream f) throws IOException {
+  private void loadDictionaryIfExists(final org.apache.drill.exec.store.parquet.columnreaders.ColumnReader<?> parentStatus,
+      final ColumnChunkMetaData columnChunkMetaData, final DirectBufInputStream f) throws IOException {
     Stopwatch timer = Stopwatch.createUnstarted();
     if (columnChunkMetaData.getDictionaryPageOffset() > 0) {
-      f.seek(columnChunkMetaData.getDictionaryPageOffset());
-      long start=f.getPos();
+      dataReader.skip(columnChunkMetaData.getDictionaryPageOffset() - dataReader.getPos());
+      long start=dataReader.getPos();
       timer.start();
       final PageHeader pageHeader = Util.readPageHeader(f);
       long timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
-      long pageHeaderBytes=f.getPos()-start;
+      long pageHeaderBytes=dataReader.getPos()-start;
       this.updateStats(pageHeader, "Page Header", start, timeToRead, pageHeaderBytes, pageHeaderBytes);
       assert pageHeader.type == PageType.DICTIONARY_PAGE;
       readDictionaryPage(pageHeader, parentStatus);
@@ -148,8 +166,8 @@ final class PageReader {
     int compressedSize = pageHeader.getCompressed_page_size();
     int uncompressedSize = pageHeader.getUncompressed_page_size();
 
-    final DrillBuf dictionaryData = allocateDictionaryBuffer(uncompressedSize);
-    readPage(pageHeader, compressedSize, uncompressedSize, dictionaryData);
+    final DrillBuf dictionaryData = readPage(pageHeader, compressedSize, uncompressedSize);
+    allocatedDictionaryBuffers.add(dictionaryData);
 
     DictionaryPage page = new DictionaryPage(
         asBytesInput(dictionaryData, 0, uncompressedSize),
@@ -160,34 +178,41 @@ final class PageReader {
     this.dictionary = page.getEncoding().initDictionary(parentStatus.columnDescriptor, page);
   }
 
-  public void readPage(PageHeader pageHeader, int compressedSize, int uncompressedSize, DrillBuf dest) throws IOException {
+  public DrillBuf readPage(PageHeader pageHeader, int compressedSize, int uncompressedSize) throws IOException {
+    DrillBuf pageDataBuf = null;
     Stopwatch timer = Stopwatch.createUnstarted();
     long timeToRead;
-    long start=inputStream.getPos();
+    long start=dataReader.getPos();
     if (parentColumnReader.columnChunkMetaData.getCodec() == CompressionCodecName.UNCOMPRESSED) {
       timer.start();
-      dataReader.loadPage(dest, compressedSize);
+      pageDataBuf = dataReader.getNext(compressedSize);
       timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
       this.updateStats(pageHeader, "Page Read", start, timeToRead, compressedSize, uncompressedSize);
     } else {
-      final DrillBuf compressedData = allocateTemporaryBuffer(compressedSize);
+      DrillBuf compressedData = null;
+      pageDataBuf=allocateTemporaryBuffer(uncompressedSize);
+
       try {
       timer.start();
-      dataReader.loadPage(compressedData, compressedSize);
+      compressedData = dataReader.getNext(compressedSize);
+       // dataReader.loadPage(compressedData, compressedSize);
       timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
       timer.reset();
       this.updateStats(pageHeader, "Page Read", start, timeToRead, compressedSize, compressedSize);
-      start = inputStream.getPos();
+      start=dataReader.getPos();
       timer.start();
       codecFactory.getDecompressor(parentColumnReader.columnChunkMetaData
           .getCodec()).decompress(compressedData.nioBuffer(0, compressedSize), compressedSize,
-          dest.nioBuffer(0, uncompressedSize), uncompressedSize);
+          pageDataBuf.nioBuffer(0, uncompressedSize), uncompressedSize);
         timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
         this.updateStats(pageHeader, "Decompress", start, timeToRead, compressedSize, uncompressedSize);
       } finally {
-        compressedData.release();
+        if(compressedData != null) {
+          compressedData.release();
+        }
       }
     }
+    return pageDataBuf;
   }
 
   public static BytesInput asBytesInput(DrillBuf buf, int offset, int length) throws IOException {
@@ -198,7 +223,7 @@ final class PageReader {
    * Grab the next page.
    *
    * @return - if another page was present
-   * @throws java.io.IOException
+   * @throws IOException
    */
   public boolean next() throws IOException {
     Stopwatch timer = Stopwatch.createUnstarted();
@@ -217,11 +242,12 @@ final class PageReader {
     // TODO - figure out if we need multiple dictionary pages, I believe it may be limited to one
     // I think we are clobbering parts of the dictionary if there can be multiple pages of dictionary
     do {
-      long start=inputStream.getPos();
+      long start=dataReader.getPos();
       timer.start();
-      pageHeader = dataReader.readPageHeader();
+      pageHeader = Util.readPageHeader(dataReader);
       long timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
-      this.updateStats(pageHeader, "Page Header Read", start, timeToRead, 0,0);
+      long pageHeaderBytes=dataReader.getPos()-start;
+      this.updateStats(pageHeader, "Page Header", start, timeToRead, pageHeaderBytes, pageHeaderBytes);
       logger.trace("ParquetTrace,{},{},{},{},{},{},{},{}","Page Header Read","",
           this.parentColumnReader.parentReader.hadoopPath,
           this.parentColumnReader.columnDescriptor.toString(), start, 0, 0, timeToRead);
@@ -233,12 +259,16 @@ final class PageReader {
 
     //TODO: Handle buffer allocation exception
 
-    allocatePageData(pageHeader.getUncompressed_page_size());
+    //allocatePageData(pageHeader.getUncompressed_page_size());
     int compressedSize = pageHeader.getCompressed_page_size();
     int uncompressedSize = pageHeader.getUncompressed_page_size();
-    readPage(pageHeader, compressedSize, uncompressedSize, pageData);
+    pageData = readPage(pageHeader, compressedSize, uncompressedSize);
 
     currentPageCount = pageHeader.data_page_header.num_values;
+    final int uncompressedPageSize = pageHeader.uncompressed_page_size;
+    final Statistics<?> stats = fromParquetStatistics(pageHeader.data_page_header.getStatistics(), parentColumnReader
+        .getColumnDescriptor().getType());
+
 
     final Encoding rlEncoding = METADATA_CONVERTER.getEncoding(pageHeader.data_page_header.repetition_level_encoding);
 
@@ -370,7 +400,11 @@ final class PageReader {
   }
 
   public void clear(){
-    this.dataReader.clear();
+    try {
+      this.dataReader.close();
+    } catch (IOException e) {
+      //TODO: Throw UserException
+    }
     // Free all memory, including fixed length types. (Data is being copied for all types not just var length types)
     //if(!this.parentColumnReader.isFixedLength) {
     clearBuffers();

http://git-wip-us.apache.org/repos/asf/drill/blob/fe2334ee/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
index f095a8a..924887e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
@@ -118,12 +118,12 @@ public class ParquetRecordReader extends AbstractRecordReader {
   public ParquetReaderStats parquetReaderStats = new ParquetReaderStats();
 
   public ParquetRecordReader(FragmentContext fragmentContext,
-                             String path,
-                             int rowGroupIndex,
+      String path,
+      int rowGroupIndex,
                              long numRecordsToRead,
-                             FileSystem fs,
-                             CodecFactory codecFactory,
-                             ParquetMetadata footer,
+      FileSystem fs,
+      CodecFactory codecFactory,
+      ParquetMetadata footer,
                              List<SchemaPath> columns,
                              ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus)
                              throws ExecutionSetupException {
@@ -316,11 +316,11 @@ public class ParquetRecordReader extends AbstractRecordReader {
       columnsToScan++;
       int dataTypeLength = getDataTypeLength(column, se);
       if (dataTypeLength == -1) {
-        allFieldsFixedLength = false;
-      } else {
+          allFieldsFixedLength = false;
+        } else {
         bitWidthAllFixedFields += dataTypeLength;
+        }
       }
-    }
 //    rowGroupOffset = footer.getBlocks().get(rowGroupIndex).getColumns().get(0).getFirstDataPageOffset();
 
     if (columnsToScan != 0  && allFieldsFixedLength) {

http://git-wip-us.apache.org/repos/asf/drill/blob/fe2334ee/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
new file mode 100644
index 0000000..6aa968a
--- /dev/null
+++ b/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);
+    // The DFS can return *more* bytes than requested if the capacity of the buffer is greater.
+    // i.e 'n' can be greater than bytes requested which is pretty stupid and violates
+    // the API contract; but we still have to deal with it. So we make sure the size of the
+    // buffer is exactly the same as the number of bytes requested
+    int bytesRead = -1;
+    int nBytes = 0;
+    if (bytesToRead > 0) {
+      try {
+        nBytes = CompatibilityUtil.getBuf(getInputStream(), directBuffer, bytesToRead);
+      } catch (Exception e) {
+        logger.error("Error reading from stream {}. Error was : {}", this.streamId, e.getMessage());
+      }
+      if (nBytes > 0) {
+        buffer.writerIndex(nBytes);
+        this.count = nBytes + this.curPosInBuffer;
+        this.curPosInStream = getInputStream().getPos();
+        bytesRead = nBytes;
+        logger.trace(
+            "Stream: {}, StartOffset: {}, TotalByteSize: {}, BufferSize: {}, BytesRead: {}, Count: {}, " +
+            "CurPosInStream: {}, CurPosInBuffer: {}",
+            this.streamId, this.startOffset, this.totalByteSize, this.bufSize, bytesRead, this.count,
+            this.curPosInStream, this.curPosInBuffer);
+      }
+    }
+    return this.count - this.curPosInBuffer;
+  }
+
+  // Reads from the internal Buffer into the output buffer
+  // May read less than the requested size if the remaining data in the buffer
+  // is less than the requested amount
+  private int readInternal(DrillBuf buf, int off, int len) throws IOException {
+    // check how many bytes are available in the buffer.
+    int bytesAvailable = this.count - this.curPosInBuffer;
+    if (bytesAvailable <= 0) {
+      // read more
+      int bytesRead = getNextBlock();
+      if (bytesRead <= 0) { // End of stream
+        return -1;
+      }
+    }
+    bytesAvailable = this.count - this.curPosInBuffer;
+    //copy into output buffer
+    int copyBytes = bytesAvailable < len ? bytesAvailable : len;
+    getBuf().getBytes(curPosInBuffer, buf, off, copyBytes);
+    buf.writerIndex(off + copyBytes);
+    this.curPosInBuffer += copyBytes;
+
+    return copyBytes;
+  }
+
+  // Reads from the internal Buffer into the output buffer
+  // May read less than the requested size if the remaining data in the buffer
+  // is less than the requested amount
+  // Does not make a copy but returns a slice of the internal buffer.
+  // Returns null if end of stream is reached
+  private DrillBuf readInternal(int off, int len) throws IOException {
+    // check how many bytes are available in the buffer.
+    int bytesAvailable = this.count - this.curPosInBuffer;
+    if (bytesAvailable <= 0) {
+      // read more
+      int bytesRead = getNextBlock();
+      if (bytesRead <= 0) { // End of stream
+        return null;
+      }
+    }
+    bytesAvailable = this.count - this.curPosInBuffer;
+    // return a slice as the  output
+    int bytesToRead = bytesAvailable < len ? bytesAvailable : len;
+    DrillBuf newBuf = this.getBuf().slice(off, bytesToRead);
+    newBuf.retain();
+    return newBuf;
+  }
+
+  /**
+   * Implements the  <code>read</code> method of <code>InputStream</code>.
+   * returns one more byte or -1 if end of stream is reached.
+   */
+  public synchronized int read() throws IOException {
+    if (this.count - this.curPosInBuffer <= 0) {
+      int bytesRead = getNextBlock();
+      // reached end of stream
+      if (bytesRead <= 0) {
+        return -1;
+      }
+    }
+    this.curPosInBuffer++;
+    return getBuf().nioBuffer().get() & 0xff;
+  }
+
+  /**
+   Has the same contract as {@link java.io.InputStream#read(byte[], int, int)}
+   Except with DrillBuf
+   */
+  public synchronized int read(DrillBuf buf, int off, int len) throws IOException {
+    checkInputStreamState();
+    Preconditions.checkArgument((off >= 0) && (len >= 0) && (buf.capacity()) >= (off + len));
+    int bytesRead = 0;
+    do {
+      int readStart = off + bytesRead;
+      int lenToRead = len - bytesRead;
+      int nRead = readInternal(buf, readStart, lenToRead);
+      if (nRead <= 0) {// if End of stream
+        if (bytesRead == 0) { // no bytes read at all
+          return -1;
+        } else {
+          return bytesRead;
+        }
+      } else {
+        bytesRead += nRead;
+        // If the last read caused us to reach the end of stream
+        // we are done
+        InputStream input = in;
+        if (input != null && input.available() <= 0) {
+          return bytesRead;
+        }
+      }
+    } while (bytesRead < len);
+    return bytesRead;
+  }
+
+
+  @Override public int read(byte[] b) throws IOException {
+    return b.length == 1 ? read() : read(b, (int) 0, b.length);
+  }
+
+
+  @Override public int read(byte[] buf, int off, int len) throws IOException {
+    checkInputStreamState();
+    Preconditions.checkArgument((off >= 0) && (len >= 0) && (buf.length) >= (off + len));
+    int bytesRead = 0;
+    if (len == 0) {
+      return 0;
+    }
+    DrillBuf byteBuf;
+    if(len <= defaultTempBufferSize){
+      byteBuf = tempBuffer;
+    } else {
+      byteBuf = this.allocator.buffer(len);
+    }
+    do {
+      int readStart = off + bytesRead;
+      int lenToRead = len - bytesRead;
+      int nRead = readInternal(byteBuf, readStart, lenToRead);
+      if (nRead <= 0) {// if End of stream
+        if (bytesRead == 0) { // no bytes read at all
+          return -1;
+        } else {
+          return bytesRead;
+        }
+      } else {
+        byteBuf.nioBuffer().get(buf, off + bytesRead, len - bytesRead);
+        byteBuf.clear();
+        bytesRead += nRead;
+      }
+    } while (bytesRead < len);
+
+    if(len > defaultTempBufferSize){
+      byteBuf.release();
+    }
+
+    return bytesRead;
+  }
+
+
+  /**
+   Has the same contract as {@link java.io.InputStream#skip(long)}
+   * Skips upto the next n bytes.
+   * Skip may return with less than n bytes skipped
+   */
+  @Override
+  public synchronized long skip(long n) throws IOException {
+    checkInputStreamState();
+    long bytesAvailable = this.count - this.curPosInBuffer;
+    long bytesSkipped = 0;
+    if (n <= 0) {
+      return 0;
+    }
+    if (bytesAvailable <= 0) {
+      checkInputStreamState();
+      bytesAvailable = getNextBlock();
+      if (bytesAvailable <= 0) { // End of stream
+        return 0;
+      }
+    }
+    bytesSkipped = bytesAvailable < n ? bytesAvailable : n;
+    this.curPosInBuffer += bytesSkipped;
+
+    return bytesSkipped;
+  }
+
+
+  @Override
+  public synchronized int available() throws IOException {
+    checkInputStreamState();
+    int bytesAvailable = this.count - this.curPosInBuffer;
+    int underlyingAvailable = getInputStream().available();
+    int available = bytesAvailable + underlyingAvailable;
+    if (available < 0) { // overflow
+      return Integer.MAX_VALUE;
+    }
+    return available;
+  }
+
+  @Override
+  public synchronized void mark(int readlimit) {
+    throw new UnsupportedOperationException("Mark/reset is not supported.");
+  }
+
+  @Override
+  public synchronized void reset() throws IOException {
+    throw new UnsupportedOperationException("Mark/reset is not supported.");
+  }
+
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+
+  /*
+    Returns the current position from the beginning of the underlying input stream
+   */
+  public long getPos() throws IOException {
+    return curPosInBuffer+startOffset;
+  }
+
+  public boolean hasRemainder() throws IOException {
+    return available() > 0;
+  }
+
+  public void close() throws IOException {
+    DrillBuf buffer;
+    InputStream inp;
+    if ((inp = in) != null) {
+      in = null;
+      inp.close();
+    }
+    if ((buffer = this.internalBuffer) != null) {
+      synchronized (this) {
+        this.internalBuffer = null;
+        buffer.release();
+      }
+    }
+    if ((buffer = this.tempBuffer) != null) {
+      synchronized (this) {
+        this.tempBuffer = null;
+        buffer.release();
+      }
+    }
+  }
+
+  public static void main(String[] args) {
+    final DrillConfig config = DrillConfig.create();
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(config);
+    final Configuration dfsConfig = new Configuration();
+    String fileName = args[0];
+    Path filePath = new Path(fileName);
+    final int BUFSZ = 8 * 1024 * 1024;
+    try {
+      List<Footer> footers = ParquetFileReader.readFooters(dfsConfig, filePath);
+      Footer footer = (Footer) footers.iterator().next();
+      FileSystem fs = FileSystem.get(dfsConfig);
+      int rowGroupIndex = 0;
+      List<BlockMetaData> blocks = footer.getParquetMetadata().getBlocks();
+      for (BlockMetaData block : blocks) {
+        List<ColumnChunkMetaData> columns = block.getColumns();
+        for (ColumnChunkMetaData columnMetadata : columns) {
+          FSDataInputStream inputStream = fs.open(filePath);
+          long startOffset = columnMetadata.getStartingPos();
+          long totalByteSize = columnMetadata.getTotalSize();
+          String streamId = fileName + ":" + columnMetadata.toString();
+          BufferedDirectBufInputStream reader =
+              new BufferedDirectBufInputStream(inputStream, allocator, streamId, startOffset,
+                  totalByteSize, BUFSZ, true);
+          reader.init();
+          while (true) {
+            try {
+              DrillBuf buf = reader.getNext(BUFSZ - 1);
+              if (buf == null) {
+                break;
+              }
+              buf.release();
+            } catch (Exception e) {
+              e.printStackTrace();
+              break;
+            }
+          }
+          reader.close();
+        }
+      } // for each Block
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+    allocator.close();
+    return;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/fe2334ee/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/DirectBufInputStream.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/DirectBufInputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/DirectBufInputStream.java
new file mode 100644
index 0000000..71c36e6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/DirectBufInputStream.java
@@ -0,0 +1,166 @@
+/**
+ * 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.exec.memory.BufferAllocator;
+import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.parquet.hadoop.util.CompatibilityUtil;
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+
+public class DirectBufInputStream extends FilterInputStream {
+
+  private static final org.slf4j.Logger logger =
+      org.slf4j.LoggerFactory.getLogger(DirectBufInputStream.class);
+
+  protected boolean enableHints = true;
+  protected String streamId; // a name for logging purposes only
+  protected BufferAllocator allocator;
+  /**
+   * The length of the data we expect to read. The caller may, in fact,
+   * ask for more or less bytes. However this is useful for providing hints where
+   * the underlying InputStream supports hints (e.g. fadvise)
+   */
+  protected final long totalByteSize;
+
+  /**
+   * The offset in the underlying stream to start reading from
+   */
+  protected final long startOffset;
+
+  public DirectBufInputStream(InputStream in, BufferAllocator allocator, String id, long startOffset,
+      long totalByteSize, boolean enableHints) {
+    super(in);
+    Preconditions.checkArgument(startOffset >= 0);
+    Preconditions.checkArgument(totalByteSize >= 0);
+    this.streamId = id;
+    this.allocator = allocator;
+    this.startOffset = startOffset;
+    this.totalByteSize = totalByteSize;
+    this.enableHints = enableHints;
+  }
+
+  public void init() throws IOException, UnsupportedOperationException {
+    checkStreamSupportsByteBuffer();
+    if (enableHints) {
+      fadviseIfAvailable(getInputStream(), this.startOffset, this.totalByteSize);
+    }
+    getInputStream().seek(this.startOffset);
+    return;
+  }
+
+  public int read() throws IOException {
+    return getInputStream().read();
+  }
+
+  public synchronized int read(DrillBuf buf, int off, int len) throws IOException {
+    buf.clear();
+    ByteBuffer directBuffer = buf.nioBuffer(0, len);
+    int lengthLeftToRead = len;
+    while (lengthLeftToRead > 0) {
+      lengthLeftToRead -= CompatibilityUtil.getBuf(getInputStream(), directBuffer, lengthLeftToRead);
+    }
+    buf.writerIndex(len);
+    return len;
+  }
+
+  public synchronized DrillBuf getNext(int bytes) throws IOException {
+    DrillBuf b = allocator.buffer(bytes);
+    int bytesRead = read(b, 0, bytes);
+    if (bytesRead <= -1) {
+      b.release();
+      return null;
+    }
+    return b;
+  }
+
+  public long getPos() throws IOException {
+    return getInputStream().getPos();
+  }
+
+  public boolean hasRemainder() throws IOException {
+    return getInputStream().available() > 0;
+  }
+
+  protected FSDataInputStream getInputStream() throws IOException {
+    // Make sure stream is open
+    checkInputStreamState();
+    return (FSDataInputStream) in;
+  }
+
+  protected void checkInputStreamState() throws IOException {
+    if (in == null) {
+      throw new IOException("Input stream is closed.");
+    }
+  }
+
+  protected void checkStreamSupportsByteBuffer() throws UnsupportedOperationException {
+    // Check input stream supports ByteBuffer
+    if (!(in instanceof ByteBufferReadable)) {
+      throw new UnsupportedOperationException("The input stream is not ByteBuffer readable.");
+    }
+  }
+
+  protected static void fadviseIfAvailable(FSDataInputStream inputStream, long off, long n) {
+    Method readAhead;
+    final Class adviceType;
+
+    try {
+      adviceType = Class.forName("org.apache.hadoop.fs.FSDataInputStream$FadviseType");
+    } catch (ClassNotFoundException e) {
+      logger.info("Unable to call fadvise due to: {}", e.toString());
+      readAhead = null;
+      return;
+    }
+    try {
+      Class<? extends FSDataInputStream> inputStreamClass = inputStream.getClass();
+      readAhead =
+          inputStreamClass.getMethod("adviseFile", new Class[] {adviceType, long.class, long.class});
+    } catch (NoSuchMethodException e) {
+      logger.info("Unable to call fadvise due to: {}", e.toString());
+      readAhead = null;
+      return;
+    }
+    if (readAhead != null) {
+      Object[] adviceTypeValues = adviceType.getEnumConstants();
+      for (int idx = 0; idx < adviceTypeValues.length; idx++) {
+        if ((adviceTypeValues[idx]).toString().contains("SEQUENTIAL")) {
+          try {
+            readAhead.invoke(inputStream, adviceTypeValues[idx], off, n);
+          } catch (IllegalAccessException e) {
+            logger.info("Unable to call fadvise due to: {}", e.toString());
+          } catch (InvocationTargetException e) {
+            logger.info("Unable to call fadvise due to: {}", e.toString());
+          }
+          break;
+        }
+      }
+    }
+    return;
+  }
+
+
+}


[05/10] drill git commit: DRILL-5400: Fix NPE in corrupt date detection

Posted by pa...@apache.org.
DRILL-5400: Fix NPE in corrupt date detection

This closes #646


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/a459e4db
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/a459e4db
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/a459e4db

Branch: refs/heads/master
Commit: a459e4dbbc242fe0d06b221afe5f830780fc682e
Parents: ee3489c
Author: Parth Chandra <pa...@apache.org>
Authored: Fri Nov 4 11:05:40 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Nov 4 15:59:11 2016 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/store/parquet/ParquetReaderUtility.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a459e4db/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
index 470cc00..767c98d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
@@ -196,7 +196,7 @@ public class ParquetReaderUtility {
     } else {
       // Possibly an old, un-migrated Drill file, check the column statistics to see if min/max values look corrupt
       // only applies if there is a date column selected
-      if (createdBy.equals("parquet-mr")) {
+      if (createdBy == null || createdBy.equals("parquet-mr")) {
         // loop through parquet column metadata to find date columns, check for corrupt values
         return checkForCorruptDateValuesInStatistics(footer, columns, autoCorrectCorruptDates);
       } else {


[03/10] drill git commit: DRILL-4800: Parallelize column reading. Read/Decode fixed width fields in parallel Decoding var length columns in parallel Use simplified decompress method for Gzip and Snappy decompression. Avoids concurrency issue with Parquet

Posted by pa...@apache.org.
DRILL-4800: Parallelize column reading. Read/Decode fixed width fields in parallel Decoding var length columns in parallel Use simplified decompress method for Gzip and Snappy decompression. Avoids concurrency issue with Parquet decompression. (It's also faster). Stress test Parquet read write Parallel column reader is disabled by default (may perform less well under higher concurrency)


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/7f5acf8f
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/7f5acf8f
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/7f5acf8f

Branch: refs/heads/master
Commit: 7f5acf8f06f4ab2a2efc9801d322b81436794004
Parents: f9a443d
Author: Parth Chandra <pa...@apache.org>
Authored: Wed Aug 24 10:46:37 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Wed Nov 2 18:00:55 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/drill/test/DrillTest.java   |   2 +-
 .../org/apache/drill/exec/ExecConstants.java    |   3 +
 .../server/options/SystemOptionManager.java     |   1 +
 .../parquet/columnreaders/AsyncPageReader.java  |  88 +++++++++++-
 .../parquet/columnreaders/ColumnReader.java     | 113 ++++++++++++++-
 .../columnreaders/FixedByteAlignedReader.java   |   2 +-
 .../store/parquet/columnreaders/PageReader.java |   1 +
 .../columnreaders/ParquetRecordReader.java      |  34 ++++-
 .../columnreaders/VarLenBinaryReader.java       | 140 +++++++++++++++++--
 .../physical/impl/writer/TestParquetWriter.java | 109 ++++++++++-----
 .../src/test/resources/supplier_snappy.parquet  | Bin 0 -> 10467 bytes
 11 files changed, 436 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/common/src/test/java/org/apache/drill/test/DrillTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/test/DrillTest.java b/common/src/test/java/org/apache/drill/test/DrillTest.java
index 18c2c1a..ccc297d 100644
--- a/common/src/test/java/org/apache/drill/test/DrillTest.java
+++ b/common/src/test/java/org/apache/drill/test/DrillTest.java
@@ -55,7 +55,7 @@ public class DrillTest {
   static MemWatcher memWatcher;
   static String className;
 
-  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(50000);
+  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(50000*10000);
   @Rule public final TestLogReporter logOutcome = LOG_OUTCOME;
 
   @Rule public final TestRule REPEAT_RULE = TestTools.getRepeatRule(false);

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index a13fd71..5f62781 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -154,6 +154,9 @@ public interface ExecConstants {
   String PARQUET_PAGEREADER_ASYNC = "store.parquet.reader.pagereader.async";
   OptionValidator PARQUET_PAGEREADER_ASYNC_VALIDATOR = new BooleanValidator(PARQUET_PAGEREADER_ASYNC, true);
 
+  String PARQUET_COLUMNREADER_ASYNC = "store.parquet.reader.columnreader.async";
+  OptionValidator PARQUET_COLUMNREADER_ASYNC_VALIDATOR = new BooleanValidator(PARQUET_COLUMNREADER_ASYNC, false);
+
   // Use a buffering reader for parquet page reader
   String PARQUET_PAGEREADER_USE_BUFFERED_READ = "store.parquet.reader.pagereader.bufferedread";
   OptionValidator PARQUET_PAGEREADER_USE_BUFFERED_READ_VALIDATOR = new  BooleanValidator(PARQUET_PAGEREADER_USE_BUFFERED_READ, true);

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 1981d24..d803fa3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -101,6 +101,7 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       ExecConstants.PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR,
       ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR,
       ExecConstants.PARQUET_PAGEREADER_ASYNC_VALIDATOR,
+      ExecConstants.PARQUET_COLUMNREADER_ASYNC_VALIDATOR,
       ExecConstants.PARQUET_PAGEREADER_USE_BUFFERED_READ_VALIDATOR,
       ExecConstants.PARQUET_PAGEREADER_BUFFER_SIZE_VALIDATOR,
       ExecConstants.PARQUET_PAGEREADER_USE_FADVISE_VALIDATOR,

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
index 3f47f04..b2bdef3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
@@ -19,8 +19,14 @@ package org.apache.drill.exec.store.parquet.columnreaders;
 
 import com.google.common.base.Stopwatch;
 import io.netty.buffer.DrillBuf;
+import io.netty.buffer.ByteBufUtil;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DirectDecompressor;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.codec.SnappyCodec;
 import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
 import org.apache.drill.exec.util.filereader.DirectBufInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -30,8 +36,10 @@ import org.apache.parquet.format.PageHeader;
 import org.apache.parquet.format.PageType;
 import org.apache.parquet.format.Util;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.xerial.snappy.Snappy;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -152,10 +160,24 @@ class AsyncPageReader extends PageReader {
     pageDataBuf = allocateTemporaryBuffer(uncompressedSize);
     try {
       timer.start();
-      codecFactory.getDecompressor(parentColumnReader.columnChunkMetaData.getCodec())
-          .decompress(compressedData.nioBuffer(0, compressedSize), compressedSize,
-              pageDataBuf.nioBuffer(0, uncompressedSize), uncompressedSize);
-      timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
+      if (logger.isTraceEnabled()) {
+        logger.trace("Decompress (1)==> Col: {}  readPos: {}  compressed_size: {}  compressedPageData: {}",
+            parentColumnReader.columnChunkMetaData.toString(), dataReader.getPos(),
+            pageHeader.getCompressed_page_size(), ByteBufUtil.hexDump(compressedData));
+      }
+      CompressionCodecName codecName = parentColumnReader.columnChunkMetaData.getCodec();
+      ByteBuffer input = compressedData.nioBuffer(0, compressedSize);
+      ByteBuffer output = pageDataBuf.nioBuffer(0, uncompressedSize);
+      DecompressionHelper decompressionHelper = new DecompressionHelper(codecName);
+      decompressionHelper.decompress(input, compressedSize, output, uncompressedSize);
+      pageDataBuf.writerIndex(uncompressedSize);
+      if (logger.isTraceEnabled()) {
+        logger.trace(
+            "Decompress (2)==> Col: {}  readPos: {}  uncompressed_size: {}  uncompressedPageData: {}",
+            parentColumnReader.columnChunkMetaData.toString(), dataReader.getPos(),
+            pageHeader.getUncompressed_page_size(), ByteBufUtil.hexDump(pageDataBuf));
+      }
+      timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
       this.updateStats(pageHeader, "Decompress", 0, timeToRead, compressedSize, uncompressedSize);
     } catch (IOException e) {
       handleAndThrowException(e, "Error decompressing data.");
@@ -204,6 +226,13 @@ class AsyncPageReader extends PageReader {
 
     pageHeader = readStatus.getPageHeader();
     pageData = getDecompressedPageData(readStatus);
+    if (logger.isTraceEnabled()) {
+      logger.trace("AsyncPageReader: Col: {}  pageData: {}",
+          this.parentColumnReader.columnChunkMetaData.toString(), ByteBufUtil.hexDump(pageData));
+      logger.trace("AsyncPageReaderTask==> Col: {}  readPos: {}  Uncompressed_size: {}  pageData: {}",
+          parentColumnReader.columnChunkMetaData.toString(), dataReader.getPos(),
+          pageHeader.getUncompressed_page_size(), ByteBufUtil.hexDump(pageData));
+    }
 
   }
 
@@ -324,9 +353,60 @@ class AsyncPageReader extends PageReader {
         throw e;
       }
       Thread.currentThread().setName(oldname);
+      if(logger.isTraceEnabled()) {
+        logger.trace("AsyncPageReaderTask==> Col: {}  readPos: {}  bytesRead: {}  pageData: {}", parent.parentColumnReader.columnChunkMetaData.toString(),
+            parent.dataReader.getPos(), bytesRead, ByteBufUtil.hexDump(pageData));
+      }
       return readStatus;
     }
 
   }
 
+  private class DecompressionHelper {
+    final CompressionCodecName codecName;
+
+    public DecompressionHelper(CompressionCodecName codecName){
+      this.codecName = codecName;
+    }
+
+    public void decompress (ByteBuffer input, int compressedSize, ByteBuffer output, int uncompressedSize)
+        throws IOException {
+      // GZip != thread_safe, so we go off and do our own thing.
+      // The hadoop interface does not support ByteBuffer so we incur some
+      // expensive copying.
+      if (codecName == CompressionCodecName.GZIP) {
+        GzipCodec codec = new GzipCodec();
+        DirectDecompressor directDecompressor = codec.createDirectDecompressor();
+        if (directDecompressor != null) {
+          logger.debug("Using GZIP direct decompressor.");
+          directDecompressor.decompress(input, output);
+        } else {
+          logger.debug("Using GZIP (in)direct decompressor.");
+          Decompressor decompressor = codec.createDecompressor();
+          decompressor.reset();
+          byte[] inputBytes = new byte[compressedSize];
+          input.position(0);
+          input.get(inputBytes);
+          decompressor.setInput(inputBytes, 0, inputBytes.length);
+          byte[] outputBytes = new byte[uncompressedSize];
+          decompressor.decompress(outputBytes, 0, uncompressedSize);
+          output.clear();
+          output.put(outputBytes);
+        }
+      } else if (codecName == CompressionCodecName.SNAPPY) {
+        // For Snappy, just call the Snappy decompressor directly.
+        // It is thread safe. The Hadoop layers though, appear to be
+        // not quite reliable in a multithreaded environment
+        output.clear();
+        int size = Snappy.uncompress(input, output);
+        output.limit(size);
+      } else {
+        CodecFactory.BytesDecompressor decompressor = codecFactory.getDecompressor(parentColumnReader.columnChunkMetaData.getCodec());
+        decompressor.decompress(input, compressedSize, output, uncompressedSize);
+      }
+    }
+
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
index 6572c78..29e23bc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
@@ -20,6 +20,9 @@ package org.apache.drill.exec.store.parquet.columnreaders;
 import io.netty.buffer.DrillBuf;
 
 import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
@@ -77,6 +80,7 @@ public abstract class ColumnReader<V extends ValueVector> {
 
   // variables for a single read pass
   long readStartInBytes = 0, readLength = 0, readLengthInBits = 0, recordsReadInThisIteration = 0;
+  private ExecutorService threadPool;
 
   protected ColumnReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
       ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, V v, SchemaElement schemaElement) throws ExecutionSetupException {
@@ -105,13 +109,20 @@ public abstract class ColumnReader<V extends ValueVector> {
         dataTypeLengthInBits = ParquetRecordReader.getTypeLengthInBits(columnDescriptor.getType());
       }
     }
-
+    if(threadPool == null) {
+      threadPool = parentReader.getOperatorContext().getScanDecodeExecutor();
+    }
   }
 
   public int getRecordsReadInCurrentPass() {
     return valuesReadInCurrentPass;
   }
 
+  public Future<Long> processPagesAsync(long recordsToReadInThisPass){
+    Future<Long> r = threadPool.submit(new ColumnReaderProcessPagesTask(recordsToReadInThisPass));
+    return r;
+  }
+
   public void processPages(long recordsToReadInThisPass) throws IOException {
     reset();
     if(recordsToReadInThisPass>0) {
@@ -120,6 +131,8 @@ public abstract class ColumnReader<V extends ValueVector> {
 
       } while (valuesReadInCurrentPass < recordsToReadInThisPass && pageReader.hasPage());
     }
+    logger.trace("Column Reader: {} - Values read in this pass: {} - ",
+        this.getColumnDescriptor().toString(), valuesReadInCurrentPass);
     valueVec.getMutator().setValueCount(valuesReadInCurrentPass);
   }
 
@@ -150,13 +163,22 @@ public abstract class ColumnReader<V extends ValueVector> {
 
   protected abstract void readField(long recordsToRead);
 
+  /*
+  public Future<Boolean> determineSizeAsync(long recordsReadInCurrentPass,
+      Integer lengthVarFieldsInCurrentRecord) throws IOException {
+    Future<Boolean> r = threadPool.submit(
+        new ColumnReaderDetermineSizeTask(recordsReadInCurrentPass, lengthVarFieldsInCurrentRecord));
+    return r;
+  }
+  */
+
   /**
    * Determines the size of a single value in a variable column.
    *
    * Return value indicates if we have finished a row group and should stop reading
    *
    * @param recordsReadInCurrentPass
-   * @param lengthVarFieldsInCurrentRecord
+   * @ param lengthVarFieldsInCurrentRecord
    * @return - true if we should stop reading
    * @throws IOException
    */
@@ -172,7 +194,8 @@ public abstract class ColumnReader<V extends ValueVector> {
       return true;
     }
 
-    lengthVarFieldsInCurrentRecord += dataTypeLengthInBits;
+    //lengthVarFieldsInCurrentRecord += dataTypeLengthInBits;
+    lengthVarFieldsInCurrentRecord = -1;
 
     doneReading = checkVectorCapacityReached();
     if (doneReading) {
@@ -182,6 +205,11 @@ public abstract class ColumnReader<V extends ValueVector> {
     return false;
   }
 
+  protected Future<Integer> readRecordsAsync(int recordsToRead){
+    Future<Integer> r = threadPool.submit(new ColumnReaderReadRecordsTask(recordsToRead));
+    return r;
+  }
+
   protected void readRecords(int recordsToRead) {
     for (int i = 0; i < recordsToRead; i++) {
       readField(i);
@@ -211,6 +239,15 @@ public abstract class ColumnReader<V extends ValueVector> {
     return (int) (valueVec.getValueCapacity() * dataTypeLengthInBits / 8.0);
   }
 
+  public Future<Boolean> readPageAsync() {
+    Future<Boolean> f = threadPool.submit(new Callable<Boolean>() {
+      @Override public Boolean call() throws Exception {
+        return new Boolean(readPage());
+      }
+    });
+    return f;
+  }
+
   // Read a page if we need more data, returns true if we need to exit the read loop
   public boolean readPage() throws IOException {
     if (!pageReader.hasPage()
@@ -258,4 +295,74 @@ public abstract class ColumnReader<V extends ValueVector> {
     return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
   }
 
+  private class ColumnReaderProcessPagesTask implements Callable<Long> {
+
+    private final ColumnReader parent = ColumnReader.this;
+    private final long recordsToReadInThisPass;
+
+    public ColumnReaderProcessPagesTask(long recordsToReadInThisPass){
+      this.recordsToReadInThisPass = recordsToReadInThisPass;
+    }
+
+    @Override public Long call() throws IOException{
+
+      String oldname = Thread.currentThread().getName();
+      Thread.currentThread().setName(oldname+"Decode-"+this.parent.columnChunkMetaData.toString());
+
+      this.parent.processPages(recordsToReadInThisPass);
+
+      Thread.currentThread().setName(oldname);
+      return recordsToReadInThisPass;
+    }
+
+  }
+
+  /*
+  private class ColumnReaderDetermineSizeTask implements Callable<Boolean> {
+
+    private final ColumnReader parent = ColumnReader.this;
+    private final long recordsReadInCurrentPass;
+    private final Integer lengthVarFieldsInCurrentRecord;
+
+    public ColumnReaderDetermineSizeTask(long recordsReadInCurrentPass, Integer lengthVarFieldsInCurrentRecord){
+      this.recordsReadInCurrentPass = recordsReadInCurrentPass;
+      this.lengthVarFieldsInCurrentRecord = lengthVarFieldsInCurrentRecord;
+    }
+
+    @Override public Boolean call() throws IOException{
+
+      String oldname = Thread.currentThread().getName();
+      Thread.currentThread().setName(oldname+"Decode-"+this.parent.columnChunkMetaData.toString());
+
+      boolean b = this.parent.determineSize(recordsReadInCurrentPass, lengthVarFieldsInCurrentRecord);
+
+      Thread.currentThread().setName(oldname);
+      return b;
+    }
+
+  }
+  */
+
+  private class ColumnReaderReadRecordsTask implements Callable<Integer> {
+
+    private final ColumnReader parent = ColumnReader.this;
+    private final int recordsToRead;
+
+    public ColumnReaderReadRecordsTask(int recordsToRead){
+      this.recordsToRead = recordsToRead;
+    }
+
+    @Override public Integer call() throws IOException{
+
+      String oldname = Thread.currentThread().getName();
+      Thread.currentThread().setName("Decode-"+this.parent.columnChunkMetaData.toString());
+
+      this.parent.readRecords(recordsToRead);
+
+      Thread.currentThread().setName(oldname);
+      return recordsToRead;
+    }
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
index f806ee4..0416a05 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
@@ -251,4 +251,4 @@ class FixedByteAlignedReader<V extends ValueVector> extends ColumnReader<V> {
       valueVec.getMutator().setSafe(index, bytebuf.getInt(start), bytebuf.getInt(start + 4), bytebuf.getInt(start + 8));
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
index c34ebd1..0736f01 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
@@ -206,6 +206,7 @@ class PageReader {
       codecFactory.getDecompressor(parentColumnReader.columnChunkMetaData
           .getCodec()).decompress(compressedData.nioBuffer(0, compressedSize), compressedSize,
           pageDataBuf.nioBuffer(0, uncompressedSize), uncompressedSize);
+        pageDataBuf.writerIndex(uncompressedSize);
         timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
         this.updateStats(pageHeader, "Decompress", start, timeToRead, compressedSize, uncompressedSize);
       } finally {

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
index 1eca00f..4f0e3b5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
@@ -23,6 +23,8 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 
 import com.google.common.collect.ImmutableList;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -32,6 +34,7 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.ops.FragmentContext;
@@ -468,12 +471,37 @@ public class ParquetRecordReader extends AbstractRecordReader {
   }
 
  public void readAllFixedFields(long recordsToRead) throws IOException {
-
-   for (ColumnReader<?> crs : columnStatuses) {
-     crs.processPages(recordsToRead);
+   boolean useAsyncColReader =
+       fragmentContext.getOptions().getOption(ExecConstants.PARQUET_COLUMNREADER_ASYNC).bool_val;
+   if(useAsyncColReader){
+    readAllFixedFieldsParallel(recordsToRead) ;
+   } else {
+     readAllFixedFieldsiSerial(recordsToRead); ;
    }
  }
 
+  public void readAllFixedFieldsiSerial(long recordsToRead) throws IOException {
+    for (ColumnReader<?> crs : columnStatuses) {
+      crs.processPages(recordsToRead);
+    }
+  }
+
+  public void readAllFixedFieldsParallel(long recordsToRead) throws IOException {
+    ArrayList<Future<Long>> futures = Lists.newArrayList();
+    for (ColumnReader<?> crs : columnStatuses) {
+      Future<Long> f = crs.processPagesAsync(recordsToRead);
+      futures.add(f);
+    }
+    for(Future f: futures){
+      try {
+        f.get();
+      } catch (Exception e) {
+        f.cancel(true);
+        handleAndRaise(null, e);
+      }
+    }
+  }
+
   @Override
   public int next() {
     resetBatch();

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
index e03d930..c78dc7a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
@@ -17,19 +17,27 @@
  */
 package org.apache.drill.exec.store.parquet.columnreaders;
 
+import com.google.common.collect.Lists;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.vector.ValueVector;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.Future;
 
 public class VarLenBinaryReader {
 
   ParquetRecordReader parentReader;
   final List<VarLengthColumn<? extends ValueVector>> columns;
+  final boolean useAsyncTasks;
 
   public VarLenBinaryReader(ParquetRecordReader parentReader, List<VarLengthColumn<? extends ValueVector>> columns) {
     this.parentReader = parentReader;
     this.columns = columns;
+    useAsyncTasks = parentReader.getFragmentContext().getOptions()
+        .getOption(ExecConstants.PARQUET_COLUMNREADER_ASYNC).bool_val;
   }
 
   /**
@@ -43,29 +51,92 @@ public class VarLenBinaryReader {
   public long readFields(long recordsToReadInThisPass, ColumnReader<?> firstColumnStatus) throws IOException {
 
     long recordsReadInCurrentPass = 0;
-    int lengthVarFieldsInCurrentRecord;
-    long totalVariableLengthData = 0;
-    boolean exitLengthDeterminingLoop = false;
+
     // write the first 0 offset
     for (VarLengthColumn<?> columnReader : columns) {
       columnReader.reset();
     }
 
+    //if(useAsyncTasks){
+    //  recordsReadInCurrentPass = determineSizesParallel(recordsToReadInThisPass);
+    //} else {
+      recordsReadInCurrentPass = determineSizesSerial(recordsToReadInThisPass);
+    //}
+    if(useAsyncTasks){
+      readRecordsParallel(recordsReadInCurrentPass);
+    }else{
+      readRecordsSerial(recordsReadInCurrentPass);
+    }
+    return recordsReadInCurrentPass;
+  }
+
+
+  private long determineSizesSerial(long recordsToReadInThisPass) throws IOException {
+    int lengthVarFieldsInCurrentRecord = 0;
+    boolean exitLengthDeterminingLoop = false;
+    long totalVariableLengthData = 0;
+    long recordsReadInCurrentPass = 0;
     do {
-      lengthVarFieldsInCurrentRecord = 0;
       for (VarLengthColumn<?> columnReader : columns) {
-        if ( !exitLengthDeterminingLoop ) {
-          exitLengthDeterminingLoop = columnReader.determineSize(recordsReadInCurrentPass, lengthVarFieldsInCurrentRecord);
+        if (!exitLengthDeterminingLoop) {
+          exitLengthDeterminingLoop =
+              columnReader.determineSize(recordsReadInCurrentPass, lengthVarFieldsInCurrentRecord);
         } else {
           break;
         }
       }
       // check that the next record will fit in the batch
-      if (exitLengthDeterminingLoop || (recordsReadInCurrentPass + 1) * parentReader.getBitWidthAllFixedFields() + totalVariableLengthData
-          + lengthVarFieldsInCurrentRecord > parentReader.getBatchSize()) {
+      if (exitLengthDeterminingLoop ||
+          (recordsReadInCurrentPass + 1) * parentReader.getBitWidthAllFixedFields()
+              + totalVariableLengthData + lengthVarFieldsInCurrentRecord > parentReader.getBatchSize()) {
+        break;
+      }
+      for (VarLengthColumn<?> columnReader : columns) {
+        columnReader.updateReadyToReadPosition();
+        columnReader.currDefLevel = -1;
+      }
+      recordsReadInCurrentPass++;
+      totalVariableLengthData += lengthVarFieldsInCurrentRecord;
+    } while (recordsReadInCurrentPass < recordsToReadInThisPass);
+
+    return recordsReadInCurrentPass;
+  }
+
+
+  public long determineSizesParallel(long recordsToReadInThisPass ) throws IOException {
+    boolean doneReading = false;
+    int lengthVarFieldsInCurrentRecord = 0;
+    boolean exitLengthDeterminingLoop = false;
+    long totalVariableLengthData = 0;
+    long recordsReadInCurrentPass = 0;
+
+    do {
+    doneReading = readPagesParallel();
+
+    if (!doneReading) {
+      lengthVarFieldsInCurrentRecord = 0;
+      for (VarLengthColumn<?> columnReader : columns) {
+        doneReading = columnReader.processPageData((int) recordsReadInCurrentPass);
+        if(doneReading) {
+          break;
+        }
+        lengthVarFieldsInCurrentRecord += columnReader.dataTypeLengthInBits;
+        doneReading = columnReader.checkVectorCapacityReached();
+        if(doneReading) {
+          break;
+        }
+      }
+    }
+
+    exitLengthDeterminingLoop = doneReading;
+
+      // check that the next record will fit in the batch
+      if (exitLengthDeterminingLoop ||
+          (recordsReadInCurrentPass + 1) * parentReader.getBitWidthAllFixedFields()
+              + totalVariableLengthData + lengthVarFieldsInCurrentRecord > parentReader.getBatchSize()) {
         break;
       }
-      for (VarLengthColumn<?> columnReader : columns ) {
+      for (VarLengthColumn<?> columnReader : columns) {
         columnReader.updateReadyToReadPosition();
         columnReader.currDefLevel = -1;
       }
@@ -73,13 +144,60 @@ public class VarLenBinaryReader {
       totalVariableLengthData += lengthVarFieldsInCurrentRecord;
     } while (recordsReadInCurrentPass < recordsToReadInThisPass);
 
+    return recordsReadInCurrentPass;
+  }
+
+  public boolean readPagesParallel() {
+
+    boolean isDone = false;
+    ArrayList<Future<Boolean>> futures = Lists.newArrayList();
+    for (VarLengthColumn<?> columnReader : columns) {
+      Future<Boolean> f = columnReader.readPageAsync();
+      futures.add(f);
+    }
+    for (Future<Boolean> f : futures) {
+      try {
+        isDone = isDone || f.get().booleanValue();
+      } catch (Exception e) {
+        f.cancel(true);
+        handleAndRaise(null, e);
+      }
+    }
+    return isDone;
+  }
+
+
+  private void readRecordsSerial(long recordsReadInCurrentPass) {
     for (VarLengthColumn<?> columnReader : columns) {
       columnReader.readRecords(columnReader.pageReader.valuesReadyToRead);
     }
     for (VarLengthColumn<?> columnReader : columns) {
-      columnReader.valueVec.getMutator().setValueCount((int) recordsReadInCurrentPass);
+      columnReader.valueVec.getMutator().setValueCount((int)recordsReadInCurrentPass);
     }
-    return recordsReadInCurrentPass;
+  }
+
+  private void readRecordsParallel(long recordsReadInCurrentPass){
+    ArrayList<Future<Integer>> futures = Lists.newArrayList();
+    for (VarLengthColumn<?> columnReader : columns) {
+      Future<Integer> f = columnReader.readRecordsAsync(columnReader.pageReader.valuesReadyToRead);
+      futures.add(f);
+    }
+    for (Future f : futures) {
+      try {
+        f.get();
+      } catch (Exception e) {
+        f.cancel(true);
+        handleAndRaise(null, e);
+      }
+    }
+    for (VarLengthColumn<?> columnReader : columns) {
+      columnReader.valueVec.getMutator().setValueCount((int)recordsReadInCurrentPass);
+    }
+  }
+
+  protected void handleAndRaise(String s, Exception e) {
+    String message = "Error in parquet record reader.\nMessage: " + s;
+    throw new DrillRuntimeException(message, e);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
index 4556b4e..56b94d7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
@@ -25,7 +25,9 @@ import java.io.File;
 import java.io.FileWriter;
 import java.math.BigDecimal;
 import java.sql.Date;
+import java.util.Arrays;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -38,13 +40,12 @@ import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.fn.interp.TestConstantFolding;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
-import org.apache.drill.exec.store.parquet.ParquetRecordWriter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.parquet.format.converter.ParquetMetadataConverter;
-import org.apache.parquet.format.converter.ParquetMetadataConverter.MetadataFilter;
+import org.apache.log4j.Level;
+import org.apache.parquet.Log;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.joda.time.DateTime;
@@ -55,10 +56,19 @@ import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestParquetWriter extends BaseTestQuery {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestParquetWriter.class);
 
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][] { {100} });
+  }
+
   @Rule
   public TemporaryFolder folder = new TemporaryFolder();
   static FileSystem fs;
@@ -106,6 +116,9 @@ public class TestParquetWriter extends BaseTestQuery {
 
   private String allTypesTable = "cp.`/parquet/alltypes.json`";
 
+  @Parameterized.Parameter
+  public int repeat = 1;
+
   @BeforeClass
   public static void initFs() throws Exception {
     Configuration conf = new Configuration();
@@ -360,7 +373,7 @@ public class TestParquetWriter extends BaseTestQuery {
     runTestAndValidate("*", "*", inputTable, "nullable_test");
   }
 
-  @Ignore("Binary file too large for version control, TODO - make available on S3 bucket or similar service")
+  @Ignore("Test file not available")
   @Test
   public void testBitError_Drill_2031() throws Exception {
     compareParquetReadersHyperVector("*", "dfs.`/tmp/wide2/0_0_3.parquet`");
@@ -372,8 +385,7 @@ public class TestParquetWriter extends BaseTestQuery {
         "cast(salary as decimal(24,2)) as decimal24, cast(salary as decimal(38,2)) as decimal38";
     String validateSelection = "decimal8, decimal15, decimal24, decimal38";
     String inputTable = "cp.`employee.json`";
-    runTestAndValidate(selection, validateSelection, inputTable,
-        "parquet_decimal");
+    runTestAndValidate(selection, validateSelection, inputTable, "parquet_decimal");
   }
 
   @Test
@@ -440,17 +452,13 @@ public class TestParquetWriter extends BaseTestQuery {
       testBuilder()
         .ordered()
         .sqlQuery(query)
-        .optionSettingQueriesForTestQuery(
-            "alter system set `store.parquet.use_new_reader` = false")
+        .optionSettingQueriesForTestQuery("alter system set `store.parquet.use_new_reader` = false")
         .sqlBaselineQuery(query)
-        .optionSettingQueriesForBaseline(
-            "alter system set `store.parquet.use_new_reader` = true")
+        .optionSettingQueriesForBaseline("alter system set `store.parquet.use_new_reader` = true")
         .build().run();
     } finally {
-      test("alter system set `%s` = %b",
-          ExecConstants.PARQUET_NEW_RECORD_READER,
-          ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR
-              .getDefault().bool_val);
+      test("alter system set `%s` = %b", ExecConstants.PARQUET_NEW_RECORD_READER,
+          ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR.getDefault().bool_val);
     }
   }
 
@@ -476,47 +484,44 @@ public class TestParquetWriter extends BaseTestQuery {
     }
   }
 
-  @Ignore
+  @Ignore("Binary file too large for version control")
   @Test
   public void testReadVoter() throws Exception {
     compareParquetReadersHyperVector("*", "dfs.`/tmp/voter.parquet`");
   }
 
-  @Ignore
+  @Ignore("Test file not available")
   @Test
   public void testReadSf_100_supplier() throws Exception {
     compareParquetReadersHyperVector("*", "dfs.`/tmp/sf100_supplier.parquet`");
   }
 
-  @Ignore
+  @Ignore("Binary file too large for version control")
   @Test
   public void testParquetRead_checkNulls_NullsFirst() throws Exception {
     compareParquetReadersColumnar("*",
         "dfs.`/tmp/parquet_with_nulls_should_sum_100000_nulls_first.parquet`");
   }
 
-  @Ignore
+  @Ignore("Test file not available")
   @Test
   public void testParquetRead_checkNulls() throws Exception {
-    compareParquetReadersColumnar("*",
-        "dfs.`/tmp/parquet_with_nulls_should_sum_100000.parquet`");
+    compareParquetReadersColumnar("*", "dfs.`/tmp/parquet_with_nulls_should_sum_100000.parquet`");
   }
 
-  @Ignore
+  @Ignore("Binary file too large for version control")
   @Test
   public void test958_sql() throws Exception {
-    compareParquetReadersHyperVector("ss_ext_sales_price",
-        "dfs.`/tmp/store_sales`");
+    compareParquetReadersHyperVector("ss_ext_sales_price", "dfs.`/tmp/store_sales`");
   }
 
-  @Ignore
+  @Ignore("Binary file too large for version control")
   @Test
   public void testReadSf_1_supplier() throws Exception {
-    compareParquetReadersHyperVector("*",
-        "dfs.`/tmp/orders_part-m-00001.parquet`");
+    compareParquetReadersHyperVector("*", "dfs.`/tmp/orders_part-m-00001.parquet`");
   }
 
-  @Ignore
+  @Ignore("Binary file too large for version control")
   @Test
   public void test958_sql_all_columns() throws Exception {
     compareParquetReadersHyperVector("*", "dfs.`/tmp/store_sales`");
@@ -527,13 +532,13 @@ public class TestParquetWriter extends BaseTestQuery {
 //        "dfs.`/tmp/store_sales`");
   }
 
-  @Ignore
+  @Ignore("Binary file too large for version control")
   @Test
   public void testDrill_1314() throws Exception {
     compareParquetReadersColumnar("l_partkey ", "dfs.`/tmp/drill_1314.parquet`");
   }
 
-  @Ignore
+  @Ignore("Binary file too large for version control")
   @Test
   public void testDrill_1314_all_columns() throws Exception {
     compareParquetReadersHyperVector("*", "dfs.`/tmp/drill_1314.parquet`");
@@ -542,19 +547,19 @@ public class TestParquetWriter extends BaseTestQuery {
         "dfs.`/tmp/drill_1314.parquet`");
   }
 
-  @Ignore
+  @Ignore("Test file not available")
   @Test
   public void testParquetRead_checkShortNullLists() throws Exception {
     compareParquetReadersColumnar("*", "dfs.`/tmp/short_null_lists.parquet`");
   }
 
-  @Ignore
+  @Ignore("Test file not available")
   @Test
   public void testParquetRead_checkStartWithNull() throws Exception {
     compareParquetReadersColumnar("*", "dfs.`/tmp/start_with_null.parquet`");
   }
 
-  @Ignore
+  @Ignore("Binary file too large for version control")
   @Test
   public void testParquetReadWebReturns() throws Exception {
     compareParquetReadersColumnar("wr_returning_customer_sk", "dfs.`/tmp/web_returns`");
@@ -753,7 +758,7 @@ public class TestParquetWriter extends BaseTestQuery {
       compareParquetReadersColumnar("field_impala_ts", "cp.`parquet/int96_impala_1.parquet`");
     } finally {
       test("alter session reset %s", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP);
-    }
+  }
   }
 
   /*
@@ -823,7 +828,8 @@ public class TestParquetWriter extends BaseTestQuery {
   */
   @Test
   public void testHiveParquetTimestampAsInt96_compare() throws Exception {
-    compareParquetReadersColumnar("convert_from(timestamp_field, 'TIMESTAMP_IMPALA')", "cp.`parquet/part1/hive_all_types.parquet`");
+    compareParquetReadersColumnar("convert_from(timestamp_field, 'TIMESTAMP_IMPALA')",
+        "cp.`parquet/part1/hive_all_types.parquet`");
   }
 
   /*
@@ -923,5 +929,40 @@ public class TestParquetWriter extends BaseTestQuery {
     }
   }
 
+  @Test
+  public void testTPCHReadWriteRunRepeated() throws Exception {
+    for (int i = 1; i <= repeat; i++) {
+      if(i%100 == 0) {
+        System.out.println("\n\n Iteration : "+i +"\n");
+      }
+      testTPCHReadWriteGzip();
+      testTPCHReadWriteSnappy();
+    }
+  }
+
+  @Test
+  public void testTPCHReadWriteGzip() throws Exception {
+    try {
+      test(String.format("alter session set `%s` = 'gzip'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE));
+      String inputTable = "cp.`tpch/supplier.parquet`";
+//      runTestAndValidate("s_suppkey, s_nationkey, s_acctbal", "s_suppkey, s_nationkey, s_acctbal", inputTable, "suppkey_parquet_dict_gzip");
+        runTestAndValidate("*", "*", inputTable, "suppkey_parquet_dict_gzip");
+    } finally {
+      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE_VALIDATOR.getDefault().string_val));
+    }
+  }
+
+  @Test
+  public void testTPCHReadWriteSnappy() throws Exception {
+    try {
+      test(String.format("alter session set `%s` = 'snappy'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE));
+      String inputTable = "cp.`supplier_snappy.parquet`";
+      //      runTestAndValidate("s_suppkey, s_nationkey, s_acctbal", "s_suppkey, s_nationkey, s_acctbal", inputTable, "suppkey_parquet_dict_gzip");
+      runTestAndValidate("*", "*", inputTable, "suppkey_parquet_dict_snappy");
+    } finally {
+      test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE_VALIDATOR.getDefault().string_val));
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/7f5acf8f/exec/java-exec/src/test/resources/supplier_snappy.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/supplier_snappy.parquet b/exec/java-exec/src/test/resources/supplier_snappy.parquet
new file mode 100644
index 0000000..5a01d9a
Binary files /dev/null and b/exec/java-exec/src/test/resources/supplier_snappy.parquet differ


[08/10] drill git commit: DRILL-1950: Parquet rowgroup level filter pushdown in query planning time.

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java b/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
index adada23..70d31b1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
@@ -1090,7 +1090,7 @@ public class TestUnionAll extends BaseTestQuery{
 
     final String query = String.format("SELECT o_custkey FROM \n" +
         " (select o1.o_custkey from dfs_test.`%s` o1 inner join dfs_test.`%s` o2 on o1.o_orderkey = o2.o_custkey) \n" +
-        " Union All SELECT o_custkey FROM dfs_test.`%s` where o_custkey < 10", l, r, l);
+        " Union All SELECT o_custkey FROM dfs_test.`%s` where o_custkey > 10", l, r, l);
 
     // Validate the plan
     final String[] expectedPlan = {"(?s)UnionExchange.*UnionAll.*HashJoin.*"};

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
index c13dc48..bfecf52 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
@@ -21,7 +21,13 @@ import com.codahale.metrics.MetricRegistry;
 import com.google.common.io.Files;
 import mockit.NonStrictExpectations;
 import org.apache.commons.io.FileUtils;
+import org.antlr.runtime.ANTLRStringStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.antlr.runtime.RecognitionException;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.parser.ExprLexer;
+import org.apache.drill.common.expression.parser.ExprParser;
 import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.exec.compile.CodeCompilerTestFactory;
 import org.apache.drill.exec.memory.RootAllocatorFactory;
@@ -91,4 +97,12 @@ public class ExecTest extends DrillTest {
     }};
   }
 
+  protected LogicalExpression parseExpr(String expr) throws RecognitionException {
+    final ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
+    final CommonTokenStream tokens = new CommonTokenStream(lexer);
+    final ExprParser parser = new ExprParser(tokens);
+    final ExprParser.parse_return ret = parser.parse();
+    return ret.e;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
index a8e8814..af2ee46 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
@@ -120,14 +120,6 @@ public class ExpressionTest extends ExecTest {
 
   // HELPER METHODS //
 
-  private LogicalExpression parseExpr(String expr) throws RecognitionException {
-    final ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-    final CommonTokenStream tokens = new CommonTokenStream(lexer);
-    final ExprParser parser = new ExprParser(tokens);
-    parse_return ret = parser.parse();
-    return ret.e;
-  }
-
   private String getExpressionCode(String expression, RecordBatch batch) throws Exception {
     final LogicalExpression expr = parseExpr(expression);
     final ErrorCollector error = new ErrorCollectorImpl();

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
index 4d2ad02..722d45e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
@@ -203,14 +203,6 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
     }
   }
 
-  private LogicalExpression parseExpr(String expr) throws RecognitionException {
-    final ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-    final CommonTokenStream tokens = new CommonTokenStream(lexer);
-    final ExprParser parser = new ExprParser(tokens);
-    final ExprParser.parse_return ret = parser.parse();
-    return ret.e;
-  }
-
   private ValueVector evalExprWithInterpreter(String expression, RecordBatch batch, Drillbit bit) throws Exception {
     final LogicalExpression expr = parseExpr(expression);
     final ErrorCollector error = new ErrorCollectorImpl();

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
index 6059a5b..fa28001 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
@@ -149,7 +149,7 @@ public class TestNestedLoopJoin extends PlanTestBase {
   public void testNlJoinEqualityNonScalar_2_planning() throws Exception {
     String query = String.format("select n.n_nationkey from cp.`tpch/nation.parquet` n, "
         + " dfs_test.`%s/multilevel/parquet` o "
-        + " where n.n_regionkey = o.o_orderkey and o.o_custkey < 5", TEST_RES_PATH);
+        + " where n.n_regionkey = o.o_orderkey and o.o_custkey > 5", TEST_RES_PATH);
     test("alter session set `planner.slice_target` = 1");
     test(DISABLE_HJ);
     test(DISABLE_MJ);

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
index b4a9e79..162b5bf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
@@ -265,7 +265,7 @@ public class TestFileGenerator {
       w.endBlock();
     }
     w.end(new HashMap<String, String>());
-    logger.debug("Finished generating parquet file.");
+    logger.debug("Finished generating parquet file {}", path.getName());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
new file mode 100644
index 0000000..1ad000e
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
@@ -0,0 +1,413 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet;
+
+import org.apache.drill.PlanTestBase;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.util.TestTools;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.proto.BitControl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.zookeeper.ZooDefs.OpCode.create;
+import static org.junit.Assert.assertEquals;
+
+public class TestParquetFilterPushDown extends PlanTestBase {
+
+  private static final String WORKING_PATH = TestTools.getWorkingPath();
+  private static final String TEST_RES_PATH = WORKING_PATH + "/src/test/resources";
+  private static FragmentContext fragContext;
+
+  static FileSystem fs;
+
+  @BeforeClass
+  public static void initFSAndCreateFragContext() throws Exception {
+    fragContext = new FragmentContext(bits[0].getContext(),
+        BitControl.PlanFragment.getDefaultInstance(), null, bits[0].getContext().getFunctionImplementationRegistry());
+
+    Configuration conf = new Configuration();
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "local");
+
+    fs = FileSystem.get(conf);
+  }
+
+  @AfterClass
+  public static void close() throws Exception {
+    fragContext.close();
+    fs.close();
+  }
+
+  @Test
+  // Test filter evaluation directly without go through SQL queries.
+  public void testIntPredicateWithEval() throws Exception {
+    // intTbl.parquet has only one int column
+    //    intCol : [0, 100].
+    final String filePath = String.format("%s/parquetFilterPush/intTbl/intTbl.parquet", TEST_RES_PATH);
+    ParquetMetadata footer = getParquetMetaData(filePath);
+
+    testParquetRowGroupFilterEval(footer, "intCol = 100", false);
+    testParquetRowGroupFilterEval(footer, "intCol = 0", false);
+    testParquetRowGroupFilterEval(footer, "intCol = 50", false);
+
+    testParquetRowGroupFilterEval(footer, "intCol = -1", true);
+    testParquetRowGroupFilterEval(footer, "intCol = 101", true);
+
+    testParquetRowGroupFilterEval(footer, "intCol > 100", true);
+    testParquetRowGroupFilterEval(footer, "intCol > 99", false);
+
+    testParquetRowGroupFilterEval(footer, "intCol >= 100", false);
+    testParquetRowGroupFilterEval(footer, "intCol >= 101", true);
+
+    testParquetRowGroupFilterEval(footer, "intCol < 100", false);
+    testParquetRowGroupFilterEval(footer, "intCol < 1", false);
+    testParquetRowGroupFilterEval(footer, "intCol < 0", true);
+
+    testParquetRowGroupFilterEval(footer, "intCol <= 100", false);
+    testParquetRowGroupFilterEval(footer, "intCol <= 1", false);
+    testParquetRowGroupFilterEval(footer, "intCol <= 0", false);
+    testParquetRowGroupFilterEval(footer, "intCol <= -1", true);
+
+    // "and"
+    testParquetRowGroupFilterEval(footer, "intCol > 100 and intCol  < 200", true);
+    testParquetRowGroupFilterEval(footer, "intCol > 50 and intCol < 200", false);
+    testParquetRowGroupFilterEval(footer, "intCol > 50 and intCol > 200", true); // essentially, intCol > 200
+
+    // "or"
+    testParquetRowGroupFilterEval(footer, "intCol = 150 or intCol = 160", true);
+    testParquetRowGroupFilterEval(footer, "intCol = 50 or intCol = 160", false);
+
+    //"nonExistCol" does not exist in the table. "AND" with a filter on exist column
+    testParquetRowGroupFilterEval(footer, "intCol > 100 and nonExistCol = 100", true);
+    testParquetRowGroupFilterEval(footer, "intCol > 50 and nonExistCol = 100", true); // since nonExistCol = 100 -> Unknown -> could drop.
+    testParquetRowGroupFilterEval(footer, "nonExistCol = 100 and intCol > 50", true); // since nonExistCol = 100 -> Unknown -> could drop.
+    testParquetRowGroupFilterEval(footer, "intCol > 100 and nonExistCol < 'abc'", true);
+    testParquetRowGroupFilterEval(footer, "nonExistCol < 'abc' and intCol > 100", true); // nonExistCol < 'abc' hit NumberException and is ignored, but intCol >100 will say "drop".
+    testParquetRowGroupFilterEval(footer, "intCol > 50 and nonExistCol < 'abc'", false); // because nonExistCol < 'abc' hit NumberException and is ignored.
+
+    //"nonExistCol" does not exist in the table. "OR" with a filter on exist column
+    testParquetRowGroupFilterEval(footer, "intCol > 100 or nonExistCol = 100", true); // nonExistCol = 100 -> could drop.
+    testParquetRowGroupFilterEval(footer, "nonExistCol = 100 or intCol > 100", true); // nonExistCol = 100 -> could drop.
+    testParquetRowGroupFilterEval(footer, "intCol > 50 or nonExistCol < 100", false);
+    testParquetRowGroupFilterEval(footer, "nonExistCol < 100 or intCol > 50", false);
+
+    // cast function on column side (LHS)
+    testParquetRowGroupFilterEval(footer, "cast(intCol as bigint) = 100", false);
+    testParquetRowGroupFilterEval(footer, "cast(intCol as bigint) = 0", false);
+    testParquetRowGroupFilterEval(footer, "cast(intCol as bigint) = 50", false);
+    testParquetRowGroupFilterEval(footer, "cast(intCol as bigint) = 101", true);
+    testParquetRowGroupFilterEval(footer, "cast(intCol as bigint) = -1", true);
+
+    // cast function on constant side (RHS)
+    testParquetRowGroupFilterEval(footer, "intCol = cast(100 as bigint)", false);
+    testParquetRowGroupFilterEval(footer, "intCol = cast(0 as bigint)", false);
+    testParquetRowGroupFilterEval(footer, "intCol = cast(50 as bigint)", false);
+    testParquetRowGroupFilterEval(footer, "intCol = cast(101 as bigint)", true);
+    testParquetRowGroupFilterEval(footer, "intCol = cast(-1 as bigint)", true);
+
+    // cast into float4/float8
+    testParquetRowGroupFilterEval(footer, "cast(intCol as float4) = cast(101.0 as float4)", true);
+    testParquetRowGroupFilterEval(footer, "cast(intCol as float4) = cast(-1.0 as float4)", true);
+    testParquetRowGroupFilterEval(footer, "cast(intCol as float4) = cast(1.0 as float4)", false);
+
+    testParquetRowGroupFilterEval(footer, "cast(intCol as float8) = 101.0", true);
+    testParquetRowGroupFilterEval(footer, "cast(intCol as float8) = -1.0", true);
+    testParquetRowGroupFilterEval(footer, "cast(intCol as float8) = 1.0", false);
+  }
+
+  @Test
+  public void testIntPredicateAgainstAllNullColWithEval() throws Exception {
+    // intAllNull.parquet has only one int column with all values being NULL.
+    // column values statistics: num_nulls: 25, min/max is not defined
+    final String filePath = String.format("%s/parquetFilterPush/intTbl/intAllNull.parquet", TEST_RES_PATH);
+    ParquetMetadata footer = getParquetMetaData(filePath);
+
+    testParquetRowGroupFilterEval(footer, "intCol = 100", true);
+    testParquetRowGroupFilterEval(footer, "intCol = 0", true);
+    testParquetRowGroupFilterEval(footer, "intCol = -100", true);
+
+    testParquetRowGroupFilterEval(footer, "intCol > 10", true);
+    testParquetRowGroupFilterEval(footer, "intCol >= 10", true);
+
+    testParquetRowGroupFilterEval(footer, "intCol < 10", true);
+    testParquetRowGroupFilterEval(footer, "intCol <= 10", true);
+  }
+
+  @Test
+  public void testDatePredicateAgainstDrillCTAS1_8WithEval() throws Exception {
+    // The parquet file is created on drill 1.8.0 with DRILL CTAS:
+    //   create table dfs.tmp.`dateTblCorrupted/t1` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-01' and date '1992-01-03';
+
+    final String filePath = String.format("%s/parquetFilterPush/dateTblCorrupted/t1/0_0_0.parquet", TEST_RES_PATH);
+    ParquetMetadata footer = getParquetMetaData(filePath);
+
+    testDatePredicateAgainstDrillCTASHelper(footer);
+  }
+
+  @Test
+  public void testDatePredicateAgainstDrillCTASPost1_8WithEval() throws Exception {
+    // The parquet file is created on drill 1.9.0-SNAPSHOT (commit id:03e8f9f3e01c56a9411bb4333e4851c92db6e410) with DRILL CTAS:
+    //   create table dfs.tmp.`dateTbl1_9/t1` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-01' and date '1992-01-03';
+
+    final String filePath = String.format("%s/parquetFilterPush/dateTbl1_9/t1/0_0_0.parquet", TEST_RES_PATH);
+    ParquetMetadata footer = getParquetMetaData(filePath);
+
+    testDatePredicateAgainstDrillCTASHelper(footer);
+  }
+
+
+  private void testDatePredicateAgainstDrillCTASHelper(ParquetMetadata footer) throws Exception{
+    testParquetRowGroupFilterEval(footer, "o_orderdate = cast('1992-01-01' as date)", false);
+    testParquetRowGroupFilterEval(footer, "o_orderdate = cast('1991-12-31' as date)", true);
+
+    testParquetRowGroupFilterEval(footer, "o_orderdate >= cast('1991-12-31' as date)", false);
+    testParquetRowGroupFilterEval(footer, "o_orderdate >= cast('1992-01-03' as date)", false);
+    testParquetRowGroupFilterEval(footer, "o_orderdate >= cast('1992-01-04' as date)", true);
+
+    testParquetRowGroupFilterEval(footer, "o_orderdate > cast('1992-01-01' as date)", false);
+    testParquetRowGroupFilterEval(footer, "o_orderdate > cast('1992-01-03' as date)", true);
+
+    testParquetRowGroupFilterEval(footer, "o_orderdate <= cast('1992-01-01' as date)", false);
+    testParquetRowGroupFilterEval(footer, "o_orderdate <= cast('1991-12-31' as date)", true);
+
+    testParquetRowGroupFilterEval(footer, "o_orderdate < cast('1992-01-02' as date)", false);
+    testParquetRowGroupFilterEval(footer, "o_orderdate < cast('1992-01-01' as date)", true);
+  }
+
+  @Test
+  public void testTimeStampPredicateWithEval() throws Exception {
+    // Table dateTblCorrupted is created by CTAS in drill 1.8.0.
+    //    create table dfs.tmp.`tsTbl/t1` as select DATE_ADD(cast(o_orderdate as date), INTERVAL '0 10:20:30' DAY TO SECOND) as o_ordertimestamp from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-01' and date '1992-01-03';
+    final String filePath = String.format("%s/parquetFilterPush/tsTbl/t1/0_0_0.parquet", TEST_RES_PATH);
+    ParquetMetadata footer = getParquetMetaData(filePath);
+
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp = cast('1992-01-01 10:20:30' as timestamp)", false);
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp = cast('1992-01-01 10:20:29' as timestamp)", true);
+
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp >= cast('1992-01-01 10:20:29' as timestamp)", false);
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp >= cast('1992-01-03 10:20:30' as timestamp)", false);
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp >= cast('1992-01-03 10:20:31' as timestamp)", true);
+
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp > cast('1992-01-03 10:20:29' as timestamp)", false);
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp > cast('1992-01-03 10:20:30' as timestamp)", true);
+
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp <= cast('1992-01-01 10:20:30' as timestamp)", false);
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp <= cast('1992-01-01 10:20:29' as timestamp)", true);
+
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp < cast('1992-01-01 10:20:31' as timestamp)", false);
+    testParquetRowGroupFilterEval(footer, "o_ordertimestamp < cast('1992-01-01 10:20:30' as timestamp)", true);
+
+  }
+
+  @Test
+  // Test against parquet files from Drill CTAS post 1.8.0 release.
+  public void testDatePredicateAgaistDrillCTASPost1_8() throws  Exception {
+    String tableName = "order_ctas";
+
+    try {
+      deleteTableIfExists(tableName);
+
+      test("use dfs_test.tmp");
+      test(String.format("create table `%s/t1` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-01' and date '1992-01-03'", tableName));
+      test(String.format("create table `%s/t2` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-04' and date '1992-01-06'", tableName));
+      test(String.format("create table `%s/t3` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-07' and date '1992-01-09'", tableName));
+
+      final String query1 = "select o_orderdate from dfs_test.tmp.order_ctas where o_orderdate = date '1992-01-01'";
+      testParquetFilterPD(query1, 9, 1, false);
+
+      final String query2 = "select o_orderdate from dfs_test.tmp.order_ctas where o_orderdate < date '1992-01-01'";
+      testParquetFilterPD(query2, 0, 1, false);
+
+      final String query3 = "select o_orderdate from dfs_test.tmp.order_ctas where o_orderdate between date '1992-01-01' and date '1992-01-03'";
+      testParquetFilterPD(query3, 22, 1, false);
+
+      final String query4 = "select o_orderdate from dfs_test.tmp.order_ctas where o_orderdate between date '1992-01-01' and date '1992-01-04'";
+      testParquetFilterPD(query4, 33, 2, false);
+
+      final String query5 = "select o_orderdate from dfs_test.tmp.order_ctas where o_orderdate between date '1992-01-01' and date '1992-01-06'";
+      testParquetFilterPD(query5, 49, 2, false);
+
+      final String query6 = "select o_orderdate from dfs_test.tmp.order_ctas where o_orderdate > date '1992-01-10'";
+      testParquetFilterPD(query6, 0, 1, false);
+
+      // Test parquet files with metadata cache files available.
+      // Now, create parquet metadata cache files, and run the above queries again. Flag "usedMetadataFile" should be true.
+      test(String.format("refresh table metadata %s", tableName));
+
+      testParquetFilterPD(query1, 9, 1, true);
+
+      testParquetFilterPD(query2, 0, 1, true);
+
+      testParquetFilterPD(query3, 22, 1, true);
+
+      testParquetFilterPD(query4, 33, 2, true);
+
+      testParquetFilterPD(query5, 49, 2, true);
+
+      testParquetFilterPD(query6, 0, 1, true);
+    } finally {
+      deleteTableIfExists(tableName);
+    }
+  }
+
+  @Test
+  public void testParquetFilterPDOptionsDisabled() throws Exception {
+    final String tableName = "order_ctas";
+
+    try {
+      deleteTableIfExists(tableName);
+
+      test("alter session set `" + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY  + "` = false");
+
+      test("use dfs_test.tmp");
+      test(String.format("create table `%s/t1` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-01' and date '1992-01-03'", tableName));
+      test(String.format("create table `%s/t2` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-04' and date '1992-01-06'", tableName));
+      test(String.format("create table `%s/t3` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-07' and date '1992-01-09'", tableName));
+
+      final String query1 = "select o_orderdate from dfs_test.tmp.order_ctas where o_orderdate = date '1992-01-01'";
+      testParquetFilterPD(query1, 9, 3, false);
+
+    } finally {
+      test("alter session set `" + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY  + "` = " + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING.getDefault().bool_val);
+      deleteTableIfExists(tableName);
+    }
+  }
+
+  @Test
+  public void testParquetFilterPDOptionsThreshold() throws Exception {
+    final String tableName = "order_ctas";
+
+    try {
+      deleteTableIfExists(tableName);
+
+      test("alter session set `" + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY  + "` = 2 ");
+
+      test("use dfs_test.tmp");
+      test(String.format("create table `%s/t1` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-01' and date '1992-01-03'", tableName));
+      test(String.format("create table `%s/t2` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-04' and date '1992-01-06'", tableName));
+      test(String.format("create table `%s/t3` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-07' and date '1992-01-09'", tableName));
+
+      final String query1 = "select o_orderdate from dfs_test.tmp.order_ctas where o_orderdate = date '1992-01-01'";
+      testParquetFilterPD(query1, 9, 3, false);
+
+    } finally {
+      test("alter session set `" + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY  + "` = " + PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD.getDefault().num_val);
+      deleteTableIfExists(tableName);
+    }
+  }
+
+  @Test
+  public void testDatePredicateAgainstCorruptedDateCol() throws Exception {
+    // Table dateTblCorrupted is created by CTAS in drill 1.8.0. Per DRILL-4203, the date column is shifted by some value.
+    // The CTAS are the following, then copy to drill test resource directory.
+    //    create table dfs.tmp.`dateTblCorrupted/t1` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-01' and date '1992-01-03';
+    //    create table dfs.tmp.`dateTblCorrupted/t2` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-04' and date '1992-01-06';
+    //    create table dfs.tmp.`dateTblCorrupted/t3` as select cast(o_orderdate as date) as o_orderdate from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-07' and date '1992-01-09';
+
+    final String query1 = String.format("select o_orderdate from dfs_test.`%s/parquetFilterPush/dateTblCorrupted` where o_orderdate = date '1992-01-01'", TEST_RES_PATH);
+    testParquetFilterPD(query1, 9, 1, false);
+
+    final String query2 = String.format("select o_orderdate from dfs_test.`%s/parquetFilterPush/dateTblCorrupted` where o_orderdate < date '1992-01-01'", TEST_RES_PATH);
+    testParquetFilterPD(query2, 0, 1, false);
+
+    final String query3 = String.format("select o_orderdate from dfs_test.`%s/parquetFilterPush/dateTblCorrupted` where o_orderdate between date '1992-01-01' and date '1992-01-03'", TEST_RES_PATH);
+    testParquetFilterPD(query3, 22, 1, false);
+
+    final String query4 = String.format("select o_orderdate from dfs_test.`%s/parquetFilterPush/dateTblCorrupted` where o_orderdate between date '1992-01-01' and date '1992-01-04'", TEST_RES_PATH);
+    testParquetFilterPD(query4, 33, 2, false);
+
+    final String query5 = String.format("select o_orderdate from dfs_test.`%s/parquetFilterPush/dateTblCorrupted` where o_orderdate between date '1992-01-01' and date '1992-01-06'", TEST_RES_PATH);
+    testParquetFilterPD(query5, 49, 2, false);
+
+    final String query6 = String.format("select o_orderdate from dfs_test.`%s/parquetFilterPush/dateTblCorrupted` where o_orderdate > date '1992-01-10'", TEST_RES_PATH);
+
+    testParquetFilterPD(query6, 0, 1, false);
+  }
+
+  @Test
+  public void testTimeStampPredicate() throws Exception {
+    // Table dateTblCorrupted is created by CTAS in drill 1.8.0.
+    //    create table dfs.tmp.`tsTbl/t1` as select DATE_ADD(cast(o_orderdate as date), INTERVAL '0 10:20:30' DAY TO SECOND) as o_ordertimestamp from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-01' and date '1992-01-03';
+    //    create table dfs.tmp.`tsTbl/t2` as select DATE_ADD(cast(o_orderdate as date), INTERVAL '0 10:20:30' DAY TO SECOND) as o_ordertimestamp from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-04' and date '1992-01-06';
+    //    create table dfs.tmp.`tsTbl/t3` as select DATE_ADD(cast(o_orderdate as date), INTERVAL '0 10:20:30' DAY TO SECOND) as o_ordertimestamp from cp.`tpch/orders.parquet` where o_orderdate between date '1992-01-07' and date '1992-01-09';
+
+    final String query1 = String.format("select o_ordertimestamp from dfs_test.`%s/parquetFilterPush/tsTbl` where o_ordertimestamp = timestamp '1992-01-01 10:20:30'", TEST_RES_PATH);
+    testParquetFilterPD(query1, 9, 1, false);
+
+    final String query2 = String.format("select o_ordertimestamp from dfs_test.`%s/parquetFilterPush/tsTbl` where o_ordertimestamp < timestamp '1992-01-01 10:20:30'", TEST_RES_PATH);
+    testParquetFilterPD(query2, 0, 1, false);
+
+    final String query3 = String.format("select o_ordertimestamp from dfs_test.`%s/parquetFilterPush/tsTbl` where o_ordertimestamp between timestamp '1992-01-01 00:00:00' and timestamp '1992-01-06 10:20:30'", TEST_RES_PATH);
+    testParquetFilterPD(query3, 49, 2, false);
+  }
+
+
+  //////////////////////////////////////////////////////////////////////////////////////////////////
+  // Some test helper functions.
+  //////////////////////////////////////////////////////////////////////////////////////////////////
+
+  private void testParquetFilterPD(final String query, int expectedRowCount, int expectedNumFiles, boolean usedMetadataFile) throws Exception{
+    int actualRowCount = testSql(query);
+    assertEquals(expectedRowCount, actualRowCount);
+    String numFilesPattern = "numFiles=" + expectedNumFiles;
+    String usedMetaPattern = "usedMetadataFile=" + usedMetadataFile;
+
+    testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern}, new String[] {});
+  }
+
+  private void testParquetRowGroupFilterEval(final ParquetMetadata footer, final String exprStr,
+      boolean canDropExpected) throws Exception{
+    final LogicalExpression filterExpr = parseExpr(exprStr);
+    testParquetRowGroupFilterEval(footer, 0, filterExpr, canDropExpected);
+  }
+
+  private void testParquetRowGroupFilterEval(final ParquetMetadata footer, final int rowGroupIndex,
+      final LogicalExpression filterExpr, boolean canDropExpected) throws Exception {
+    boolean canDrop = ParquetRGFilterEvaluator.evalFilter(filterExpr, footer, rowGroupIndex,
+        fragContext.getOptions(), fragContext);
+    Assert.assertEquals(canDropExpected, canDrop);
+  }
+
+  private ParquetMetadata getParquetMetaData(String filePathStr) throws IOException{
+    Configuration fsConf = new Configuration();
+    ParquetMetadata footer = ParquetFileReader.readFooter(fsConf, new Path(filePathStr));
+    return footer;
+  }
+
+  private static void deleteTableIfExists(String tableName) {
+    try {
+      Path path = new Path(getDfsTestTmpSchemaLocation(), tableName);
+      if (fs.exists(path)) {
+        fs.delete(path, true);
+      }
+    } catch (Exception e) {
+      // ignore exceptions.
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/resources/parquetFilterPush/dateTbl1_9/t1/0_0_0.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/dateTbl1_9/t1/0_0_0.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/dateTbl1_9/t1/0_0_0.parquet
new file mode 100644
index 0000000..bd4f8e7
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/dateTbl1_9/t1/0_0_0.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t1/0_0_0.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t1/0_0_0.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t1/0_0_0.parquet
new file mode 100644
index 0000000..0ff9bdd
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t1/0_0_0.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t2/0_0_0.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t2/0_0_0.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t2/0_0_0.parquet
new file mode 100644
index 0000000..cf28b54
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t2/0_0_0.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t3/0_0_0.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t3/0_0_0.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t3/0_0_0.parquet
new file mode 100644
index 0000000..5fd9853
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/dateTblCorrupted/t3/0_0_0.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/resources/parquetFilterPush/intTbl/intAllNull.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/intTbl/intAllNull.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/intTbl/intAllNull.parquet
new file mode 100644
index 0000000..06eb81d
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/intTbl/intAllNull.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/resources/parquetFilterPush/intTbl/intTbl.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/intTbl/intTbl.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/intTbl/intTbl.parquet
new file mode 100644
index 0000000..9943078
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/intTbl/intTbl.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t1/0_0_0.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t1/0_0_0.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t1/0_0_0.parquet
new file mode 100644
index 0000000..f0498c6
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t1/0_0_0.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t2/0_0_0.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t2/0_0_0.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t2/0_0_0.parquet
new file mode 100644
index 0000000..4da4e6b
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t2/0_0_0.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t3/0_0_0.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t3/0_0_0.parquet b/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t3/0_0_0.parquet
new file mode 100644
index 0000000..ee0c92c
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquetFilterPush/tsTbl/t3/0_0_0.parquet differ

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/logical/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/expression/ValueExpressions.java b/logical/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
index 2fd8e67..662258d 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
@@ -59,6 +59,10 @@ public class ValueExpressions {
     return new org.apache.drill.common.expression.ValueExpressions.DateExpression(date.getTimeInMillis());
   }
 
+  public static LogicalExpression getDate(long milliSecond){
+    return new org.apache.drill.common.expression.ValueExpressions.DateExpression(milliSecond);
+  }
+
   public static LogicalExpression getTime(GregorianCalendar time) {
       int millis = time.get(GregorianCalendar.HOUR_OF_DAY) * 60 * 60 * 1000 +
                    time.get(GregorianCalendar.MINUTE) * 60 * 1000 +
@@ -68,9 +72,18 @@ public class ValueExpressions {
       return new TimeExpression(millis);
   }
 
+  public static LogicalExpression getTime(int milliSeconds) {
+    return new TimeExpression(milliSeconds);
+  }
+
   public static LogicalExpression getTimeStamp(GregorianCalendar date) {
     return new org.apache.drill.common.expression.ValueExpressions.TimeStampExpression(date.getTimeInMillis());
   }
+
+  public static LogicalExpression getTimeStamp(long milliSeconds) {
+    return new org.apache.drill.common.expression.ValueExpressions.TimeStampExpression(milliSeconds);
+  }
+
   public static LogicalExpression getIntervalYear(int months) {
     return new IntervalYearExpression(months);
   }
@@ -140,6 +153,8 @@ public class ValueExpressions {
 
   public static class BooleanExpression extends ValueExpression<Boolean> {
 
+    public static final BooleanExpression TRUE = new BooleanExpression("true", ExpressionPosition.UNKNOWN);
+    public static final BooleanExpression FALSE = new BooleanExpression("false", ExpressionPosition.UNKNOWN);
 
     public BooleanExpression(String value, ExpressionPosition pos) {
       super(value, pos);

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java b/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
index af25dd7..43b3b2e 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
@@ -177,6 +177,15 @@ public class CastFunctions {
         CAST_FUNC_REPLACEMENT_NEEDED.contains(originalfunction);
   }
 
+  /**
+   * Check if a funcName is one of the cast function.
+   * @param funcName
+   * @return
+   */
+  public static boolean isCastFunction(String funcName) {
+    return TYPE2FUNC.values().contains(funcName);
+  }
+
   private static String getReplacingCastFunctionFromNonNullable(String originalCastFunction, MinorType inputType) {
     if(inputType == MinorType.VARCHAR && CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.containsKey(originalCastFunction)) {
       return CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.get(originalCastFunction);


[07/10] drill git commit: DRILL-1950: Update parquet metadata cache format to include both min/max and additional column type information.

Posted by pa...@apache.org.
DRILL-1950: Update parquet metadata cache format to include both min/max and additional column type information.

    Parquet meta cache format change:
    1. include both min/max in ColumnMetaData if column statistics is available,
    2. include precision/scale/repetitionLevel/definitionLevel in ColumnTypeMetaData (precision/scale/definitionLevel is for future use).


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/0d4319b2
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/0d4319b2
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/0d4319b2

Branch: refs/heads/master
Commit: 0d4319b25274ba4661a40766302bab318d20709b
Parents: 8ce1f03
Author: Jinfeng Ni <jn...@apache.org>
Authored: Sun Oct 30 21:30:12 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Nov 4 15:59:32 2016 -0700

----------------------------------------------------------------------
 .../drill/exec/store/parquet/Metadata.java      | 567 +++++++++++++++++--
 1 file changed, 519 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/0d4319b2/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java
index f5554c9..ead0a8f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java
@@ -47,6 +47,7 @@ import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.schema.GroupType;
 import org.apache.parquet.schema.MessageType;
 import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
 import org.apache.parquet.schema.Type;
 import org.apache.commons.lang3.tuple.Pair;
@@ -107,7 +108,7 @@ public class Metadata {
    * @return
    * @throws IOException
    */
-  public static ParquetTableMetadata_v2 getParquetTableMetadata(FileSystem fs, String path, ParquetFormatConfig formatConfig)
+  public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs, String path, ParquetFormatConfig formatConfig)
       throws IOException {
     Metadata metadata = new Metadata(fs, formatConfig);
     return metadata.getParquetTableMetadata(path);
@@ -121,7 +122,7 @@ public class Metadata {
    * @return
    * @throws IOException
    */
-  public static ParquetTableMetadata_v2 getParquetTableMetadata(FileSystem fs,
+  public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       List<FileStatus> fileStatuses, ParquetFormatConfig formatConfig) throws IOException {
     Metadata metadata = new Metadata(fs, formatConfig);
     return metadata.getParquetTableMetadata(fileStatuses);
@@ -158,12 +159,12 @@ public class Metadata {
    * @param path
    * @throws IOException
    */
-  private Pair<ParquetTableMetadata_v2, ParquetTableMetadataDirs>
+  private Pair<ParquetTableMetadata_v3, ParquetTableMetadataDirs>
   createMetaFilesRecursively(final String path) throws IOException {
     Stopwatch timer = Stopwatch.createStarted();
-    List<ParquetFileMetadata_v2> metaDataList = Lists.newArrayList();
+    List<ParquetFileMetadata_v3> metaDataList = Lists.newArrayList();
     List<String> directoryList = Lists.newArrayList();
-    ConcurrentHashMap<ColumnTypeMetadata_v2.Key, ColumnTypeMetadata_v2> columnTypeInfoSet =
+    ConcurrentHashMap<ColumnTypeMetadata_v3.Key, ColumnTypeMetadata_v3> columnTypeInfoSet =
         new ConcurrentHashMap<>();
     Path p = new Path(path);
     FileStatus fileStatus = fs.getFileStatus(p);
@@ -173,7 +174,7 @@ public class Metadata {
 
     for (final FileStatus file : fs.listStatus(p, new DrillPathFilter())) {
       if (file.isDirectory()) {
-        ParquetTableMetadata_v2 subTableMetadata = (createMetaFilesRecursively(file.getPath().toString())).getLeft();
+        ParquetTableMetadata_v3 subTableMetadata = (createMetaFilesRecursively(file.getPath().toString())).getLeft();
         metaDataList.addAll(subTableMetadata.files);
         directoryList.addAll(subTableMetadata.directories);
         directoryList.add(file.getPath().toString());
@@ -184,10 +185,10 @@ public class Metadata {
         childFiles.add(file);
       }
     }
-    ParquetTableMetadata_v2 parquetTableMetadata = new ParquetTableMetadata_v2(true);
+    ParquetTableMetadata_v3 parquetTableMetadata = new ParquetTableMetadata_v3(true);
     if (childFiles.size() > 0) {
-      List<ParquetFileMetadata_v2> childFilesMetadata =
-          getParquetFileMetadata_v2(parquetTableMetadata, childFiles);
+      List<ParquetFileMetadata_v3> childFilesMetadata =
+          getParquetFileMetadata_v3(parquetTableMetadata, childFiles);
       metaDataList.addAll(childFilesMetadata);
       // Note that we do not need to merge the columnInfo at this point. The columnInfo is already added
       // to the parquetTableMetadata.
@@ -226,7 +227,7 @@ public class Metadata {
    * @return
    * @throws IOException
    */
-  private ParquetTableMetadata_v2 getParquetTableMetadata(String path) throws IOException {
+  private ParquetTableMetadata_v3 getParquetTableMetadata(String path) throws IOException {
     Path p = new Path(path);
     FileStatus fileStatus = fs.getFileStatus(p);
     final Stopwatch watch = Stopwatch.createStarted();
@@ -234,9 +235,9 @@ public class Metadata {
     logger.info("Took {} ms to get file statuses", watch.elapsed(TimeUnit.MILLISECONDS));
     watch.reset();
     watch.start();
-    ParquetTableMetadata_v2 metadata_v1 = getParquetTableMetadata(fileStatuses);
+    ParquetTableMetadata_v3 metadata_v3 = getParquetTableMetadata(fileStatuses);
     logger.info("Took {} ms to read file metadata", watch.elapsed(TimeUnit.MILLISECONDS));
-    return metadata_v1;
+    return metadata_v3;
   }
 
   /**
@@ -246,10 +247,10 @@ public class Metadata {
    * @return
    * @throws IOException
    */
-  private ParquetTableMetadata_v2 getParquetTableMetadata(List<FileStatus> fileStatuses)
+  private ParquetTableMetadata_v3 getParquetTableMetadata(List<FileStatus> fileStatuses)
       throws IOException {
-    ParquetTableMetadata_v2 tableMetadata = new ParquetTableMetadata_v2();
-    List<ParquetFileMetadata_v2> fileMetadataList = getParquetFileMetadata_v2(tableMetadata, fileStatuses);
+    ParquetTableMetadata_v3 tableMetadata = new ParquetTableMetadata_v3();
+    List<ParquetFileMetadata_v3> fileMetadataList = getParquetFileMetadata_v3(tableMetadata, fileStatuses);
     tableMetadata.files = fileMetadataList;
     tableMetadata.directories = new ArrayList<String>();
     return tableMetadata;
@@ -262,14 +263,14 @@ public class Metadata {
    * @return
    * @throws IOException
    */
-  private List<ParquetFileMetadata_v2> getParquetFileMetadata_v2(
-      ParquetTableMetadata_v2 parquetTableMetadata_v1, List<FileStatus> fileStatuses) throws IOException {
-    List<TimedRunnable<ParquetFileMetadata_v2>> gatherers = Lists.newArrayList();
+  private List<ParquetFileMetadata_v3> getParquetFileMetadata_v3(
+      ParquetTableMetadata_v3 parquetTableMetadata_v3, List<FileStatus> fileStatuses) throws IOException {
+    List<TimedRunnable<ParquetFileMetadata_v3>> gatherers = Lists.newArrayList();
     for (FileStatus file : fileStatuses) {
-      gatherers.add(new MetadataGatherer(parquetTableMetadata_v1, file));
+      gatherers.add(new MetadataGatherer(parquetTableMetadata_v3, file));
     }
 
-    List<ParquetFileMetadata_v2> metaDataList = Lists.newArrayList();
+    List<ParquetFileMetadata_v3> metaDataList = Lists.newArrayList();
     metaDataList.addAll(TimedRunnable.run("Fetch parquet metadata", logger, gatherers, 16));
     return metaDataList;
   }
@@ -296,19 +297,19 @@ public class Metadata {
   /**
    * TimedRunnable that reads the footer from parquet and collects file metadata
    */
-  private class MetadataGatherer extends TimedRunnable<ParquetFileMetadata_v2> {
+  private class MetadataGatherer extends TimedRunnable<ParquetFileMetadata_v3> {
 
     private FileStatus fileStatus;
-    private ParquetTableMetadata_v2 parquetTableMetadata;
+    private ParquetTableMetadata_v3 parquetTableMetadata;
 
-    public MetadataGatherer(ParquetTableMetadata_v2 parquetTableMetadata, FileStatus fileStatus) {
+    public MetadataGatherer(ParquetTableMetadata_v3 parquetTableMetadata, FileStatus fileStatus) {
       this.fileStatus = fileStatus;
       this.parquetTableMetadata = parquetTableMetadata;
     }
 
     @Override
-    protected ParquetFileMetadata_v2 runInner() throws Exception {
-      return getParquetFileMetadata_v2(parquetTableMetadata, fileStatus);
+    protected ParquetFileMetadata_v3 runInner() throws Exception {
+      return getParquetFileMetadata_v3(parquetTableMetadata, fileStatus);
     }
 
     @Override
@@ -329,6 +330,41 @@ public class Metadata {
     return getOriginalType(t, path, depth + 1);
   }
 
+  private ColTypeInfo getColTypeInfo(MessageType schema, Type type, String[] path, int depth) {
+    if (type.isPrimitive()) {
+      PrimitiveType primitiveType = (PrimitiveType) type;
+      int precision = 0;
+      int scale = 0;
+      if (primitiveType.getDecimalMetadata() != null) {
+        precision = primitiveType.getDecimalMetadata().getPrecision();
+        scale = primitiveType.getDecimalMetadata().getScale();
+      }
+
+      int repetitionLevel = schema.getMaxRepetitionLevel(path);
+      int definitionLevel = schema.getMaxDefinitionLevel(path);
+
+      return new ColTypeInfo(type.getOriginalType(), precision, scale, repetitionLevel, definitionLevel);
+    }
+    Type t = ((GroupType) type).getType(path[depth]);
+    return getColTypeInfo(schema, t, path, depth + 1);
+  }
+
+  private class ColTypeInfo {
+    public OriginalType originalType;
+    public int precision;
+    public int scale;
+    public int repetitionLevel;
+    public int definitionLevel;
+
+    public ColTypeInfo(OriginalType originalType, int precision, int scale, int repetitionLevel, int definitionLevel) {
+      this.originalType = originalType;
+      this.precision = precision;
+      this.scale = scale;
+      this.repetitionLevel = repetitionLevel;
+      this.definitionLevel = definitionLevel;
+    }
+  }
+
   /**
    * Get the metadata for a single file
    *
@@ -336,18 +372,19 @@ public class Metadata {
    * @return
    * @throws IOException
    */
-  private ParquetFileMetadata_v2 getParquetFileMetadata_v2(ParquetTableMetadata_v2 parquetTableMetadata,
+  private ParquetFileMetadata_v3 getParquetFileMetadata_v3(ParquetTableMetadata_v3 parquetTableMetadata,
       FileStatus file) throws IOException {
     ParquetMetadata metadata = ParquetFileReader.readFooter(fs.getConf(), file);
     MessageType schema = metadata.getFileMetaData().getSchema();
 
-    Map<SchemaPath, OriginalType> originalTypeMap = Maps.newHashMap();
+//    Map<SchemaPath, OriginalType> originalTypeMap = Maps.newHashMap();
+    Map<SchemaPath, ColTypeInfo> colTypeInfoMap = Maps.newHashMap();
     schema.getPaths();
     for (String[] path : schema.getPaths()) {
-      originalTypeMap.put(SchemaPath.getCompoundPath(path), getOriginalType(schema, path, 0));
+      colTypeInfoMap.put(SchemaPath.getCompoundPath(path), getColTypeInfo(schema, schema, path, 0));
     }
 
-    List<RowGroupMetadata_v2> rowGroupMetadataList = Lists.newArrayList();
+    List<RowGroupMetadata_v3> rowGroupMetadataList = Lists.newArrayList();
 
     ArrayList<SchemaPath> ALL_COLS = new ArrayList<>();
     ALL_COLS.add(AbstractRecordReader.STAR_COLUMN);
@@ -355,53 +392,60 @@ public class Metadata {
     ParquetReaderUtility.DateCorruptionStatus containsCorruptDates = ParquetReaderUtility.detectCorruptDates(metadata, ALL_COLS, autoCorrectCorruptDates);
     logger.info(containsCorruptDates.toString());
     for (BlockMetaData rowGroup : metadata.getBlocks()) {
-      List<ColumnMetadata_v2> columnMetadataList = Lists.newArrayList();
+      List<ColumnMetadata_v3> columnMetadataList = Lists.newArrayList();
       long length = 0;
       for (ColumnChunkMetaData col : rowGroup.getColumns()) {
-        ColumnMetadata_v2 columnMetadata;
+        ColumnMetadata_v3 columnMetadata;
 
         boolean statsAvailable = (col.getStatistics() != null && !col.getStatistics().isEmpty());
 
         Statistics<?> stats = col.getStatistics();
         String[] columnName = col.getPath().toArray();
         SchemaPath columnSchemaName = SchemaPath.getCompoundPath(columnName);
-        ColumnTypeMetadata_v2 columnTypeMetadata =
-            new ColumnTypeMetadata_v2(columnName, col.getType(), originalTypeMap.get(columnSchemaName));
+        ColTypeInfo colTypeInfo = colTypeInfoMap.get(columnSchemaName);
+
+        ColumnTypeMetadata_v3 columnTypeMetadata =
+            new ColumnTypeMetadata_v3(columnName, col.getType(), colTypeInfo.originalType,
+                colTypeInfo.precision, colTypeInfo.scale, colTypeInfo.repetitionLevel, colTypeInfo.definitionLevel);
+
         if (parquetTableMetadata.columnTypeInfo == null) {
           parquetTableMetadata.columnTypeInfo = new ConcurrentHashMap<>();
         }
         // Save the column schema info. We'll merge it into one list
         parquetTableMetadata.columnTypeInfo
-            .put(new ColumnTypeMetadata_v2.Key(columnTypeMetadata.name), columnTypeMetadata);
+            .put(new ColumnTypeMetadata_v3.Key(columnTypeMetadata.name), columnTypeMetadata);
         if (statsAvailable) {
-          // Write stats only if minVal==maxVal. Also, we then store only maxVal
-          Object mxValue = null;
-          if (stats.genericGetMax() != null && stats.genericGetMin() != null &&
-              stats.genericGetMax().equals(stats.genericGetMin())) {
-            mxValue = stats.genericGetMax();
+          // Write stats when they are not null
+          Object minValue = null;
+          Object maxValue = null;
+          if (stats.genericGetMax() != null && stats.genericGetMin() != null ) {
+            minValue = stats.genericGetMin();
+            maxValue = stats.genericGetMax();
             if (containsCorruptDates == ParquetReaderUtility.DateCorruptionStatus.META_SHOWS_CORRUPTION
                 && columnTypeMetadata.originalType == OriginalType.DATE) {
-              mxValue = ParquetReaderUtility.autoCorrectCorruptedDate((Integer) mxValue);
+              minValue = ParquetReaderUtility.autoCorrectCorruptedDate((Integer) minValue);
+              maxValue = ParquetReaderUtility.autoCorrectCorruptedDate((Integer) maxValue);
             }
+
           }
           columnMetadata =
-              new ColumnMetadata_v2(columnTypeMetadata.name, col.getType(), mxValue, stats.getNumNulls());
+              new ColumnMetadata_v3(columnTypeMetadata.name, col.getType(), minValue, maxValue, stats.getNumNulls());
         } else {
-          columnMetadata = new ColumnMetadata_v2(columnTypeMetadata.name, col.getType(), null, null);
+          columnMetadata = new ColumnMetadata_v3(columnTypeMetadata.name, col.getType(), null, null, null);
         }
         columnMetadataList.add(columnMetadata);
         length += col.getTotalSize();
       }
 
-      RowGroupMetadata_v2 rowGroupMeta =
-          new RowGroupMetadata_v2(rowGroup.getStartingPos(), length, rowGroup.getRowCount(),
+      RowGroupMetadata_v3 rowGroupMeta =
+          new RowGroupMetadata_v3(rowGroup.getStartingPos(), length, rowGroup.getRowCount(),
               getHostAffinity(file, rowGroup.getStartingPos(), length), columnMetadataList);
 
       rowGroupMetadataList.add(rowGroupMeta);
     }
     String path = Path.getPathWithoutSchemeAndAuthority(file.getPath()).toString();
 
-    return new ParquetFileMetadata_v2(path, file.getLen(), rowGroupMetadataList);
+    return new ParquetFileMetadata_v3(path, file.getLen(), rowGroupMetadataList);
   }
 
   /**
@@ -442,13 +486,13 @@ public class Metadata {
    * @param p
    * @throws IOException
    */
-  private void writeFile(ParquetTableMetadata_v2 parquetTableMetadata, Path p) throws IOException {
+  private void writeFile(ParquetTableMetadata_v3 parquetTableMetadata, Path p) throws IOException {
     JsonFactory jsonFactory = new JsonFactory();
     jsonFactory.configure(Feature.AUTO_CLOSE_TARGET, false);
     jsonFactory.configure(JsonParser.Feature.AUTO_CLOSE_SOURCE, false);
     ObjectMapper mapper = new ObjectMapper(jsonFactory);
     SimpleModule module = new SimpleModule();
-    module.addSerializer(ColumnMetadata_v2.class, new ColumnMetadata_v2.Serializer());
+    module.addSerializer(ColumnMetadata_v3.class, new ColumnMetadata_v3.Serializer());
     mapper.registerModule(module);
     FSDataOutputStream os = fs.create(p);
     mapper.writerWithDefaultPrettyPrinter().writeValue(os, parquetTableMetadata);
@@ -487,6 +531,7 @@ public class Metadata {
     final SimpleModule serialModule = new SimpleModule();
     serialModule.addDeserializer(SchemaPath.class, new SchemaPath.De());
     serialModule.addKeyDeserializer(ColumnTypeMetadata_v2.Key.class, new ColumnTypeMetadata_v2.Key.DeSerializer());
+    serialModule.addKeyDeserializer(ColumnTypeMetadata_v3.Key.class, new ColumnTypeMetadata_v3.Key.DeSerializer());
 
     AfterburnerModule module = new AfterburnerModule();
     module.setUseOptimizedBeanDeserializer(true);
@@ -534,6 +579,7 @@ public class Metadata {
    * Check if the parquet metadata needs to be updated by comparing the modification time of the directories with
    * the modification time of the metadata file
    *
+   * @param directories
    * @param metaFilePath
    * @return
    * @throws IOException
@@ -580,7 +626,8 @@ public class Metadata {
   @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "metadata_version")
   @JsonSubTypes({
       @JsonSubTypes.Type(value = ParquetTableMetadata_v1.class, name="v1"),
-      @JsonSubTypes.Type(value = ParquetTableMetadata_v2.class, name="v2")
+      @JsonSubTypes.Type(value = ParquetTableMetadata_v2.class, name="v2"),
+      @JsonSubTypes.Type(value = ParquetTableMetadata_v3.class, name="v3")
       })
   public static abstract class ParquetTableMetadataBase {
 
@@ -596,11 +643,18 @@ public class Metadata {
 
     @JsonIgnore public abstract OriginalType getOriginalType(String[] columnName);
 
+    @JsonIgnore public abstract Integer getRepetitionLevel(String[] columnName);
+
+    @JsonIgnore public abstract Integer getDefinitionLevel(String[] columnName);
+
+    @JsonIgnore public abstract boolean isRowGroupPrunable();
+
     @JsonIgnore public abstract ParquetTableMetadataBase clone();
 
     @JsonIgnore public abstract String getDrillVersion();
 
     @JsonIgnore public abstract boolean isDateCorrect();
+
   }
 
   public static abstract class ParquetFileMetadata {
@@ -632,6 +686,8 @@ public class Metadata {
 
     public abstract boolean hasSingleValue();
 
+    public abstract Object getMinValue();
+
     public abstract Object getMaxValue();
 
     /**
@@ -710,6 +766,21 @@ public class Metadata {
       return null;
     }
 
+    @JsonIgnore @Override
+    public Integer getRepetitionLevel(String[] columnName) {
+      return null;
+    }
+
+    @JsonIgnore @Override
+    public Integer getDefinitionLevel(String[] columnName) {
+      return null;
+    }
+
+    @JsonIgnore @Override
+    public boolean isRowGroupPrunable() {
+      return false;
+    }
+
     @JsonIgnore @Override public ParquetTableMetadataBase clone() {
       return new ParquetTableMetadata_v1(files, directories);
     }
@@ -723,6 +794,7 @@ public class Metadata {
     public boolean isDateCorrect() {
       return false;
     }
+
   }
 
 
@@ -908,6 +980,10 @@ public class Metadata {
       return (max != null && min != null && max.equals(min));
     }
 
+    @Override public Object getMinValue() {
+      return min;
+    }
+
     @Override public Object getMaxValue() {
       return max;
     }
@@ -982,6 +1058,21 @@ public class Metadata {
       return getColumnTypeInfo(columnName).originalType;
     }
 
+    @JsonIgnore @Override
+    public Integer getRepetitionLevel(String[] columnName) {
+      return null;
+    }
+
+    @JsonIgnore @Override
+    public Integer getDefinitionLevel(String[] columnName) {
+      return null;
+    }
+
+    @JsonIgnore @Override
+    public boolean isRowGroupPrunable() {
+      return false;
+    }
+
     @JsonIgnore @Override public ParquetTableMetadataBase clone() {
       return new ParquetTableMetadata_v2(files, directories, columnTypeInfo);
     }
@@ -1197,6 +1288,10 @@ public class Metadata {
       return (mxValue != null);
     }
 
+    @Override public Object getMinValue() {
+      return mxValue;
+    }
+
     @Override public Object getMaxValue() {
       return mxValue;
     }
@@ -1251,5 +1346,381 @@ public class Metadata {
 
   }
 
+  /**
+   * Struct which contains the metadata for an entire parquet directory structure
+   *
+   * Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel
+   */
+  @JsonTypeName("v3") public static class ParquetTableMetadata_v3 extends ParquetTableMetadataBase {
+    /*
+     ColumnTypeInfo is schema information from all the files and row groups, merged into
+     one. To get this info, we pass the ParquetTableMetadata object all the way dow to the
+     RowGroup and the column type is built there as it is read from the footer.
+     */
+    @JsonProperty public ConcurrentHashMap<ColumnTypeMetadata_v3.Key, ColumnTypeMetadata_v3> columnTypeInfo;
+    @JsonProperty List<ParquetFileMetadata_v3> files;
+    @JsonProperty List<String> directories;
+    @JsonProperty String drillVersion;
+    @JsonProperty boolean isDateCorrect;
+
+    public ParquetTableMetadata_v3() {
+      super();
+    }
+
+    public ParquetTableMetadata_v3(boolean isDateCorrect) {
+      this.drillVersion = DrillVersionInfo.getVersion();
+      this.isDateCorrect = isDateCorrect;
+    }
+
+    public ParquetTableMetadata_v3(ParquetTableMetadataBase parquetTable,
+        List<ParquetFileMetadata_v3> files, List<String> directories) {
+      this.files = files;
+      this.directories = directories;
+      this.columnTypeInfo = ((ParquetTableMetadata_v3) parquetTable).columnTypeInfo;
+      this.isDateCorrect = true;
+    }
+
+    public ParquetTableMetadata_v3(List<ParquetFileMetadata_v3> files, List<String> directories,
+        ConcurrentHashMap<ColumnTypeMetadata_v3.Key, ColumnTypeMetadata_v3> columnTypeInfo) {
+      this.files = files;
+      this.directories = directories;
+      this.columnTypeInfo = columnTypeInfo;
+    }
+
+    public ColumnTypeMetadata_v3 getColumnTypeInfo(String[] name) {
+      return columnTypeInfo.get(new ColumnTypeMetadata_v3.Key(name));
+    }
+
+    @JsonIgnore @Override public List<String> getDirectories() {
+      return directories;
+    }
+
+    @JsonIgnore @Override public List<? extends ParquetFileMetadata> getFiles() {
+      return files;
+    }
+
+    @JsonIgnore @Override public void assignFiles(List<? extends ParquetFileMetadata> newFiles) {
+      this.files = (List<ParquetFileMetadata_v3>) newFiles;
+    }
+
+    @Override public boolean hasColumnMetadata() {
+      return true;
+    }
+
+    @JsonIgnore @Override public PrimitiveTypeName getPrimitiveType(String[] columnName) {
+      return getColumnTypeInfo(columnName).primitiveType;
+    }
+
+    @JsonIgnore @Override public OriginalType getOriginalType(String[] columnName) {
+      return getColumnTypeInfo(columnName).originalType;
+    }
+
+    @JsonIgnore @Override
+    public Integer getRepetitionLevel(String[] columnName) {
+      return getColumnTypeInfo(columnName).repetitionLevel;
+    }
+
+    @JsonIgnore @Override
+    public Integer getDefinitionLevel(String[] columnName) {
+      return getColumnTypeInfo(columnName).definitionLevel;
+    }
+
+    @JsonIgnore @Override
+    public boolean isRowGroupPrunable() {
+      return true;
+    }
+
+    @JsonIgnore @Override public ParquetTableMetadataBase clone() {
+      return new ParquetTableMetadata_v3(files, directories, columnTypeInfo);
+    }
+
+    @JsonIgnore @Override
+    public String getDrillVersion() {
+      return drillVersion;
+    }
+
+    @JsonIgnore @Override public boolean isDateCorrect() {
+      return isDateCorrect;
+    }
+
+  }
+
+
+  /**
+   * Struct which contains the metadata for a single parquet file
+   */
+  public static class ParquetFileMetadata_v3 extends ParquetFileMetadata {
+    @JsonProperty public String path;
+    @JsonProperty public Long length;
+    @JsonProperty public List<RowGroupMetadata_v3> rowGroups;
+
+    public ParquetFileMetadata_v3() {
+      super();
+    }
+
+    public ParquetFileMetadata_v3(String path, Long length, List<RowGroupMetadata_v3> rowGroups) {
+      this.path = path;
+      this.length = length;
+      this.rowGroups = rowGroups;
+    }
+
+    @Override public String toString() {
+      return String.format("path: %s rowGroups: %s", path, rowGroups);
+    }
+
+    @JsonIgnore @Override public String getPath() {
+      return path;
+    }
+
+    @JsonIgnore @Override public Long getLength() {
+      return length;
+    }
+
+    @JsonIgnore @Override public List<? extends RowGroupMetadata> getRowGroups() {
+      return rowGroups;
+    }
+  }
+
+
+  /**
+   * A struct that contains the metadata for a parquet row group
+   */
+  public static class RowGroupMetadata_v3 extends RowGroupMetadata {
+    @JsonProperty public Long start;
+    @JsonProperty public Long length;
+    @JsonProperty public Long rowCount;
+    @JsonProperty public Map<String, Float> hostAffinity;
+    @JsonProperty public List<ColumnMetadata_v3> columns;
+
+    public RowGroupMetadata_v3() {
+      super();
+    }
+
+    public RowGroupMetadata_v3(Long start, Long length, Long rowCount, Map<String, Float> hostAffinity,
+        List<ColumnMetadata_v3> columns) {
+      this.start = start;
+      this.length = length;
+      this.rowCount = rowCount;
+      this.hostAffinity = hostAffinity;
+      this.columns = columns;
+    }
+
+    @Override public Long getStart() {
+      return start;
+    }
+
+    @Override public Long getLength() {
+      return length;
+    }
+
+    @Override public Long getRowCount() {
+      return rowCount;
+    }
+
+    @Override public Map<String, Float> getHostAffinity() {
+      return hostAffinity;
+    }
+
+    @Override public List<? extends ColumnMetadata> getColumns() {
+      return columns;
+    }
+  }
+
+
+  public static class ColumnTypeMetadata_v3 {
+    @JsonProperty public String[] name;
+    @JsonProperty public PrimitiveTypeName primitiveType;
+    @JsonProperty public OriginalType originalType;
+    @JsonProperty public int precision;
+    @JsonProperty public int scale;
+    @JsonProperty public int repetitionLevel;
+    @JsonProperty public int definitionLevel;
+
+    // Key to find by name only
+    @JsonIgnore private Key key;
+
+    public ColumnTypeMetadata_v3() {
+      super();
+    }
+
+    public ColumnTypeMetadata_v3(String[] name, PrimitiveTypeName primitiveType, OriginalType originalType, int precision, int scale, int repetitionLevel, int definitionLevel) {
+      this.name = name;
+      this.primitiveType = primitiveType;
+      this.originalType = originalType;
+      this.precision = precision;
+      this.scale = scale;
+      this.repetitionLevel = repetitionLevel;
+      this.definitionLevel = definitionLevel;
+      this.key = new Key(name);
+    }
+
+    @JsonIgnore private Key key() {
+      return this.key;
+    }
+
+    private static class Key {
+      private String[] name;
+      private int hashCode = 0;
+
+      public Key(String[] name) {
+        this.name = name;
+      }
+
+      @Override public int hashCode() {
+        if (hashCode == 0) {
+          hashCode = Arrays.hashCode(name);
+        }
+        return hashCode;
+      }
+
+      @Override public boolean equals(Object obj) {
+        if (obj == null) {
+          return false;
+        }
+        if (getClass() != obj.getClass()) {
+          return false;
+        }
+        final Key other = (Key) obj;
+        return Arrays.equals(this.name, other.name);
+      }
+
+      @Override public String toString() {
+        String s = null;
+        for (String namePart : name) {
+          if (s != null) {
+            s += ".";
+            s += namePart;
+          } else {
+            s = namePart;
+          }
+        }
+        return s;
+      }
+
+      public static class DeSerializer extends KeyDeserializer {
+
+        public DeSerializer() {
+          super();
+        }
+
+        @Override
+        public Object deserializeKey(String key, com.fasterxml.jackson.databind.DeserializationContext ctxt)
+            throws IOException, com.fasterxml.jackson.core.JsonProcessingException {
+          return new Key(key.split("\\."));
+        }
+      }
+    }
+  }
+
+
+  /**
+   * A struct that contains the metadata for a column in a parquet file
+   */
+  public static class ColumnMetadata_v3 extends ColumnMetadata {
+    // Use a string array for name instead of Schema Path to make serialization easier
+    @JsonProperty public String[] name;
+    @JsonProperty public Long nulls;
+
+    public Object minValue;
+    public Object maxValue;
+
+    @JsonIgnore private PrimitiveTypeName primitiveType;
+
+    public ColumnMetadata_v3() {
+      super();
+    }
+
+    public ColumnMetadata_v3(String[] name, PrimitiveTypeName primitiveType, Object minValue, Object maxValue, Long nulls) {
+      this.name = name;
+      this.minValue = minValue;
+      this.maxValue = maxValue;
+      this.nulls = nulls;
+      this.primitiveType = primitiveType;
+    }
+
+    @JsonProperty(value = "minValue") public void setMin(Object minValue) {
+      this.minValue = minValue;
+    }
+
+    @JsonProperty(value = "maxValue") public void setMax(Object maxValue) {
+      this.maxValue = maxValue;
+    }
+
+    @Override public String[] getName() {
+      return name;
+    }
+
+    @Override public Long getNulls() {
+      return nulls;
+    }
+
+    @Override
+    public boolean hasSingleValue() {
+      return (minValue !=null && maxValue != null && minValue.equals(maxValue));
+    }
+
+    @Override public Object getMinValue() {
+      return minValue;
+    }
+
+    @Override public Object getMaxValue() {
+      return maxValue;
+    }
+
+    @Override public PrimitiveTypeName getPrimitiveType() {
+      return null;
+    }
+
+    @Override public OriginalType getOriginalType() {
+      return null;
+    }
+
+    public static class DeSerializer extends JsonDeserializer<ColumnMetadata_v3> {
+      @Override public ColumnMetadata_v3 deserialize(JsonParser jp, DeserializationContext ctxt)
+          throws IOException, JsonProcessingException {
+        return null;
+      }
+    }
+
+
+    // We use a custom serializer and write only non null values.
+    public static class Serializer extends JsonSerializer<ColumnMetadata_v3> {
+      @Override
+      public void serialize(ColumnMetadata_v3 value, JsonGenerator jgen, SerializerProvider provider)
+          throws IOException, JsonProcessingException {
+        jgen.writeStartObject();
+        jgen.writeArrayFieldStart("name");
+        for (String n : value.name) {
+          jgen.writeString(n);
+        }
+        jgen.writeEndArray();
+        if (value.minValue != null) {
+          Object val;
+          if (value.primitiveType == PrimitiveTypeName.BINARY && value.minValue != null) {
+            val = new String(((Binary) value.minValue).getBytes());
+          } else {
+            val = value.minValue;
+          }
+          jgen.writeObjectField("minValue", val);
+        }
+        if (value.maxValue != null) {
+          Object val;
+          if (value.primitiveType == PrimitiveTypeName.BINARY && value.maxValue != null) {
+            val = new String(((Binary) value.maxValue).getBytes());
+          } else {
+            val = value.maxValue;
+          }
+          jgen.writeObjectField("maxValue", val);
+        }
+
+        if (value.nulls != null) {
+          jgen.writeObjectField("nulls", value.nulls);
+        }
+        jgen.writeEndObject();
+      }
+    }
+
+  }
+
 }
 


[09/10] drill git commit: DRILL-1950: Parquet rowgroup level filter pushdown in query planning time.

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
index b613707..37a57dc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
@@ -6,9 +6,7 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
  * http://www.apache.org/licenses/LICENSE-2.0
- *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -17,268 +15,281 @@
  */
 package org.apache.drill.exec.store.parquet;
 
-import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import org.apache.drill.common.expression.BooleanOperator;
-import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.FunctionHolderExpression;
 import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.expression.fn.CastFunctions;
+import org.apache.drill.common.expression.fn.FuncHolder;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
-import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.expr.fn.DrillSimpleFuncHolder;
+import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
+import org.apache.drill.exec.expr.fn.interpreter.InterpreterEvaluator;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.DateHolder;
+import org.apache.drill.exec.expr.holders.Float4Holder;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.TimeHolder;
+import org.apache.drill.exec.expr.holders.TimeStampHolder;
+import org.apache.drill.exec.expr.holders.ValueHolder;
+import org.apache.drill.exec.expr.stat.ParquetPredicates;
+import org.apache.drill.exec.expr.stat.TypedFieldExpr;
+import org.apache.drill.exec.ops.UdfUtilities;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.parquet.filter2.predicate.FilterApi;
-import org.apache.parquet.filter2.predicate.FilterPredicate;
 
-import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
-public class ParquetFilterBuilder extends
-        AbstractExprVisitor<FilterPredicate, Void, RuntimeException> {
-    static final Logger logger = LoggerFactory
-            .getLogger(ParquetFilterBuilder.class);
-    private LogicalExpression le;
-    private boolean allExpressionsConverted = true;
-    private ParquetGroupScan groupScan;
-
-    public ParquetFilterBuilder(ParquetGroupScan groupScan, LogicalExpression conditionExp) {
-        this.le = conditionExp;
-        this.groupScan = groupScan;
+/**
+ * A visitor which visits a materialized logical expression, and build ParquetFilterPredicate
+ * If a visitXXX method returns null, that means the corresponding filter branch is not qualified for pushdown.
+ */
+public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression, Set<LogicalExpression>, RuntimeException> {
+  static final Logger logger = LoggerFactory.getLogger(ParquetFilterBuilder.class);
+
+  private final UdfUtilities udfUtilities;
+
+  /**
+   * @param expr materialized filter expression
+   * @param constantBoundaries set of constant expressions
+   * @param udfUtilities
+   */
+  public static LogicalExpression buildParquetFilterPredicate(LogicalExpression expr, final Set<LogicalExpression> constantBoundaries, UdfUtilities udfUtilities) {
+    final LogicalExpression predicate = expr.accept(new ParquetFilterBuilder(udfUtilities), constantBoundaries);
+    return predicate;
+  }
+
+  private ParquetFilterBuilder(UdfUtilities udfUtilities) {
+    this.udfUtilities = udfUtilities;
+  }
+
+  @Override
+  public LogicalExpression visitUnknown(LogicalExpression e, Set<LogicalExpression> value) {
+    if (e instanceof TypedFieldExpr &&
+        ! containsArraySeg(((TypedFieldExpr) e).getPath()) &&
+        e.getMajorType().getMode() != TypeProtos.DataMode.REPEATED) {
+      // A filter is not qualified for push down, if
+      // 1. it contains an array segment : a.b[1], a.b[1].c.d
+      // 2. it's repeated type.
+      return e;
     }
 
-    public ParquetGroupScan parseTree() {
-        FilterPredicate predicate = le.accept(this, null);
-        try {
-            return this.groupScan.clone(predicate);
-        } catch (IOException e) {
-            logger.error("Failed to set Parquet filter", e);
-            return null;
+    return null;
+  }
+
+  @Override
+  public LogicalExpression visitIntConstant(ValueExpressions.IntExpression intExpr, Set<LogicalExpression> value)
+      throws RuntimeException {
+    return intExpr;
+  }
+
+  @Override
+  public LogicalExpression visitDoubleConstant(ValueExpressions.DoubleExpression dExpr, Set<LogicalExpression> value)
+      throws RuntimeException {
+    return dExpr;
+  }
+
+  @Override
+  public LogicalExpression visitFloatConstant(ValueExpressions.FloatExpression fExpr, Set<LogicalExpression> value)
+      throws RuntimeException {
+    return fExpr;
+  }
+
+  @Override
+  public LogicalExpression visitLongConstant(ValueExpressions.LongExpression intExpr, Set<LogicalExpression> value)
+      throws RuntimeException {
+    return intExpr;
+  }
+
+  @Override
+  public LogicalExpression visitDateConstant(ValueExpressions.DateExpression dateExpr, Set<LogicalExpression> value) throws RuntimeException {
+    return dateExpr;
+  }
+
+  @Override
+  public LogicalExpression visitTimeStampConstant(ValueExpressions.TimeStampExpression tsExpr, Set<LogicalExpression> value) throws RuntimeException {
+    return tsExpr;
+  }
+
+  @Override
+  public LogicalExpression visitTimeConstant(ValueExpressions.TimeExpression timeExpr, Set<LogicalExpression> value) throws RuntimeException {
+    return timeExpr;
+  }
+
+  @Override
+  public LogicalExpression visitBooleanOperator(BooleanOperator op, Set<LogicalExpression> value) {
+    List<LogicalExpression> childPredicates = new ArrayList<>();
+    String functionName = op.getName();
+
+    for (LogicalExpression arg : op.args) {
+      LogicalExpression childPredicate = arg.accept(this, value);
+      if (childPredicate == null) {
+        if (functionName.equals("booleanOr")) {
+          // we can't include any leg of the OR if any of the predicates cannot be converted
+          return null;
         }
+      } else {
+        childPredicates.add(childPredicate);
+      }
     }
 
-    public boolean areAllExpressionsConverted() {
-        return allExpressionsConverted;
+    if (childPredicates.size() == 0) {
+      return null; // none leg is qualified, return null.
+    } else if (childPredicates.size() == 1) {
+      return childPredicates.get(0); // only one leg is qualified, remove boolean op.
+    } else {
+      if (functionName.equals("booleanOr")) {
+        return new ParquetPredicates.OrPredicate(op.getName(), childPredicates, op.getPosition());
+      } else {
+        return new ParquetPredicates.AndPredicate(op.getName(), childPredicates, op.getPosition());
+      }
     }
+  }
 
-    @Override
-    public FilterPredicate visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
-        allExpressionsConverted = false;
-        return null;
+  private boolean containsArraySeg(final SchemaPath schemaPath) {
+    PathSegment seg = schemaPath.getRootSegment();
+
+    while (seg != null) {
+      if (seg.isArray()) {
+        return true;
+      }
+      seg = seg.getChild();
     }
+    return false;
+  }
 
-    @Override
-    public FilterPredicate visitBooleanOperator(BooleanOperator op, Void value) {
-        List<LogicalExpression> args = op.args;
-        FilterPredicate nodePredicate = null;
-        String functionName = op.getName();
-        for (LogicalExpression arg : args) {
-            switch (functionName) {
-                case "booleanAnd":
-                case "booleanOr":
-                    if (nodePredicate == null) {
-                        nodePredicate = arg.accept(this, null);
-                    } else {
-                        FilterPredicate predicate = arg.accept(this, null);
-                        if (predicate != null) {
-                            nodePredicate = mergePredicates(functionName, nodePredicate, predicate);
-                        } else {
-                            // we can't include any part of the OR if any of the predicates cannot be converted
-                            if (functionName == "booleanOr") {
-                                nodePredicate = null;
-                            }
-                            allExpressionsConverted = false;
-                        }
-                    }
-                    break;
-            }
-        }
-        return nodePredicate;
+  private LogicalExpression getValueExpressionFromConst(ValueHolder holder, TypeProtos.MinorType type) {
+    switch (type) {
+    case INT:
+      return ValueExpressions.getInt(((IntHolder) holder).value);
+    case BIGINT:
+      return ValueExpressions.getBigInt(((BigIntHolder) holder).value);
+    case FLOAT4:
+      return ValueExpressions.getFloat4(((Float4Holder) holder).value);
+    case FLOAT8:
+      return ValueExpressions.getFloat8(((Float8Holder) holder).value);
+    case DATE:
+      return ValueExpressions.getDate(((DateHolder) holder).value);
+    case TIMESTAMP:
+      return ValueExpressions.getTimeStamp(((TimeStampHolder) holder).value);
+    case TIME:
+      return ValueExpressions.getTime(((TimeHolder) holder).value);
+    default:
+      return null;
     }
+  }
 
-    private FilterPredicate mergePredicates(String functionName,
-                                            FilterPredicate leftPredicate, FilterPredicate rightPredicate) {
-        if (leftPredicate != null && rightPredicate != null) {
-            if (functionName == "booleanAnd") {
-                return FilterApi.and(leftPredicate, rightPredicate);
-            }
-            else {
-                return FilterApi.or(leftPredicate, rightPredicate);
-            }
-        } else {
-            allExpressionsConverted = false;
-            if ("booleanAnd".equals(functionName)) {
-                return leftPredicate == null ? rightPredicate : leftPredicate;
-            }
-        }
+  @Override
+  public LogicalExpression visitFunctionHolderExpression(FunctionHolderExpression funcHolderExpr, Set<LogicalExpression> value)
+      throws RuntimeException {
+    FuncHolder holder = funcHolderExpr.getHolder();
 
+    if (! (holder instanceof DrillSimpleFuncHolder)) {
+      return null;
+    }
+
+    if (value.contains(funcHolderExpr)) {
+      ValueHolder result ;
+      try {
+        result = InterpreterEvaluator.evaluateConstantExpr(udfUtilities, funcHolderExpr);
+      } catch (Exception e) {
+        logger.warn("Error in evaluating function of {}", funcHolderExpr.getName());
         return null;
+      }
+
+      logger.debug("Reduce a constant function expression into a value expression");
+      return getValueExpressionFromConst(result, funcHolderExpr.getMajorType().getMinorType());
     }
 
-    @Override
-    public FilterPredicate visitFunctionCall(FunctionCall call, Void value) throws RuntimeException {
-        FilterPredicate predicate = null;
-        String functionName = call.getName();
-        ImmutableList<LogicalExpression> args = call.args;
-
-        if (ParquetCompareFunctionProcessor.isCompareFunction(functionName)) {
-            ParquetCompareFunctionProcessor processor = ParquetCompareFunctionProcessor
-                    .process(call);
-            if (processor.isSuccess()) {
-                try {
-                    predicate = createFilterPredicate(processor.getFunctionName(),
-                            processor.getPath(), processor.getValue());
-                } catch (Exception e) {
-                    logger.error("Failed to create Parquet filter", e);
-                }
-            }
-        } else {
-            switch (functionName) {
-                case "booleanAnd":
-                case "booleanOr":
-                    FilterPredicate leftPredicate = args.get(0).accept(this, null);
-                    FilterPredicate rightPredicate = args.get(1).accept(this, null);
-                    predicate = mergePredicates(functionName, leftPredicate, rightPredicate);
-                    break;
-            }
-        }
+    final String funcName = ((DrillSimpleFuncHolder) holder).getRegisteredNames()[0];
+
+    if (isCompareFunction(funcName)) {
+      return handleCompareFunction(funcHolderExpr, value);
+    }
 
-        if (predicate == null) {
-            allExpressionsConverted = false;
+    if (CastFunctions.isCastFunction(funcName)) {
+      List<LogicalExpression> newArgs = new ArrayList();
+      for (LogicalExpression arg : funcHolderExpr.args) {
+        final LogicalExpression newArg = arg.accept(this, value);
+        if (newArg == null) {
+          return null;
         }
+        newArgs.add(newArg);
+      }
 
-        return predicate;
+      return funcHolderExpr.copy(newArgs);
+    } else {
+      return null;
     }
+  }
 
-    private FilterPredicate createFilterPredicate(String functionName,
-                                                  SchemaPath field, Object fieldValue) {
-        FilterPredicate filter = null;
-
-        // extract the field name
-        String fieldName = field.getAsUnescapedPath();
-        switch (functionName) {
-            case "equal":
-                if (fieldValue instanceof Long) {
-                    filter = FilterApi.eq(FilterApi.longColumn(fieldName), (Long) fieldValue);
-                }
-                else if (fieldValue instanceof Integer) {
-                    filter = FilterApi.eq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
-                }
-                else if (fieldValue instanceof Float) {
-                    filter = FilterApi.eq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
-                }
-                else if (fieldValue instanceof Double) {
-                    filter = FilterApi.eq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
-                }
-                else if (fieldValue instanceof Boolean) {
-                    filter = FilterApi.eq(FilterApi.booleanColumn(fieldName), (Boolean) fieldValue);
-                }
-                break;
-            case "not_equal":
-                if (fieldValue instanceof Long) {
-                    filter = FilterApi.notEq(FilterApi.longColumn(fieldName), (Long) fieldValue);
-                }
-                else if (fieldValue instanceof Integer) {
-                    filter = FilterApi.notEq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
-                }
-                else if (fieldValue instanceof Float) {
-                    filter = FilterApi.notEq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
-                }
-                else if (fieldValue instanceof Double) {
-                    filter = FilterApi.notEq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
-                }
-                else if (fieldValue instanceof Boolean) {
-                    filter = FilterApi.notEq(FilterApi.booleanColumn(fieldName), (Boolean) fieldValue);
-                }
-                break;
-            case "greater_than_or_equal_to":
-                if (fieldValue instanceof Long) {
-                    filter = FilterApi.gtEq(FilterApi.longColumn(fieldName), (Long) fieldValue);
-                }
-                else if (fieldValue instanceof Integer) {
-                    filter = FilterApi.gtEq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
-                }
-                else if (fieldValue instanceof Float) {
-                    filter = FilterApi.gtEq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
-                }
-                else if (fieldValue instanceof Double) {
-                    filter = FilterApi.gtEq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
-                }
-                break;
-            case "greater_than":
-                if (fieldValue instanceof Long) {
-                    filter = FilterApi.gt(FilterApi.longColumn(fieldName), (Long) fieldValue);
-                }
-                else if (fieldValue instanceof Integer) {
-                    filter = FilterApi.gt(FilterApi.intColumn(fieldName), (Integer) fieldValue);
-                }
-                else if (fieldValue instanceof Float) {
-                    filter = FilterApi.gt(FilterApi.floatColumn(fieldName), (Float) fieldValue);
-                }
-                else if (fieldValue instanceof Double) {
-                    filter = FilterApi.gt(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
-                }
-                break;
-            case "less_than_or_equal_to":
-                if (fieldValue instanceof Long) {
-                    filter = FilterApi.ltEq(FilterApi.longColumn(fieldName), (Long) fieldValue);
-                }
-                else if (fieldValue instanceof Integer) {
-                    filter = FilterApi.ltEq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
-                }
-                else if (fieldValue instanceof Float) {
-                    filter = FilterApi.ltEq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
-                }
-                else if (fieldValue instanceof Double) {
-                    filter = FilterApi.ltEq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
-                }
-                break;
-            case "less_than":
-                if (fieldValue instanceof Long) {
-                    filter = FilterApi.lt(FilterApi.longColumn(fieldName), (Long) fieldValue);
-                }
-                else if (fieldValue instanceof Integer) {
-                    filter = FilterApi.lt(FilterApi.intColumn(fieldName), (Integer) fieldValue);
-                }
-                else if (fieldValue instanceof Float) {
-                    filter = FilterApi.lt(FilterApi.floatColumn(fieldName), (Float) fieldValue);
-                }
-                else if (fieldValue instanceof Double) {
-                    filter = FilterApi.lt(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
-                }
-                break;
-            case "isnull":
-            case "isNull":
-            case "is null":
-                if (fieldValue instanceof Long) {
-                    filter = FilterApi.eq(FilterApi.longColumn(fieldName), null);
-                }
-                else if (fieldValue instanceof Integer) {
-                    filter = FilterApi.eq(FilterApi.intColumn(fieldName), null);
-                }
-                else if (fieldValue instanceof Float) {
-                    filter = FilterApi.eq(FilterApi.floatColumn(fieldName), null);
-                }
-                else if (fieldValue instanceof Double) {
-                    filter = FilterApi.eq(FilterApi.doubleColumn(fieldName), null);
-                }
-                break;
-            case "isnotnull":
-            case "isNotNull":
-            case "is not null":
-                if (fieldValue instanceof Long) {
-                    filter = FilterApi.notEq(FilterApi.longColumn(fieldName), null);
-                }
-                else if (fieldValue instanceof Integer) {
-                    filter = FilterApi.notEq(FilterApi.intColumn(fieldName), null);
-                }
-                else if (fieldValue instanceof Float) {
-                    filter = FilterApi.notEq(FilterApi.floatColumn(fieldName), null);
-                }
-                else if (fieldValue instanceof Double) {
-                    filter = FilterApi.notEq(FilterApi.doubleColumn(fieldName), null);
-                }
-                break;
-        }
+  private LogicalExpression handleCompareFunction(FunctionHolderExpression functionHolderExpression, Set<LogicalExpression> value) {
+    List<LogicalExpression> newArgs = new ArrayList();
+
+    for (LogicalExpression arg : functionHolderExpression.args) {
+      LogicalExpression newArg = arg.accept(this, value);
+      if (newArg == null) {
+        return null;
+      }
+      newArgs.add(newArg);
+    }
+
+    String funcName = ((DrillSimpleFuncHolder) functionHolderExpression.getHolder()).getRegisteredNames()[0];
 
-        return filter;
+    switch (funcName) {
+    case FunctionGenerationHelper.EQ :
+      return new ParquetPredicates.EqualPredicate(newArgs.get(0), newArgs.get(1));
+    case FunctionGenerationHelper.GT :
+      return new ParquetPredicates.GTPredicate(newArgs.get(0), newArgs.get(1));
+    case FunctionGenerationHelper.GE :
+      return new ParquetPredicates.GEPredicate(newArgs.get(0), newArgs.get(1));
+    case FunctionGenerationHelper.LT :
+      return new ParquetPredicates.LTPredicate(newArgs.get(0), newArgs.get(1));
+    case FunctionGenerationHelper.LE :
+      return new ParquetPredicates.LEPredicate(newArgs.get(0), newArgs.get(1));
+    case FunctionGenerationHelper.NE :
+      return new ParquetPredicates.NEPredicate(newArgs.get(0), newArgs.get(1));
+    default:
+      return null;
     }
+  }
+
+  private LogicalExpression handleCastFunction(FunctionHolderExpression functionHolderExpression, Set<LogicalExpression> value) {
+    for (LogicalExpression arg : functionHolderExpression.args) {
+      LogicalExpression newArg = arg.accept(this, value);
+      if (newArg == null) {
+        return null;
+      }
+    }
+
+    String funcName = ((DrillSimpleFuncHolder) functionHolderExpression.getHolder()).getRegisteredNames()[0];
+
+    return null;
+  }
+
+  private static boolean isCompareFunction(String funcName) {
+    return COMPARE_FUNCTIONS_SET.contains(funcName);
+  }
+
+  private static final ImmutableSet<String> COMPARE_FUNCTIONS_SET;
+
+  static {
+    ImmutableSet.Builder<String> builder = ImmutableSet.builder();
+    COMPARE_FUNCTIONS_SET = builder
+        .add(FunctionGenerationHelper.EQ)
+        .add(FunctionGenerationHelper.GT)
+        .add(FunctionGenerationHelper.GE)
+        .add(FunctionGenerationHelper.LT)
+        .add(FunctionGenerationHelper.LE)
+        .add(FunctionGenerationHelper.NE)
+        .build();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index a8e55b7..71e681b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -26,14 +26,26 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.avro.generic.GenericData;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.ExpressionStringBuilder;
+import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.compile.sig.ConstantExpressionIdentifier;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.expr.stat.ParquetFilterPredicate;
+import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.ops.UdfUtilities;
 import org.apache.drill.exec.physical.EndpointAffinity;
 import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
 import org.apache.drill.exec.physical.base.AbstractFileGroupScan;
@@ -42,7 +54,11 @@ import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.store.ImplicitColumnExplorer;
+import org.apache.drill.exec.store.ParquetOutputRecordWriter;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.DrillPathFilter;
@@ -56,6 +72,8 @@ import org.apache.drill.exec.store.parquet.Metadata.ColumnMetadata;
 import org.apache.drill.exec.store.parquet.Metadata.ParquetFileMetadata;
 import org.apache.drill.exec.store.parquet.Metadata.ParquetTableMetadataBase;
 import org.apache.drill.exec.store.parquet.Metadata.RowGroupMetadata;
+import org.apache.drill.exec.store.parquet.stat.ColumnStatistics;
+import org.apache.drill.exec.store.parquet.stat.ParquetMetaStatCollector;
 import org.apache.drill.exec.store.schedule.AffinityCreator;
 import org.apache.drill.exec.store.schedule.AssignmentCreator;
 import org.apache.drill.exec.store.schedule.CompleteWork;
@@ -112,6 +130,13 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
   private List<SchemaPath> columns;
   private ListMultimap<Integer, RowGroupInfo> mappings;
   private List<RowGroupInfo> rowGroupInfos;
+  private LogicalExpression filter;
+
+  /**
+   * The parquet table metadata may have already been read
+   * from a metadata cache file earlier; we can re-use during
+   * the ParquetGroupScan and avoid extra loading time.
+   */
   private Metadata.ParquetTableMetadataBase parquetTableMetadata = null;
   private String cacheFileRoot = null;
 
@@ -133,7 +158,8 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
       @JacksonInject StoragePluginRegistry engineRegistry, //
       @JsonProperty("columns") List<SchemaPath> columns, //
       @JsonProperty("selectionRoot") String selectionRoot, //
-      @JsonProperty("cacheFileRoot") String cacheFileRoot //
+      @JsonProperty("cacheFileRoot") String cacheFileRoot, //
+      @JsonProperty("filter") LogicalExpression filter
   ) throws IOException, ExecutionSetupException {
     super(ImpersonationUtil.resolveUserName(userName));
     this.columns = columns;
@@ -149,6 +175,7 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
     this.entries = entries;
     this.selectionRoot = selectionRoot;
     this.cacheFileRoot = cacheFileRoot;
+    this.filter = filter;
 
     init(null);
   }
@@ -159,7 +186,18 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
       ParquetFormatPlugin formatPlugin, //
       String selectionRoot,
       String cacheFileRoot,
-      List<SchemaPath> columns) //
+      List<SchemaPath> columns) throws IOException{
+    this(userName, selection, formatPlugin, selectionRoot, cacheFileRoot, columns, ValueExpressions.BooleanExpression.TRUE);
+  }
+
+  public ParquetGroupScan( //
+      String userName,
+      FileSelection selection, //
+      ParquetFormatPlugin formatPlugin, //
+      String selectionRoot,
+      String cacheFileRoot,
+      List<SchemaPath> columns,
+      LogicalExpression filter) //
       throws IOException {
     super(userName);
     this.formatPlugin = formatPlugin;
@@ -187,6 +225,8 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
       }
     }
 
+    this.filter = filter;
+
     init(fileSelection.getMetaContext());
   }
 
@@ -206,11 +246,12 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
     this.rowGroupInfos = that.rowGroupInfos == null ? null : Lists.newArrayList(that.rowGroupInfos);
     this.selectionRoot = that.selectionRoot;
     this.columnValueCounts = that.columnValueCounts == null ? null : new HashMap<>(that.columnValueCounts);
-    this.columnTypeMap = that.columnTypeMap == null ? null : new HashMap<>(that.columnTypeMap);
+    this.partitionColTypeMap = that.partitionColTypeMap == null ? null : new HashMap<>(that.partitionColTypeMap);
     this.partitionValueMap = that.partitionValueMap == null ? null : new HashMap<>(that.partitionValueMap);
     this.fileSet = that.fileSet == null ? null : new HashSet<>(that.fileSet);
     this.usedMetadataCache = that.usedMetadataCache;
     this.parquetTableMetadata = that.parquetTableMetadata;
+    this.filter = that.filter;
     this.cacheFileRoot = that.cacheFileRoot;
   }
 
@@ -260,6 +301,14 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
     return fileSet;
   }
 
+  public LogicalExpression getFilter() {
+    return this.filter;
+  }
+
+  public void setFilter(LogicalExpression filter) {
+    this.filter = filter;
+  }
+
   @Override
   public boolean hasFiles() {
     return true;
@@ -273,7 +322,8 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
   private Set<String> fileSet;
 
   @JsonIgnore
-  private Map<SchemaPath, MajorType> columnTypeMap = Maps.newHashMap();
+  // only for partition columns : value is unique for each partition
+  private Map<SchemaPath, MajorType> partitionColTypeMap = Maps.newHashMap();
 
   /**
    * When reading the very first footer, any column is a potential partition column. So for the first footer, we check
@@ -295,21 +345,21 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
     }
     if (first) {
       if (hasSingleValue(columnMetadata)) {
-        columnTypeMap.put(schemaPath, getType(primitiveType, originalType));
+        partitionColTypeMap.put(schemaPath, getType(primitiveType, originalType));
         return true;
       } else {
         return false;
       }
     } else {
-      if (!columnTypeMap.keySet().contains(schemaPath)) {
+      if (!partitionColTypeMap.keySet().contains(schemaPath)) {
         return false;
       } else {
         if (!hasSingleValue(columnMetadata)) {
-          columnTypeMap.remove(schemaPath);
+          partitionColTypeMap.remove(schemaPath);
           return false;
         }
-        if (!getType(primitiveType, originalType).equals(columnTypeMap.get(schemaPath))) {
-          columnTypeMap.remove(schemaPath);
+        if (!getType(primitiveType, originalType).equals(partitionColTypeMap.get(schemaPath))) {
+          partitionColTypeMap.remove(schemaPath);
           return false;
         }
       }
@@ -317,7 +367,7 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
     return true;
   }
 
-  private MajorType getType(PrimitiveTypeName type, OriginalType originalType) {
+  public static MajorType getType(PrimitiveTypeName type, OriginalType originalType) {
     if (originalType != null) {
       switch (originalType) {
         case DECIMAL:
@@ -390,7 +440,7 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
   }
 
   public MajorType getTypeForColumn(SchemaPath schemaPath) {
-    return columnTypeMap.get(schemaPath);
+    return partitionColTypeMap.get(schemaPath);
   }
 
   // Map from file names to maps of column name to partition value mappings
@@ -771,13 +821,13 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
             Object currentValue = column.getMaxValue();
             if (value != null) {
               if (value != currentValue) {
-                columnTypeMap.remove(schemaPath);
+                partitionColTypeMap.remove(schemaPath);
               }
             } else {
               map.put(schemaPath, currentValue);
             }
           } else {
-            columnTypeMap.remove(schemaPath);
+            partitionColTypeMap.remove(schemaPath);
           }
         }
         this.rowCount += rowGroup.getRowCount();
@@ -839,7 +889,7 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
         String.format("MinorFragmentId %d has no read entries assigned", minorFragmentId));
 
     return new ParquetRowGroupScan(
-        getUserName(), formatPlugin, convertToReadEntries(rowGroupsForMinor), columns, selectionRoot);
+        getUserName(), formatPlugin, convertToReadEntries(rowGroupsForMinor), columns, selectionRoot, filter);
   }
 
   private List<RowGroupReadEntry> convertToReadEntries(List<RowGroupInfo> rowGroups) {
@@ -893,12 +943,16 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
             Path.getPathWithoutSchemeAndAuthority(new Path(cacheFileRoot)).toString();
       cacheFileString = ", cacheFileRoot=" + str;
     }
+    final String filterStr = filter == null || filter.equals(ValueExpressions.BooleanExpression.TRUE) ? "" : ", filter=" + ExpressionStringBuilder.toString(this.filter);
+
     return "ParquetGroupScan [entries=" + entries
         + ", selectionRoot=" + selectionRoot
         + ", numFiles=" + getEntries().size()
         + ", usedMetadataFile=" + usedMetadataCache
+        + filterStr
         + cacheFileString
-        + ", columns=" + columns + "]";
+        + ", columns=" + columns
+        + "]";
   }
 
   @Override
@@ -1000,6 +1054,88 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
 
   @Override
   public List<SchemaPath> getPartitionColumns() {
-    return new ArrayList<>(columnTypeMap.keySet());
+    return new ArrayList<>(partitionColTypeMap.keySet());
+  }
+
+  public GroupScan applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities,
+      FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) {
+    if (fileSet.size() == 1 ||
+        ! (parquetTableMetadata.isRowGroupPrunable()) ||
+        rowGroupInfos.size() > optionManager.getOption(PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD)
+        ) {
+      // Stop pruning for 3 cases:
+      //    -  1 single parquet file,
+      //    -  metadata does not have proper format to support row group level filter pruning,
+      //    -  # of row groups is beyond PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD.
+      return null;
+    }
+
+    final Set<SchemaPath> schemaPathsInExpr = filterExpr.accept(new ParquetRGFilterEvaluator.FieldReferenceFinder(), null);
+
+    final List<RowGroupMetadata> qualifiedRGs = new ArrayList<>(parquetTableMetadata.getFiles().size());
+    Set<String> qualifiedFileNames = Sets.newHashSet(); // HashSet keeps a fileName unique.
+
+    ParquetFilterPredicate filterPredicate = null;
+
+    for (ParquetFileMetadata file : parquetTableMetadata.getFiles()) {
+      final ImplicitColumnExplorer columnExplorer = new ImplicitColumnExplorer(optionManager, this.columns);
+      Map<String, String> implicitColValues = columnExplorer.populateImplicitColumns(file.getPath(), selectionRoot);
+
+      for (RowGroupMetadata rowGroup : file.getRowGroups()) {
+        ParquetMetaStatCollector statCollector = new ParquetMetaStatCollector(
+            parquetTableMetadata,
+            rowGroup.getColumns(),
+            implicitColValues);
+
+        Map<SchemaPath, ColumnStatistics> columnStatisticsMap = statCollector.collectColStat(schemaPathsInExpr);
+
+        if (filterPredicate == null) {
+          ErrorCollector errorCollector = new ErrorCollectorImpl();
+          LogicalExpression materializedFilter = ExpressionTreeMaterializer.materializeFilterExpr(
+              filterExpr, columnStatisticsMap, errorCollector, functionImplementationRegistry);
+
+          if (errorCollector.hasErrors()) {
+            logger.error("{} error(s) encountered when materialize filter expression : {}",
+                errorCollector.getErrorCount(), errorCollector.toErrorString());
+            return null;
+          }
+          //    logger.debug("materializedFilter : {}", ExpressionStringBuilder.toString(materializedFilter));
+
+          Set<LogicalExpression> constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(materializedFilter);
+          filterPredicate = (ParquetFilterPredicate) ParquetFilterBuilder.buildParquetFilterPredicate(
+              materializedFilter, constantBoundaries, udfUtilities);
+
+          if (filterPredicate == null) {
+            return null;
+          }
+        }
+
+        if (ParquetRGFilterEvaluator.canDrop(filterPredicate, columnStatisticsMap, rowGroup.getRowCount())) {
+          continue;
+        }
+
+        qualifiedRGs.add(rowGroup);
+        qualifiedFileNames.add(file.getPath());  // TODO : optimize when 1 file contains m row groups.
+      }
+    }
+
+    if (qualifiedFileNames.size() == fileSet.size() ) {
+      // There is no reduction of rowGroups. Return the original groupScan.
+      logger.debug("applyFilter does not have any pruning!");
+      return null;
+    } else if (qualifiedFileNames.size() == 0) {
+      logger.warn("All rowgroups have been filtered out. Add back one to get schema from scannner");
+      qualifiedFileNames.add(fileSet.iterator().next());
+    }
+
+    try {
+      FileSelection newSelection = new FileSelection(null, Lists.newArrayList(qualifiedFileNames), getSelectionRoot(), cacheFileRoot, false);
+      logger.info("applyFilter {} reduce parquet file # from {} to {}", ExpressionStringBuilder.toString(filterExpr), fileSet.size(), qualifiedFileNames.size());
+      return this.clone(newSelection);
+    } catch (IOException e) {
+      logger.warn("Could not apply filter prune due to Exception : {}", e);
+      return null;
+    }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
index 10c817b..6f870f7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
@@ -6,9 +6,7 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
  * http://www.apache.org/licenses/LICENSE-2.0
- *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -17,15 +15,18 @@
  */
 package org.apache.drill.exec.store.parquet;
 
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexNode;
 import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.common.expression.ValueExpressions;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
-import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.planner.logical.DrillOptiq;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
@@ -35,108 +36,108 @@ import org.apache.drill.exec.planner.physical.ProjectPrel;
 import org.apache.drill.exec.planner.physical.ScanPrel;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 
-import com.google.common.collect.ImmutableList;
+import java.util.concurrent.TimeUnit;
 
 public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
-    public static final StoragePluginOptimizerRule getFilterOnProject(final OptimizerRulesContext context){
-        return new ParquetPushDownFilter(
-                RelOptHelper.some(FilterPrel.class, RelOptHelper.some(ProjectPrel.class, RelOptHelper.any(ScanPrel.class))),
-                "ParquetPushDownFilter:Filter_On_Project", context) {
-
-            @Override
-            public boolean matches(RelOptRuleCall call) {
-                if (!enabled) {
-                    return false;
-                }
-                final ScanPrel scan = call.rel(2);
-                if (scan.getGroupScan() instanceof ParquetGroupScan) {
-                    return super.matches(call);
-                }
-                return false;
-            }
-
-            @Override
-            public void onMatch(RelOptRuleCall call) {
-                final FilterPrel filterRel = call.rel(0);
-                final ProjectPrel projectRel = call.rel(1);
-                final ScanPrel scanRel = call.rel(2);
-                doOnMatch(call, filterRel, projectRel, scanRel);
-            };
-        };
-    }
 
-    public static final StoragePluginOptimizerRule getFilterOnScan(final OptimizerRulesContext context){
-        return new ParquetPushDownFilter(
-                RelOptHelper.some(FilterPrel.class, RelOptHelper.any(ScanPrel.class)),
-                "ParquetPushDownFilter:Filter_On_Scan", context) {
-
-            @Override
-            public boolean matches(RelOptRuleCall call) {
-                if (!enabled) {
-                    return false;
-                }
-                final ScanPrel scan = call.rel(1);
-                if (scan.getGroupScan() instanceof ParquetGroupScan) {
-                    return super.matches(call);
-                }
-                return false;
-            }
-
-            @Override
-            public void onMatch(RelOptRuleCall call) {
-                final FilterPrel filterRel = call.rel(0);
-                final ScanPrel scanRel = call.rel(1);
-                doOnMatch(call, filterRel, null, scanRel);
-            }
-        };
-    }
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetPushDownFilter.class);
 
-    private final OptimizerRulesContext context;
-    // private final boolean useNewReader;
-    protected final boolean enabled;
+  public static RelOptRule getFilterOnProject(OptimizerRulesContext optimizerRulesContext) {
+    return new ParquetPushDownFilter(
+        RelOptHelper.some(FilterPrel.class, RelOptHelper.some(ProjectPrel.class, RelOptHelper.any(ScanPrel.class))),
+        "ParquetPushDownFilter:Filter_On_Project", optimizerRulesContext) {
 
-    private ParquetPushDownFilter(RelOptRuleOperand operand, String id, OptimizerRulesContext context) {
-        super(operand, id);
-        this.context = context;
-        this.enabled = context.getPlannerSettings().isParquetFilterPushEnabled();
-        // this.useNewReader = context.getPlannerSettings()getOptions().getOption(ExecConstants.PARQUET_NEW_RECORD_READER).bool_val;
+      @Override
+      public boolean matches(RelOptRuleCall call) {
+        final ScanPrel scan = call.rel(2);
+        if (scan.getGroupScan() instanceof ParquetGroupScan) {
+          return super.matches(call);
+        }
+        return false;
+      }
+
+      @Override
+      public void onMatch(RelOptRuleCall call) {
+        final FilterPrel filterRel = call.rel(0);
+        final ProjectPrel projectRel = call.rel(1);
+        final ScanPrel scanRel = call.rel(2);
+        doOnMatch(call, filterRel, projectRel, scanRel);
+      }
+
+    };
+  }
+
+  public static StoragePluginOptimizerRule getFilterOnScan(OptimizerRulesContext optimizerContext) {
+    return new ParquetPushDownFilter(
+        RelOptHelper.some(FilterPrel.class, RelOptHelper.any(ScanPrel.class)),
+        "ParquetPushDownFilter:Filter_On_Scan", optimizerContext) {
+
+      @Override
+      public boolean matches(RelOptRuleCall call) {
+        final ScanPrel scan = call.rel(1);
+        if (scan.getGroupScan() instanceof ParquetGroupScan) {
+          return super.matches(call);
+        }
+        return false;
+      }
+
+      @Override
+      public void onMatch(RelOptRuleCall call) {
+        final FilterPrel filterRel = call.rel(0);
+        final ScanPrel scanRel = call.rel(1);
+        doOnMatch(call, filterRel, null, scanRel);
+      }
+    };
+  }
+
+  // private final boolean useNewReader;
+  protected final OptimizerRulesContext optimizerContext;
+
+  private ParquetPushDownFilter(RelOptRuleOperand operand, String id, OptimizerRulesContext optimizerContext) {
+    super(operand, id);
+    this.optimizerContext = optimizerContext;
+  }
+
+  protected void doOnMatch(RelOptRuleCall call, FilterPrel filter, ProjectPrel project, ScanPrel scan) {
+    ParquetGroupScan groupScan = (ParquetGroupScan) scan.getGroupScan();
+    if (groupScan.getFilter() != null && !groupScan.getFilter().equals(ValueExpressions.BooleanExpression.TRUE)) {
+      return;
     }
 
-    protected void doOnMatch(RelOptRuleCall call, FilterPrel filter, ProjectPrel project, ScanPrel scan) {
-        ParquetGroupScan groupScan = (ParquetGroupScan) scan.getGroupScan();
-        if (groupScan.getFilter() != null) {
-            return;
-        }
+    RexNode condition = null;
+    if (project == null) {
+      condition = filter.getCondition();
+    } else {
+      // get the filter as if it were below the projection.
+      condition = RelOptUtil.pushFilterPastProject(filter.getCondition(), project);
+    }
 
-        RexNode condition = null;
-        if(project == null){
-            condition = filter.getCondition();
-        }else{
-            // get the filter as if it were below the projection.
-            condition = RelOptUtil.pushFilterPastProject(filter.getCondition(), project);
-        }
+    if (condition == null || condition.equals(ValueExpressions.BooleanExpression.TRUE)) {
+      return;
+    }
 
-        LogicalExpression conditionExp = DrillOptiq.toDrill(
-                new DrillParseContext(PrelUtil.getPlannerSettings(call.getPlanner())), scan, condition);
-        ParquetFilterBuilder parquetFilterBuilder = new ParquetFilterBuilder(groupScan,
-                conditionExp);
-        ParquetGroupScan newGroupScan = parquetFilterBuilder.parseTree();
-        if (newGroupScan == null) {
-            return; // no filter pushdown so nothing to apply.
-        }
+    LogicalExpression conditionExp = DrillOptiq.toDrill(
+        new DrillParseContext(PrelUtil.getPlannerSettings(call.getPlanner())), scan, condition);
+
+    Stopwatch timer = Stopwatch.createStarted();
+    final GroupScan newGroupScan = groupScan.applyFilter(conditionExp,optimizerContext,
+        optimizerContext.getFunctionRegistry(), optimizerContext.getPlannerSettings().getOptions());
+    logger.info("Took {} ms to apply filter on parquet row groups. ", timer.elapsed(TimeUnit.MILLISECONDS));
 
-        final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(),
-                newGroupScan, scan.getRowType());
+    if (newGroupScan == null ) {
+      return;
+    }
 
-        RelNode inputPrel = newScanPrel;
+    final ScanPrel newScanRel = ScanPrel.create(scan, scan.getTraitSet(), newGroupScan, scan.getRowType());
 
-        if(project != null){
-            inputPrel = project.copy(project.getTraitSet(), ImmutableList.of(inputPrel));
-        }
+    RelNode inputRel = newScanRel;
 
-        // Normally we could eliminate the filter if all expressions were pushed down;
-        // however, the Parquet filter implementation is type specific (whereas Drill is not)
-        final RelNode newFilter = filter.copy(filter.getTraitSet(), ImmutableList.of(inputPrel));
-        call.transformTo(newFilter);
+    if (project != null) {
+      inputRel = project.copy(project.getTraitSet(), ImmutableList.of(inputRel));
     }
+
+    final RelNode newFilter = filter.copy(filter.getTraitSet(), ImmutableList.<RelNode>of(inputRel));
+
+    call.transformTo(newFilter);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRGFilterEvaluator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRGFilterEvaluator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRGFilterEvaluator.java
new file mode 100644
index 0000000..bc4be13
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRGFilterEvaluator.java
@@ -0,0 +1,115 @@
+/**
+ * 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.store.parquet;
+
+import com.google.common.collect.Sets;
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+import org.apache.drill.exec.compile.sig.ConstantExpressionIdentifier;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.expr.stat.ParquetFilterPredicate;
+import org.apache.drill.exec.expr.stat.RangeExprEvaluator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.UdfUtilities;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.store.parquet.stat.ColumnStatCollector;
+import org.apache.drill.exec.store.parquet.stat.ColumnStatistics;
+import org.apache.drill.exec.store.parquet.stat.ParquetFooterStatCollector;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+public class ParquetRGFilterEvaluator {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRGFilterEvaluator.class);
+
+  public static boolean evalFilter(LogicalExpression expr, ParquetMetadata footer, int rowGroupIndex,
+      OptionManager options, FragmentContext fragmentContext) {
+    final HashMap<String, String> emptyMap = new HashMap<String, String>();
+    return evalFilter(expr, footer, rowGroupIndex, options, fragmentContext, emptyMap);
+  }
+
+  public static boolean evalFilter(LogicalExpression expr, ParquetMetadata footer, int rowGroupIndex,
+      OptionManager options, FragmentContext fragmentContext, Map<String, String> implicitColValues) {
+    // figure out the set of columns referenced in expression.
+    final Set<SchemaPath> schemaPathsInExpr = expr.accept(new FieldReferenceFinder(), null);
+    final ColumnStatCollector columnStatCollector = new ParquetFooterStatCollector(footer, rowGroupIndex, implicitColValues,true, options);
+
+    Map<SchemaPath, ColumnStatistics> columnStatisticsMap = columnStatCollector.collectColStat(schemaPathsInExpr);
+
+    boolean canDrop = canDrop(expr, columnStatisticsMap, footer.getBlocks().get(rowGroupIndex).getRowCount(), fragmentContext, fragmentContext.getFunctionRegistry());
+    return canDrop;
+  }
+
+
+  public static boolean canDrop(ParquetFilterPredicate parquetPredicate, Map<SchemaPath,
+      ColumnStatistics> columnStatisticsMap, long rowCount) {
+    boolean canDrop = false;
+    if (parquetPredicate != null) {
+      RangeExprEvaluator rangeExprEvaluator = new RangeExprEvaluator(columnStatisticsMap, rowCount);
+      canDrop = parquetPredicate.canDrop(rangeExprEvaluator);
+    }
+    return canDrop;
+  }
+
+
+  public static boolean canDrop(LogicalExpression expr, Map<SchemaPath, ColumnStatistics> columnStatisticsMap,
+      long rowCount, UdfUtilities udfUtilities, FunctionImplementationRegistry functionImplementationRegistry) {
+    ErrorCollector errorCollector = new ErrorCollectorImpl();
+    LogicalExpression materializedFilter = ExpressionTreeMaterializer.materializeFilterExpr(
+        expr, columnStatisticsMap, errorCollector, functionImplementationRegistry);
+
+    if (errorCollector.hasErrors()) {
+      logger.error("{} error(s) encountered when materialize filter expression : {}",
+          errorCollector.getErrorCount(), errorCollector.toErrorString());
+      return false;
+    }
+
+    Set<LogicalExpression> constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(materializedFilter);
+    ParquetFilterPredicate parquetPredicate = (ParquetFilterPredicate) ParquetFilterBuilder.buildParquetFilterPredicate(
+        materializedFilter, constantBoundaries, udfUtilities);
+
+    return canDrop(parquetPredicate, columnStatisticsMap, rowCount);
+  }
+
+  /**
+   * Search through a LogicalExpression, finding all internal schema path references and returning them in a set.
+   */
+  public static class FieldReferenceFinder extends AbstractExprVisitor<Set<SchemaPath>, Void, RuntimeException> {
+    @Override
+    public Set<SchemaPath> visitSchemaPath(SchemaPath path, Void value) {
+      Set<SchemaPath> set = Sets.newHashSet();
+      set.add(path);
+      return set;
+    }
+
+    @Override
+    public Set<SchemaPath> visitUnknown(LogicalExpression e, Void value) {
+      Set<SchemaPath> paths = Sets.newHashSet();
+      for (LogicalExpression ex : e) {
+        paths.addAll(ex.accept(this, null));
+      }
+      return paths;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
index cffcdac..f62efb5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
@@ -22,6 +22,7 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
@@ -50,6 +51,7 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
   private final ParquetFormatPlugin formatPlugin;
   private final List<RowGroupReadEntry> rowGroupReadEntries;
   private final List<SchemaPath> columns;
+  private LogicalExpression filter;
   private String selectionRoot;
 
   @JsonCreator
@@ -60,11 +62,12 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
       @JsonProperty("format") FormatPluginConfig formatConfig, //
       @JsonProperty("entries") LinkedList<RowGroupReadEntry> rowGroupReadEntries, //
       @JsonProperty("columns") List<SchemaPath> columns, //
-      @JsonProperty("selectionRoot") String selectionRoot //
+      @JsonProperty("selectionRoot") String selectionRoot, //
+      @JsonProperty("filter") LogicalExpression filter
   ) throws ExecutionSetupException {
     this(userName, (ParquetFormatPlugin) registry.getFormatPlugin(Preconditions.checkNotNull(storageConfig),
             formatConfig == null ? new ParquetFormatConfig() : formatConfig),
-        rowGroupReadEntries, columns, selectionRoot);
+        rowGroupReadEntries, columns, selectionRoot, filter);
   }
 
   public ParquetRowGroupScan( //
@@ -72,7 +75,8 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
       ParquetFormatPlugin formatPlugin, //
       List<RowGroupReadEntry> rowGroupReadEntries, //
       List<SchemaPath> columns, //
-      String selectionRoot //
+      String selectionRoot, //
+      LogicalExpression filter
   ) {
     super(userName);
     this.formatPlugin = Preconditions.checkNotNull(formatPlugin);
@@ -80,6 +84,7 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
     this.rowGroupReadEntries = rowGroupReadEntries;
     this.columns = columns == null ? GroupScan.ALL_COLUMNS : columns;
     this.selectionRoot = selectionRoot;
+    this.filter = filter;
   }
 
   @JsonProperty("entries")
@@ -114,7 +119,7 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
     Preconditions.checkArgument(children.isEmpty());
-    return new ParquetRowGroupScan(getUserName(), formatPlugin, rowGroupReadEntries, columns, selectionRoot);
+    return new ParquetRowGroupScan(getUserName(), formatPlugin, rowGroupReadEntries, columns, selectionRoot, filter);
   }
 
   @Override
@@ -126,6 +131,10 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
     return columns;
   }
 
+  public LogicalExpression getFilter() {
+    return filter;
+  }
+
   @Override
   public int getOperatorType() {
     return CoreOperatorType.PARQUET_ROW_GROUP_SCAN_VALUE;

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
index a14bab5..fa7f44e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
@@ -67,7 +67,8 @@ public class ParquetScanBatchCreator implements BatchCreator<ParquetRowGroupScan
 
     if (!columnExplorer.isStarQuery()) {
       rowGroupScan = new ParquetRowGroupScan(rowGroupScan.getUserName(), rowGroupScan.getStorageEngine(),
-          rowGroupScan.getRowGroupReadEntries(), columnExplorer.getTableColumns(), rowGroupScan.getSelectionRoot());
+          rowGroupScan.getRowGroupReadEntries(), columnExplorer.getTableColumns(), rowGroupScan.getSelectionRoot(),
+          rowGroupScan.getFilter());
       rowGroupScan.setOperatorId(rowGroupScan.getOperatorId());
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
index 57c0a66..be27f3e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
@@ -32,14 +32,15 @@ import static com.google.common.base.Preconditions.checkArgument;
 
 public class ParquetToDrillTypeConverter {
 
-  private static TypeProtos.MinorType getDecimalType(SchemaElement schemaElement) {
-    return schemaElement.getPrecision() <= 28 ? TypeProtos.MinorType.DECIMAL28SPARSE : MinorType.DECIMAL38SPARSE;
+
+  private static TypeProtos.MinorType getDecimalType(int precision) {
+    return precision <= 28 ? TypeProtos.MinorType.DECIMAL28SPARSE : MinorType.DECIMAL38SPARSE;
   }
 
   private static TypeProtos.MinorType getMinorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
-                                                   SchemaElement schemaElement, OptionManager options) {
+                                                   ConvertedType convertedType, int precision, int scale,
+      OptionManager options) {
 
-    ConvertedType convertedType = schemaElement.getConverted_type();
 
     switch (primitiveTypeName) {
       case BINARY:
@@ -51,7 +52,7 @@ public class ParquetToDrillTypeConverter {
             return (TypeProtos.MinorType.VARCHAR);
           case DECIMAL:
             ParquetReaderUtility.checkDecimalTypeEnabled(options);
-            return (getDecimalType(schemaElement));
+            return (getDecimalType(precision));
           default:
             return (TypeProtos.MinorType.VARBINARY);
         }
@@ -106,7 +107,7 @@ public class ParquetToDrillTypeConverter {
           return TypeProtos.MinorType.VARBINARY;
         } else if (convertedType == ConvertedType.DECIMAL) {
           ParquetReaderUtility.checkDecimalTypeEnabled(options);
-          return getDecimalType(schemaElement);
+          return getDecimalType(precision);
         } else if (convertedType == ConvertedType.INTERVAL) {
           return TypeProtos.MinorType.INTERVAL;
         }
@@ -118,12 +119,20 @@ public class ParquetToDrillTypeConverter {
   public static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
                                           TypeProtos.DataMode mode, SchemaElement schemaElement,
                                           OptionManager options) {
-    MinorType minorType = getMinorType(primitiveTypeName, length, schemaElement, options);
+    return toMajorType(primitiveTypeName, length, mode, schemaElement.getConverted_type(),
+        schemaElement.getPrecision(), schemaElement.getScale(), options);
+  }
+
+  public static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
+      TypeProtos.DataMode mode, ConvertedType convertedType, int precision, int scale,
+      OptionManager options) {
+    MinorType minorType = getMinorType(primitiveTypeName, length, convertedType, precision, scale, options);
     TypeProtos.MajorType.Builder typeBuilder = TypeProtos.MajorType.newBuilder().setMinorType(minorType).setMode(mode);
 
     if (CoreDecimalUtility.isDecimalType(minorType)) {
-      typeBuilder.setPrecision(schemaElement.getPrecision()).setScale(schemaElement.getScale());
+      typeBuilder.setPrecision(precision).setScale(scale);
     }
     return typeBuilder.build();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ColumnStatCollector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ColumnStatCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ColumnStatCollector.java
new file mode 100644
index 0000000..8f93c8a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ColumnStatCollector.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.store.parquet.stat;
+
+import org.apache.drill.common.expression.SchemaPath;
+
+import java.util.Map;
+import java.util.Set;
+
+public interface ColumnStatCollector {
+  /**
+   * Given a list of fields (SchemaPath), return mapping from field to its corresponding ColumnStatistics
+   * @return
+   */
+  Map<SchemaPath, ColumnStatistics> collectColStat(Set<SchemaPath> fields);
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ColumnStatistics.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ColumnStatistics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ColumnStatistics.java
new file mode 100644
index 0000000..7bad491
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ColumnStatistics.java
@@ -0,0 +1,40 @@
+/**
+ * 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.store.parquet.stat;
+
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.parquet.column.statistics.Statistics;
+
+public class ColumnStatistics {
+  private final Statistics statistics;
+  private final TypeProtos.MajorType majorType;
+
+  public ColumnStatistics(final Statistics statistics, final TypeProtos.MajorType majorType) {
+    this.statistics = statistics;
+    this.majorType = majorType;
+  }
+
+  public Statistics getStatistics() {
+    return this.statistics;
+  }
+
+  public TypeProtos.MajorType getMajorType() {
+    return this.majorType;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetFooterStatCollector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetFooterStatCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetFooterStatCollector.java
new file mode 100644
index 0000000..6294655
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetFooterStatCollector.java
@@ -0,0 +1,199 @@
+/**
+ * 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.store.parquet.stat;
+
+import com.google.common.base.Stopwatch;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.store.ParquetOutputRecordWriter;
+import org.apache.drill.exec.store.parquet.ParquetGroupScan;
+import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
+import org.apache.drill.exec.store.parquet.columnreaders.ParquetToDrillTypeConverter;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.statistics.BinaryStatistics;
+import org.apache.parquet.column.statistics.IntStatistics;
+import org.apache.parquet.column.statistics.LongStatistics;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.format.SchemaElement;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.joda.time.DateTimeConstants;
+import org.joda.time.DateTimeUtils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public class ParquetFooterStatCollector implements ColumnStatCollector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetFooterStatCollector.class);
+
+  private final ParquetMetadata footer;
+  private final int rowGroupIndex;
+  private final OptionManager options;
+  private final Map<String, String> implicitColValues;
+  private final boolean autoCorrectCorruptDates;
+
+  public ParquetFooterStatCollector(ParquetMetadata footer, int rowGroupIndex, Map<String, String> implicitColValues,
+      boolean autoCorrectCorruptDates, OptionManager options) {
+    this.footer = footer;
+    this.rowGroupIndex = rowGroupIndex;
+
+    // Reasons to pass implicit columns and their values:
+    // 1. Differentiate implicit columns from regular non-exist columns. Implicit columns do not
+    //    exist in parquet metadata. Without such knowledge, implicit columns is treated as non-exist
+    //    column.  A condition on non-exist column would lead to canDrop = true, which is not the
+    //    right behavior for condition on implicit columns.
+
+    // 2. Pass in the implicit column name with corresponding values, and wrap them in Statistics with
+    //    min and max having same value. This expands the possibility of pruning.
+    //    For example, regCol = 5 or dir0 = 1995. If regCol is not a partition column, we would not do
+    //    any partition pruning in the current partition pruning logical. Pass the implicit column values
+    //    may allow us to prune some row groups using condition regCol = 5 or dir0 = 1995.
+
+    this.implicitColValues = implicitColValues;
+    this.autoCorrectCorruptDates = autoCorrectCorruptDates;
+    this.options = options;
+  }
+
+  @Override
+  public Map<SchemaPath, ColumnStatistics> collectColStat(Set<SchemaPath> fields) {
+    Stopwatch timer = Stopwatch.createStarted();
+
+    ParquetReaderUtility.DateCorruptionStatus containsCorruptDates =
+        ParquetReaderUtility.detectCorruptDates(footer, new ArrayList<>(fields), autoCorrectCorruptDates);
+
+    // map from column name to ColumnDescriptor
+    Map<SchemaPath, ColumnDescriptor> columnDescMap = new HashMap<>();
+
+    // map from column name to ColumnChunkMetaData
+    final Map<SchemaPath, ColumnChunkMetaData> columnChkMetaMap = new HashMap<>();
+
+    // map from column name to MajorType
+    final Map<SchemaPath, TypeProtos.MajorType> columnTypeMap = new HashMap<>();
+
+    // map from column name to SchemaElement
+    final Map<SchemaPath, SchemaElement> schemaElementMap = new HashMap<>();
+
+    // map from column name to column statistics.
+    final Map<SchemaPath, ColumnStatistics> statMap = new HashMap<>();
+
+    final org.apache.parquet.format.FileMetaData fileMetaData = new ParquetMetadataConverter().toParquetMetadata(ParquetFileWriter.CURRENT_VERSION, footer);
+
+    for (final ColumnDescriptor column : footer.getFileMetaData().getSchema().getColumns()) {
+      final SchemaPath schemaPath = SchemaPath.getCompoundPath(column.getPath());
+      if (fields.contains(schemaPath)) {
+        columnDescMap.put(schemaPath, column);
+      }
+    }
+
+    for (final SchemaElement se : fileMetaData.getSchema()) {
+      final SchemaPath schemaPath = SchemaPath.getSimplePath(se.getName());
+      if (fields.contains(schemaPath)) {
+        schemaElementMap.put(schemaPath, se);
+      }
+    }
+
+    for (final ColumnChunkMetaData colMetaData: footer.getBlocks().get(rowGroupIndex).getColumns()) {
+      final SchemaPath schemaPath = SchemaPath.getCompoundPath(colMetaData.getPath().toArray());
+      if (fields.contains(schemaPath)) {
+        columnChkMetaMap.put(schemaPath, colMetaData);
+      }
+    }
+
+    for (final SchemaPath path : fields) {
+      if (columnDescMap.containsKey(path) && schemaElementMap.containsKey(path) && columnChkMetaMap.containsKey(path)) {
+        ColumnDescriptor columnDesc =  columnDescMap.get(path);
+        SchemaElement se = schemaElementMap.get(path);
+        ColumnChunkMetaData metaData = columnChkMetaMap.get(path);
+
+        TypeProtos.MajorType type = ParquetToDrillTypeConverter.toMajorType(columnDesc.getType(), se.getType_length(),
+            getDataMode(columnDesc), se, options);
+
+        columnTypeMap.put(path, type);
+
+        Statistics stat = metaData.getStatistics();
+        if (type.getMinorType() == TypeProtos.MinorType.DATE) {
+          stat = convertDateStatIfNecessary(metaData.getStatistics(), containsCorruptDates);
+        }
+
+        statMap.put(path, new ColumnStatistics(stat, type));
+      } else {
+        final String columnName = path.getRootSegment().getPath();
+        if (implicitColValues.containsKey(columnName)) {
+          TypeProtos.MajorType type = Types.required(TypeProtos.MinorType.VARCHAR);
+          Statistics stat = new BinaryStatistics();
+          stat.setNumNulls(0);
+          byte[] val = implicitColValues.get(columnName).getBytes();
+          stat.setMinMaxFromBytes(val, val);
+          statMap.put(path, new ColumnStatistics(stat, type));
+        }
+      }
+    }
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("Took {} ms to column statistics for row group", timer.elapsed(TimeUnit.MILLISECONDS));
+    }
+
+    return statMap;
+  }
+
+  private static TypeProtos.DataMode getDataMode(ColumnDescriptor column) {
+    if (column.getMaxRepetitionLevel() > 0 ) {
+      return TypeProtos.DataMode.REPEATED;
+    } else if (column.getMaxDefinitionLevel() == 0) {
+      return TypeProtos.DataMode.REQUIRED;
+    } else {
+      return TypeProtos.DataMode.OPTIONAL;
+    }
+  }
+
+  public static Statistics convertDateStatIfNecessary(Statistics stat,
+      ParquetReaderUtility.DateCorruptionStatus containsCorruptDates) {
+    IntStatistics dateStat = (IntStatistics) stat;
+    LongStatistics dateMLS = new LongStatistics();
+
+    boolean isDateCorrect = containsCorruptDates == ParquetReaderUtility.DateCorruptionStatus.META_SHOWS_NO_CORRUPTION;
+
+    // Only do conversion when stat is NOT empty.
+    if (!dateStat.isEmpty()) {
+        dateMLS.setMinMax(
+            convertToDrillDateValue(dateStat.getMin(), isDateCorrect),
+            convertToDrillDateValue(dateStat.getMax(), isDateCorrect));
+        dateMLS.setNumNulls(dateStat.getNumNulls());
+    }
+
+    return dateMLS;
+
+  }
+
+  private static long convertToDrillDateValue(int dateValue, boolean isDateCorrect) {
+    // See DRILL-4203 for the background regarding date type corruption issue in Drill CTAS prior to 1.9.0 release.
+    if (isDateCorrect) {
+      return dateValue * (long) DateTimeConstants.MILLIS_PER_DAY;
+    } else {
+      return (dateValue - ParquetReaderUtility.CORRECT_CORRUPT_DATE_SHIFT) * DateTimeConstants.MILLIS_PER_DAY;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java
new file mode 100644
index 0000000..3fe10c8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet.stat;
+
+import com.google.common.base.Stopwatch;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.store.parquet.Metadata;
+import org.apache.drill.exec.store.parquet.ParquetGroupScan;
+import org.apache.parquet.column.statistics.BinaryStatistics;
+import org.apache.parquet.column.statistics.DoubleStatistics;
+import org.apache.parquet.column.statistics.FloatStatistics;
+import org.apache.parquet.column.statistics.IntStatistics;
+import org.apache.parquet.column.statistics.LongStatistics;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.joda.time.DateTimeConstants;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public class ParquetMetaStatCollector implements  ColumnStatCollector{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetMetaStatCollector.class);
+
+  private  final Metadata.ParquetTableMetadataBase parquetTableMetadata;
+  private  final List<? extends Metadata.ColumnMetadata> columnMetadataList;
+  final Map<String, String> implicitColValues;
+
+  public ParquetMetaStatCollector(Metadata.ParquetTableMetadataBase parquetTableMetadata,
+      List<? extends Metadata.ColumnMetadata> columnMetadataList, Map<String, String> implicitColValues) {
+    this.parquetTableMetadata = parquetTableMetadata;
+    this.columnMetadataList = columnMetadataList;
+
+    // Reasons to pass implicit columns and their values:
+    // 1. Differentiate implicit columns from regular non-exist columns. Implicit columns do not
+    //    exist in parquet metadata. Without such knowledge, implicit columns is treated as non-exist
+    //    column.  A condition on non-exist column would lead to canDrop = true, which is not the
+    //    right behavior for condition on implicit columns.
+
+    // 2. Pass in the implicit column name with corresponding values, and wrap them in Statistics with
+    //    min and max having same value. This expands the possibility of pruning.
+    //    For example, regCol = 5 or dir0 = 1995. If regCol is not a partition column, we would not do
+    //    any partition pruning in the current partition pruning logical. Pass the implicit column values
+    //    may allow us to prune some row groups using condition regCol = 5 or dir0 = 1995.
+
+    this.implicitColValues = implicitColValues;
+  }
+
+  @Override
+  public Map<SchemaPath, ColumnStatistics> collectColStat(Set<SchemaPath> fields) {
+    Stopwatch timer = Stopwatch.createStarted();
+
+    // map from column to ColumnMetadata
+    final Map<SchemaPath, Metadata.ColumnMetadata> columnMetadataMap = new HashMap<>();
+
+    // map from column name to column statistics.
+    final Map<SchemaPath, ColumnStatistics> statMap = new HashMap<>();
+
+    for (final Metadata.ColumnMetadata columnMetadata : columnMetadataList) {
+      SchemaPath schemaPath = SchemaPath.getCompoundPath(columnMetadata.getName());
+      columnMetadataMap.put(schemaPath, columnMetadata);
+    }
+
+    for (final SchemaPath schemaPath : fields) {
+      final PrimitiveType.PrimitiveTypeName primitiveType;
+      final OriginalType originalType;
+
+      final Metadata.ColumnMetadata columnMetadata = columnMetadataMap.get(schemaPath);
+
+      if (columnMetadata != null) {
+        final Object min = columnMetadata.getMinValue();
+        final Object max = columnMetadata.getMaxValue();
+        final Long numNull = columnMetadata.getNulls();
+
+        primitiveType = this.parquetTableMetadata.getPrimitiveType(columnMetadata.getName());
+        originalType = this.parquetTableMetadata.getOriginalType(columnMetadata.getName());
+        final Integer repetitionLevel = this.parquetTableMetadata.getRepetitionLevel(columnMetadata.getName());
+
+        statMap.put(schemaPath, getStat(min, max, numNull, primitiveType, originalType, repetitionLevel));
+      } else {
+        final String columnName = schemaPath.getRootSegment().getPath();
+        if (implicitColValues.containsKey(columnName)) {
+          TypeProtos.MajorType type = Types.required(TypeProtos.MinorType.VARCHAR);
+          Statistics stat = new BinaryStatistics();
+          stat.setNumNulls(0);
+          byte[] val = implicitColValues.get(columnName).getBytes();
+          stat.setMinMaxFromBytes(val, val);
+          statMap.put(schemaPath, new ColumnStatistics(stat, type));
+        }
+      }
+    }
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("Took {} ms to column statistics for row group", timer.elapsed(TimeUnit.MILLISECONDS));
+    }
+
+    return statMap;
+  }
+
+  private ColumnStatistics getStat(Object min, Object max, Long numNull,
+      PrimitiveType.PrimitiveTypeName primitiveType, OriginalType originalType, Integer repetitionLevel) {
+    Statistics stat = Statistics.getStatsBasedOnType(primitiveType);
+    Statistics convertedStat = stat;
+
+    TypeProtos.MajorType type = ParquetGroupScan.getType(primitiveType, originalType);
+
+    // Change to repeated if repetitionLevel > 0
+    if (repetitionLevel != null && repetitionLevel > 0) {
+      type = TypeProtos.MajorType.newBuilder().setMinorType(type.getMinorType()).setMode(TypeProtos.DataMode.REPEATED).build();
+    }
+
+    if (numNull != null) {
+      stat.setNumNulls(numNull.longValue());
+    }
+
+    if (min != null && max != null ) {
+      switch (type.getMinorType()) {
+      case INT :
+      case TIME:
+        ((IntStatistics) stat).setMinMax(((Integer) min).intValue(), ((Integer) max).intValue());
+        break;
+      case BIGINT:
+      case TIMESTAMP:
+        ((LongStatistics) stat).setMinMax(((Long) min).longValue(), ((Long) max).longValue());
+        break;
+      case FLOAT4:
+        ((FloatStatistics) stat).setMinMax(((Float) min).floatValue(), ((Float) max).floatValue());
+        break;
+      case FLOAT8:
+        ((DoubleStatistics) stat).setMinMax(((Double) min).doubleValue(), ((Double) max).doubleValue());
+        break;
+      case DATE:
+        convertedStat = new LongStatistics();
+        convertedStat.setNumNulls(stat.getNumNulls());
+        final long minMS = convertToDrillDateValue(((Integer) min).intValue());
+        final long maxMS = convertToDrillDateValue(((Integer) max).intValue());
+        ((LongStatistics) convertedStat ).setMinMax(minMS, maxMS);
+        break;
+      default:
+      }
+    }
+
+    return new ColumnStatistics(convertedStat, type);
+  }
+
+  private static long convertToDrillDateValue(int dateValue) {
+      return dateValue * (long) DateTimeConstants.MILLIS_PER_DAY;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/test/java/org/apache/drill/TestPartitionFilter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestPartitionFilter.java b/exec/java-exec/src/test/java/org/apache/drill/TestPartitionFilter.java
index b1d833b..7d029ea 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestPartitionFilter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestPartitionFilter.java
@@ -202,7 +202,8 @@ public class TestPartitionFilter extends PlanTestBase {
   @Test // Parquet: one side of OR has partition filter only, other side has both partition filter and non-partition filter
   public void testPartitionFilter6_Parquet_from_CTAS() throws Exception {
     String query = String.format("select * from dfs_test.tmp.parquet where (yr=1995 and o_totalprice < 40000) or yr=1996", TEST_RES_PATH);
-    testIncludeFilter(query, 8, "Filter", 46);
+    // Parquet RG filter pushdown further reduces to 6 files.
+    testIncludeFilter(query, 6, "Filter", 46);
   }
 
   @Test // Parquet: trivial case with 1 partition filter
@@ -232,13 +233,15 @@ public class TestPartitionFilter extends PlanTestBase {
   @Test // Parquet: partition filter on subdirectory only plus non-partition filter
   public void testPartitionFilter9_Parquet() throws Exception {
     String query = String.format("select * from dfs_test.`%s/multilevel/parquet` where dir1 in ('Q1','Q4') and o_totalprice < 40000", TEST_RES_PATH);
-    testIncludeFilter(query, 6, "Filter", 9);
+    // Parquet RG filter pushdown further reduces to 4 files.
+    testIncludeFilter(query, 4, "Filter", 9);
   }
 
   @Test
   public void testPartitionFilter9_Parquet_from_CTAS() throws Exception {
     String query = String.format("select * from dfs_test.tmp.parquet where qrtr in ('Q1','Q4') and o_totalprice < 40000", TEST_RES_PATH);
-    testIncludeFilter(query, 6, "Filter", 9);
+    // Parquet RG filter pushdown further reduces to 4 files.
+    testIncludeFilter(query, 4, "Filter", 9);
   }
 
   @Test
@@ -272,7 +275,8 @@ public class TestPartitionFilter extends PlanTestBase {
   public void testMainQueryFilterRegularColumn() throws Exception {
     String root = FileUtils.getResourceAsFile("/multilevel/parquet").toURI().toString();
     String query =  String.format("select * from (select dir0, o_custkey from dfs_test.`%s` where dir0='1994' and o_custkey = 10) t limit 0", root);
-    testIncludeFilter(query, 4, "Filter", 0);
+    // with Parquet RG filter pushdown, reduce to 1 file ( o_custkey all > 10).
+    testIncludeFilter(query, 1, "Filter", 0);
   }
 
   @Test // see DRILL-2852 and DRILL-3591


[04/10] drill git commit: DRILL-4800: Various fixes. Fix buffer underflow exception in BufferedDirectBufInputStream. Fix writer index for in64 dictionary encoded types. Added logging to help debug. Fix memory leaks. Work around issues with of InputStream

Posted by pa...@apache.org.
DRILL-4800: Various fixes. Fix buffer underflow exception in BufferedDirectBufInputStream. Fix writer index for in64 dictionary encoded types. Added logging to help debug. Fix memory leaks. Work around issues with of InputStream.available() ( Do not use hasRemainder; Remove check for EOF in BufferedDirectBufInputStream.read() ). Finalize defaults. Remove commented code.
Addressed review comments

This closes #611


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/ee3489ce
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/ee3489ce
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/ee3489ce

Branch: refs/heads/master
Commit: ee3489ce3b6e5ad53e5c3a59b6e2e4e50773c630
Parents: 7f5acf8
Author: Parth Chandra <pa...@apache.org>
Authored: Tue Sep 13 21:47:49 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Nov 4 15:57:44 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/drill/test/DrillTest.java   |  2 +-
 .../drill/exec/ops/OperatorContextImpl.java     |  3 +
 .../drill/exec/server/BootStrapContext.java     |  7 +-
 .../parquet/columnreaders/AsyncPageReader.java  | 71 +++++++-------
 .../parquet/columnreaders/ColumnReader.java     | 61 ++++--------
 .../NullableFixedByteAlignedReaders.java        |  3 +
 .../store/parquet/columnreaders/PageReader.java | 16 ++--
 .../columnreaders/ParquetRecordReader.java      | 20 ++--
 .../columnreaders/VarLenBinaryReader.java       | 86 +++--------------
 .../BufferedDirectBufInputStream.java           | 97 ++++++++++----------
 .../util/filereader/DirectBufInputStream.java   | 21 ++++-
 .../physical/impl/writer/TestParquetWriter.java |  3 +-
 12 files changed, 165 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/common/src/test/java/org/apache/drill/test/DrillTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/test/DrillTest.java b/common/src/test/java/org/apache/drill/test/DrillTest.java
index ccc297d..18c2c1a 100644
--- a/common/src/test/java/org/apache/drill/test/DrillTest.java
+++ b/common/src/test/java/org/apache/drill/test/DrillTest.java
@@ -55,7 +55,7 @@ public class DrillTest {
   static MemWatcher memWatcher;
   static String className;
 
-  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(50000*10000);
+  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(50000);
   @Rule public final TestLogReporter logOutcome = LOG_OUTCOME;
 
   @Rule public final TestRule REPEAT_RULE = TestTools.getRepeatRule(false);

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
index 38ddd16..390b71c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
@@ -192,6 +192,9 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
   }
 
   @Override
+  /*
+     Creates a DrillFileSystem that does not automatically track operator stats.
+   */
   public DrillFileSystem newNonTrackingFileSystem(Configuration conf) throws IOException {
     Preconditions.checkState(fs == null, "Tried to create a second FileSystem. Can only be called once per OperatorContext");
     fs = new DrillFileSystem(conf, null);

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
index adb6323..c498185 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
@@ -53,8 +53,7 @@ public class BootStrapContext implements AutoCloseable {
     this.config = config;
     this.classpathScan = classpathScan;
     this.loop = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), "BitServer-");
-    this.loop2 = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS),
-        "BitClient-");
+    this.loop2 = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), "BitClient-");
     // Note that metrics are stored in a static instance
     this.metrics = DrillMetrics.getRegistry();
     this.allocator = RootAllocatorFactory.newRoot(config);
@@ -79,8 +78,8 @@ public class BootStrapContext implements AutoCloseable {
     final int numScanDecodeThreads = (int) config.getDouble(ExecConstants.SCAN_DECODE_THREADPOOL_SIZE);
     final int scanThreadPoolSize =
         MIN_SCAN_THREADPOOL_SIZE > numScanThreads ? MIN_SCAN_THREADPOOL_SIZE : numScanThreads;
-    final int scanDecodeThreadPoolSize = numCores > numScanDecodeThreads ? numCores : numScanDecodeThreads;
-
+    final int scanDecodeThreadPoolSize =
+        (numCores + 1) / 2 > numScanDecodeThreads ? (numCores + 1) / 2 : numScanDecodeThreads;
     this.scanExecutor = Executors.newFixedThreadPool(scanThreadPoolSize, new NamedThreadFactory("scan-"));
     this.scanDecodeExecutor =
         Executors.newFixedThreadPool(scanDecodeThreadPoolSize, new NamedThreadFactory("scan-decode-"));

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
index b2bdef3..e2ba865 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
@@ -63,10 +63,12 @@ class AsyncPageReader extends PageReader {
     asyncPageRead = threadPool.submit(new AsyncPageReaderTask());
   }
 
-  @Override protected void loadDictionaryIfExists(final ColumnReader<?> parentStatus,
+  @Override
+  protected void loadDictionaryIfExists(final ColumnReader<?> parentStatus,
       final ColumnChunkMetaData columnChunkMetaData, final DirectBufInputStream f) throws UserException {
     if (columnChunkMetaData.getDictionaryPageOffset() > 0) {
       try {
+        assert(columnChunkMetaData.getDictionaryPageOffset() >= dataReader.getPos() );
         dataReader.skip(columnChunkMetaData.getDictionaryPageOffset() - dataReader.getPos());
       } catch (IOException e) {
         handleAndThrowException(e, "Error Reading dictionary page.");
@@ -90,12 +92,12 @@ class AsyncPageReader extends PageReader {
       isDictionary = readStatus.isDictionaryPage;
     }
     if (parentColumnReader.columnChunkMetaData.getCodec() != CompressionCodecName.UNCOMPRESSED) {
-      DrillBuf uncompressedData = data;
-      data = decompress(readStatus.getPageHeader(), uncompressedData);
+      DrillBuf compressedData = data;
+      data = decompress(readStatus.getPageHeader(), compressedData);
       synchronized (this) {
         readStatus.setPageData(null);
       }
-      uncompressedData.release();
+      compressedData.release();
     } else {
       if (isDictionary) {
         stats.totalDictPageReadBytes.addAndGet(readStatus.bytesRead);
@@ -160,23 +162,12 @@ class AsyncPageReader extends PageReader {
     pageDataBuf = allocateTemporaryBuffer(uncompressedSize);
     try {
       timer.start();
-      if (logger.isTraceEnabled()) {
-        logger.trace("Decompress (1)==> Col: {}  readPos: {}  compressed_size: {}  compressedPageData: {}",
-            parentColumnReader.columnChunkMetaData.toString(), dataReader.getPos(),
-            pageHeader.getCompressed_page_size(), ByteBufUtil.hexDump(compressedData));
-      }
       CompressionCodecName codecName = parentColumnReader.columnChunkMetaData.getCodec();
       ByteBuffer input = compressedData.nioBuffer(0, compressedSize);
       ByteBuffer output = pageDataBuf.nioBuffer(0, uncompressedSize);
       DecompressionHelper decompressionHelper = new DecompressionHelper(codecName);
       decompressionHelper.decompress(input, compressedSize, output, uncompressedSize);
       pageDataBuf.writerIndex(uncompressedSize);
-      if (logger.isTraceEnabled()) {
-        logger.trace(
-            "Decompress (2)==> Col: {}  readPos: {}  uncompressed_size: {}  uncompressedPageData: {}",
-            parentColumnReader.columnChunkMetaData.toString(), dataReader.getPos(),
-            pageHeader.getUncompressed_page_size(), ByteBufUtil.hexDump(pageDataBuf));
-      }
       timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
       this.updateStats(pageHeader, "Decompress", 0, timeToRead, compressedSize, uncompressedSize);
     } catch (IOException e) {
@@ -219,30 +210,23 @@ class AsyncPageReader extends PageReader {
       }
     } while (pageHeader.getType() == PageType.DICTIONARY_PAGE);
 
-    if (dataReader.hasRemainder() && parentColumnReader.totalValuesRead + readStatus.getValuesRead()
+    if (parentColumnReader.totalValuesRead + readStatus.getValuesRead()
         < parentColumnReader.columnChunkMetaData.getValueCount()) {
       asyncPageRead = threadPool.submit(new AsyncPageReaderTask());
     }
 
     pageHeader = readStatus.getPageHeader();
     pageData = getDecompressedPageData(readStatus);
-    if (logger.isTraceEnabled()) {
-      logger.trace("AsyncPageReader: Col: {}  pageData: {}",
-          this.parentColumnReader.columnChunkMetaData.toString(), ByteBufUtil.hexDump(pageData));
-      logger.trace("AsyncPageReaderTask==> Col: {}  readPos: {}  Uncompressed_size: {}  pageData: {}",
-          parentColumnReader.columnChunkMetaData.toString(), dataReader.getPos(),
-          pageHeader.getUncompressed_page_size(), ByteBufUtil.hexDump(pageData));
-    }
+
 
   }
 
 
   @Override public void clear() {
     if (asyncPageRead != null) {
-      asyncPageRead.cancel(true);
       try {
-        ReadStatus r = asyncPageRead.get();
-        r.getPageData().release();
+        final ReadStatus readStatus = asyncPageRead.get();
+        readStatus.getPageData().release();
       } catch (Exception e) {
         // Do nothing.
       }
@@ -319,7 +303,8 @@ class AsyncPageReader extends PageReader {
       ReadStatus readStatus = new ReadStatus();
 
       String oldname = Thread.currentThread().getName();
-      Thread.currentThread().setName(parent.parentColumnReader.columnChunkMetaData.toString());
+      String name = parent.parentColumnReader.columnChunkMetaData.toString();
+      Thread.currentThread().setName(name);
 
       long bytesRead = 0;
       long valuesRead = 0;
@@ -327,10 +312,28 @@ class AsyncPageReader extends PageReader {
 
       DrillBuf pageData = null;
       try {
+        long s = parent.dataReader.getPos();
         PageHeader pageHeader = Util.readPageHeader(parent.dataReader);
+        long e = parent.dataReader.getPos();
+        if (logger.isTraceEnabled()) {
+          logger.trace("[{}]: Read Page Header : ReadPos = {} : Bytes Read = {} ", name, s, e - s);
+        }
         int compressedSize = pageHeader.getCompressed_page_size();
+        s = parent.dataReader.getPos();
         pageData = parent.dataReader.getNext(compressedSize);
+        e = parent.dataReader.getPos();
         bytesRead = compressedSize;
+
+        if (logger.isTraceEnabled()) {
+          DrillBuf bufStart = pageData.slice(0, compressedSize>100?100:compressedSize);
+          int endOffset = compressedSize>100?compressedSize-100:0;
+          DrillBuf bufEnd = pageData.slice(endOffset, compressedSize-endOffset);
+          logger
+              .trace("[{}]: Read Page Data : ReadPos = {} : Bytes Read = {} : Buf Start = {} : Buf End = {} ",
+                  name, s, e - s, ByteBufUtil.hexDump(bufStart), ByteBufUtil.hexDump(bufEnd));
+
+        }
+
         synchronized (parent) {
           if (pageHeader.getType() == PageType.DICTIONARY_PAGE) {
             readStatus.setIsDictionaryPage(true);
@@ -353,10 +356,6 @@ class AsyncPageReader extends PageReader {
         throw e;
       }
       Thread.currentThread().setName(oldname);
-      if(logger.isTraceEnabled()) {
-        logger.trace("AsyncPageReaderTask==> Col: {}  readPos: {}  bytesRead: {}  pageData: {}", parent.parentColumnReader.columnChunkMetaData.toString(),
-            parent.dataReader.getPos(), bytesRead, ByteBufUtil.hexDump(pageData));
-      }
       return readStatus;
     }
 
@@ -365,7 +364,7 @@ class AsyncPageReader extends PageReader {
   private class DecompressionHelper {
     final CompressionCodecName codecName;
 
-    public DecompressionHelper(CompressionCodecName codecName){
+    public DecompressionHelper(CompressionCodecName codecName) {
       this.codecName = codecName;
     }
 
@@ -376,6 +375,7 @@ class AsyncPageReader extends PageReader {
       // expensive copying.
       if (codecName == CompressionCodecName.GZIP) {
         GzipCodec codec = new GzipCodec();
+        // DirectDecompressor: @see https://hadoop.apache.org/docs/r2.7.2/api/org/apache/hadoop/io/compress/DirectDecompressor.html
         DirectDecompressor directDecompressor = codec.createDirectDecompressor();
         if (directDecompressor != null) {
           logger.debug("Using GZIP direct decompressor.");
@@ -394,9 +394,10 @@ class AsyncPageReader extends PageReader {
           output.put(outputBytes);
         }
       } else if (codecName == CompressionCodecName.SNAPPY) {
-        // For Snappy, just call the Snappy decompressor directly.
-        // It is thread safe. The Hadoop layers though, appear to be
-        // not quite reliable in a multithreaded environment
+        // For Snappy, just call the Snappy decompressor directly instead
+        // of going thru the DirectDecompressor class.
+        // The Snappy codec is itself thread safe, while going thru the DirectDecompressor path
+        // seems to have concurrency issues.
         output.clear();
         int size = Snappy.uncompress(input, output);
         output.limit(size);

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
index 29e23bc..73cbc3d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
@@ -163,22 +163,13 @@ public abstract class ColumnReader<V extends ValueVector> {
 
   protected abstract void readField(long recordsToRead);
 
-  /*
-  public Future<Boolean> determineSizeAsync(long recordsReadInCurrentPass,
-      Integer lengthVarFieldsInCurrentRecord) throws IOException {
-    Future<Boolean> r = threadPool.submit(
-        new ColumnReaderDetermineSizeTask(recordsReadInCurrentPass, lengthVarFieldsInCurrentRecord));
-    return r;
-  }
-  */
-
   /**
    * Determines the size of a single value in a variable column.
    *
    * Return value indicates if we have finished a row group and should stop reading
    *
    * @param recordsReadInCurrentPass
-   * @ param lengthVarFieldsInCurrentRecord
+   * @param lengthVarFieldsInCurrentRecord
    * @return - true if we should stop reading
    * @throws IOException
    */
@@ -194,7 +185,7 @@ public abstract class ColumnReader<V extends ValueVector> {
       return true;
     }
 
-    //lengthVarFieldsInCurrentRecord += dataTypeLengthInBits;
+    // Never used in this code path. Hard to remove because the method is overidden by subclasses
     lengthVarFieldsInCurrentRecord = -1;
 
     doneReading = checkVectorCapacityReached();
@@ -307,41 +298,18 @@ public abstract class ColumnReader<V extends ValueVector> {
     @Override public Long call() throws IOException{
 
       String oldname = Thread.currentThread().getName();
-      Thread.currentThread().setName(oldname+"Decode-"+this.parent.columnChunkMetaData.toString());
-
-      this.parent.processPages(recordsToReadInThisPass);
-
-      Thread.currentThread().setName(oldname);
-      return recordsToReadInThisPass;
-    }
-
-  }
-
-  /*
-  private class ColumnReaderDetermineSizeTask implements Callable<Boolean> {
-
-    private final ColumnReader parent = ColumnReader.this;
-    private final long recordsReadInCurrentPass;
-    private final Integer lengthVarFieldsInCurrentRecord;
-
-    public ColumnReaderDetermineSizeTask(long recordsReadInCurrentPass, Integer lengthVarFieldsInCurrentRecord){
-      this.recordsReadInCurrentPass = recordsReadInCurrentPass;
-      this.lengthVarFieldsInCurrentRecord = lengthVarFieldsInCurrentRecord;
-    }
-
-    @Override public Boolean call() throws IOException{
-
-      String oldname = Thread.currentThread().getName();
-      Thread.currentThread().setName(oldname+"Decode-"+this.parent.columnChunkMetaData.toString());
+      try {
+        Thread.currentThread().setName(oldname + "Decode-" + this.parent.columnChunkMetaData.toString());
 
-      boolean b = this.parent.determineSize(recordsReadInCurrentPass, lengthVarFieldsInCurrentRecord);
+        this.parent.processPages(recordsToReadInThisPass);
+        return recordsToReadInThisPass;
 
-      Thread.currentThread().setName(oldname);
-      return b;
+      } finally {
+        Thread.currentThread().setName(oldname);
+      }
     }
 
   }
-  */
 
   private class ColumnReaderReadRecordsTask implements Callable<Integer> {
 
@@ -355,12 +323,15 @@ public abstract class ColumnReader<V extends ValueVector> {
     @Override public Integer call() throws IOException{
 
       String oldname = Thread.currentThread().getName();
-      Thread.currentThread().setName("Decode-"+this.parent.columnChunkMetaData.toString());
+      try {
+        Thread.currentThread().setName("Decode-" + this.parent.columnChunkMetaData.toString());
 
-      this.parent.readRecords(recordsToRead);
+        this.parent.readRecords(recordsToRead);
+        return recordsToRead;
 
-      Thread.currentThread().setName(oldname);
-      return recordsToRead;
+      } finally {
+        Thread.currentThread().setName(oldname);
+      }
     }
 
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
index f4fe5ee..e20504f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
@@ -150,7 +150,10 @@ public class NullableFixedByteAlignedReaders {
         for (int i = 0; i < recordsToReadInThisPass; i++){
           valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, pageReader.dictionaryValueReader.readInteger());
         }
+        int writerIndex = castedBaseVector.getBuffer().writerIndex();
+        castedBaseVector.getBuffer().setIndex(0, writerIndex + (int)readLength);
       } else {
+
         for (int i = 0; i < recordsToReadInThisPass; i++){
           valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, pageReader.valueReader.readInteger());
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
index 0736f01..f71eeae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.parquet.columnreaders;
 
 import com.google.common.base.Stopwatch;
+import io.netty.buffer.ByteBufUtil;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.util.filereader.BufferedDirectBufInputStream;
 import io.netty.buffer.ByteBuf;
@@ -63,9 +64,8 @@ class PageReader {
   public static final ParquetMetadataConverter METADATA_CONVERTER = ParquetFormatPlugin.parquetMetadataConverter;
 
   protected final org.apache.drill.exec.store.parquet.columnreaders.ColumnReader<?> parentColumnReader;
-  //private final ColumnDataReader dataReader;
   protected final DirectBufInputStream dataReader;
-  //der; buffer to store bytes of current page
+  //buffer to store bytes of current page
   protected DrillBuf pageData;
 
   // for variable length data we need to keep track of our current position in the page data
@@ -189,6 +189,11 @@ class PageReader {
     if (parentColumnReader.columnChunkMetaData.getCodec() == CompressionCodecName.UNCOMPRESSED) {
       timer.start();
       pageDataBuf = dataReader.getNext(compressedSize);
+      if (logger.isTraceEnabled()) {
+        logger.trace("PageReaderTask==> Col: {}  readPos: {}  Uncompressed_size: {}  pageData: {}",
+            parentColumnReader.columnChunkMetaData.toString(), dataReader.getPos(),
+            pageHeader.getUncompressed_page_size(), ByteBufUtil.hexDump(pageData));
+      }
       timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
       this.updateStats(pageHeader, "Page Read", start, timeToRead, compressedSize, uncompressedSize);
     } else {
@@ -247,9 +252,6 @@ class PageReader {
       }
     } while (pageHeader.getType() == PageType.DICTIONARY_PAGE);
 
-    //TODO: Handle buffer allocation exception
-
-    //allocatePageData(pageHeader.getUncompressed_page_size());
     int compressedSize = pageHeader.getCompressed_page_size();
     int uncompressedSize = pageHeader.getUncompressed_page_size();
     pageData = readPage(pageHeader, compressedSize, uncompressedSize);
@@ -270,7 +272,7 @@ class PageReader {
 
     // TODO - the metatdata for total size appears to be incorrect for impala generated files, need to find cause
     // and submit a bug report
-    if(!dataReader.hasRemainder() || parentColumnReader.totalValuesRead == parentColumnReader.columnChunkMetaData.getValueCount()) {
+    if(parentColumnReader.totalValuesRead == parentColumnReader.columnChunkMetaData.getValueCount()) {
       return false;
     }
     clearBuffers();
@@ -395,7 +397,7 @@ class PageReader {
 
   public void clear(){
     try {
-      this.inputStream.close();
+      // data reader also owns the input stream and will close it.
       this.dataReader.close();
     } catch (IOException e) {
       //Swallow the exception which is OK for input streams

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
index 4f0e3b5..69f6a62 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
@@ -476,11 +476,11 @@ public class ParquetRecordReader extends AbstractRecordReader {
    if(useAsyncColReader){
     readAllFixedFieldsParallel(recordsToRead) ;
    } else {
-     readAllFixedFieldsiSerial(recordsToRead); ;
+     readAllFixedFieldsSerial(recordsToRead); ;
    }
  }
 
-  public void readAllFixedFieldsiSerial(long recordsToRead) throws IOException {
+  public void readAllFixedFieldsSerial(long recordsToRead) throws IOException {
     for (ColumnReader<?> crs : columnStatuses) {
       crs.processPages(recordsToRead);
     }
@@ -492,14 +492,22 @@ public class ParquetRecordReader extends AbstractRecordReader {
       Future<Long> f = crs.processPagesAsync(recordsToRead);
       futures.add(f);
     }
+    Exception exception = null;
     for(Future f: futures){
-      try {
-        f.get();
-      } catch (Exception e) {
+      if(exception != null) {
         f.cancel(true);
-        handleAndRaise(null, e);
+      } else {
+        try {
+          f.get();
+        } catch (Exception e) {
+          f.cancel(true);
+          exception = e;
+        }
       }
     }
+    if(exception != null){
+      handleAndRaise(null, exception);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
index c78dc7a..7bcce11 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
@@ -57,11 +57,7 @@ public class VarLenBinaryReader {
       columnReader.reset();
     }
 
-    //if(useAsyncTasks){
-    //  recordsReadInCurrentPass = determineSizesParallel(recordsToReadInThisPass);
-    //} else {
-      recordsReadInCurrentPass = determineSizesSerial(recordsToReadInThisPass);
-    //}
+    recordsReadInCurrentPass = determineSizesSerial(recordsToReadInThisPass);
     if(useAsyncTasks){
       readRecordsParallel(recordsReadInCurrentPass);
     }else{
@@ -102,71 +98,6 @@ public class VarLenBinaryReader {
     return recordsReadInCurrentPass;
   }
 
-
-  public long determineSizesParallel(long recordsToReadInThisPass ) throws IOException {
-    boolean doneReading = false;
-    int lengthVarFieldsInCurrentRecord = 0;
-    boolean exitLengthDeterminingLoop = false;
-    long totalVariableLengthData = 0;
-    long recordsReadInCurrentPass = 0;
-
-    do {
-    doneReading = readPagesParallel();
-
-    if (!doneReading) {
-      lengthVarFieldsInCurrentRecord = 0;
-      for (VarLengthColumn<?> columnReader : columns) {
-        doneReading = columnReader.processPageData((int) recordsReadInCurrentPass);
-        if(doneReading) {
-          break;
-        }
-        lengthVarFieldsInCurrentRecord += columnReader.dataTypeLengthInBits;
-        doneReading = columnReader.checkVectorCapacityReached();
-        if(doneReading) {
-          break;
-        }
-      }
-    }
-
-    exitLengthDeterminingLoop = doneReading;
-
-      // check that the next record will fit in the batch
-      if (exitLengthDeterminingLoop ||
-          (recordsReadInCurrentPass + 1) * parentReader.getBitWidthAllFixedFields()
-              + totalVariableLengthData + lengthVarFieldsInCurrentRecord > parentReader.getBatchSize()) {
-        break;
-      }
-      for (VarLengthColumn<?> columnReader : columns) {
-        columnReader.updateReadyToReadPosition();
-        columnReader.currDefLevel = -1;
-      }
-      recordsReadInCurrentPass++;
-      totalVariableLengthData += lengthVarFieldsInCurrentRecord;
-    } while (recordsReadInCurrentPass < recordsToReadInThisPass);
-
-    return recordsReadInCurrentPass;
-  }
-
-  public boolean readPagesParallel() {
-
-    boolean isDone = false;
-    ArrayList<Future<Boolean>> futures = Lists.newArrayList();
-    for (VarLengthColumn<?> columnReader : columns) {
-      Future<Boolean> f = columnReader.readPageAsync();
-      futures.add(f);
-    }
-    for (Future<Boolean> f : futures) {
-      try {
-        isDone = isDone || f.get().booleanValue();
-      } catch (Exception e) {
-        f.cancel(true);
-        handleAndRaise(null, e);
-      }
-    }
-    return isDone;
-  }
-
-
   private void readRecordsSerial(long recordsReadInCurrentPass) {
     for (VarLengthColumn<?> columnReader : columns) {
       columnReader.readRecords(columnReader.pageReader.valuesReadyToRead);
@@ -182,12 +113,17 @@ public class VarLenBinaryReader {
       Future<Integer> f = columnReader.readRecordsAsync(columnReader.pageReader.valuesReadyToRead);
       futures.add(f);
     }
-    for (Future f : futures) {
-      try {
-        f.get();
-      } catch (Exception e) {
+    Exception exception = null;
+    for(Future f: futures){
+      if(exception != null) {
         f.cancel(true);
-        handleAndRaise(null, e);
+      } else {
+        try {
+          f.get();
+        } catch (Exception e) {
+          f.cancel(true);
+          exception = e;
+        }
       }
     }
     for (VarLengthColumn<?> columnReader : columns) {

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
index a5a6b81..327c9a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
@@ -19,24 +19,13 @@ 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.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
@@ -52,8 +41,9 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
   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
+  private static final int DEFAULT_BUFFER_SIZE = 8192 * 1024; // 8 MiB
+  private static final int DEFAULT_TEMP_BUFFER_SIZE = 8192; // 8 KiB
+  private static final int SMALL_BUFFER_SIZE = 64 * 1024; // 64 KiB
 
   /**
    * The internal buffer to keep data read from the underlying inputStream.
@@ -82,11 +72,10 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
 
   protected long curPosInStream; // current offset in the input stream
 
-  private final int bufSize;
+  private 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) {
@@ -101,7 +90,7 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
    */
   public BufferedDirectBufInputStream(InputStream in, BufferAllocator allocator, String id,
       long startOffset, long totalByteSize, boolean enableHints) {
-    this(in, allocator, id, startOffset, totalByteSize, defaultBufferSize, enableHints);
+    this(in, allocator, id, startOffset, totalByteSize, DEFAULT_BUFFER_SIZE, enableHints);
   }
 
   /**
@@ -130,13 +119,21 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
   @Override public void init() throws UnsupportedOperationException, IOException {
     super.init();
     this.internalBuffer = this.allocator.buffer(this.bufSize);
-    this.tempBuffer = this.allocator.buffer(defaultTempBufferSize);
+    this.tempBuffer = this.allocator.buffer(DEFAULT_TEMP_BUFFER_SIZE);
     int bytesRead = getNextBlock();
     if (bytesRead <= 0) {
       throw new IOException("End of stream reached while initializing buffered reader.");
     }
   }
 
+  private DrillBuf reallocBuffer(int newSize ){
+    this.internalBuffer.release();
+    this.bufSize = newSize;
+    this.internalBuffer = this.allocator.buffer(this.bufSize);
+    logger.debug("Internal buffer resized to {}", newSize);
+    return this.internalBuffer;
+  }
+
   /**
    * Read one more block from the underlying stream.
    * Assumes we have reached the end of buffered data
@@ -152,11 +149,14 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
     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 );
+    // metadata for Parquet files is incorrect (sometimes). So we read
+    // beyond the totalByteSize parameter. However, to prevent ourselves from reading too
+    // much data, we reduce the size of the buffer, down to 64KiB.
+    if (buffer.capacity() >= (totalByteSize + startOffset - curPosInStream)) {
+      if (buffer.capacity() > SMALL_BUFFER_SIZE) {
+        buffer = this.reallocBuffer(SMALL_BUFFER_SIZE);
+      }
+    }
     int bytesToRead = buffer.capacity();
 
     ByteBuffer directBuffer = buffer.nioBuffer(curPosInBuffer, bytesToRead);
@@ -171,6 +171,7 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
         nBytes = CompatibilityUtil.getBuf(getInputStream(), directBuffer, bytesToRead);
       } catch (Exception e) {
         logger.error("Error reading from stream {}. Error was : {}", this.streamId, e.getMessage());
+        throw new IOException((e));
       }
       if (nBytes > 0) {
         buffer.writerIndex(nBytes);
@@ -269,12 +270,13 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
         }
       } else {
         bytesRead += nRead;
+        //TODO: Uncomment this when the InputStream.available() call is fixed.
         // If the last read caused us to reach the end of stream
-        // we are done
-        InputStream input = in;
-        if (input != null && input.available() <= 0) {
-          return bytesRead;
-        }
+        // we are done.
+        //InputStream input = in;
+        //if (input != null && input.available() <= 0) {
+        //  return bytesRead;
+        //}
       }
     } while (bytesRead < len);
     return bytesRead;
@@ -294,7 +296,7 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
       return 0;
     }
     DrillBuf byteBuf;
-    if (len <= defaultTempBufferSize) {
+    if (len <= DEFAULT_TEMP_BUFFER_SIZE) {
       byteBuf = tempBuffer;
     } else {
       byteBuf = this.allocator.buffer(len);
@@ -310,13 +312,13 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
           return bytesRead;
         }
       } else {
-        byteBuf.nioBuffer().get(buf, off + bytesRead, len - bytesRead);
+        byteBuf.nioBuffer().get(buf, off + bytesRead, nRead);
         byteBuf.clear();
         bytesRead += nRead;
       }
     } while (bytesRead < len);
 
-    if (len > defaultTempBufferSize) {
+    if (len > DEFAULT_TEMP_BUFFER_SIZE) {
       byteBuf.release();
     }
 
@@ -380,27 +382,26 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
     return curPosInBuffer + startOffset;
   }
 
-  public boolean hasRemainder() throws IOException {
-    return available() > 0;
-  }
-
   public void close() throws IOException {
     DrillBuf buffer;
     InputStream inp;
-    if ((inp = in) != null) {
-      in = null;
-      inp.close();
-    }
-    if ((buffer = this.internalBuffer) != null) {
-      synchronized (this) {
-        this.internalBuffer = null;
-        buffer.release();
-      }
-    }
-    if ((buffer = this.tempBuffer) != null) {
-      synchronized (this) {
-        this.tempBuffer = null;
-        buffer.release();
+    synchronized (this) {
+      try {
+        if ((inp = in) != null) {
+          in = null;
+          inp.close();
+        }
+      } catch (IOException e) {
+        throw e;
+      } finally {
+        if ((buffer = this.internalBuffer) != null) {
+          this.internalBuffer = null;
+          buffer.release();
+        }
+        if ((buffer = this.tempBuffer) != null) {
+          this.tempBuffer = null;
+          buffer.release();
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/DirectBufInputStream.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/DirectBufInputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/DirectBufInputStream.java
index 71c36e6..f11ad1f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/DirectBufInputStream.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/DirectBufInputStream.java
@@ -89,7 +89,13 @@ public class DirectBufInputStream extends FilterInputStream {
 
   public synchronized DrillBuf getNext(int bytes) throws IOException {
     DrillBuf b = allocator.buffer(bytes);
-    int bytesRead = read(b, 0, bytes);
+    int bytesRead = -1;
+    try {
+    bytesRead = read(b, 0, bytes);
+    } catch (IOException e){
+      b.release();
+      throw e;
+    }
     if (bytesRead <= -1) {
       b.release();
       return null;
@@ -102,7 +108,10 @@ public class DirectBufInputStream extends FilterInputStream {
   }
 
   public boolean hasRemainder() throws IOException {
-    return getInputStream().available() > 0;
+    // We use the following instead of "getInputStream.available() > 0" because
+    // available() on HDFS seems to have issues with file sizes
+    // that are greater than Integer.MAX_VALUE
+    return (this.getPos() < (this.startOffset + this.totalByteSize));
   }
 
   protected FSDataInputStream getInputStream() throws IOException {
@@ -117,6 +126,14 @@ public class DirectBufInputStream extends FilterInputStream {
     }
   }
 
+  public synchronized void close() throws IOException {
+    InputStream inp;
+    if ((inp = in) != null) {
+      in = null;
+      inp.close();
+    }
+  }
+
   protected void checkStreamSupportsByteBuffer() throws UnsupportedOperationException {
     // Check input stream supports ByteBuffer
     if (!(in instanceof ByteBufferReadable)) {

http://git-wip-us.apache.org/repos/asf/drill/blob/ee3489ce/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
index 56b94d7..ae0e699 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
@@ -929,6 +929,7 @@ public class TestParquetWriter extends BaseTestQuery {
     }
   }
 
+  @Ignore ("Used to test decompression in AsyncPageReader. Takes too long.")
   @Test
   public void testTPCHReadWriteRunRepeated() throws Exception {
     for (int i = 1; i <= repeat; i++) {
@@ -945,7 +946,6 @@ public class TestParquetWriter extends BaseTestQuery {
     try {
       test(String.format("alter session set `%s` = 'gzip'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE));
       String inputTable = "cp.`tpch/supplier.parquet`";
-//      runTestAndValidate("s_suppkey, s_nationkey, s_acctbal", "s_suppkey, s_nationkey, s_acctbal", inputTable, "suppkey_parquet_dict_gzip");
         runTestAndValidate("*", "*", inputTable, "suppkey_parquet_dict_gzip");
     } finally {
       test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE_VALIDATOR.getDefault().string_val));
@@ -957,7 +957,6 @@ public class TestParquetWriter extends BaseTestQuery {
     try {
       test(String.format("alter session set `%s` = 'snappy'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE));
       String inputTable = "cp.`supplier_snappy.parquet`";
-      //      runTestAndValidate("s_suppkey, s_nationkey, s_acctbal", "s_suppkey, s_nationkey, s_acctbal", inputTable, "suppkey_parquet_dict_gzip");
       runTestAndValidate("*", "*", inputTable, "suppkey_parquet_dict_snappy");
     } finally {
       test(String.format("alter session set `%s` = '%s'", ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE, ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE_VALIDATOR.getDefault().string_val));


[10/10] drill git commit: DRILL-1950: Parquet rowgroup level filter pushdown in query planning time.

Posted by pa...@apache.org.
DRILL-1950: Parquet rowgroup level filter pushdown in query planning time.

Implement Parquet rowgroup level filter pushdown. The filter pushdown is performed in
in Drill physical planning phase.

Only a local filter, which refers to columns in a single table, is qualified for filter pushdown.

A filter may be qualified if it is a simple comparison filter, or a compound "and/or" filter consists of
simple comparison filter. Data types allowed in comparison filter are int, bigint, float, double, date,
timestamp, time. Comparison operators are =, !=, <, <=, >, >=. Operands have to be a column of the above
data types, or an explicit cast or implicit cast function, or a constant expressions.

This closes #637


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/9411b26e
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/9411b26e
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/9411b26e

Branch: refs/heads/master
Commit: 9411b26ece34ed8b2f498deea5e41f1901eb1013
Parents: 0d4319b
Author: Jinfeng Ni <jn...@apache.org>
Authored: Fri Nov 4 14:56:52 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Nov 4 16:07:00 2016 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/types/Types.java    |   1 +
 .../exec/expr/ExpressionTreeMaterializer.java   |  78 ++-
 .../exec/expr/fn/FunctionGenerationHelper.java  |   7 +
 .../fn/interpreter/InterpreterEvaluator.java    |  83 ++--
 .../exec/expr/stat/ParquetFilterPredicate.java  |  23 +
 .../drill/exec/expr/stat/ParquetPredicates.java | 352 ++++++++++++++
 .../exec/expr/stat/RangeExprEvaluator.java      | 275 +++++++++++
 .../drill/exec/expr/stat/TypedFieldExpr.java    |  63 +++
 .../apache/drill/exec/planner/PlannerPhase.java |  39 +-
 .../exec/planner/physical/PlannerSettings.java  |  15 +
 .../planner/sql/handlers/DefaultSqlHandler.java |   6 +
 .../server/options/SystemOptionManager.java     |   2 +
 .../exec/store/ImplicitColumnExplorer.java      |  25 +-
 .../ParquetCompareFunctionProcessor.java        | 280 -----------
 .../store/parquet/ParquetFilterBuilder.java     | 481 ++++++++++---------
 .../exec/store/parquet/ParquetGroupScan.java    | 168 ++++++-
 .../store/parquet/ParquetPushDownFilter.java    | 189 ++++----
 .../store/parquet/ParquetRGFilterEvaluator.java | 115 +++++
 .../exec/store/parquet/ParquetRowGroupScan.java |  17 +-
 .../store/parquet/ParquetScanBatchCreator.java  |   3 +-
 .../ParquetToDrillTypeConverter.java            |  25 +-
 .../store/parquet/stat/ColumnStatCollector.java |  32 ++
 .../store/parquet/stat/ColumnStatistics.java    |  40 ++
 .../stat/ParquetFooterStatCollector.java        | 199 ++++++++
 .../parquet/stat/ParquetMetaStatCollector.java  | 170 +++++++
 .../org/apache/drill/TestPartitionFilter.java   |  12 +-
 .../java/org/apache/drill/TestUnionAll.java     |   2 +-
 .../java/org/apache/drill/exec/ExecTest.java    |  14 +
 .../apache/drill/exec/expr/ExpressionTest.java  |   8 -
 .../fn/interp/ExpressionInterpreterTest.java    |   8 -
 .../physical/impl/join/TestNestedLoopJoin.java  |   2 +-
 .../exec/store/parquet/TestFileGenerator.java   |   2 +-
 .../parquet/TestParquetFilterPushDown.java      | 413 ++++++++++++++++
 .../dateTbl1_9/t1/0_0_0.parquet                 | Bin 0 -> 337 bytes
 .../dateTblCorrupted/t1/0_0_0.parquet           | Bin 0 -> 304 bytes
 .../dateTblCorrupted/t2/0_0_0.parquet           | Bin 0 -> 307 bytes
 .../dateTblCorrupted/t3/0_0_0.parquet           | Bin 0 -> 292 bytes
 .../parquetFilterPush/intTbl/intAllNull.parquet | Bin 0 -> 232 bytes
 .../parquetFilterPush/intTbl/intTbl.parquet     | Bin 0 -> 341 bytes
 .../parquetFilterPush/tsTbl/t1/0_0_0.parquet    | Bin 0 -> 339 bytes
 .../parquetFilterPush/tsTbl/t2/0_0_0.parquet    | Bin 0 -> 353 bytes
 .../parquetFilterPush/tsTbl/t3/0_0_0.parquet    | Bin 0 -> 337 bytes
 .../common/expression/ValueExpressions.java     |  15 +
 .../common/expression/fn/CastFunctions.java     |   9 +
 44 files changed, 2447 insertions(+), 726 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/types/Types.java b/common/src/main/java/org/apache/drill/common/types/Types.java
index 116d0d1..ce2abf9 100644
--- a/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -35,6 +35,7 @@ public class Types {
   public static final MajorType LATE_BIND_TYPE = optional(MinorType.LATE);
   public static final MajorType REQUIRED_BIT = required(MinorType.BIT);
   public static final MajorType OPTIONAL_BIT = optional(MinorType.BIT);
+  public static final MajorType OPTIONAL_INT = optional(MinorType.INT);
 
   public static boolean isUnion(MajorType toType) {
     return toType.getMinorType() == MinorType.UNION;

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index bece93d..b70ad26 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -24,6 +24,7 @@ import java.util.Deque;
 import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
 
@@ -78,6 +79,8 @@ import org.apache.drill.exec.expr.fn.DrillComplexWriterFuncHolder;
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
 import org.apache.drill.exec.expr.fn.ExceptionFunction;
 import org.apache.drill.exec.expr.fn.FunctionLookupContext;
+import org.apache.drill.exec.expr.stat.TypedFieldExpr;
+import org.apache.drill.exec.record.MaterializeVisitor;
 import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.resolver.FunctionResolver;
@@ -89,6 +92,7 @@ import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import org.apache.drill.exec.store.parquet.stat.ColumnStatistics;
 import org.apache.drill.exec.util.DecimalUtility;
 
 public class ExpressionTreeMaterializer {
@@ -116,6 +120,12 @@ public class ExpressionTreeMaterializer {
     return materialize(expr, batch, errorCollector, functionLookupContext, allowComplexWriterExpr, false);
   }
 
+  public static LogicalExpression materializeFilterExpr(LogicalExpression expr, Map<SchemaPath, ColumnStatistics> fieldTypes, ErrorCollector errorCollector, FunctionLookupContext functionLookupContext) {
+    final FilterMaterializeVisitor filterMaterializeVisitor = new FilterMaterializeVisitor(fieldTypes, errorCollector);
+    LogicalExpression out =  expr.accept(filterMaterializeVisitor, functionLookupContext);
+    return out;
+  }
+
   public static LogicalExpression materialize(LogicalExpression expr, VectorAccessible batch, ErrorCollector errorCollector, FunctionLookupContext functionLookupContext,
       boolean allowComplexWriterExpr, boolean unionTypeEnabled) {
     LogicalExpression out =  expr.accept(new MaterializeVisitor(batch, errorCollector, allowComplexWriterExpr, unionTypeEnabled), functionLookupContext);
@@ -214,12 +224,58 @@ public class ExpressionTreeMaterializer {
     errorCollector.addGeneralError(call.getPosition(), sb.toString());
   }
 
+  private static class MaterializeVisitor extends AbstractMaterializeVisitor {
+    private final VectorAccessible batch;
+
+    public MaterializeVisitor(VectorAccessible batch, ErrorCollector errorCollector, boolean allowComplexWriter, boolean unionTypeEnabled) {
+      super(errorCollector, allowComplexWriter, unionTypeEnabled);
+      this.batch = batch;
+    }
 
-  private static class MaterializeVisitor extends AbstractExprVisitor<LogicalExpression, FunctionLookupContext, RuntimeException> {
+    @Override
+    public LogicalExpression visitSchemaPath(SchemaPath path, FunctionLookupContext functionLookupContext) {
+      //      logger.debug("Visiting schema path {}", path);
+      TypedFieldId tfId = batch.getValueVectorId(path);
+      if (tfId == null) {
+        logger.warn("Unable to find value vector of path {}, returning null instance.", path);
+        return NullExpression.INSTANCE;
+      } else {
+        ValueVectorReadExpression e = new ValueVectorReadExpression(tfId);
+        return e;
+      }
+    }
+  }
+
+  private static class FilterMaterializeVisitor extends AbstractMaterializeVisitor {
+    private final Map<SchemaPath, ColumnStatistics> stats;
+
+    public FilterMaterializeVisitor(Map<SchemaPath, ColumnStatistics> stats, ErrorCollector errorCollector) {
+      super(errorCollector, false, false);
+      this.stats = stats;
+    }
+
+    @Override
+    public LogicalExpression visitSchemaPath(SchemaPath path, FunctionLookupContext functionLookupContext) {
+      MajorType type = null;
+
+      if (stats.containsKey(path)) {
+        type = stats.get(path).getMajorType();
+      }
+
+      if (type != null) {
+        return new TypedFieldExpr(path, type);
+      } else {
+        logger.warn("Unable to find value vector of path {}, returning null-int instance.", path);
+        return new TypedFieldExpr(path, Types.OPTIONAL_INT);
+        // return NullExpression.INSTANCE;
+      }
+    }
+  }
+
+  private static abstract class AbstractMaterializeVisitor extends AbstractExprVisitor<LogicalExpression, FunctionLookupContext, RuntimeException> {
     private ExpressionValidator validator = new ExpressionValidator();
     private ErrorCollector errorCollector;
     private Deque<ErrorCollector> errorCollectors = new ArrayDeque<>();
-    private final VectorAccessible batch;
     private final boolean allowComplexWriter;
     /**
      * If this is false, the materializer will not handle or create UnionTypes
@@ -231,8 +287,7 @@ public class ExpressionTreeMaterializer {
      */
     private Set<LogicalExpression> materializedExpressions = Sets.newIdentityHashSet();
 
-    public MaterializeVisitor(VectorAccessible batch, ErrorCollector errorCollector, boolean allowComplexWriter, boolean unionTypeEnabled) {
-      this.batch = batch;
+    public AbstractMaterializeVisitor(ErrorCollector errorCollector, boolean allowComplexWriter, boolean unionTypeEnabled) {
       this.errorCollector = errorCollector;
       this.allowComplexWriter = allowComplexWriter;
       this.unionTypeEnabled = unionTypeEnabled;
@@ -243,6 +298,8 @@ public class ExpressionTreeMaterializer {
       return newExpr;
     }
 
+    abstract public LogicalExpression visitSchemaPath(SchemaPath path, FunctionLookupContext functionLookupContext);
+
     @Override
     public LogicalExpression visitUnknown(LogicalExpression e, FunctionLookupContext functionLookupContext)
       throws RuntimeException {
@@ -635,19 +692,6 @@ public class ExpressionTreeMaterializer {
     }
 
     @Override
-    public LogicalExpression visitSchemaPath(SchemaPath path, FunctionLookupContext functionLookupContext) {
-//      logger.debug("Visiting schema path {}", path);
-      TypedFieldId tfId = batch.getValueVectorId(path);
-      if (tfId == null) {
-        logger.warn("Unable to find value vector of path {}, returning null instance.", path);
-        return NullExpression.INSTANCE;
-      } else {
-        ValueVectorReadExpression e = new ValueVectorReadExpression(tfId);
-        return e;
-      }
-    }
-
-    @Override
     public LogicalExpression visitIntConstant(IntExpression intExpr, FunctionLookupContext functionLookupContext) {
       return intExpr;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
index 90b0816..b83350d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
@@ -44,6 +44,13 @@ public class FunctionGenerationHelper {
   public static final String COMPARE_TO_NULLS_HIGH = "compare_to_nulls_high";
   public static final String COMPARE_TO_NULLS_LOW = "compare_to_nulls_low";
 
+  public static final String EQ = "equal";
+  public static final String NE = "not_equal";
+  public static final String GT = "greater_than";
+  public static final String GE = "greater_than_or_equal_to";
+  public static final String LT = "less_than";
+  public static final String LE = "less_than_or_equal_to";
+
   /**
    * Finds ordering comparator ("compare_to...") FunctionHolderExpression with
    * a specified ordering for NULL (and considering NULLS <i>equal</i>).

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/interpreter/InterpreterEvaluator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/interpreter/InterpreterEvaluator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/interpreter/InterpreterEvaluator.java
index d3a5573..d106887 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/interpreter/InterpreterEvaluator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/interpreter/InterpreterEvaluator.java
@@ -85,6 +85,49 @@ public class InterpreterEvaluator {
 
   }
 
+  public static ValueHolder evaluateFunction(DrillSimpleFunc interpreter, ValueHolder[] args, String funcName) throws Exception {
+    Preconditions.checkArgument(interpreter != null, "interpreter could not be null when use interpreted model to evaluate function " + funcName);
+
+    // the current input index to assign into the next available parameter, found using the @Param notation
+    // the order parameters are declared in the java class for the DrillFunc is meaningful
+    int currParameterIndex = 0;
+    Field outField = null;
+    try {
+      Field[] fields = interpreter.getClass().getDeclaredFields();
+      for (Field f : fields) {
+        // if this is annotated as a parameter to the function
+        if ( f.getAnnotation(Param.class) != null ) {
+          f.setAccessible(true);
+          if (currParameterIndex < args.length) {
+            f.set(interpreter, args[currParameterIndex]);
+          }
+          currParameterIndex++;
+        } else if ( f.getAnnotation(Output.class) != null ) {
+          f.setAccessible(true);
+          outField = f;
+          // create an instance of the holder for the output to be stored in
+          f.set(interpreter, f.getType().newInstance());
+        }
+      }
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException(e);
+    }
+    if (args.length != currParameterIndex ) {
+      throw new DrillRuntimeException(
+          String.format("Wrong number of parameters provided to interpreted expression evaluation " +
+                  "for function %s, expected %d parameters, but received %d.",
+              funcName, currParameterIndex, args.length));
+    }
+    if (outField == null) {
+      throw new DrillRuntimeException("Malformed DrillFunction without a return type: " + funcName);
+    }
+    interpreter.setup();
+    interpreter.eval();
+    ValueHolder out = (ValueHolder) outField.get(interpreter);
+
+    return out;
+  }
+
   private static class InitVisitor extends AbstractExprVisitor<LogicalExpression, VectorAccessible, RuntimeException> {
 
     private UdfUtilities udfUtilities;
@@ -270,44 +313,7 @@ public class InterpreterEvaluator {
       try {
         DrillSimpleFunc interpreter =  ((DrillFuncHolderExpr) holderExpr).getInterpreter();
 
-        Preconditions.checkArgument(interpreter != null, "interpreter could not be null when use interpreted model to evaluate function " + holder.getRegisteredNames()[0]);
-
-        // the current input index to assign into the next available parameter, found using the @Param notation
-        // the order parameters are declared in the java class for the DrillFunc is meaningful
-        int currParameterIndex = 0;
-        Field outField = null;
-        try {
-          Field[] fields = interpreter.getClass().getDeclaredFields();
-          for (Field f : fields) {
-            // if this is annotated as a parameter to the function
-            if ( f.getAnnotation(Param.class) != null ) {
-              f.setAccessible(true);
-              if (currParameterIndex < args.length) {
-                f.set(interpreter, args[currParameterIndex]);
-              }
-              currParameterIndex++;
-            } else if ( f.getAnnotation(Output.class) != null ) {
-              f.setAccessible(true);
-              outField = f;
-              // create an instance of the holder for the output to be stored in
-              f.set(interpreter, f.getType().newInstance());
-            }
-          }
-        } catch (IllegalAccessException e) {
-            throw new RuntimeException(e);
-        }
-        if (args.length != currParameterIndex ) {
-          throw new DrillRuntimeException(
-              String.format("Wrong number of parameters provided to interpreted expression evaluation " +
-                  "for function %s, expected %d parameters, but received %d.",
-                  holderExpr.getName(), currParameterIndex, args.length));
-        }
-        if (outField == null) {
-          throw new DrillRuntimeException("Malformed DrillFunction without a return type: " + holderExpr.getName());
-        }
-        interpreter.setup();
-        interpreter.eval();
-        ValueHolder out = (ValueHolder) outField.get(interpreter);
+        ValueHolder out = evaluateFunction(interpreter, args, holderExpr.getName());
 
         if (TypeHelper.getValueHolderType(out).getMode() == TypeProtos.DataMode.OPTIONAL &&
             holderExpr.getMajorType().getMode() == TypeProtos.DataMode.REQUIRED) {
@@ -325,6 +331,7 @@ public class InterpreterEvaluator {
 
     }
 
+
     @Override
     public ValueHolder visitBooleanOperator(BooleanOperator op, Integer inIndex) {
       // Apply short circuit evaluation to boolean operator.

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetFilterPredicate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetFilterPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetFilterPredicate.java
new file mode 100644
index 0000000..2711faa
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetFilterPredicate.java
@@ -0,0 +1,23 @@
+/**
+ * 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.expr.stat;
+
+public interface ParquetFilterPredicate {
+  boolean canDrop(RangeExprEvaluator evaluator);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicates.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicates.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicates.java
new file mode 100644
index 0000000..54f703a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/ParquetPredicates.java
@@ -0,0 +1,352 @@
+/**
+ * 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.expr.stat;
+
+import org.apache.drill.common.expression.BooleanOperator;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.LogicalExpressionBase;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public abstract  class ParquetPredicates {
+  public static abstract  class ParquetCompPredicate extends LogicalExpressionBase implements ParquetFilterPredicate {
+    protected final LogicalExpression left;
+    protected final LogicalExpression right;
+
+    public ParquetCompPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left.getPosition());
+      this.left = left;
+      this.right = right;
+    }
+
+    @Override
+    public Iterator<LogicalExpression> iterator() {
+      final List<LogicalExpression> args = new ArrayList<>();
+      args.add(left);
+      args.add(right);
+      return args.iterator();
+    }
+
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitUnknown(this, value);
+    }
+
+  }
+
+  public static abstract class ParquetBooleanPredicate extends BooleanOperator implements ParquetFilterPredicate {
+    public ParquetBooleanPredicate(String name, List<LogicalExpression> args, ExpressionPosition pos) {
+      super(name, args, pos);
+    }
+
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitBooleanOperator(this, value);
+    }
+  }
+
+  public static class AndPredicate extends ParquetBooleanPredicate {
+    public AndPredicate(String name, List<LogicalExpression> args, ExpressionPosition pos) {
+      super(name, args, pos);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      // "and" : as long as one branch is OK to drop, we can drop it.
+      for (LogicalExpression child : this) {
+        if (((ParquetFilterPredicate) child).canDrop(evaluator)) {
+          return true;
+        }
+      }
+      return false;
+    }
+  }
+
+  public static class OrPredicate extends ParquetBooleanPredicate {
+    public OrPredicate(String name, List<LogicalExpression> args, ExpressionPosition pos) {
+      super(name, args, pos);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      for (LogicalExpression child : this) {
+        // "long" : as long as one branch is NOT ok to drop, we can NOT drop it.
+        if (! ((ParquetFilterPredicate) child).canDrop(evaluator)) {
+          return false;
+        }
+      }
+
+      return true;
+    }
+  }
+
+  // is this column chunk composed entirely of nulls?
+  // assumes the column chunk's statistics is not empty
+  protected static boolean isAllNulls(Statistics stat, long rowCount) {
+    return stat.getNumNulls() == rowCount;
+  }
+
+  // are there any nulls in this column chunk?
+  // assumes the column chunk's statistics is not empty
+  protected static boolean hasNulls(Statistics stat) {
+    return stat.getNumNulls() > 0;
+  }
+
+  /**
+   * EQ (=) predicate
+   */
+  public static class EqualPredicate extends ParquetCompPredicate {
+    public EqualPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    /**
+        Semantics of canDrop() is very similar to what is implemented in Parquet library's
+        {@link org.apache.parquet.filter2.statisticslevel.StatisticsFilter} and
+        {@link org.apache.parquet.filter2.predicate.FilterPredicate}
+
+        Main difference :
+     1. A RangeExprEvaluator is used to compute the min/max of an expression, such as CAST function
+        of a column. CAST function could be explicitly added by Drill user (It's recommended to use CAST
+        function after DRILL-4372, if user wants to reduce planning time for limit 0 query), or implicitly
+        inserted by Drill, when the types of compare operands are not identical. Therefore, it's important
+         to allow CAST function to appear in the filter predicate.
+     2. We do not require list of ColumnChunkMetaData to do the evaluation, while Parquet library's
+        StatisticsFilter has such requirement. Drill's ParquetTableMetaData does not maintain ColumnChunkMetaData,
+        making it impossible to directly use Parquet library's StatisticFilter in query planning time.
+     3. We allows both sides of comparison operator to be a min/max range. As such, we support
+           expression_of(Column1)   <   expression_of(Column2),
+        where Column1 and Column2 are from same parquet table.
+     */
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
+          isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when left's max < right's min, or right's max < left's min
+      if ( ( leftStat.genericGetMax().compareTo(rightStat.genericGetMin()) < 0
+            || rightStat.genericGetMax().compareTo(leftStat.genericGetMin()) < 0)) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public String toString() {
+      return left.toString()  + " = " + right.toString();
+    }
+  }
+
+  /**
+   * GT (>) predicate.
+   */
+  public static class GTPredicate extends ParquetCompPredicate {
+    public GTPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
+          isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when left's max <= right's min.
+      if ( leftStat.genericGetMax().compareTo(rightStat.genericGetMin()) <= 0 ) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * GE (>=) predicate.
+   */
+  public static class GEPredicate extends ParquetCompPredicate {
+    public GEPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
+          isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when left's max < right's min.
+      if ( leftStat.genericGetMax().compareTo(rightStat.genericGetMin()) < 0 ) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * LT (<) predicate.
+   */
+  public static class LTPredicate extends ParquetCompPredicate {
+    public LTPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
+          isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when right's max <= left's min.
+      if ( rightStat.genericGetMax().compareTo(leftStat.genericGetMin()) <= 0 ) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * LE (<=) predicate.
+   */
+  public static class LEPredicate extends ParquetCompPredicate {
+    public LEPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, = is evaluated to UNKNOW -> canDrop
+      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
+          isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when right's max < left's min.
+      if ( rightStat.genericGetMax().compareTo(leftStat.genericGetMin()) < 0 ) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * NE (!=) predicate.
+   */
+  public static class NEPredicate extends ParquetCompPredicate {
+    public NEPredicate(LogicalExpression left, LogicalExpression right) {
+      super(left, right);
+    }
+
+    @Override
+    public boolean canDrop(RangeExprEvaluator evaluator) {
+      Statistics leftStat = left.accept(evaluator, null);
+      Statistics rightStat = right.accept(evaluator, null);
+
+      if (leftStat == null ||
+          rightStat == null ||
+          leftStat.isEmpty() ||
+          rightStat.isEmpty()) {
+        return false;
+      }
+
+      // if either side is ALL null, comparison is evaluated to UNKNOW -> canDrop
+      if (isAllNulls(leftStat, evaluator.getRowCount()) ||
+          isAllNulls(rightStat, evaluator.getRowCount())) {
+        return true;
+      }
+
+      // can drop when there is only one unique value.
+      if ( leftStat.genericGetMin().compareTo(leftStat.genericGetMax()) == 0 &&
+           rightStat.genericGetMin().compareTo(rightStat.genericGetMax()) ==0 &&
+           leftStat.genericGetMax().compareTo(rightStat.genericGetMax()) == 0) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
new file mode 100644
index 0000000..8f77070
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
@@ -0,0 +1,275 @@
+/**
+ * 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.expr.stat;
+
+import com.google.common.base.Preconditions;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.expression.FunctionHolderExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.expression.fn.CastFunctions;
+import org.apache.drill.common.expression.fn.FuncHolder;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.fn.DrillSimpleFuncHolder;
+import org.apache.drill.exec.expr.fn.interpreter.InterpreterEvaluator;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.Float4Holder;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.ValueHolder;
+import org.apache.drill.exec.store.parquet.stat.ColumnStatistics;
+import org.apache.drill.exec.vector.ValueHolderHelper;
+import org.apache.parquet.column.statistics.DoubleStatistics;
+import org.apache.parquet.column.statistics.FloatStatistics;
+import org.apache.parquet.column.statistics.IntStatistics;
+import org.apache.parquet.column.statistics.LongStatistics;
+import org.apache.parquet.column.statistics.Statistics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class RangeExprEvaluator extends AbstractExprVisitor<Statistics, Void, RuntimeException> {
+  static final Logger logger = LoggerFactory.getLogger(RangeExprEvaluator.class);
+
+  private final Map<SchemaPath, ColumnStatistics> columnStatMap;
+  private final long rowCount;
+
+  public RangeExprEvaluator(final Map<SchemaPath, ColumnStatistics> columnStatMap, long rowCount) {
+    this.columnStatMap = columnStatMap;
+    this.rowCount = rowCount;
+  }
+
+  public long getRowCount() {
+    return this.rowCount;
+  }
+
+  @Override
+  public Statistics visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
+    if (e instanceof TypedFieldExpr) {
+      TypedFieldExpr fieldExpr = (TypedFieldExpr) e;
+      final ColumnStatistics columnStatistics = columnStatMap.get(fieldExpr.getPath());
+      if (columnStatistics != null) {
+        return columnStatistics.getStatistics();
+      } else {
+        // field does not exist.
+        Preconditions.checkArgument(fieldExpr.getMajorType().equals(Types.OPTIONAL_INT));
+        IntStatistics intStatistics = new IntStatistics();
+        intStatistics.setNumNulls(rowCount); // all values are nulls
+        return intStatistics;
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public Statistics visitIntConstant(ValueExpressions.IntExpression expr, Void value) throws RuntimeException {
+    return getStatistics(expr.getInt());
+  }
+
+  @Override
+  public Statistics visitLongConstant(ValueExpressions.LongExpression expr, Void value) throws RuntimeException {
+    return getStatistics(expr.getLong());
+  }
+
+  @Override
+  public Statistics visitFloatConstant(ValueExpressions.FloatExpression expr, Void value) throws RuntimeException {
+    return getStatistics(expr.getFloat());
+  }
+
+  @Override
+  public Statistics visitDoubleConstant(ValueExpressions.DoubleExpression expr, Void value) throws RuntimeException {
+    return getStatistics(expr.getDouble());
+  }
+
+  @Override
+  public Statistics visitDateConstant(ValueExpressions.DateExpression expr, Void value) throws RuntimeException {
+    long dateInMillis = expr.getDate();
+    return getStatistics(dateInMillis);
+  }
+
+  @Override
+  public Statistics visitTimeStampConstant(ValueExpressions.TimeStampExpression tsExpr, Void value) throws RuntimeException {
+    long tsInMillis = tsExpr.getTimeStamp();
+    return getStatistics(tsInMillis);
+  }
+
+  @Override
+  public Statistics visitTimeConstant(ValueExpressions.TimeExpression timeExpr, Void value) throws RuntimeException {
+    int milliSeconds = timeExpr.getTime();
+    return getStatistics(milliSeconds);
+  }
+
+  @Override
+  public Statistics visitFunctionHolderExpression(FunctionHolderExpression holderExpr, Void value) throws RuntimeException {
+    FuncHolder funcHolder = holderExpr.getHolder();
+
+    if (! (funcHolder instanceof DrillSimpleFuncHolder)) {
+      // Only Drill function is allowed.
+      return null;
+    }
+
+    final String funcName = ((DrillSimpleFuncHolder) funcHolder).getRegisteredNames()[0];
+
+    if (CastFunctions.isCastFunction(funcName)) {
+      Statistics stat = holderExpr.args.get(0).accept(this, null);
+      if (stat != null && ! stat.isEmpty()) {
+        return evalCastFunc(holderExpr, stat);
+      }
+    }
+    return null;
+  }
+
+  private IntStatistics getStatistics(int value) {
+    return getStatistics(value, value);
+  }
+
+  private IntStatistics getStatistics(int min, int max) {
+    final IntStatistics intStatistics = new IntStatistics();
+    intStatistics.setMinMax(min, max);
+    return intStatistics;
+  }
+
+  private LongStatistics getStatistics(long value) {
+    return getStatistics(value, value);
+  }
+
+  private LongStatistics getStatistics(long min, long max) {
+    final LongStatistics longStatistics = new LongStatistics();
+    longStatistics.setMinMax(min, max);
+    return longStatistics;
+  }
+
+  private DoubleStatistics getStatistics(double value) {
+    return getStatistics(value, value);
+  }
+
+  private DoubleStatistics getStatistics(double min, double max) {
+    final DoubleStatistics doubleStatistics = new DoubleStatistics();
+    doubleStatistics.setMinMax(min, max);
+    return doubleStatistics;
+  }
+
+  private FloatStatistics getStatistics(float value) {
+    return getStatistics(value, value);
+  }
+
+  private FloatStatistics getStatistics(float min, float max) {
+    final FloatStatistics floatStatistics = new FloatStatistics();
+    floatStatistics.setMinMax(min, max);
+    return floatStatistics;
+  }
+
+  private Statistics evalCastFunc(FunctionHolderExpression holderExpr, Statistics input) {
+    try {
+      DrillSimpleFuncHolder funcHolder = (DrillSimpleFuncHolder) holderExpr.getHolder();
+
+      DrillSimpleFunc interpreter = funcHolder.createInterpreter();
+
+      final ValueHolder minHolder, maxHolder;
+
+      TypeProtos.MinorType srcType = holderExpr.args.get(0).getMajorType().getMinorType();
+      TypeProtos.MinorType destType = holderExpr.getMajorType().getMinorType();
+
+      if (srcType.equals(destType)) {
+        // same type cast ==> NoOp.
+        return input;
+      } else if (!CAST_FUNC.containsKey(srcType) || !CAST_FUNC.get(srcType).contains(destType)) {
+        return null; // cast func between srcType and destType is NOT allowed.
+      }
+
+      switch (srcType) {
+      case INT :
+        minHolder = ValueHolderHelper.getIntHolder(((IntStatistics)input).getMin());
+        maxHolder = ValueHolderHelper.getIntHolder(((IntStatistics)input).getMax());
+        break;
+      case BIGINT:
+        minHolder = ValueHolderHelper.getBigIntHolder(((LongStatistics)input).getMin());
+        maxHolder = ValueHolderHelper.getBigIntHolder(((LongStatistics)input).getMax());
+        break;
+      case FLOAT4:
+        minHolder = ValueHolderHelper.getFloat4Holder(((FloatStatistics)input).getMin());
+        maxHolder = ValueHolderHelper.getFloat4Holder(((FloatStatistics)input).getMax());
+        break;
+      case FLOAT8:
+        minHolder = ValueHolderHelper.getFloat8Holder(((DoubleStatistics)input).getMin());
+        maxHolder = ValueHolderHelper.getFloat8Holder(((DoubleStatistics)input).getMax());
+        break;
+      default:
+        return null;
+      }
+
+      final ValueHolder[] args1 = {minHolder};
+      final ValueHolder[] args2 = {maxHolder};
+
+      final ValueHolder minFuncHolder = InterpreterEvaluator.evaluateFunction(interpreter, args1, holderExpr.getName());
+      final ValueHolder maxFuncHolder = InterpreterEvaluator.evaluateFunction(interpreter, args2, holderExpr.getName());
+
+      switch (destType) {
+      //TODO : need handle # of nulls.
+      case INT:
+        return getStatistics( ((IntHolder)minFuncHolder).value, ((IntHolder)maxFuncHolder).value);
+      case BIGINT:
+        return getStatistics( ((BigIntHolder)minFuncHolder).value, ((BigIntHolder)maxFuncHolder).value);
+      case FLOAT4:
+        return getStatistics( ((Float4Holder)minFuncHolder).value, ((Float4Holder)maxFuncHolder).value);
+      case FLOAT8:
+        return getStatistics( ((Float8Holder)minFuncHolder).value, ((Float8Holder)maxFuncHolder).value);
+      default:
+        return null;
+      }
+    } catch (Exception e) {
+      throw new DrillRuntimeException("Error in evaluating function of " + holderExpr.getName() );
+    }
+  }
+
+  static Map<TypeProtos.MinorType, Set<TypeProtos.MinorType>> CAST_FUNC = new HashMap<>();
+  static {
+    // float -> double , int, bigint
+    CAST_FUNC.put(TypeProtos.MinorType.FLOAT4, new HashSet<TypeProtos.MinorType>());
+    CAST_FUNC.get(TypeProtos.MinorType.FLOAT4).add(TypeProtos.MinorType.FLOAT8);
+    CAST_FUNC.get(TypeProtos.MinorType.FLOAT4).add(TypeProtos.MinorType.INT);
+    CAST_FUNC.get(TypeProtos.MinorType.FLOAT4).add(TypeProtos.MinorType.BIGINT);
+
+    // double -> float, int, bigint
+    CAST_FUNC.put(TypeProtos.MinorType.FLOAT8, new HashSet<TypeProtos.MinorType>());
+    CAST_FUNC.get(TypeProtos.MinorType.FLOAT8).add(TypeProtos.MinorType.FLOAT4);
+    CAST_FUNC.get(TypeProtos.MinorType.FLOAT8).add(TypeProtos.MinorType.INT);
+    CAST_FUNC.get(TypeProtos.MinorType.FLOAT8).add(TypeProtos.MinorType.BIGINT);
+
+    // int -> float, double, bigint
+    CAST_FUNC.put(TypeProtos.MinorType.INT, new HashSet<TypeProtos.MinorType>());
+    CAST_FUNC.get(TypeProtos.MinorType.INT).add(TypeProtos.MinorType.FLOAT4);
+    CAST_FUNC.get(TypeProtos.MinorType.INT).add(TypeProtos.MinorType.FLOAT8);
+    CAST_FUNC.get(TypeProtos.MinorType.INT).add(TypeProtos.MinorType.BIGINT);
+
+    // bigint -> int, float, double
+    CAST_FUNC.put(TypeProtos.MinorType.BIGINT, new HashSet<TypeProtos.MinorType>());
+    CAST_FUNC.get(TypeProtos.MinorType.BIGINT).add(TypeProtos.MinorType.INT);
+    CAST_FUNC.get(TypeProtos.MinorType.BIGINT).add(TypeProtos.MinorType.FLOAT4);
+    CAST_FUNC.get(TypeProtos.MinorType.BIGINT).add(TypeProtos.MinorType.FLOAT8);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/TypedFieldExpr.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/TypedFieldExpr.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/TypedFieldExpr.java
new file mode 100644
index 0000000..4287929
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/TypedFieldExpr.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.stat;
+
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.LogicalExpressionBase;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos;
+
+import java.util.Iterator;
+
+public class TypedFieldExpr extends LogicalExpressionBase {
+  TypeProtos.MajorType type;
+  SchemaPath path;
+
+  public TypedFieldExpr(SchemaPath path, TypeProtos.MajorType type) {
+    super(path.getPosition());
+    this.path = path;
+    this.type = type;
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitUnknown(this, value);
+  }
+
+  @Override
+  public Iterator<LogicalExpression> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public TypeProtos.MajorType getMajorType() {
+    return this.type;
+  }
+
+  @Override
+  public String toString() {
+    return this.path.getRootSegment().getPath() + "(" + type.getMinorType() + "_" + type.getMode() +")";
+  }
+
+  public SchemaPath getPath() {
+    return this.path;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
index 22a8b6f..1551040 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
@@ -17,16 +17,13 @@
  */
 package org.apache.drill.exec.planner;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableSet.Builder;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.volcano.AbstractConverter.ExpandConversionRule;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateRemoveRule;
-import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
 import org.apache.calcite.rel.rules.FilterMergeRule;
 import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
 import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
@@ -88,9 +85,11 @@ import org.apache.drill.exec.planner.physical.WindowPrule;
 import org.apache.drill.exec.planner.physical.WriterPrule;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.parquet.ParquetPushDownFilter;
 
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.ImmutableSet.Builder;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
 
 public enum PlannerPhase {
   //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRuleSets.class);
@@ -154,6 +153,12 @@ public enum PlannerPhase {
     }
   },
 
+  PHYSICAL_PARTITION_PRUNING("Physical Partition Prune Planning") {
+    public RuleSet getRules(OptimizerRulesContext context, Collection<StoragePlugin> plugins) {
+      return PlannerPhase.mergedRuleSets(getPhysicalPruneScanRules(context), getStorageRules(context, plugins, this));
+    }
+  },
+
   DIRECTORY_PRUNING("Directory Prune Planning") {
     public RuleSet getRules(OptimizerRulesContext context, Collection<StoragePlugin> plugins) {
       return PlannerPhase.mergedRuleSets(getDirPruneScanRules(context), getStorageRules(context, plugins, this));
@@ -346,6 +351,26 @@ public enum PlannerPhase {
   }
 
   /**
+   *   Get an immutable list of pruning rules that will be used post physical planning.
+   */
+  static RuleSet getPhysicalPruneScanRules(OptimizerRulesContext optimizerRulesContext) {
+    final ImmutableSet<RelOptRule> pruneRules = ImmutableSet.<RelOptRule>builder()
+        .add(
+            // See DRILL-4998 for more detail.
+            // Main reason for doing this is we want to reduce the performance regression possibility
+            // caused by a different join order, as a result of reduced row count in scan operator.
+            // Ideally this should be done in logical planning, before join order planning is done.
+            // Before we can make such change, we have to figure out how to adjust the selectivity
+            // estimation of filter operator, after filter is pushed down to scan.
+            ParquetPushDownFilter.getFilterOnProject(optimizerRulesContext),
+            ParquetPushDownFilter.getFilterOnScan(optimizerRulesContext)
+        )
+        .build();
+
+    return RuleSets.ofList(pruneRules);
+  }
+
+  /**
    *  Get an immutable list of directory-based partition pruing rules that will be used in Calcite logical planning.
    * @param optimizerRulesContext
    * @return

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index 218bf5b..b3dc7d6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -99,6 +99,13 @@ public class PlannerSettings implements Context{
   public static final LongValidator IN_SUBQUERY_THRESHOLD =
       new PositiveLongValidator("planner.in_subquery_threshold", Integer.MAX_VALUE, 20); /* Same as Calcite's default IN List subquery size */
 
+  public static final String PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY = "planner.store.parquet.rowgroup.filter.pushdown";
+  public static final BooleanValidator PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING = new BooleanValidator(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY, true);
+  public static final String PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY = "planner.store.parquet.rowgroup.filter.pushdown.threshold";
+  public static final PositiveLongValidator PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD = new PositiveLongValidator(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY,
+      Long.MAX_VALUE, 10000);
+
+
   public OptionManager options = null;
   public FunctionImplementationRegistry functionImplementationRegistry = null;
 
@@ -247,6 +254,14 @@ public class PlannerSettings implements Context{
     return options.getOption(UNIONALL_DISTRIBUTE);
   }
 
+  public boolean isParquetRowGroupFilterPushdownPlanningEnabled() {
+    return options.getOption(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING);
+  }
+
+  public long getParquetRowGroupFilterPushDownThreshold() {
+    return options.getOption(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD);
+  }
+
   @Override
   public <T> T unwrap(Class<T> clazz) {
     if(clazz == PlannerSettings.class){

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 2d0c069..15702e3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -486,6 +486,12 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
           .getHashJoinSwapMarginFactor()));
     }
 
+    /* Parquet row group filter pushdown in planning time */
+
+    if (context.getPlannerSettings().isParquetRowGroupFilterPushdownPlanningEnabled()) {
+      phyRelNode = (Prel) transform(PlannerType.HEP_BOTTOM_UP, PlannerPhase.PHYSICAL_PARTITION_PRUNING, phyRelNode);
+    }
+
     /*
      * 1.2) Break up all expressions with complex outputs into their own project operations
      */

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index d803fa3..2c322c7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -90,6 +90,8 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       PlannerSettings.TYPE_INFERENCE,
       PlannerSettings.IN_SUBQUERY_THRESHOLD,
       PlannerSettings.UNIONALL_DISTRIBUTE,
+      PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING,
+      PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD,
       ExecConstants.CAST_TO_NULLABLE_NUMERIC_OPTION,
       ExecConstants.OUTPUT_FORMAT_VALIDATOR,
       ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR,

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/ImplicitColumnExplorer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ImplicitColumnExplorer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ImplicitColumnExplorer.java
index b67d8b5..42ff827 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ImplicitColumnExplorer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ImplicitColumnExplorer.java
@@ -52,12 +52,21 @@ public class ImplicitColumnExplorer {
    * Also populates map with implicit columns names as keys and their values
    */
   public ImplicitColumnExplorer(FragmentContext context, List<SchemaPath> columns) {
-    this.partitionDesignator = context.getOptions().getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
+    this(context.getOptions(), columns);
+  }
+
+  /**
+   * Helper class that encapsulates logic for sorting out columns
+   * between actual table columns, partition columns and implicit file columns.
+   * Also populates map with implicit columns names as keys and their values
+   */
+  public ImplicitColumnExplorer(OptionManager optionManager, List<SchemaPath> columns) {
+    this.partitionDesignator = optionManager.getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
     this.columns = columns;
     this.isStarQuery = columns != null && AbstractRecordReader.isStarQuery(columns);
     this.selectedPartitionColumns = Lists.newArrayList();
     this.tableColumns = Lists.newArrayList();
-    this.allImplicitColumns = initImplicitFileColumns(context.getOptions());
+    this.allImplicitColumns = initImplicitFileColumns(optionManager);
     this.selectedImplicitColumns = CaseInsensitiveMap.newHashMap();
 
     init();
@@ -84,10 +93,20 @@ public class ImplicitColumnExplorer {
    * @return map with columns names as keys and their values
    */
   public Map<String, String> populateImplicitColumns(FileWork work, String selectionRoot) {
+    return populateImplicitColumns(work.getPath(), selectionRoot);
+  }
+
+  /**
+   * Compares selection root and actual file path to determine partition columns values.
+   * Adds implicit file columns according to columns list.
+   *
+   * @return map with columns names as keys and their values
+   */
+  public Map<String, String> populateImplicitColumns(String filePath, String selectionRoot) {
     Map<String, String> implicitValues = Maps.newLinkedHashMap();
     if (selectionRoot != null) {
       String[] r = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)).toString().split("/");
-      Path path = Path.getPathWithoutSchemeAndAuthority(new Path(work.getPath()));
+      Path path = Path.getPathWithoutSchemeAndAuthority(new Path(filePath));
       String[] p = path.toString().split("/");
       if (p.length > r.length) {
         String[] q = ArrayUtils.subarray(p, r.length, p.length - 1);

http://git-wip-us.apache.org/repos/asf/drill/blob/9411b26e/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java
deleted file mode 100644
index bd59021..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java
+++ /dev/null
@@ -1,280 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.parquet;
-
-import org.apache.drill.common.expression.CastExpression;
-import org.apache.drill.common.expression.ConvertExpression;
-import org.apache.drill.common.expression.FunctionCall;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.ValueExpressions.BooleanExpression;
-import org.apache.drill.common.expression.ValueExpressions.DateExpression;
-import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
-import org.apache.drill.common.expression.ValueExpressions.FloatExpression;
-import org.apache.drill.common.expression.ValueExpressions.IntExpression;
-import org.apache.drill.common.expression.ValueExpressions.LongExpression;
-import org.apache.drill.common.expression.ValueExpressions.QuotedString;
-import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
-import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
-import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import org.joda.time.DateTimeUtils;
-
-public class ParquetCompareFunctionProcessor extends
-        AbstractExprVisitor<Boolean, LogicalExpression, RuntimeException> {
-    private Object value;
-    private boolean success;
-    private boolean isEqualityFn;
-    private SchemaPath path;
-    private String functionName;
-
-    public static final long JULIAN_DAY_EPOC = DateTimeUtils.toJulianDayNumber(0);
-
-    public static boolean isCompareFunction(String functionName) {
-        return COMPARE_FUNCTIONS_TRANSPOSE_MAP.keySet().contains(functionName);
-    }
-
-    public static ParquetCompareFunctionProcessor process(FunctionCall call) {
-        String functionName = call.getName();
-        LogicalExpression nameArg = call.args.get(0);
-        LogicalExpression valueArg = call.args.size() == 2 ? call.args.get(1)
-                : null;
-        ParquetCompareFunctionProcessor evaluator = new ParquetCompareFunctionProcessor(
-                functionName);
-
-        if (valueArg != null) { // binary function
-            if (VALUE_EXPRESSION_CLASSES.contains(nameArg.getClass())) {
-                LogicalExpression swapArg = valueArg;
-                valueArg = nameArg;
-                nameArg = swapArg;
-                evaluator.functionName = COMPARE_FUNCTIONS_TRANSPOSE_MAP
-                        .get(functionName);
-            }
-            evaluator.success = nameArg.accept(evaluator, valueArg);
-        } else if (call.args.get(0) instanceof SchemaPath) {
-            evaluator.success = true;
-            evaluator.path = (SchemaPath) nameArg;
-        }
-
-        return evaluator;
-    }
-
-    public ParquetCompareFunctionProcessor(String functionName) {
-        this.success = false;
-        this.functionName = functionName;
-        this.isEqualityFn = COMPARE_FUNCTIONS_TRANSPOSE_MAP
-                .containsKey(functionName)
-                && COMPARE_FUNCTIONS_TRANSPOSE_MAP.get(functionName).equals(
-                functionName);
-    }
-
-    public Object getValue() {
-        return value;
-    }
-
-    public boolean isSuccess() {
-        return success;
-    }
-
-    public SchemaPath getPath() {
-        return path;
-    }
-
-    public String getFunctionName() {
-        return functionName;
-    }
-
-    @Override
-    public Boolean visitCastExpression(CastExpression e,
-                                       LogicalExpression valueArg) throws RuntimeException {
-        if (e.getInput() instanceof CastExpression
-                || e.getInput() instanceof SchemaPath) {
-            return e.getInput().accept(this, valueArg);
-        }
-        return false;
-    }
-
-    @Override
-    public Boolean visitConvertExpression(ConvertExpression e,
-                                          LogicalExpression valueArg) throws RuntimeException {
-        if (e.getConvertFunction() == ConvertExpression.CONVERT_FROM
-                && e.getInput() instanceof SchemaPath) {
-            String encodingType = e.getEncodingType();
-            switch (encodingType) {
-                case "INT_BE":
-                case "INT":
-                case "UINT_BE":
-                case "UINT":
-                case "UINT4_BE":
-                case "UINT4":
-                    if (valueArg instanceof IntExpression
-                            && (isEqualityFn || encodingType.startsWith("U"))) {
-                        this.value = ((IntExpression) valueArg).getInt();
-                    }
-                    break;
-                case "BIGINT_BE":
-                case "BIGINT":
-                case "UINT8_BE":
-                case "UINT8":
-                    if (valueArg instanceof LongExpression
-                            && (isEqualityFn || encodingType.startsWith("U"))) {
-                        this.value = ((LongExpression) valueArg).getLong();
-                    }
-                    break;
-                case "FLOAT":
-                    if (valueArg instanceof FloatExpression && isEqualityFn) {
-                        this.value = ((FloatExpression) valueArg).getFloat();
-                    }
-                    break;
-                case "DOUBLE":
-                    if (valueArg instanceof DoubleExpression && isEqualityFn) {
-                        this.value = ((DoubleExpression) valueArg).getDouble();
-                    }
-                    break;
-                case "TIME_EPOCH":
-                case "TIME_EPOCH_BE":
-                    if (valueArg instanceof TimeExpression) {
-                        this.value = ((TimeExpression) valueArg).getTime();
-                    }
-                    break;
-                case "DATE_EPOCH":
-                case "DATE_EPOCH_BE":
-                    if (valueArg instanceof DateExpression) {
-                        long dateInMillis = ((DateExpression) valueArg).getDate();
-                        this.value = (int) (DateTimeUtils.toJulianDayNumber(dateInMillis) + JULIAN_DAY_EPOC);
-                    }
-                    break;
-                case "BOOLEAN_BYTE":
-                    if (valueArg instanceof BooleanExpression) {
-                        this.value = ((BooleanExpression) valueArg).getBoolean();
-                    }
-                    break;
-                case "UTF8":
-                    // let visitSchemaPath() handle this.
-                    return e.getInput().accept(this, valueArg);
-            }
-
-            if (value != null) {
-                this.path = (SchemaPath) e.getInput();
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public Boolean visitUnknown(LogicalExpression e, LogicalExpression valueArg)
-            throws RuntimeException {
-        return false;
-    }
-
-    @Override
-    public Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg)
-            throws RuntimeException {
-        if (valueArg instanceof QuotedString) {
-            this.value = ((QuotedString) valueArg).value;
-            this.path = path;
-            return true;
-        }
-
-        if (valueArg instanceof IntExpression) {
-            this.value = ((IntExpression) valueArg).getInt();
-            this.path = path;
-            return true;
-        }
-
-        if (valueArg instanceof LongExpression) {
-            this.value = ((LongExpression) valueArg).getLong();
-            this.path = path;
-            return true;
-        }
-
-        if (valueArg instanceof FloatExpression) {
-            this.value = ((FloatExpression) valueArg).getFloat();
-            this.path = path;
-            return true;
-        }
-
-        if (valueArg instanceof DoubleExpression) {
-            this.value = ((DoubleExpression) valueArg).getDouble();
-            this.path = path;
-            return true;
-        }
-
-        if (valueArg instanceof BooleanExpression) {
-            this.value = ((BooleanExpression) valueArg).getBoolean();
-            this.path = path;
-            return true;
-        }
-
-        if (valueArg instanceof TimeExpression) {
-            this.value = ((TimeExpression) valueArg).getTime();
-            this.path = path;
-            return true;
-        }
-
-        if (valueArg instanceof TimeStampExpression) {
-            this.value = ((TimeStampExpression) valueArg).getTimeStamp();
-            this.path = path;
-            return true;
-        }
-
-        if (valueArg instanceof DateExpression) {
-            long dateInMillis = ((DateExpression) valueArg).getDate();
-            this.value = (int) (DateTimeUtils.toJulianDayNumber(dateInMillis) + JULIAN_DAY_EPOC);
-            this.path = path;
-            return true;
-        }
-
-        return false;
-    }
-
-    private static final ImmutableSet<Class<? extends LogicalExpression>> VALUE_EXPRESSION_CLASSES;
-    static {
-        ImmutableSet.Builder<Class<? extends LogicalExpression>> builder = ImmutableSet
-                .builder();
-        VALUE_EXPRESSION_CLASSES = builder.add(BooleanExpression.class)
-                .add(DateExpression.class).add(DoubleExpression.class)
-                .add(FloatExpression.class).add(IntExpression.class)
-                .add(LongExpression.class).add(QuotedString.class)
-                .add(TimeExpression.class).add(TimeStampExpression.class)
-                .add(DateExpression.class).build();
-    }
-
-    private static final ImmutableMap<String, String> COMPARE_FUNCTIONS_TRANSPOSE_MAP;
-    static {
-        ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
-        COMPARE_FUNCTIONS_TRANSPOSE_MAP = builder
-                // unary functions
-                .put("isnotnull", "isnotnull")
-                .put("isNotNull", "isNotNull")
-                .put("is not null", "is not null")
-                .put("isnull", "isnull")
-                .put("isNull", "isNull")
-                .put("is null", "is null")
-                        // binary functions
-                .put("equal", "equal").put("not_equal", "not_equal")
-                .put("greater_than_or_equal_to", "less_than_or_equal_to")
-                .put("greater_than", "less_than")
-                .put("less_than_or_equal_to", "greater_than_or_equal_to")
-                .put("less_than", "greater_than").build();
-    }
-
-}


[06/10] drill git commit: DRILL-1950: Initial prototype patch for parquet filter pushdown.

Posted by pa...@apache.org.
DRILL-1950: Initial prototype patch for parquet filter pushdown.

Use three new classes from Adam's patch.


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/8ce1f033
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/8ce1f033
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/8ce1f033

Branch: refs/heads/master
Commit: 8ce1f03331970172dfd9d967504c60fd9752716a
Parents: a459e4d
Author: AdamPD <ad...@pharmadata.net.au>
Authored: Thu Jun 30 14:09:53 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Nov 4 15:59:31 2016 -0700

----------------------------------------------------------------------
 .../ParquetCompareFunctionProcessor.java        | 280 ++++++++++++++++++
 .../store/parquet/ParquetFilterBuilder.java     | 284 +++++++++++++++++++
 .../store/parquet/ParquetPushDownFilter.java    | 142 ++++++++++
 3 files changed, 706 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/8ce1f033/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java
new file mode 100644
index 0000000..bd59021
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetCompareFunctionProcessor.java
@@ -0,0 +1,280 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet;
+
+import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions.BooleanExpression;
+import org.apache.drill.common.expression.ValueExpressions.DateExpression;
+import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
+import org.apache.drill.common.expression.ValueExpressions.FloatExpression;
+import org.apache.drill.common.expression.ValueExpressions.IntExpression;
+import org.apache.drill.common.expression.ValueExpressions.LongExpression;
+import org.apache.drill.common.expression.ValueExpressions.QuotedString;
+import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
+import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.joda.time.DateTimeUtils;
+
+public class ParquetCompareFunctionProcessor extends
+        AbstractExprVisitor<Boolean, LogicalExpression, RuntimeException> {
+    private Object value;
+    private boolean success;
+    private boolean isEqualityFn;
+    private SchemaPath path;
+    private String functionName;
+
+    public static final long JULIAN_DAY_EPOC = DateTimeUtils.toJulianDayNumber(0);
+
+    public static boolean isCompareFunction(String functionName) {
+        return COMPARE_FUNCTIONS_TRANSPOSE_MAP.keySet().contains(functionName);
+    }
+
+    public static ParquetCompareFunctionProcessor process(FunctionCall call) {
+        String functionName = call.getName();
+        LogicalExpression nameArg = call.args.get(0);
+        LogicalExpression valueArg = call.args.size() == 2 ? call.args.get(1)
+                : null;
+        ParquetCompareFunctionProcessor evaluator = new ParquetCompareFunctionProcessor(
+                functionName);
+
+        if (valueArg != null) { // binary function
+            if (VALUE_EXPRESSION_CLASSES.contains(nameArg.getClass())) {
+                LogicalExpression swapArg = valueArg;
+                valueArg = nameArg;
+                nameArg = swapArg;
+                evaluator.functionName = COMPARE_FUNCTIONS_TRANSPOSE_MAP
+                        .get(functionName);
+            }
+            evaluator.success = nameArg.accept(evaluator, valueArg);
+        } else if (call.args.get(0) instanceof SchemaPath) {
+            evaluator.success = true;
+            evaluator.path = (SchemaPath) nameArg;
+        }
+
+        return evaluator;
+    }
+
+    public ParquetCompareFunctionProcessor(String functionName) {
+        this.success = false;
+        this.functionName = functionName;
+        this.isEqualityFn = COMPARE_FUNCTIONS_TRANSPOSE_MAP
+                .containsKey(functionName)
+                && COMPARE_FUNCTIONS_TRANSPOSE_MAP.get(functionName).equals(
+                functionName);
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    public boolean isSuccess() {
+        return success;
+    }
+
+    public SchemaPath getPath() {
+        return path;
+    }
+
+    public String getFunctionName() {
+        return functionName;
+    }
+
+    @Override
+    public Boolean visitCastExpression(CastExpression e,
+                                       LogicalExpression valueArg) throws RuntimeException {
+        if (e.getInput() instanceof CastExpression
+                || e.getInput() instanceof SchemaPath) {
+            return e.getInput().accept(this, valueArg);
+        }
+        return false;
+    }
+
+    @Override
+    public Boolean visitConvertExpression(ConvertExpression e,
+                                          LogicalExpression valueArg) throws RuntimeException {
+        if (e.getConvertFunction() == ConvertExpression.CONVERT_FROM
+                && e.getInput() instanceof SchemaPath) {
+            String encodingType = e.getEncodingType();
+            switch (encodingType) {
+                case "INT_BE":
+                case "INT":
+                case "UINT_BE":
+                case "UINT":
+                case "UINT4_BE":
+                case "UINT4":
+                    if (valueArg instanceof IntExpression
+                            && (isEqualityFn || encodingType.startsWith("U"))) {
+                        this.value = ((IntExpression) valueArg).getInt();
+                    }
+                    break;
+                case "BIGINT_BE":
+                case "BIGINT":
+                case "UINT8_BE":
+                case "UINT8":
+                    if (valueArg instanceof LongExpression
+                            && (isEqualityFn || encodingType.startsWith("U"))) {
+                        this.value = ((LongExpression) valueArg).getLong();
+                    }
+                    break;
+                case "FLOAT":
+                    if (valueArg instanceof FloatExpression && isEqualityFn) {
+                        this.value = ((FloatExpression) valueArg).getFloat();
+                    }
+                    break;
+                case "DOUBLE":
+                    if (valueArg instanceof DoubleExpression && isEqualityFn) {
+                        this.value = ((DoubleExpression) valueArg).getDouble();
+                    }
+                    break;
+                case "TIME_EPOCH":
+                case "TIME_EPOCH_BE":
+                    if (valueArg instanceof TimeExpression) {
+                        this.value = ((TimeExpression) valueArg).getTime();
+                    }
+                    break;
+                case "DATE_EPOCH":
+                case "DATE_EPOCH_BE":
+                    if (valueArg instanceof DateExpression) {
+                        long dateInMillis = ((DateExpression) valueArg).getDate();
+                        this.value = (int) (DateTimeUtils.toJulianDayNumber(dateInMillis) + JULIAN_DAY_EPOC);
+                    }
+                    break;
+                case "BOOLEAN_BYTE":
+                    if (valueArg instanceof BooleanExpression) {
+                        this.value = ((BooleanExpression) valueArg).getBoolean();
+                    }
+                    break;
+                case "UTF8":
+                    // let visitSchemaPath() handle this.
+                    return e.getInput().accept(this, valueArg);
+            }
+
+            if (value != null) {
+                this.path = (SchemaPath) e.getInput();
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Boolean visitUnknown(LogicalExpression e, LogicalExpression valueArg)
+            throws RuntimeException {
+        return false;
+    }
+
+    @Override
+    public Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg)
+            throws RuntimeException {
+        if (valueArg instanceof QuotedString) {
+            this.value = ((QuotedString) valueArg).value;
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof IntExpression) {
+            this.value = ((IntExpression) valueArg).getInt();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof LongExpression) {
+            this.value = ((LongExpression) valueArg).getLong();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof FloatExpression) {
+            this.value = ((FloatExpression) valueArg).getFloat();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof DoubleExpression) {
+            this.value = ((DoubleExpression) valueArg).getDouble();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof BooleanExpression) {
+            this.value = ((BooleanExpression) valueArg).getBoolean();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof TimeExpression) {
+            this.value = ((TimeExpression) valueArg).getTime();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof TimeStampExpression) {
+            this.value = ((TimeStampExpression) valueArg).getTimeStamp();
+            this.path = path;
+            return true;
+        }
+
+        if (valueArg instanceof DateExpression) {
+            long dateInMillis = ((DateExpression) valueArg).getDate();
+            this.value = (int) (DateTimeUtils.toJulianDayNumber(dateInMillis) + JULIAN_DAY_EPOC);
+            this.path = path;
+            return true;
+        }
+
+        return false;
+    }
+
+    private static final ImmutableSet<Class<? extends LogicalExpression>> VALUE_EXPRESSION_CLASSES;
+    static {
+        ImmutableSet.Builder<Class<? extends LogicalExpression>> builder = ImmutableSet
+                .builder();
+        VALUE_EXPRESSION_CLASSES = builder.add(BooleanExpression.class)
+                .add(DateExpression.class).add(DoubleExpression.class)
+                .add(FloatExpression.class).add(IntExpression.class)
+                .add(LongExpression.class).add(QuotedString.class)
+                .add(TimeExpression.class).add(TimeStampExpression.class)
+                .add(DateExpression.class).build();
+    }
+
+    private static final ImmutableMap<String, String> COMPARE_FUNCTIONS_TRANSPOSE_MAP;
+    static {
+        ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
+        COMPARE_FUNCTIONS_TRANSPOSE_MAP = builder
+                // unary functions
+                .put("isnotnull", "isnotnull")
+                .put("isNotNull", "isNotNull")
+                .put("is not null", "is not null")
+                .put("isnull", "isnull")
+                .put("isNull", "isNull")
+                .put("is null", "is null")
+                        // binary functions
+                .put("equal", "equal").put("not_equal", "not_equal")
+                .put("greater_than_or_equal_to", "less_than_or_equal_to")
+                .put("greater_than", "less_than")
+                .put("less_than_or_equal_to", "greater_than_or_equal_to")
+                .put("less_than", "greater_than").build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8ce1f033/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
new file mode 100644
index 0000000..b613707
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
@@ -0,0 +1,284 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.drill.common.expression.BooleanOperator;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.parquet.filter2.predicate.FilterApi;
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+
+import java.io.IOException;
+import java.util.List;
+
+public class ParquetFilterBuilder extends
+        AbstractExprVisitor<FilterPredicate, Void, RuntimeException> {
+    static final Logger logger = LoggerFactory
+            .getLogger(ParquetFilterBuilder.class);
+    private LogicalExpression le;
+    private boolean allExpressionsConverted = true;
+    private ParquetGroupScan groupScan;
+
+    public ParquetFilterBuilder(ParquetGroupScan groupScan, LogicalExpression conditionExp) {
+        this.le = conditionExp;
+        this.groupScan = groupScan;
+    }
+
+    public ParquetGroupScan parseTree() {
+        FilterPredicate predicate = le.accept(this, null);
+        try {
+            return this.groupScan.clone(predicate);
+        } catch (IOException e) {
+            logger.error("Failed to set Parquet filter", e);
+            return null;
+        }
+    }
+
+    public boolean areAllExpressionsConverted() {
+        return allExpressionsConverted;
+    }
+
+    @Override
+    public FilterPredicate visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
+        allExpressionsConverted = false;
+        return null;
+    }
+
+    @Override
+    public FilterPredicate visitBooleanOperator(BooleanOperator op, Void value) {
+        List<LogicalExpression> args = op.args;
+        FilterPredicate nodePredicate = null;
+        String functionName = op.getName();
+        for (LogicalExpression arg : args) {
+            switch (functionName) {
+                case "booleanAnd":
+                case "booleanOr":
+                    if (nodePredicate == null) {
+                        nodePredicate = arg.accept(this, null);
+                    } else {
+                        FilterPredicate predicate = arg.accept(this, null);
+                        if (predicate != null) {
+                            nodePredicate = mergePredicates(functionName, nodePredicate, predicate);
+                        } else {
+                            // we can't include any part of the OR if any of the predicates cannot be converted
+                            if (functionName == "booleanOr") {
+                                nodePredicate = null;
+                            }
+                            allExpressionsConverted = false;
+                        }
+                    }
+                    break;
+            }
+        }
+        return nodePredicate;
+    }
+
+    private FilterPredicate mergePredicates(String functionName,
+                                            FilterPredicate leftPredicate, FilterPredicate rightPredicate) {
+        if (leftPredicate != null && rightPredicate != null) {
+            if (functionName == "booleanAnd") {
+                return FilterApi.and(leftPredicate, rightPredicate);
+            }
+            else {
+                return FilterApi.or(leftPredicate, rightPredicate);
+            }
+        } else {
+            allExpressionsConverted = false;
+            if ("booleanAnd".equals(functionName)) {
+                return leftPredicate == null ? rightPredicate : leftPredicate;
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public FilterPredicate visitFunctionCall(FunctionCall call, Void value) throws RuntimeException {
+        FilterPredicate predicate = null;
+        String functionName = call.getName();
+        ImmutableList<LogicalExpression> args = call.args;
+
+        if (ParquetCompareFunctionProcessor.isCompareFunction(functionName)) {
+            ParquetCompareFunctionProcessor processor = ParquetCompareFunctionProcessor
+                    .process(call);
+            if (processor.isSuccess()) {
+                try {
+                    predicate = createFilterPredicate(processor.getFunctionName(),
+                            processor.getPath(), processor.getValue());
+                } catch (Exception e) {
+                    logger.error("Failed to create Parquet filter", e);
+                }
+            }
+        } else {
+            switch (functionName) {
+                case "booleanAnd":
+                case "booleanOr":
+                    FilterPredicate leftPredicate = args.get(0).accept(this, null);
+                    FilterPredicate rightPredicate = args.get(1).accept(this, null);
+                    predicate = mergePredicates(functionName, leftPredicate, rightPredicate);
+                    break;
+            }
+        }
+
+        if (predicate == null) {
+            allExpressionsConverted = false;
+        }
+
+        return predicate;
+    }
+
+    private FilterPredicate createFilterPredicate(String functionName,
+                                                  SchemaPath field, Object fieldValue) {
+        FilterPredicate filter = null;
+
+        // extract the field name
+        String fieldName = field.getAsUnescapedPath();
+        switch (functionName) {
+            case "equal":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.eq(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.eq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.eq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.eq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                else if (fieldValue instanceof Boolean) {
+                    filter = FilterApi.eq(FilterApi.booleanColumn(fieldName), (Boolean) fieldValue);
+                }
+                break;
+            case "not_equal":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.notEq(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.notEq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.notEq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.notEq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                else if (fieldValue instanceof Boolean) {
+                    filter = FilterApi.notEq(FilterApi.booleanColumn(fieldName), (Boolean) fieldValue);
+                }
+                break;
+            case "greater_than_or_equal_to":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.gtEq(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.gtEq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.gtEq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.gtEq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                break;
+            case "greater_than":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.gt(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.gt(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.gt(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.gt(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                break;
+            case "less_than_or_equal_to":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.ltEq(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.ltEq(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.ltEq(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.ltEq(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                break;
+            case "less_than":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.lt(FilterApi.longColumn(fieldName), (Long) fieldValue);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.lt(FilterApi.intColumn(fieldName), (Integer) fieldValue);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.lt(FilterApi.floatColumn(fieldName), (Float) fieldValue);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.lt(FilterApi.doubleColumn(fieldName), (Double) fieldValue);
+                }
+                break;
+            case "isnull":
+            case "isNull":
+            case "is null":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.eq(FilterApi.longColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.eq(FilterApi.intColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.eq(FilterApi.floatColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.eq(FilterApi.doubleColumn(fieldName), null);
+                }
+                break;
+            case "isnotnull":
+            case "isNotNull":
+            case "is not null":
+                if (fieldValue instanceof Long) {
+                    filter = FilterApi.notEq(FilterApi.longColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Integer) {
+                    filter = FilterApi.notEq(FilterApi.intColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Float) {
+                    filter = FilterApi.notEq(FilterApi.floatColumn(fieldName), null);
+                }
+                else if (fieldValue instanceof Double) {
+                    filter = FilterApi.notEq(FilterApi.doubleColumn(fieldName), null);
+                }
+                break;
+        }
+
+        return filter;
+    }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8ce1f033/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
new file mode 100644
index 0000000..10c817b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.PrelUtil;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+
+import com.google.common.collect.ImmutableList;
+
+public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
+    public static final StoragePluginOptimizerRule getFilterOnProject(final OptimizerRulesContext context){
+        return new ParquetPushDownFilter(
+                RelOptHelper.some(FilterPrel.class, RelOptHelper.some(ProjectPrel.class, RelOptHelper.any(ScanPrel.class))),
+                "ParquetPushDownFilter:Filter_On_Project", context) {
+
+            @Override
+            public boolean matches(RelOptRuleCall call) {
+                if (!enabled) {
+                    return false;
+                }
+                final ScanPrel scan = call.rel(2);
+                if (scan.getGroupScan() instanceof ParquetGroupScan) {
+                    return super.matches(call);
+                }
+                return false;
+            }
+
+            @Override
+            public void onMatch(RelOptRuleCall call) {
+                final FilterPrel filterRel = call.rel(0);
+                final ProjectPrel projectRel = call.rel(1);
+                final ScanPrel scanRel = call.rel(2);
+                doOnMatch(call, filterRel, projectRel, scanRel);
+            };
+        };
+    }
+
+    public static final StoragePluginOptimizerRule getFilterOnScan(final OptimizerRulesContext context){
+        return new ParquetPushDownFilter(
+                RelOptHelper.some(FilterPrel.class, RelOptHelper.any(ScanPrel.class)),
+                "ParquetPushDownFilter:Filter_On_Scan", context) {
+
+            @Override
+            public boolean matches(RelOptRuleCall call) {
+                if (!enabled) {
+                    return false;
+                }
+                final ScanPrel scan = call.rel(1);
+                if (scan.getGroupScan() instanceof ParquetGroupScan) {
+                    return super.matches(call);
+                }
+                return false;
+            }
+
+            @Override
+            public void onMatch(RelOptRuleCall call) {
+                final FilterPrel filterRel = call.rel(0);
+                final ScanPrel scanRel = call.rel(1);
+                doOnMatch(call, filterRel, null, scanRel);
+            }
+        };
+    }
+
+    private final OptimizerRulesContext context;
+    // private final boolean useNewReader;
+    protected final boolean enabled;
+
+    private ParquetPushDownFilter(RelOptRuleOperand operand, String id, OptimizerRulesContext context) {
+        super(operand, id);
+        this.context = context;
+        this.enabled = context.getPlannerSettings().isParquetFilterPushEnabled();
+        // this.useNewReader = context.getPlannerSettings()getOptions().getOption(ExecConstants.PARQUET_NEW_RECORD_READER).bool_val;
+    }
+
+    protected void doOnMatch(RelOptRuleCall call, FilterPrel filter, ProjectPrel project, ScanPrel scan) {
+        ParquetGroupScan groupScan = (ParquetGroupScan) scan.getGroupScan();
+        if (groupScan.getFilter() != null) {
+            return;
+        }
+
+        RexNode condition = null;
+        if(project == null){
+            condition = filter.getCondition();
+        }else{
+            // get the filter as if it were below the projection.
+            condition = RelOptUtil.pushFilterPastProject(filter.getCondition(), project);
+        }
+
+        LogicalExpression conditionExp = DrillOptiq.toDrill(
+                new DrillParseContext(PrelUtil.getPlannerSettings(call.getPlanner())), scan, condition);
+        ParquetFilterBuilder parquetFilterBuilder = new ParquetFilterBuilder(groupScan,
+                conditionExp);
+        ParquetGroupScan newGroupScan = parquetFilterBuilder.parseTree();
+        if (newGroupScan == null) {
+            return; // no filter pushdown so nothing to apply.
+        }
+
+        final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(),
+                newGroupScan, scan.getRowType());
+
+        RelNode inputPrel = newScanPrel;
+
+        if(project != null){
+            inputPrel = project.copy(project.getTraitSet(), ImmutableList.of(inputPrel));
+        }
+
+        // Normally we could eliminate the filter if all expressions were pushed down;
+        // however, the Parquet filter implementation is type specific (whereas Drill is not)
+        final RelNode newFilter = filter.copy(filter.getTraitSet(), ImmutableList.of(inputPrel));
+        call.transformTo(newFilter);
+    }
+}


[02/10] drill git commit: DRILL-4800: Add AsyncPageReader to pipeline PageRead Use non tracking input stream for Parquet scans. Make choice between async and sync reader configurable. Make various options user configurable - choose between sync and async

Posted by pa...@apache.org.
DRILL-4800: Add AsyncPageReader to pipeline PageRead Use non tracking input stream for Parquet scans. Make choice between async and sync reader configurable. Make various options user configurable - choose between sync and async page reader, enable/disable fadvise Add Parquet Scan metrics to track time spent in various operations


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/f9a443d8
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/f9a443d8
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/f9a443d8

Branch: refs/heads/master
Commit: f9a443d8a3d8e81b7e76f161b611003d16a53a4d
Parents: fe2334e
Author: Parth Chandra <pa...@apache.org>
Authored: Tue Sep 27 14:03:35 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Wed Nov 2 17:57:46 2016 -0700

----------------------------------------------------------------------
 .../src/resources/drill-override-example.conf   |   4 +
 .../org/apache/drill/exec/ExecConstants.java    |  15 +
 .../apache/drill/exec/ops/OperatorContext.java  |   6 +
 .../drill/exec/ops/OperatorContextImpl.java     |  21 +-
 .../drill/exec/ops/OperatorMetricRegistry.java  |   2 +
 .../apache/drill/exec/ops/OperatorStats.java    |  16 +-
 .../drill/exec/server/BootStrapContext.java     |  33 +-
 .../drill/exec/server/DrillbitContext.java      |   6 +
 .../server/options/SystemOptionManager.java     |   5 +-
 .../exec/store/parquet/ParquetReaderStats.java  |  42 +--
 .../store/parquet/ParquetScanBatchCreator.java  |  14 +-
 .../parquet/columnreaders/AsyncPageReader.java  | 332 +++++++++++++++++++
 .../parquet/columnreaders/ColumnReader.java     |  36 +-
 .../store/parquet/columnreaders/PageReader.java | 160 +++++----
 .../columnreaders/ParquetRecordReader.java      | 102 +++++-
 .../columnreaders/VarLenBinaryReader.java       |   6 +-
 .../BufferedDirectBufInputStream.java           |  51 ++-
 .../src/main/resources/drill-module.conf        |   4 +
 .../src/main/resources/rest/profile/profile.ftl |   2 +-
 19 files changed, 684 insertions(+), 173 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/distribution/src/resources/drill-override-example.conf
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-override-example.conf b/distribution/src/resources/drill-override-example.conf
index 52949db..4be4aa2 100644
--- a/distribution/src/resources/drill-override-example.conf
+++ b/distribution/src/resources/drill-override-example.conf
@@ -166,6 +166,10 @@ drill.exec: {
       initial: 20000000
     }
   },
+  scan: {
+    threadpool_size: 8,
+    decode_threadpool_size: 1
+  },
   debug.error_on_leak: true
 }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index ba6b084..a13fd71 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -96,6 +96,10 @@ public interface ExecConstants {
   /** Size of JDBC batch queue (in batches) above which throttling begins. */
   String JDBC_BATCH_QUEUE_THROTTLING_THRESHOLD =
       "drill.jdbc.batch_queue_throttling_threshold";
+  // Thread pool size for scan threads. Used by the Parquet scan.
+  String SCAN_THREADPOOL_SIZE = "drill.exec.scan.threadpool_size";
+  // The size of the thread pool used by a scan to decode the data. Used by Parquet
+  String SCAN_DECODE_THREADPOOL_SIZE = "drill.exec.scan.decode_threadpool_size";
 
   /**
    * Currently if a query is cancelled, but one of the fragments reports the status as FAILED instead of CANCELLED or
@@ -147,10 +151,21 @@ public interface ExecConstants {
   String PARQUET_READER_INT96_AS_TIMESTAMP = "store.parquet.reader.int96_as_timestamp";
   OptionValidator PARQUET_READER_INT96_AS_TIMESTAMP_VALIDATOR = new BooleanValidator(PARQUET_READER_INT96_AS_TIMESTAMP, false);
 
+  String PARQUET_PAGEREADER_ASYNC = "store.parquet.reader.pagereader.async";
+  OptionValidator PARQUET_PAGEREADER_ASYNC_VALIDATOR = new BooleanValidator(PARQUET_PAGEREADER_ASYNC, true);
+
   // Use a buffering reader for parquet page reader
   String PARQUET_PAGEREADER_USE_BUFFERED_READ = "store.parquet.reader.pagereader.bufferedread";
   OptionValidator PARQUET_PAGEREADER_USE_BUFFERED_READ_VALIDATOR = new  BooleanValidator(PARQUET_PAGEREADER_USE_BUFFERED_READ, true);
 
+  // Size in MiB of the buffer the Parquet page reader will use to read from disk. Default is 8 MiB
+  String PARQUET_PAGEREADER_BUFFER_SIZE = "store.parquet.reader.pagereader.buffersize";
+  OptionValidator PARQUET_PAGEREADER_BUFFER_SIZE_VALIDATOR = new  LongValidator(PARQUET_PAGEREADER_BUFFER_SIZE, 4*1024*1024);
+
+  // try to use fadvise if available
+  String PARQUET_PAGEREADER_USE_FADVISE = "store.parquet.reader.pagereader.usefadvise";
+  OptionValidator PARQUET_PAGEREADER_USE_FADVISE_VALIDATOR = new  BooleanValidator(PARQUET_PAGEREADER_USE_FADVISE, false);
+
   OptionValidator COMPILE_SCALAR_REPLACEMENT = new BooleanValidator("exec.compile.scalar_replacement", false);
 
   String JSON_ALL_TEXT_MODE = "store.json.all_text_mode";

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
index 33fa288..92a7269 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
@@ -47,10 +47,16 @@ public abstract class OperatorContext {
 
   public abstract ExecutorService getExecutor();
 
+  public abstract ExecutorService getScanExecutor();
+
+  public abstract ExecutorService getScanDecodeExecutor();
+
   public abstract ExecutionControls getExecutionControls();
 
   public abstract DrillFileSystem newFileSystem(Configuration conf) throws IOException;
 
+  public abstract DrillFileSystem newNonTrackingFileSystem(Configuration conf) throws IOException;
+
   /**
    * Run the callable as the given proxy user.
    *

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
index 85f0ccb..38ddd16 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
@@ -50,6 +50,8 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
   private final BufferManager manager;
   private DrillFileSystem fs;
   private final ExecutorService executor;
+  private final ExecutorService scanExecutor;
+  private final ExecutorService scanDecodeExecutor;
 
   /**
    * This lazily initialized executor service is used to submit a {@link Callable task} that needs a proxy user. There
@@ -70,6 +72,8 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
     stats = context.getStats().newOperatorStats(def, allocator);
     executionControls = context.getExecutionControls();
     executor = context.getDrillbitContext().getExecutor();
+    scanExecutor = context.getDrillbitContext().getScanExecutor();
+    scanDecodeExecutor = context.getDrillbitContext().getScanDecodeExecutor();
   }
 
   public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, OperatorStats stats)
@@ -81,6 +85,8 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
     this.stats     = stats;
     executionControls = context.getExecutionControls();
     executor = context.getDrillbitContext().getExecutor();
+    scanExecutor = context.getDrillbitContext().getScanExecutor();
+    scanDecodeExecutor = context.getDrillbitContext().getScanDecodeExecutor();
   }
 
   public DrillBuf replace(DrillBuf old, int newSize) {
@@ -95,10 +101,16 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
     return manager.getManagedBuffer(size);
   }
 
-  // Allow and operator to use the thread pool
+  // Allow an operator to use the thread pool
   public ExecutorService getExecutor() {
     return executor;
   }
+  public ExecutorService getScanExecutor() {
+    return scanExecutor;
+  }
+  public ExecutorService getScanDecodeExecutor() {
+    return scanDecodeExecutor;
+  }
 
   public ExecutionControls getExecutionControls() {
     return executionControls;
@@ -179,4 +191,11 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
     return fs;
   }
 
+  @Override
+  public DrillFileSystem newNonTrackingFileSystem(Configuration conf) throws IOException {
+    Preconditions.checkState(fs == null, "Tried to create a second FileSystem. Can only be called once per OperatorContext");
+    fs = new DrillFileSystem(conf, null);
+    return fs;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorMetricRegistry.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorMetricRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorMetricRegistry.java
index b704bb6..0424332 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorMetricRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorMetricRegistry.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.physical.impl.mergereceiver.MergingRecordBatch;
 import org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec;
 import org.apache.drill.exec.physical.impl.unorderedreceiver.UnorderedReceiverBatch;
 import org.apache.drill.exec.physical.impl.xsort.ExternalSortBatch;
+import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 /**
@@ -47,6 +48,7 @@ public class OperatorMetricRegistry {
     register(CoreOperatorType.HASH_AGGREGATE_VALUE, HashAggTemplate.Metric.class);
     register(CoreOperatorType.HASH_JOIN_VALUE, HashJoinBatch.Metric.class);
     register(CoreOperatorType.EXTERNAL_SORT_VALUE, ExternalSortBatch.Metric.class);
+    register(CoreOperatorType.PARQUET_ROW_GROUP_SCAN_VALUE, ParquetRecordReader.Metric.class);
   }
 
   private static void register(final int operatorType, final Class<? extends MetricDef> metricDef) {

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
index 271f734..b565774 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
@@ -131,7 +131,7 @@ public class OperatorStats {
   /**
    * Clear stats
    */
-  public void clear() {
+  public synchronized void clear() {
     processingNanos = 0l;
     setupNanos = 0l;
     waitNanos = 0l;
@@ -139,47 +139,47 @@ public class OperatorStats {
     doubleMetrics.clear();
   }
 
-  public void startSetup() {
+  public synchronized void startSetup() {
     assert !inSetup  : assertionError("starting setup");
     stopProcessing();
     inSetup = true;
     setupMark = System.nanoTime();
   }
 
-  public void stopSetup() {
+  public synchronized void stopSetup() {
     assert inSetup :  assertionError("stopping setup");
     startProcessing();
     setupNanos += System.nanoTime() - setupMark;
     inSetup = false;
   }
 
-  public void startProcessing() {
+  public synchronized void startProcessing() {
     assert !inProcessing : assertionError("starting processing");
     processingMark = System.nanoTime();
     inProcessing = true;
   }
 
-  public void stopProcessing() {
+  public synchronized void stopProcessing() {
     assert inProcessing : assertionError("stopping processing");
     processingNanos += System.nanoTime() - processingMark;
     inProcessing = false;
   }
 
-  public void startWait() {
+  public synchronized void startWait() {
     assert !inWait : assertionError("starting waiting");
     stopProcessing();
     inWait = true;
     waitMark = System.nanoTime();
   }
 
-  public void stopWait() {
+  public synchronized void stopWait() {
     assert inWait : assertionError("stopping waiting");
     startProcessing();
     waitNanos += System.nanoTime() - waitMark;
     inWait = false;
   }
 
-  public void batchReceived(int inputIndex, long records, boolean newSchema) {
+  public synchronized void batchReceived(int inputIndex, long records, boolean newSchema) {
     recordsReceivedByInput[inputIndex] += records;
     batchesReceivedByInput[inputIndex]++;
     if(newSchema){

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
index 6554e33..adb6323 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
@@ -20,11 +20,11 @@ package org.apache.drill.exec.server;
 import com.codahale.metrics.MetricRegistry;
 import io.netty.channel.EventLoopGroup;
 
+import java.util.concurrent.Executors;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-
+import java.util.concurrent.SynchronousQueue;
 import org.apache.drill.common.DrillAutoCloseables;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.scanner.persistence.ScanResult;
@@ -37,6 +37,7 @@ import org.apache.drill.exec.rpc.TransportCheck;
 
 public class BootStrapContext implements AutoCloseable {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BootStrapContext.class);
+  private static final int MIN_SCAN_THREADPOOL_SIZE = 8; // Magic num
 
   private final DrillConfig config;
   private final EventLoopGroup loop;
@@ -45,12 +46,15 @@ public class BootStrapContext implements AutoCloseable {
   private final BufferAllocator allocator;
   private final ScanResult classpathScan;
   private final ExecutorService executor;
+  private final ExecutorService scanExecutor;
+  private final ExecutorService scanDecodeExecutor;
 
   public BootStrapContext(DrillConfig config, ScanResult classpathScan) {
     this.config = config;
     this.classpathScan = classpathScan;
     this.loop = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), "BitServer-");
-    this.loop2 = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), "BitClient-");
+    this.loop2 = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS),
+        "BitClient-");
     // Note that metrics are stored in a static instance
     this.metrics = DrillMetrics.getRegistry();
     this.allocator = RootAllocatorFactory.newRoot(config);
@@ -65,12 +69,35 @@ public class BootStrapContext implements AutoCloseable {
         super.afterExecute(r, t);
       }
     };
+    // Setup two threadpools one for reading raw data from disk and another for decoding the data
+    // A good guideline is to have the number threads in the scan pool to be a multiple (fractional
+    // numbers are ok) of the number of disks.
+    // A good guideline is to have the number threads in the decode pool to be a small multiple (fractional
+    // numbers are ok) of the number of cores.
+    final int numCores = Runtime.getRuntime().availableProcessors();
+    final int numScanThreads = (int) (config.getDouble(ExecConstants.SCAN_THREADPOOL_SIZE));
+    final int numScanDecodeThreads = (int) config.getDouble(ExecConstants.SCAN_DECODE_THREADPOOL_SIZE);
+    final int scanThreadPoolSize =
+        MIN_SCAN_THREADPOOL_SIZE > numScanThreads ? MIN_SCAN_THREADPOOL_SIZE : numScanThreads;
+    final int scanDecodeThreadPoolSize = numCores > numScanDecodeThreads ? numCores : numScanDecodeThreads;
+
+    this.scanExecutor = Executors.newFixedThreadPool(scanThreadPoolSize, new NamedThreadFactory("scan-"));
+    this.scanDecodeExecutor =
+        Executors.newFixedThreadPool(scanDecodeThreadPoolSize, new NamedThreadFactory("scan-decode-"));
   }
 
   public ExecutorService getExecutor() {
     return executor;
   }
 
+  public ExecutorService getScanExecutor() {
+    return scanExecutor;
+  }
+
+  public ExecutorService getScanDecodeExecutor() {
+    return scanDecodeExecutor;
+  }
+
   public DrillConfig getConfig() {
     return config;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index 3eb87ea..ffe6c28 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -172,6 +172,12 @@ public class DrillbitContext implements AutoCloseable {
   public ExecutorService getExecutor() {
     return context.getExecutor();
   }
+  public ExecutorService getScanExecutor() {
+    return context.getScanExecutor();
+  }
+  public ExecutorService getScanDecodeExecutor() {
+    return context.getScanDecodeExecutor();
+  }
 
   public LogicalPlanPersistence getLpPersistence() {
     return lpPersistence;

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 8b67fdb..1981d24 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -99,8 +99,11 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING_VALIDATOR,
       ExecConstants.PARQUET_VECTOR_FILL_THRESHOLD_VALIDATOR,
       ExecConstants.PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR,
-      ExecConstants.PARQUET_PAGEREADER_USE_BUFFERED_READ_VALIDATOR,
       ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR,
+      ExecConstants.PARQUET_PAGEREADER_ASYNC_VALIDATOR,
+      ExecConstants.PARQUET_PAGEREADER_USE_BUFFERED_READ_VALIDATOR,
+      ExecConstants.PARQUET_PAGEREADER_BUFFER_SIZE_VALIDATOR,
+      ExecConstants.PARQUET_PAGEREADER_USE_FADVISE_VALIDATOR,
       ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP_VALIDATOR,
       ExecConstants.JSON_READER_ALL_TEXT_MODE_VALIDATOR,
       ExecConstants.ENABLE_UNION_TYPE,

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderStats.java
index e95b0c8..c2711cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderStats.java
@@ -17,28 +17,30 @@
  */
 package org.apache.drill.exec.store.parquet;
 
+import java.util.concurrent.atomic.AtomicLong;
+
 public class ParquetReaderStats {
 
-  public long numDictPageHeaders;
-  public long numPageHeaders;
-  public long numDictPageLoads;
-  public long numPageLoads;
-  public long numDictPagesDecompressed;
-  public long numPagesDecompressed;
-
-  public long totalDictPageHeaderBytes;
-  public long totalPageHeaderBytes;
-  public long totalDictPageReadBytes;
-  public long totalPageReadBytes;
-  public long totalDictDecompressedBytes;
-  public long totalDecompressedBytes;
-
-  public long timeDictPageHeaders;
-  public long timePageHeaders;
-  public long timeDictPageLoads;
-  public long timePageLoads;
-  public long timeDictPagesDecompressed;
-  public long timePagesDecompressed;
+  public AtomicLong numDictPageLoads = new AtomicLong();
+  public AtomicLong numDataPageLoads = new AtomicLong();
+  public AtomicLong numDataPagesDecoded = new AtomicLong();
+  public AtomicLong numDictPagesDecompressed = new AtomicLong();
+  public AtomicLong numDataPagesDecompressed = new AtomicLong();
+
+  public AtomicLong totalDictPageReadBytes = new AtomicLong();
+  public AtomicLong totalDataPageReadBytes = new AtomicLong();
+  public AtomicLong totalDictDecompressedBytes = new AtomicLong();
+  public AtomicLong totalDataDecompressedBytes = new AtomicLong();
+
+  public AtomicLong timeDictPageLoads = new AtomicLong();
+  public AtomicLong timeDataPageLoads = new AtomicLong();
+  public AtomicLong timeDataPageDecode = new AtomicLong();
+  public AtomicLong timeDictPageDecode = new AtomicLong();
+  public AtomicLong timeDictPagesDecompressed = new AtomicLong();
+  public AtomicLong timeDataPagesDecompressed = new AtomicLong();
+
+  public AtomicLong timeDiskScanWait = new AtomicLong();
+  public AtomicLong timeDiskScan = new AtomicLong();
 
   public ParquetReaderStats() {
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
index a98c660..a14bab5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
@@ -73,10 +73,18 @@ public class ParquetScanBatchCreator implements BatchCreator<ParquetRowGroupScan
 
     DrillFileSystem fs;
     try {
-      fs = oContext.newFileSystem(rowGroupScan.getStorageEngine().getFsConf());
-    } catch(IOException e) {
-      throw new ExecutionSetupException(String.format("Failed to create DrillFileSystem: %s", e.getMessage()), e);
+      boolean useAsyncPageReader =
+          context.getOptions().getOption(ExecConstants.PARQUET_PAGEREADER_ASYNC).bool_val;
+      if (useAsyncPageReader) {
+        fs = oContext.newNonTrackingFileSystem(rowGroupScan.getStorageEngine().getFsConf());
+      } else {
+        fs = oContext.newFileSystem(rowGroupScan.getStorageEngine().getFsConf());
+      }
+    } catch (IOException e) {
+      throw new ExecutionSetupException(
+          String.format("Failed to create DrillFileSystem: %s", e.getMessage()), e);
     }
+
     Configuration conf = new Configuration(fs.getConf());
     conf.setBoolean(ENABLE_BYTES_READ_COUNTER, false);
     conf.setBoolean(ENABLE_BYTES_TOTAL_COUNTER, false);

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
new file mode 100644
index 0000000..3f47f04
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
@@ -0,0 +1,332 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.parquet.columnreaders;
+
+import com.google.common.base.Stopwatch;
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.drill.exec.util.filereader.DirectBufInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.PageType;
+import org.apache.parquet.format.Util;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.parquet.column.Encoding.valueOf;
+
+class AsyncPageReader extends PageReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AsyncPageReader.class);
+
+
+  private ExecutorService threadPool;
+  private Future<ReadStatus> asyncPageRead;
+
+  AsyncPageReader(ColumnReader<?> parentStatus, FileSystem fs, Path path,
+      ColumnChunkMetaData columnChunkMetaData) throws ExecutionSetupException {
+    super(parentStatus, fs, path, columnChunkMetaData);
+    if (threadPool == null) {
+      threadPool = parentColumnReader.parentReader.getOperatorContext().getScanExecutor();
+    }
+    asyncPageRead = threadPool.submit(new AsyncPageReaderTask());
+  }
+
+  @Override protected void loadDictionaryIfExists(final ColumnReader<?> parentStatus,
+      final ColumnChunkMetaData columnChunkMetaData, final DirectBufInputStream f) throws UserException {
+    if (columnChunkMetaData.getDictionaryPageOffset() > 0) {
+      try {
+        dataReader.skip(columnChunkMetaData.getDictionaryPageOffset() - dataReader.getPos());
+      } catch (IOException e) {
+        handleAndThrowException(e, "Error Reading dictionary page.");
+      }
+      // parent constructor may call this method before the thread pool is set.
+      if (threadPool == null) {
+        threadPool = parentColumnReader.parentReader.getOperatorContext().getScanExecutor();
+      }
+      asyncPageRead = threadPool.submit(new AsyncPageReaderTask());
+      readDictionaryPage(asyncPageRead, parentStatus);
+      asyncPageRead = null; // reset after consuming
+    }
+  }
+
+  private DrillBuf getDecompressedPageData(ReadStatus readStatus) {
+    DrillBuf data;
+    boolean isDictionary = false;
+    synchronized (this) {
+      data = readStatus.getPageData();
+      readStatus.setPageData(null);
+      isDictionary = readStatus.isDictionaryPage;
+    }
+    if (parentColumnReader.columnChunkMetaData.getCodec() != CompressionCodecName.UNCOMPRESSED) {
+      DrillBuf uncompressedData = data;
+      data = decompress(readStatus.getPageHeader(), uncompressedData);
+      synchronized (this) {
+        readStatus.setPageData(null);
+      }
+      uncompressedData.release();
+    } else {
+      if (isDictionary) {
+        stats.totalDictPageReadBytes.addAndGet(readStatus.bytesRead);
+      } else {
+        stats.totalDataPageReadBytes.addAndGet(readStatus.bytesRead);
+      }
+    }
+    return data;
+  }
+
+  // Read and decode the dictionary and the header
+  private void readDictionaryPage(final Future<ReadStatus> asyncPageRead,
+      final ColumnReader<?> parentStatus) throws UserException {
+    try {
+      Stopwatch timer = Stopwatch.createStarted();
+      ReadStatus readStatus = asyncPageRead.get();
+      long timeBlocked = timer.elapsed(TimeUnit.NANOSECONDS);
+      stats.timeDiskScanWait.addAndGet(timeBlocked);
+      stats.timeDiskScan.addAndGet(readStatus.getDiskScanTime());
+      stats.numDictPageLoads.incrementAndGet();
+      stats.timeDictPageLoads.addAndGet(timeBlocked+readStatus.getDiskScanTime());
+      readDictionaryPageData(readStatus, parentStatus);
+    } catch (Exception e) {
+      handleAndThrowException(e, "Error reading dictionary page.");
+    }
+  }
+
+  // Read and decode the dictionary data
+  private void readDictionaryPageData(final ReadStatus readStatus, final ColumnReader<?> parentStatus)
+      throws UserException {
+    try {
+      pageHeader = readStatus.getPageHeader();
+      int uncompressedSize = pageHeader.getUncompressed_page_size();
+      final DrillBuf dictionaryData = getDecompressedPageData(readStatus);
+      Stopwatch timer = Stopwatch.createStarted();
+      allocatedDictionaryBuffers.add(dictionaryData);
+      DictionaryPage page = new DictionaryPage(asBytesInput(dictionaryData, 0, uncompressedSize),
+          pageHeader.uncompressed_page_size, pageHeader.dictionary_page_header.num_values,
+          valueOf(pageHeader.dictionary_page_header.encoding.name()));
+      this.dictionary = page.getEncoding().initDictionary(parentStatus.columnDescriptor, page);
+      long timeToDecode = timer.elapsed(TimeUnit.NANOSECONDS);
+      stats.timeDictPageDecode.addAndGet(timeToDecode);
+    } catch (Exception e) {
+      handleAndThrowException(e, "Error decoding dictionary page.");
+    }
+  }
+
+  private void handleAndThrowException(Exception e, String msg) throws UserException {
+    UserException ex = UserException.dataReadError(e).message(msg)
+        .pushContext("Row Group Start: ", this.parentColumnReader.columnChunkMetaData.getStartingPos())
+        .pushContext("Column: ", this.parentColumnReader.schemaElement.getName())
+        .pushContext("File: ", this.fileName).build(logger);
+    throw ex;
+  }
+
+  private DrillBuf decompress(PageHeader pageHeader, DrillBuf compressedData) {
+    DrillBuf pageDataBuf = null;
+    Stopwatch timer = Stopwatch.createUnstarted();
+    long timeToRead;
+    int compressedSize = pageHeader.getCompressed_page_size();
+    int uncompressedSize = pageHeader.getUncompressed_page_size();
+    pageDataBuf = allocateTemporaryBuffer(uncompressedSize);
+    try {
+      timer.start();
+      codecFactory.getDecompressor(parentColumnReader.columnChunkMetaData.getCodec())
+          .decompress(compressedData.nioBuffer(0, compressedSize), compressedSize,
+              pageDataBuf.nioBuffer(0, uncompressedSize), uncompressedSize);
+      timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
+      this.updateStats(pageHeader, "Decompress", 0, timeToRead, compressedSize, uncompressedSize);
+    } catch (IOException e) {
+      handleAndThrowException(e, "Error decompressing data.");
+    }
+    return pageDataBuf;
+  }
+
+  @Override protected void nextInternal() throws IOException {
+    ReadStatus readStatus = null;
+    try {
+      Stopwatch timer = Stopwatch.createStarted();
+      readStatus = asyncPageRead.get();
+      long timeBlocked = timer.elapsed(TimeUnit.NANOSECONDS);
+      stats.timeDiskScanWait.addAndGet(timeBlocked);
+      stats.timeDiskScan.addAndGet(readStatus.getDiskScanTime());
+      if (readStatus.isDictionaryPage) {
+        stats.numDictPageLoads.incrementAndGet();
+        stats.timeDictPageLoads.addAndGet(timeBlocked + readStatus.getDiskScanTime());
+      } else {
+        stats.numDataPageLoads.incrementAndGet();
+        stats.timeDataPageLoads.addAndGet(timeBlocked + readStatus.getDiskScanTime());
+      }
+      pageHeader = readStatus.getPageHeader();
+      // reset this. At the time of calling close, if this is not null then a pending asyncPageRead needs to be consumed
+      asyncPageRead = null;
+    } catch (Exception e) {
+      handleAndThrowException(e, "Error reading page data.");
+    }
+
+    // TODO - figure out if we need multiple dictionary pages, I believe it may be limited to one
+    // I think we are clobbering parts of the dictionary if there can be multiple pages of dictionary
+
+    do {
+      if (pageHeader.getType() == PageType.DICTIONARY_PAGE) {
+        readDictionaryPageData(readStatus, parentColumnReader);
+        // Ugly. Use the Async task to make a synchronous read call.
+        readStatus = new AsyncPageReaderTask().call();
+        pageHeader = readStatus.getPageHeader();
+      }
+    } while (pageHeader.getType() == PageType.DICTIONARY_PAGE);
+
+    if (dataReader.hasRemainder() && parentColumnReader.totalValuesRead + readStatus.getValuesRead()
+        < parentColumnReader.columnChunkMetaData.getValueCount()) {
+      asyncPageRead = threadPool.submit(new AsyncPageReaderTask());
+    }
+
+    pageHeader = readStatus.getPageHeader();
+    pageData = getDecompressedPageData(readStatus);
+
+  }
+
+
+  @Override public void clear() {
+    if (asyncPageRead != null) {
+      asyncPageRead.cancel(true);
+      try {
+        ReadStatus r = asyncPageRead.get();
+        r.getPageData().release();
+      } catch (Exception e) {
+        // Do nothing.
+      }
+    }
+    super.clear();
+  }
+
+  public static class ReadStatus {
+    private PageHeader pageHeader;
+    private DrillBuf pageData;
+    private boolean isDictionaryPage = false;
+    private long bytesRead = 0;
+    private long valuesRead = 0;
+    private long diskScanTime = 0;
+
+    public synchronized PageHeader getPageHeader() {
+      return pageHeader;
+    }
+
+    public synchronized void setPageHeader(PageHeader pageHeader) {
+      this.pageHeader = pageHeader;
+    }
+
+    public synchronized DrillBuf getPageData() {
+      return pageData;
+    }
+
+    public synchronized void setPageData(DrillBuf pageData) {
+      this.pageData = pageData;
+    }
+
+    public synchronized boolean isDictionaryPage() {
+      return isDictionaryPage;
+    }
+
+    public synchronized void setIsDictionaryPage(boolean isDictionaryPage) {
+      this.isDictionaryPage = isDictionaryPage;
+    }
+
+    public synchronized long getBytesRead() {
+      return bytesRead;
+    }
+
+    public synchronized void setBytesRead(long bytesRead) {
+      this.bytesRead = bytesRead;
+    }
+
+    public synchronized long getValuesRead() {
+      return valuesRead;
+    }
+
+    public synchronized void setValuesRead(long valuesRead) {
+      this.valuesRead = valuesRead;
+    }
+
+    public long getDiskScanTime() {
+      return diskScanTime;
+    }
+
+    public void setDiskScanTime(long diskScanTime) {
+      this.diskScanTime = diskScanTime;
+    }
+  }
+
+
+  private class AsyncPageReaderTask implements Callable<ReadStatus> {
+
+    private final AsyncPageReader parent = AsyncPageReader.this;
+
+    public AsyncPageReaderTask() {
+    }
+
+    @Override public ReadStatus call() throws IOException {
+      ReadStatus readStatus = new ReadStatus();
+
+      String oldname = Thread.currentThread().getName();
+      Thread.currentThread().setName(parent.parentColumnReader.columnChunkMetaData.toString());
+
+      long bytesRead = 0;
+      long valuesRead = 0;
+      Stopwatch timer = Stopwatch.createStarted();
+
+      DrillBuf pageData = null;
+      try {
+        PageHeader pageHeader = Util.readPageHeader(parent.dataReader);
+        int compressedSize = pageHeader.getCompressed_page_size();
+        pageData = parent.dataReader.getNext(compressedSize);
+        bytesRead = compressedSize;
+        synchronized (parent) {
+          if (pageHeader.getType() == PageType.DICTIONARY_PAGE) {
+            readStatus.setIsDictionaryPage(true);
+            valuesRead += pageHeader.getDictionary_page_header().getNum_values();
+          } else {
+            valuesRead += pageHeader.getData_page_header().getNum_values();
+          }
+          long timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
+          readStatus.setPageHeader(pageHeader);
+          readStatus.setPageData(pageData);
+          readStatus.setBytesRead(bytesRead);
+          readStatus.setValuesRead(valuesRead);
+          readStatus.setDiskScanTime(timeToRead);
+        }
+
+      } catch (Exception e) {
+        if (pageData != null) {
+          pageData.release();
+        }
+        throw e;
+      }
+      Thread.currentThread().setName(oldname);
+      return readStatus;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
index f62f424..6572c78 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
@@ -22,6 +22,8 @@ import io.netty.buffer.DrillBuf;
 import java.io.IOException;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.ValueVector;
 
@@ -70,7 +72,7 @@ public abstract class ColumnReader<V extends ValueVector> {
   protected DrillBuf vectorData;
   // when reading definition levels for nullable columns, it is a one-way stream of integers
   // when reading var length data, where we don't know if all of the records will fit until we've read all of them
-  // we must store the last definition level an use it in at the start of the next batch
+  // we must store the last definition level and use it at the start of the next batch
   int currDefLevel;
 
   // variables for a single read pass
@@ -84,7 +86,17 @@ public abstract class ColumnReader<V extends ValueVector> {
     this.isFixedLength = fixedLength;
     this.schemaElement = schemaElement;
     this.valueVec =  v;
-    this.pageReader = new PageReader(this, parentReader.getFileSystem(), parentReader.getHadoopPath(), columnChunkMetaData);
+    boolean useAsyncPageReader  = parentReader.getFragmentContext().getOptions()
+        .getOption(ExecConstants.PARQUET_PAGEREADER_ASYNC).bool_val;
+    if (useAsyncPageReader) {
+      this.pageReader =
+          new AsyncPageReader(this, parentReader.getFileSystem(), parentReader.getHadoopPath(),
+              columnChunkMetaData);
+    } else {
+      this.pageReader =
+          new PageReader(this, parentReader.getFileSystem(), parentReader.getHadoopPath(),
+              columnChunkMetaData);
+    }
 
     if (columnDescriptor.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
       if (columnDescriptor.getType() == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
@@ -117,11 +129,23 @@ public abstract class ColumnReader<V extends ValueVector> {
   }
 
   public void readValues(long recordsToRead) {
-    readField(recordsToRead);
+    try {
+      readField(recordsToRead);
+
+      valuesReadInCurrentPass += recordsReadInThisIteration;
+      pageReader.valuesRead += recordsReadInThisIteration;
+      pageReader.readPosInBytes = readStartInBytes + readLength;
+    } catch (Exception e) {
+      UserException ex = UserException.dataReadError(e)
+          .message("Error reading from Parquet file")
+          .pushContext("Row Group Start: ", this.columnChunkMetaData.getStartingPos())
+          .pushContext("Column: ", this.schemaElement.getName())
+          .pushContext("File: ", this.parentReader.getHadoopPath().toString() )
+          .build(logger);
+      throw ex;
+
+    }
 
-    valuesReadInCurrentPass += recordsReadInThisIteration;
-    pageReader.valuesRead += recordsReadInThisIteration;
-    pageReader.readPosInBytes = readStartInBytes + readLength;
   }
 
   protected abstract void readField(long recordsToRead);

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
index 078e4ce..c34ebd1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/PageReader.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.store.parquet.columnreaders;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.util.filereader.BufferedDirectBufInputStream;
@@ -36,7 +35,6 @@ import org.apache.parquet.column.Dictionary;
 import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.ValuesType;
 import org.apache.parquet.column.page.DictionaryPage;
-import org.apache.parquet.column.statistics.Statistics;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.dictionary.DictionaryValuesReader;
 import org.apache.parquet.format.PageHeader;
@@ -58,25 +56,23 @@ import static org.apache.parquet.column.Encoding.valueOf;
 import static org.apache.parquet.format.converter.ParquetMetadataConverter.fromParquetStatistics;
 
 // class to keep track of the read position of variable length columns
-final class PageReader {
+class PageReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(
       org.apache.drill.exec.store.parquet.columnreaders.PageReader.class);
 
   public static final ParquetMetadataConverter METADATA_CONVERTER = ParquetFormatPlugin.parquetMetadataConverter;
 
-  private final org.apache.drill.exec.store.parquet.columnreaders.ColumnReader<?> parentColumnReader;
+  protected final org.apache.drill.exec.store.parquet.columnreaders.ColumnReader<?> parentColumnReader;
   //private final ColumnDataReader dataReader;
-  private final DirectBufInputStream dataReader;
+  protected final DirectBufInputStream dataReader;
   //der; buffer to store bytes of current page
-  DrillBuf pageData;
+  protected DrillBuf pageData;
 
   // for variable length data we need to keep track of our current position in the page data
   // as the values and lengths are intermixed, making random access to the length data impossible
   long readyToReadPosInBytes;
   // read position in the current page, stored in the ByteBuf in ParquetRecordReader called bufferWithAllData
   long readPosInBytes;
-  // bit shift needed for the next page if the last one did not line up with a byte boundary
-  int bitShift;
   // storage space for extra bits at the end of a page if they did not line up with a byte boundary
   // prevents the need to keep the entire last page, as these pageDataByteArray need to be added to the next batch
   //byte extraBits;
@@ -103,14 +99,18 @@ final class PageReader {
 
   int currentPageCount = -1;
 
-  private FSDataInputStream inputStream;
+  protected FSDataInputStream inputStream;
 
   // These need to be held throughout reading of the entire column chunk
   List<ByteBuf> allocatedDictionaryBuffers;
 
-  private final CodecFactory codecFactory;
+  protected final CodecFactory codecFactory;
+  protected final String fileName;
 
-  private final ParquetReaderStats stats;
+  protected final ParquetReaderStats stats;
+  private final boolean useBufferedReader;
+  private final int scanBufferSize;
+  private final boolean useFadvise;
 
   PageReader(org.apache.drill.exec.store.parquet.columnreaders.ColumnReader<?> parentStatus, FileSystem fs, Path path, ColumnChunkMetaData columnChunkMetaData)
     throws ExecutionSetupException {
@@ -118,21 +118,24 @@ final class PageReader {
     allocatedDictionaryBuffers = new ArrayList<ByteBuf>();
     codecFactory = parentColumnReader.parentReader.getCodecFactory();
     this.stats = parentColumnReader.parentReader.parquetReaderStats;
-    long start = columnChunkMetaData.getFirstDataPageOffset();
+    this.fileName = path.toString();
     try {
       inputStream  = fs.open(path);
       BufferAllocator allocator =  parentColumnReader.parentReader.getOperatorContext().getAllocator();
-      //TODO: make read batch size configurable
       columnChunkMetaData.getTotalUncompressedSize();
-      boolean useBufferedReader  = parentColumnReader.parentReader.getFragmentContext().getOptions()
+      useBufferedReader  = parentColumnReader.parentReader.getFragmentContext().getOptions()
           .getOption(ExecConstants.PARQUET_PAGEREADER_USE_BUFFERED_READ).bool_val;
+      scanBufferSize = parentColumnReader.parentReader.getFragmentContext().getOptions()
+          .getOption(ExecConstants.PARQUET_PAGEREADER_BUFFER_SIZE).num_val.intValue();
+      useFadvise = parentColumnReader.parentReader.getFragmentContext().getOptions()
+          .getOption(ExecConstants.PARQUET_PAGEREADER_USE_FADVISE).bool_val;
       if (useBufferedReader) {
         this.dataReader = new BufferedDirectBufInputStream(inputStream, allocator, path.getName(),
-            columnChunkMetaData.getStartingPos(), columnChunkMetaData.getTotalSize(), 8 * 1024 * 1024,
-            true);
+            columnChunkMetaData.getStartingPos(), columnChunkMetaData.getTotalSize(), scanBufferSize,
+            useFadvise);
       } else {
         this.dataReader = new DirectBufInputStream(inputStream, allocator, path.getName(),
-            columnChunkMetaData.getStartingPos(), columnChunkMetaData.getTotalSize(), true);
+            columnChunkMetaData.getStartingPos(), columnChunkMetaData.getTotalSize(), useFadvise);
       }
       dataReader.init();
 
@@ -145,7 +148,7 @@ final class PageReader {
 
   }
 
-  private void loadDictionaryIfExists(final org.apache.drill.exec.store.parquet.columnreaders.ColumnReader<?> parentStatus,
+  protected void loadDictionaryIfExists(final org.apache.drill.exec.store.parquet.columnreaders.ColumnReader<?> parentStatus,
       final ColumnChunkMetaData columnChunkMetaData, final DirectBufInputStream f) throws IOException {
     Stopwatch timer = Stopwatch.createUnstarted();
     if (columnChunkMetaData.getDictionaryPageOffset() > 0) {
@@ -153,7 +156,7 @@ final class PageReader {
       long start=dataReader.getPos();
       timer.start();
       final PageHeader pageHeader = Util.readPageHeader(f);
-      long timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
+      long timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
       long pageHeaderBytes=dataReader.getPos()-start;
       this.updateStats(pageHeader, "Page Header", start, timeToRead, pageHeaderBytes, pageHeaderBytes);
       assert pageHeader.type == PageType.DICTIONARY_PAGE;
@@ -178,7 +181,7 @@ final class PageReader {
     this.dictionary = page.getEncoding().initDictionary(parentStatus.columnDescriptor, page);
   }
 
-  public DrillBuf readPage(PageHeader pageHeader, int compressedSize, int uncompressedSize) throws IOException {
+  private DrillBuf readPage(PageHeader pageHeader, int compressedSize, int uncompressedSize) throws IOException {
     DrillBuf pageDataBuf = null;
     Stopwatch timer = Stopwatch.createUnstarted();
     long timeToRead;
@@ -186,7 +189,7 @@ final class PageReader {
     if (parentColumnReader.columnChunkMetaData.getCodec() == CompressionCodecName.UNCOMPRESSED) {
       timer.start();
       pageDataBuf = dataReader.getNext(compressedSize);
-      timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
+      timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
       this.updateStats(pageHeader, "Page Read", start, timeToRead, compressedSize, uncompressedSize);
     } else {
       DrillBuf compressedData = null;
@@ -195,8 +198,7 @@ final class PageReader {
       try {
       timer.start();
       compressedData = dataReader.getNext(compressedSize);
-       // dataReader.loadPage(compressedData, compressedSize);
-      timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
+      timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
       timer.reset();
       this.updateStats(pageHeader, "Page Read", start, timeToRead, compressedSize, compressedSize);
       start=dataReader.getPos();
@@ -204,7 +206,7 @@ final class PageReader {
       codecFactory.getDecompressor(parentColumnReader.columnChunkMetaData
           .getCodec()).decompress(compressedData.nioBuffer(0, compressedSize), compressedSize,
           pageDataBuf.nioBuffer(0, uncompressedSize), uncompressedSize);
-        timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
+        timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
         this.updateStats(pageHeader, "Decompress", start, timeToRead, compressedSize, uncompressedSize);
       } finally {
         if(compressedData != null) {
@@ -219,25 +221,12 @@ final class PageReader {
     return BytesInput.from(buf.nioBuffer(offset, length), 0, length);
   }
 
+
   /**
-   * Grab the next page.
-   *
-   * @return - if another page was present
-   * @throws IOException
+   * Get the page header and the pageData (uncompressed) for the next page
    */
-  public boolean next() throws IOException {
+  protected void nextInternal() throws IOException{
     Stopwatch timer = Stopwatch.createUnstarted();
-    currentPageCount = -1;
-    valuesRead = 0;
-    valuesReadyToRead = 0;
-
-    // TODO - the metatdata for total size appears to be incorrect for impala generated files, need to find cause
-    // and submit a bug report
-    if(!dataReader.hasRemainder() || parentColumnReader.totalValuesRead == parentColumnReader.columnChunkMetaData.getValueCount()) {
-      return false;
-    }
-    clearBuffers();
-
     // next, we need to decompress the bytes
     // TODO - figure out if we need multiple dictionary pages, I believe it may be limited to one
     // I think we are clobbering parts of the dictionary if there can be multiple pages of dictionary
@@ -245,7 +234,7 @@ final class PageReader {
       long start=dataReader.getPos();
       timer.start();
       pageHeader = Util.readPageHeader(dataReader);
-      long timeToRead = timer.elapsed(TimeUnit.MICROSECONDS);
+      long timeToRead = timer.elapsed(TimeUnit.NANOSECONDS);
       long pageHeaderBytes=dataReader.getPos()-start;
       this.updateStats(pageHeader, "Page Header", start, timeToRead, pageHeaderBytes, pageHeaderBytes);
       logger.trace("ParquetTrace,{},{},{},{},{},{},{},{}","Page Header Read","",
@@ -264,14 +253,33 @@ final class PageReader {
     int uncompressedSize = pageHeader.getUncompressed_page_size();
     pageData = readPage(pageHeader, compressedSize, uncompressedSize);
 
-    currentPageCount = pageHeader.data_page_header.num_values;
-    final int uncompressedPageSize = pageHeader.uncompressed_page_size;
-    final Statistics<?> stats = fromParquetStatistics(pageHeader.data_page_header.getStatistics(), parentColumnReader
-        .getColumnDescriptor().getType());
+  }
+
+  /**
+   * Grab the next page.
+   *
+   * @return - if another page was present
+   * @throws IOException
+   */
+  public boolean next() throws IOException {
+    Stopwatch timer = Stopwatch.createUnstarted();
+    currentPageCount = -1;
+    valuesRead = 0;
+    valuesReadyToRead = 0;
 
+    // TODO - the metatdata for total size appears to be incorrect for impala generated files, need to find cause
+    // and submit a bug report
+    if(!dataReader.hasRemainder() || parentColumnReader.totalValuesRead == parentColumnReader.columnChunkMetaData.getValueCount()) {
+      return false;
+    }
+    clearBuffers();
 
-    final Encoding rlEncoding = METADATA_CONVERTER.getEncoding(pageHeader.data_page_header.repetition_level_encoding);
+    nextInternal();
 
+    timer.start();
+    currentPageCount = pageHeader.data_page_header.num_values;
+
+    final Encoding rlEncoding = METADATA_CONVERTER.getEncoding(pageHeader.data_page_header.repetition_level_encoding);
     final Encoding dlEncoding = METADATA_CONVERTER.getEncoding(pageHeader.data_page_header.definition_level_encoding);
     final Encoding valueEncoding = METADATA_CONVERTER.getEncoding(pageHeader.data_page_header.encoding);
 
@@ -321,40 +329,24 @@ final class PageReader {
     // fit one record at a time, such as for variable length data. Both operations must start in the same location after the
     // definition and repetition level data which is stored alongside the page data itself
     readyToReadPosInBytes = readPosInBytes;
+    long timeDecode = timer.elapsed(TimeUnit.NANOSECONDS);
+    stats.numDataPagesDecoded.incrementAndGet();
+    stats.timeDataPageDecode.addAndGet(timeDecode);
     return true;
   }
 
   /**
-   * Allocate a page data buffer. Note that only one page data buffer should be active at a time. The reader will ensure
-   * that the page data is released after the reader is completed.
-   */
-  private void allocatePageData(int size) {
-    Preconditions.checkArgument(pageData == null);
-    pageData = parentColumnReader.parentReader.getOperatorContext().getAllocator().buffer(size);
-  }
-
-  /**
    * Allocate a buffer which the user should release immediately. The reader does not manage release of these buffers.
    */
-  private DrillBuf allocateTemporaryBuffer(int size) {
+  protected DrillBuf allocateTemporaryBuffer(int size) {
     return parentColumnReader.parentReader.getOperatorContext().getAllocator().buffer(size);
   }
 
-  /**
-   * Allocate and return a dictionary buffer. These are maintained for the life of the reader and then released when the
-   * reader is cleared.
-   */
-  private DrillBuf allocateDictionaryBuffer(int size) {
-    DrillBuf buf = parentColumnReader.parentReader.getOperatorContext().getAllocator().buffer(size);
-    allocatedDictionaryBuffers.add(buf);
-    return buf;
-  }
-
   protected boolean hasPage() {
     return currentPageCount != -1;
   }
 
-  private void updateStats(PageHeader pageHeader, String op, long start, long time, long bytesin, long bytesout) {
+  protected void updateStats(PageHeader pageHeader, String op, long start, long time, long bytesin, long bytesout) {
     String pageType = "Data Page";
     if (pageHeader.type == PageType.DICTIONARY_PAGE) {
       pageType = "Dictionary Page";
@@ -362,37 +354,38 @@ final class PageReader {
     logger.trace("ParquetTrace,{},{},{},{},{},{},{},{}", op, pageType.toString(),
         this.parentColumnReader.parentReader.hadoopPath,
         this.parentColumnReader.columnDescriptor.toString(), start, bytesin, bytesout, time);
+
     if (pageHeader.type != PageType.DICTIONARY_PAGE) {
       if (bytesin == bytesout) {
-        this.stats.timePageLoads += time;
-        this.stats.numPageLoads++;
-        this.stats.totalPageReadBytes += bytesin;
+        this.stats.timeDataPageLoads.addAndGet(time);
+        this.stats.numDataPageLoads.incrementAndGet();
+        this.stats.totalDataPageReadBytes.addAndGet(bytesin);
       } else {
-        this.stats.timePagesDecompressed += time;
-        this.stats.numPagesDecompressed++;
-        this.stats.totalDecompressedBytes += bytesin;
+        this.stats.timeDataPagesDecompressed.addAndGet(time);
+        this.stats.numDataPagesDecompressed.incrementAndGet();
+        this.stats.totalDataDecompressedBytes.addAndGet(bytesin);
       }
     } else {
       if (bytesin == bytesout) {
-        this.stats.timeDictPageLoads += time;
-        this.stats.numDictPageLoads++;
-        this.stats.totalDictPageReadBytes += bytesin;
+        this.stats.timeDictPageLoads.addAndGet(time);
+        this.stats.numDictPageLoads.incrementAndGet();
+        this.stats.totalDictPageReadBytes.addAndGet(bytesin);
       } else {
-        this.stats.timeDictPagesDecompressed += time;
-        this.stats.numDictPagesDecompressed++;
-        this.stats.totalDictDecompressedBytes += bytesin;
+        this.stats.timeDictPagesDecompressed.addAndGet(time);
+        this.stats.numDictPagesDecompressed.incrementAndGet();
+        this.stats.totalDictDecompressedBytes.addAndGet(bytesin);
       }
     }
   }
 
-  public void clearBuffers() {
+  protected void clearBuffers() {
     if (pageData != null) {
       pageData.release();
       pageData = null;
     }
   }
 
-  public void clearDictionaryBuffers() {
+  protected void clearDictionaryBuffers() {
     for (ByteBuf b : allocatedDictionaryBuffers) {
       b.release();
     }
@@ -401,15 +394,14 @@ final class PageReader {
 
   public void clear(){
     try {
+      this.inputStream.close();
       this.dataReader.close();
     } catch (IOException e) {
-      //TODO: Throw UserException
+      //Swallow the exception which is OK for input streams
     }
     // Free all memory, including fixed length types. (Data is being copied for all types not just var length types)
-    //if(!this.parentColumnReader.isFixedLength) {
     clearBuffers();
     clearDictionaryBuffers();
-    //}
   }
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
index 924887e..1eca00f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
@@ -35,6 +35,7 @@ import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.MetricDef;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.MaterializedField;
@@ -117,16 +118,39 @@ public class ParquetRecordReader extends AbstractRecordReader {
 
   public ParquetReaderStats parquetReaderStats = new ParquetReaderStats();
 
+  public enum Metric implements MetricDef {
+    NUM_DICT_PAGE_LOADS,         // Number of dictionary pages read
+    NUM_DATA_PAGE_lOADS,         // Number of data pages read
+    NUM_DATA_PAGES_DECODED,      // Number of data pages decoded
+    NUM_DICT_PAGES_DECOMPRESSED, // Number of dictionary pages decompressed
+    NUM_DATA_PAGES_DECOMPRESSED, // Number of data pages decompressed
+    TOTAL_DICT_PAGE_READ_BYTES,  // Total bytes read from disk for dictionary pages
+    TOTAL_DATA_PAGE_READ_BYTES,  // Total bytes read from disk for data pages
+    TOTAL_DICT_DECOMPRESSED_BYTES, // Total bytes decompressed for dictionary pages (same as compressed bytes on disk)
+    TOTAL_DATA_DECOMPRESSED_BYTES, // Total bytes decompressed for data pages (same as compressed bytes on disk)
+    TIME_DICT_PAGE_LOADS,          // Time in nanos in reading dictionary pages from disk
+    TIME_DATA_PAGE_LOADS,          // Time in nanos in reading data pages from disk
+    TIME_DATA_PAGE_DECODE,         // Time in nanos in decoding data pages
+    TIME_DICT_PAGE_DECODE,         // Time in nanos in decoding dictionary pages
+    TIME_DICT_PAGES_DECOMPRESSED,  // Time in nanos in decompressing dictionary pages
+    TIME_DATA_PAGES_DECOMPRESSED,  // Time in nanos in decompressing data pages
+    TIME_DISK_SCAN_WAIT,           // Time in nanos spent in waiting for an async disk read to complete
+    TIME_DISK_SCAN;                // Time in nanos spent in reading data from disk.
+
+    @Override public int metricId() {
+      return ordinal();
+    }
+  }
+
   public ParquetRecordReader(FragmentContext fragmentContext,
       String path,
       int rowGroupIndex,
-                             long numRecordsToRead,
+      long numRecordsToRead,
       FileSystem fs,
       CodecFactory codecFactory,
       ParquetMetadata footer,
-                             List<SchemaPath> columns,
-                             ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus)
-                             throws ExecutionSetupException {
+      List<SchemaPath> columns,
+      ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus) throws ExecutionSetupException {
     this(fragmentContext, DEFAULT_BATCH_LENGTH_IN_BITS, numRecordsToRead,
          path, rowGroupIndex, fs, codecFactory, footer, columns, dateCorruptionStatus);
   }
@@ -470,6 +494,7 @@ public class ParquetRecordReader extends AbstractRecordReader {
       // No columns found in the file were selected, simply return a full batch of null records for each column requested
       if (firstColumnStatus == null) {
         if (mockRecordsRead == footer.getBlocks().get(rowGroupIndex).getRowCount()) {
+          updateStats();
           return 0;
         }
         recordsToRead = Math.min(DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH, footer.getBlocks().get(rowGroupIndex).getRowCount() - mockRecordsRead);
@@ -483,6 +508,7 @@ public class ParquetRecordReader extends AbstractRecordReader {
         mockRecordsRead += recordsToRead;
         totalRecordsRead += recordsToRead;
         numRecordsToRead -= recordsToRead;
+        updateStats();
         return (int) recordsToRead;
       }
 
@@ -514,6 +540,7 @@ public class ParquetRecordReader extends AbstractRecordReader {
 //      logger.debug("So far read {} records out of row group({}) in file '{}'", totalRecordsRead, rowGroupIndex, hadoopPath.toUri().getPath());
       totalRecordsRead += firstColumnStatus.getRecordsReadInCurrentPass();
       numRecordsToRead -= firstColumnStatus.getRecordsReadInCurrentPass();
+      updateStats();
       return firstColumnStatus.getRecordsReadInCurrentPass();
     } catch (Exception e) {
       handleAndRaise("\nHadoop path: " + hadoopPath.toUri().getPath() +
@@ -530,7 +557,8 @@ public class ParquetRecordReader extends AbstractRecordReader {
 
   @Override
   public void close() {
-    logger.debug("Read {} records out of row group({}) in file '{}'", totalRecordsRead, rowGroupIndex, hadoopPath.toUri().getPath());
+    logger.debug("Read {} records out of row group({}) in file '{}'", totalRecordsRead, rowGroupIndex,
+        hadoopPath.toUri().getPath());
     // enable this for debugging when it is know that a whole file will be read
     // limit kills upstream operators once it has enough records, so this assert will fail
 //    assert totalRecordsRead == footer.getBlocks().get(rowGroupIndex).getRowCount();
@@ -552,29 +580,67 @@ public class ParquetRecordReader extends AbstractRecordReader {
       varLengthReader = null;
     }
 
+
     if(parquetReaderStats != null) {
-      logger.trace("ParquetTrace,Summary,{},{},{},{},{},{},{},{},{},{},{},{},{},{},{},{},{},{},{}",
+      logger.trace("ParquetTrace,Summary,{},{},{},{},{},{},{},{},{},{},{},{},{},{},{},{},{},{}",
           hadoopPath,
-          parquetReaderStats.numDictPageHeaders,
-          parquetReaderStats.numPageHeaders,
           parquetReaderStats.numDictPageLoads,
-          parquetReaderStats.numPageLoads,
+          parquetReaderStats.numDataPageLoads,
+          parquetReaderStats.numDataPagesDecoded,
           parquetReaderStats.numDictPagesDecompressed,
-          parquetReaderStats.numPagesDecompressed,
-          parquetReaderStats.totalDictPageHeaderBytes,
-          parquetReaderStats.totalPageHeaderBytes,
+          parquetReaderStats.numDataPagesDecompressed,
           parquetReaderStats.totalDictPageReadBytes,
-          parquetReaderStats.totalPageReadBytes,
+          parquetReaderStats.totalDataPageReadBytes,
           parquetReaderStats.totalDictDecompressedBytes,
-          parquetReaderStats.totalDecompressedBytes,
-          parquetReaderStats.timeDictPageHeaders,
-          parquetReaderStats.timePageHeaders,
+          parquetReaderStats.totalDataDecompressedBytes,
           parquetReaderStats.timeDictPageLoads,
-          parquetReaderStats.timePageLoads,
+          parquetReaderStats.timeDataPageLoads,
+          parquetReaderStats.timeDataPageDecode,
+          parquetReaderStats.timeDictPageDecode,
           parquetReaderStats.timeDictPagesDecompressed,
-          parquetReaderStats.timePagesDecompressed);
+          parquetReaderStats.timeDataPagesDecompressed,
+          parquetReaderStats.timeDiskScanWait,
+          parquetReaderStats.timeDiskScan
+      );
       parquetReaderStats=null;
     }
+
+  }
+
+  private void updateStats(){
+
+    operatorContext.getStats().setLongStat(Metric.NUM_DICT_PAGE_LOADS,
+        parquetReaderStats.numDictPageLoads.longValue());
+    operatorContext.getStats().setLongStat(Metric.NUM_DATA_PAGE_lOADS, parquetReaderStats.numDataPageLoads.longValue());
+    operatorContext.getStats().setLongStat(Metric.NUM_DATA_PAGES_DECODED, parquetReaderStats.numDataPagesDecoded.longValue());
+    operatorContext.getStats().setLongStat(Metric.NUM_DICT_PAGES_DECOMPRESSED,
+        parquetReaderStats.numDictPagesDecompressed.longValue());
+    operatorContext.getStats().setLongStat(Metric.NUM_DATA_PAGES_DECOMPRESSED,
+        parquetReaderStats.numDataPagesDecompressed.longValue());
+    operatorContext.getStats().setLongStat(Metric.TOTAL_DICT_PAGE_READ_BYTES,
+        parquetReaderStats.totalDictPageReadBytes.longValue());
+    operatorContext.getStats().setLongStat(Metric.TOTAL_DATA_PAGE_READ_BYTES,
+        parquetReaderStats.totalDataPageReadBytes.longValue());
+    operatorContext.getStats().setLongStat(Metric.TOTAL_DICT_DECOMPRESSED_BYTES,
+        parquetReaderStats.totalDictDecompressedBytes.longValue());
+    operatorContext.getStats().setLongStat(Metric.TOTAL_DATA_DECOMPRESSED_BYTES,
+        parquetReaderStats.totalDataDecompressedBytes.longValue());
+    operatorContext.getStats().setLongStat(Metric.TIME_DICT_PAGE_LOADS,
+        parquetReaderStats.timeDictPageLoads.longValue());
+    operatorContext.getStats().setLongStat(Metric.TIME_DATA_PAGE_LOADS,
+        parquetReaderStats.timeDataPageLoads.longValue());
+    operatorContext.getStats().setLongStat(Metric.TIME_DATA_PAGE_DECODE,
+        parquetReaderStats.timeDataPageDecode.longValue());
+    operatorContext.getStats().setLongStat(Metric.TIME_DICT_PAGE_DECODE,
+        parquetReaderStats.timeDictPageDecode.longValue());
+    operatorContext.getStats().setLongStat(Metric.TIME_DICT_PAGES_DECOMPRESSED,
+        parquetReaderStats.timeDictPagesDecompressed.longValue());
+    operatorContext.getStats().setLongStat(Metric.TIME_DATA_PAGES_DECOMPRESSED,
+        parquetReaderStats.timeDataPagesDecompressed.longValue());
+    operatorContext.getStats().setLongStat(Metric.TIME_DISK_SCAN_WAIT,
+        parquetReaderStats.timeDiskScanWait.longValue());
+    operatorContext.getStats().setLongStat(Metric.TIME_DISK_SCAN, parquetReaderStats.timeDiskScan.longValue());
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
index 6ca0205..e03d930 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLenBinaryReader.java
@@ -17,15 +17,17 @@
  */
 package org.apache.drill.exec.store.parquet.columnreaders;
 
+import org.apache.drill.exec.vector.ValueVector;
+
 import java.io.IOException;
 import java.util.List;
 
 public class VarLenBinaryReader {
 
   ParquetRecordReader parentReader;
-  final List<VarLengthColumn<?>> columns;
+  final List<VarLengthColumn<? extends ValueVector>> columns;
 
-  public VarLenBinaryReader(ParquetRecordReader parentReader, List<VarLengthColumn<?>> columns) {
+  public VarLenBinaryReader(ParquetRecordReader parentReader, List<VarLengthColumn<? extends ValueVector>> columns) {
     this.parentReader = parentReader;
     this.columns = columns;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
index 6aa968a..a5a6b81 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/filereader/BufferedDirectBufInputStream.java
@@ -23,7 +23,6 @@ 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;
@@ -73,7 +72,7 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
   /**
    * The current read position in the buffer; the index of the next
    * character to be read from the <code>internalBuffer</code> array.
-   * <p>
+   * <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
@@ -128,8 +127,7 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
 
   }
 
-  @Override
-  public void init() throws UnsupportedOperationException, IOException {
+  @Override public void init() throws UnsupportedOperationException, IOException {
     super.init();
     this.internalBuffer = this.allocator.buffer(this.bufSize);
     this.tempBuffer = this.allocator.buffer(defaultTempBufferSize);
@@ -180,10 +178,10 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
         this.curPosInStream = getInputStream().getPos();
         bytesRead = nBytes;
         logger.trace(
-            "Stream: {}, StartOffset: {}, TotalByteSize: {}, BufferSize: {}, BytesRead: {}, Count: {}, " +
-            "CurPosInStream: {}, CurPosInBuffer: {}",
-            this.streamId, this.startOffset, this.totalByteSize, this.bufSize, bytesRead, this.count,
-            this.curPosInStream, this.curPosInBuffer);
+            "Stream: {}, StartOffset: {}, TotalByteSize: {}, BufferSize: {}, BytesRead: {}, Count: {}, "
+                + "CurPosInStream: {}, CurPosInBuffer: {}", this.streamId, this.startOffset,
+            this.totalByteSize, this.bufSize, bytesRead, this.count, this.curPosInStream,
+            this.curPosInBuffer);
       }
     }
     return this.count - this.curPosInBuffer;
@@ -252,8 +250,8 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
   }
 
   /**
-   Has the same contract as {@link java.io.InputStream#read(byte[], int, int)}
-   Except with DrillBuf
+   * Has the same contract as {@link java.io.InputStream#read(byte[], int, int)}
+   * Except with DrillBuf
    */
   public synchronized int read(DrillBuf buf, int off, int len) throws IOException {
     checkInputStreamState();
@@ -296,7 +294,7 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
       return 0;
     }
     DrillBuf byteBuf;
-    if(len <= defaultTempBufferSize){
+    if (len <= defaultTempBufferSize) {
       byteBuf = tempBuffer;
     } else {
       byteBuf = this.allocator.buffer(len);
@@ -318,7 +316,7 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
       }
     } while (bytesRead < len);
 
-    if(len > defaultTempBufferSize){
+    if (len > defaultTempBufferSize) {
       byteBuf.release();
     }
 
@@ -327,12 +325,11 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
 
 
   /**
-   Has the same contract as {@link java.io.InputStream#skip(long)}
+   * Has the same contract as {@link java.io.InputStream#skip(long)}
    * Skips upto the next n bytes.
    * Skip may return with less than n bytes skipped
    */
-  @Override
-  public synchronized long skip(long n) throws IOException {
+  @Override public synchronized long skip(long n) throws IOException {
     checkInputStreamState();
     long bytesAvailable = this.count - this.curPosInBuffer;
     long bytesSkipped = 0;
@@ -353,8 +350,7 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
   }
 
 
-  @Override
-  public synchronized int available() throws IOException {
+  @Override public synchronized int available() throws IOException {
     checkInputStreamState();
     int bytesAvailable = this.count - this.curPosInBuffer;
     int underlyingAvailable = getInputStream().available();
@@ -365,18 +361,15 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
     return available;
   }
 
-  @Override
-  public synchronized void mark(int readlimit) {
+  @Override public synchronized void mark(int readlimit) {
     throw new UnsupportedOperationException("Mark/reset is not supported.");
   }
 
-  @Override
-  public synchronized void reset() throws IOException {
+  @Override public synchronized void reset() throws IOException {
     throw new UnsupportedOperationException("Mark/reset is not supported.");
   }
 
-  @Override
-  public boolean markSupported() {
+  @Override public boolean markSupported() {
     return false;
   }
 
@@ -384,7 +377,7 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
     Returns the current position from the beginning of the underlying input stream
    */
   public long getPos() throws IOException {
-    return curPosInBuffer+startOffset;
+    return curPosInBuffer + startOffset;
   }
 
   public boolean hasRemainder() throws IOException {
@@ -412,6 +405,11 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
     }
   }
 
+  /**
+   * Uncomment For testing Parquet files that are too big to use in unit tests
+   * @param args
+   */
+  /*
   public static void main(String[] args) {
     final DrillConfig config = DrillConfig.create();
     final BufferAllocator allocator = RootAllocatorFactory.newRoot(config);
@@ -433,8 +431,8 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
           long totalByteSize = columnMetadata.getTotalSize();
           String streamId = fileName + ":" + columnMetadata.toString();
           BufferedDirectBufInputStream reader =
-              new BufferedDirectBufInputStream(inputStream, allocator, streamId, startOffset,
-                  totalByteSize, BUFSZ, true);
+              new BufferedDirectBufInputStream(inputStream, allocator, streamId, startOffset, totalByteSize,
+                  BUFSZ, true);
           reader.init();
           while (true) {
             try {
@@ -457,4 +455,5 @@ public class BufferedDirectBufInputStream extends DirectBufInputStream implement
     allocator.close();
     return;
   }
+  */
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 60dcf15..deb31b3 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -194,6 +194,10 @@ drill.exec: {
   },
   debug: {
     return_error_for_failure_in_cancelled_fragments: false
+  },
+  scan: {
+    threadpool_size: 8,
+    decode_threadpool_size: 1
   }
   udf: {
     retry-attempts: 5,

http://git-wip-us.apache.org/repos/asf/drill/blob/f9a443d8/exec/java-exec/src/main/resources/rest/profile/profile.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/profile/profile.ftl b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
index ef971fb..c0f2d8e 100644
--- a/exec/java-exec/src/main/resources/rest/profile/profile.ftl
+++ b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
@@ -221,7 +221,7 @@
             </h4>
           </div>
           <div id="${op.getId()}-metrics" class="panel-collapse collapse">
-            <div class="panel-body">
+            <div class="panel-body" style="display:block;overflow-x:auto">
               ${op.getMetricsTable()}
             </div>
           </div>