You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by ti...@apache.org on 2014/08/18 19:38:25 UTC

git commit: PARQUET-73: Add support for FilterPredicates to cascading schemes

Repository: incubator-parquet-mr
Updated Branches:
  refs/heads/master 08a3c6a7d -> 0d497c454


PARQUET-73: Add support for FilterPredicates to cascading schemes

Author: Alex Levenson <al...@twitter.com>

Closes #34 from isnotinvain/alexlevenson/filter-cascading-scheme and squashes the following commits:

cd69a8e [Alex Levenson] Add support for FilterPredicates to cascading schemes


Project: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/commit/0d497c45
Tree: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/tree/0d497c45
Diff: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/diff/0d497c45

Branch: refs/heads/master
Commit: 0d497c4547934485f2aa9e2e9ead46f26fab7bd2
Parents: 08a3c6a
Author: Alex Levenson <al...@twitter.com>
Authored: Mon Aug 18 10:38:11 2014 -0700
Committer: Tianshuo Deng <td...@twitter.com>
Committed: Mon Aug 18 10:38:11 2014 -0700

----------------------------------------------------------------------
 .../parquet/cascading/ParquetTBaseScheme.java   | 16 ++++-
 .../parquet/cascading/ParquetTupleScheme.java   | 75 +++++++++++++-------
 .../parquet/cascading/ParquetValueScheme.java   | 27 +++++--
 .../main/scala/parquet/filter2/dsl/Dsl.scala    |  2 +
 .../parquet/scrooge/ParquetScroogeScheme.java   | 16 +++--
 5 files changed, 98 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/0d497c45/parquet-cascading/src/main/java/parquet/cascading/ParquetTBaseScheme.java
----------------------------------------------------------------------
diff --git a/parquet-cascading/src/main/java/parquet/cascading/ParquetTBaseScheme.java b/parquet-cascading/src/main/java/parquet/cascading/ParquetTBaseScheme.java
index f966dd6..111c7ab 100644
--- a/parquet-cascading/src/main/java/parquet/cascading/ParquetTBaseScheme.java
+++ b/parquet-cascading/src/main/java/parquet/cascading/ParquetTBaseScheme.java
@@ -20,6 +20,9 @@ import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.thrift.TBase;
 
+import cascading.flow.FlowProcess;
+import cascading.tap.Tap;
+import parquet.filter2.predicate.FilterPredicate;
 import parquet.hadoop.ParquetInputFormat;
 import parquet.hadoop.mapred.DeprecatedParquetInputFormat;
 import parquet.hadoop.mapred.DeprecatedParquetOutputFormat;
@@ -27,8 +30,6 @@ import parquet.hadoop.thrift.ParquetThriftInputFormat;
 import parquet.hadoop.thrift.ThriftReadSupport;
 import parquet.hadoop.thrift.ThriftWriteSupport;
 import parquet.thrift.TBaseRecordConverter;
-import cascading.flow.FlowProcess;
-import cascading.tap.Tap;
 
 public class ParquetTBaseScheme<T extends TBase<?,?>> extends ParquetValueScheme<T> {
 
@@ -42,10 +43,21 @@ public class ParquetTBaseScheme<T extends TBase<?,?>> extends ParquetValueScheme
     this.thriftClass = thriftClass;
   }
 
+  public ParquetTBaseScheme(FilterPredicate filterPredicate) {
+    super(filterPredicate);
+  }
+
+  public ParquetTBaseScheme(FilterPredicate filterPredicate, Class<T> thriftClass) {
+    super(filterPredicate);
+    this.thriftClass = thriftClass;
+  }
+
   @SuppressWarnings("rawtypes")
   @Override
   public void sourceConfInit(FlowProcess<JobConf> fp,
       Tap<JobConf, RecordReader, OutputCollector> tap, JobConf jobConf) {
+
+    super.sourceConfInit(fp, tap, jobConf);
     jobConf.setInputFormat(DeprecatedParquetInputFormat.class);
     ParquetInputFormat.setReadSupportClass(jobConf, ThriftReadSupport.class);
     ThriftReadSupport.setRecordConverterClass(jobConf, TBaseRecordConverter.class);

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/0d497c45/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java
----------------------------------------------------------------------
diff --git a/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java b/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java
index ad913ba..a093332 100644
--- a/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java
+++ b/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java
@@ -15,33 +15,36 @@
  */
  package parquet.cascading;
 
- import java.io.IOException;
- import java.util.List;
- import org.apache.hadoop.mapred.JobConf;
- import org.apache.hadoop.mapred.OutputCollector;
- import org.apache.hadoop.mapred.RecordReader;
-
- import parquet.hadoop.ParquetInputFormat;
- import parquet.hadoop.Footer;
- import parquet.hadoop.mapred.Container;
- import parquet.hadoop.mapred.DeprecatedParquetInputFormat;
- import parquet.schema.MessageType;
-
- import cascading.flow.FlowProcess;
- import cascading.scheme.SinkCall;
- import cascading.scheme.Scheme;
- import cascading.scheme.SourceCall;
- import cascading.tap.Tap;
- import cascading.tap.TapException;
- import cascading.tap.CompositeTap;
- import cascading.tap.hadoop.Hfs;
- import cascading.tuple.Tuple;
- import cascading.tuple.Fields;
- import cascading.tuple.TupleEntry;
- import parquet.hadoop.ParquetOutputFormat;
- import parquet.hadoop.mapred.DeprecatedParquetOutputFormat;
-
- /**
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordReader;
+
+import cascading.flow.FlowProcess;
+import cascading.scheme.Scheme;
+import cascading.scheme.SinkCall;
+import cascading.scheme.SourceCall;
+import cascading.tap.CompositeTap;
+import cascading.tap.Tap;
+import cascading.tap.TapException;
+import cascading.tap.hadoop.Hfs;
+import cascading.tuple.Fields;
+import cascading.tuple.Tuple;
+import cascading.tuple.TupleEntry;
+import parquet.filter2.predicate.FilterPredicate;
+import parquet.hadoop.Footer;
+import parquet.hadoop.ParquetInputFormat;
+import parquet.hadoop.ParquetOutputFormat;
+import parquet.hadoop.mapred.Container;
+import parquet.hadoop.mapred.DeprecatedParquetInputFormat;
+import parquet.hadoop.mapred.DeprecatedParquetOutputFormat;
+import parquet.schema.MessageType;
+
+import static parquet.Preconditions.checkNotNull;
+
+/**
   * A Cascading Scheme that converts Parquet groups into Cascading tuples.
   * If you provide it with sourceFields, it will selectively materialize only the columns for those fields.
   * The names must match the names in the Parquet schema.
@@ -57,13 +60,25 @@ public class ParquetTupleScheme extends Scheme<JobConf, RecordReader, OutputColl
 
   private static final long serialVersionUID = 0L;
   private String parquetSchema;
+  private final FilterPredicate filterPredicate;
 
   public ParquetTupleScheme() {
     super();
+    this.filterPredicate = null;
   }
 
   public ParquetTupleScheme(Fields sourceFields) {
     super(sourceFields);
+    this.filterPredicate = null;
+  }
+
+  public ParquetTupleScheme(FilterPredicate filterPredicate) {
+    this.filterPredicate = checkNotNull(filterPredicate, "filterPredicate");
+  }
+
+  public ParquetTupleScheme(FilterPredicate filterPredicate, Fields sourceFields) {
+    super(sourceFields);
+    this.filterPredicate = checkNotNull(filterPredicate, "filterPredicate");
   }
 
   /**
@@ -78,12 +93,18 @@ public class ParquetTupleScheme extends Scheme<JobConf, RecordReader, OutputColl
   public ParquetTupleScheme(Fields sourceFields, Fields sinkFields, final String schema) {
     super(sourceFields, sinkFields);
     parquetSchema = schema;
+    this.filterPredicate = null;
   }
 
   @SuppressWarnings("rawtypes")
   @Override
   public void sourceConfInit(FlowProcess<JobConf> fp,
       Tap<JobConf, RecordReader, OutputCollector> tap, JobConf jobConf) {
+
+    if (filterPredicate != null) {
+      ParquetInputFormat.setFilterPredicate(jobConf, filterPredicate);
+    }
+
     jobConf.setInputFormat(DeprecatedParquetInputFormat.class);
     ParquetInputFormat.setReadSupportClass(jobConf, TupleReadSupport.class);
     TupleReadSupport.setRequestedFields(jobConf, getSourceFields());

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/0d497c45/parquet-cascading/src/main/java/parquet/cascading/ParquetValueScheme.java
----------------------------------------------------------------------
diff --git a/parquet-cascading/src/main/java/parquet/cascading/ParquetValueScheme.java b/parquet-cascading/src/main/java/parquet/cascading/ParquetValueScheme.java
index be1496d..5296aee 100644
--- a/parquet-cascading/src/main/java/parquet/cascading/ParquetValueScheme.java
+++ b/parquet-cascading/src/main/java/parquet/cascading/ParquetValueScheme.java
@@ -21,14 +21,18 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.RecordReader;
 
-import parquet.hadoop.mapred.Container;
-
 import cascading.flow.FlowProcess;
 import cascading.scheme.Scheme;
-import cascading.scheme.SourceCall;
 import cascading.scheme.SinkCall;
+import cascading.scheme.SourceCall;
+import cascading.tap.Tap;
 import cascading.tuple.Tuple;
 import cascading.tuple.TupleEntry;
+import parquet.filter2.predicate.FilterPredicate;
+import parquet.hadoop.ParquetInputFormat;
+import parquet.hadoop.mapred.Container;
+
+import static parquet.Preconditions.checkNotNull;
 
 /**
  * A Cascading Scheme that returns a simple Tuple with a single value, the "value" object
@@ -38,8 +42,23 @@ import cascading.tuple.TupleEntry;
  * correctly in the respective Init methods.
  */
 public abstract class ParquetValueScheme<T> extends Scheme<JobConf, RecordReader, OutputCollector, Object[], Object[]>{
-
   private static final long serialVersionUID = 157560846420730043L;
+  private final FilterPredicate filterPredicate;
+
+  public ParquetValueScheme() {
+    this.filterPredicate = null;
+  }
+
+  public ParquetValueScheme(FilterPredicate filterPredicate) {
+    this.filterPredicate = checkNotNull(filterPredicate, "filterPredicate");
+  }
+
+  @Override
+  public void sourceConfInit(FlowProcess<JobConf> jobConfFlowProcess, Tap<JobConf, RecordReader, OutputCollector> jobConfRecordReaderOutputCollectorTap, final JobConf jobConf) {
+    if (filterPredicate != null) {
+      ParquetInputFormat.setFilterPredicate(jobConf, filterPredicate);
+    }
+  }
 
   @SuppressWarnings("unchecked")
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/0d497c45/parquet-scala/src/main/scala/parquet/filter2/dsl/Dsl.scala
----------------------------------------------------------------------
diff --git a/parquet-scala/src/main/scala/parquet/filter2/dsl/Dsl.scala b/parquet-scala/src/main/scala/parquet/filter2/dsl/Dsl.scala
index c60b804..7e39977 100644
--- a/parquet-scala/src/main/scala/parquet/filter2/dsl/Dsl.scala
+++ b/parquet-scala/src/main/scala/parquet/filter2/dsl/Dsl.scala
@@ -86,4 +86,6 @@ object Dsl {
     def unary_! = FilterApi.not(pred)
   }
 
+  implicit def stringToBinary(s: String): Binary = Binary.fromString(s)
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/0d497c45/parquet-scrooge/src/main/java/parquet/scrooge/ParquetScroogeScheme.java
----------------------------------------------------------------------
diff --git a/parquet-scrooge/src/main/java/parquet/scrooge/ParquetScroogeScheme.java b/parquet-scrooge/src/main/java/parquet/scrooge/ParquetScroogeScheme.java
index 78c2777..1fe1a6e 100644
--- a/parquet-scrooge/src/main/java/parquet/scrooge/ParquetScroogeScheme.java
+++ b/parquet-scrooge/src/main/java/parquet/scrooge/ParquetScroogeScheme.java
@@ -22,16 +22,17 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.RecordReader;
 
+import com.twitter.scrooge.ThriftStruct;
+
+import cascading.flow.FlowProcess;
+import cascading.scheme.SinkCall;
+import cascading.tap.Tap;
 import parquet.cascading.ParquetValueScheme;
+import parquet.filter2.predicate.FilterPredicate;
 import parquet.hadoop.ParquetInputFormat;
 import parquet.hadoop.mapred.DeprecatedParquetInputFormat;
 import parquet.hadoop.thrift.ParquetThriftInputFormat;
 import parquet.hadoop.thrift.ThriftReadSupport;
-import cascading.flow.FlowProcess;
-import cascading.scheme.SinkCall;
-import cascading.tap.Tap;
-
-import com.twitter.scrooge.ThriftStruct;
 
 public class ParquetScroogeScheme<T extends ThriftStruct> extends ParquetValueScheme<T> {
 
@@ -42,6 +43,11 @@ public class ParquetScroogeScheme<T extends ThriftStruct> extends ParquetValueSc
     this.klass = klass;
   }
 
+  public ParquetScroogeScheme(FilterPredicate filterPredicate, Class<T> klass) {
+    super(filterPredicate);
+    this.klass = klass;
+  }
+
   @SuppressWarnings("rawtypes")
   @Override
   public void sinkConfInit(FlowProcess<JobConf> arg0,