incubator-crunch-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jwi...@apache.org
Subject [10/28] Rename com.cloudera.crunch -> org.apache.crunch in the Java core
Date Sat, 07 Jul 2012 21:49:06 GMT
http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/fn/MapValuesFn.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/fn/MapValuesFn.java b/src/main/java/org/apache/crunch/fn/MapValuesFn.java
new file mode 100644
index 0000000..3c7065e
--- /dev/null
+++ b/src/main/java/org/apache/crunch/fn/MapValuesFn.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.crunch.fn;
+
+import org.apache.crunch.DoFn;
+import org.apache.crunch.Emitter;
+import org.apache.crunch.Pair;
+
+public abstract class MapValuesFn<K, V1, V2> extends DoFn<Pair<K, V1>, Pair<K, V2>> {
+  
+  @Override
+  public void process(Pair<K, V1> input, Emitter<Pair<K, V2>> emitter) {
+    emitter.emit(Pair.of(input.first(), map(input.second())));
+  }
+
+  public abstract V2 map(V1 v);
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/fn/PairMapFn.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/fn/PairMapFn.java b/src/main/java/org/apache/crunch/fn/PairMapFn.java
new file mode 100644
index 0000000..2cfd17b
--- /dev/null
+++ b/src/main/java/org/apache/crunch/fn/PairMapFn.java
@@ -0,0 +1,64 @@
+/**
+ * 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.crunch.fn;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.crunch.Emitter;
+import org.apache.crunch.MapFn;
+import org.apache.crunch.Pair;
+
+public class PairMapFn<K, V, S, T> extends MapFn<Pair<K, V>, Pair<S, T>> {
+  
+  private MapFn<K, S> keys;
+  private MapFn<V, T> values;
+
+  public PairMapFn(MapFn<K, S> keys, MapFn<V, T> values) {
+    this.keys = keys;
+    this.values = values;
+  }
+
+  @Override
+  public void configure(Configuration conf) {
+    keys.configure(conf);
+    values.configure(conf);
+  }
+  
+  @Override
+  public void initialize() {
+    keys.setContext(getContext());
+    values.setContext(getContext());
+  }
+
+  @Override
+  public Pair<S, T> map(Pair<K, V> input) {
+    return Pair.of(keys.map(input.first()), values.map(input.second()));
+  }
+  
+  @Override
+  public void cleanup(Emitter<Pair<S, T>> emitter) {
+    keys.cleanup(null);
+    values.cleanup(null);
+  }
+
+  @Override
+  public void setConfigurationForTest(Configuration conf) {
+    keys.setConfigurationForTest(conf);
+    values.setConfigurationForTest(conf);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mem/MemPipeline.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mem/MemPipeline.java b/src/main/java/org/apache/crunch/impl/mem/MemPipeline.java
new file mode 100644
index 0000000..6305fcb
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mem/MemPipeline.java
@@ -0,0 +1,209 @@
+/**
+ * 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.crunch.impl.mem;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.crunch.PCollection;
+import org.apache.crunch.PTable;
+import org.apache.crunch.Pair;
+import org.apache.crunch.Pipeline;
+import org.apache.crunch.PipelineResult;
+import org.apache.crunch.Source;
+import org.apache.crunch.TableSource;
+import org.apache.crunch.Target;
+import org.apache.crunch.impl.mem.collect.MemCollection;
+import org.apache.crunch.impl.mem.collect.MemTable;
+import org.apache.crunch.io.At;
+import org.apache.crunch.io.PathTarget;
+import org.apache.crunch.io.ReadableSource;
+import org.apache.crunch.types.PTableType;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public class MemPipeline implements Pipeline {
+
+  private static final Log LOG = LogFactory.getLog(MemPipeline.class);
+  
+  private static final MemPipeline INSTANCE = new MemPipeline();
+  
+  public static Pipeline getInstance() {
+    return INSTANCE;
+  }
+  
+  public static <T> PCollection<T> collectionOf(T...ts) {
+    return new MemCollection<T>(ImmutableList.copyOf(ts));  
+  }
+  
+  public static <T> PCollection<T> collectionOf(Iterable<T> collect) {
+    return new MemCollection<T>(collect);
+  }
+  
+  public static <T> PCollection<T> typedCollectionOf(PType<T> ptype, T... ts) {
+    return new MemCollection<T>(ImmutableList.copyOf(ts), ptype, null);  
+  }
+  
+  public static <T> PCollection<T> typedCollectionOf(PType<T> ptype, Iterable<T> collect) {
+    return new MemCollection<T>(collect, ptype, null);  
+  }
+  
+  public static <S, T> PTable<S, T> tableOf(S s, T t, Object... more) {
+    List<Pair<S, T>> pairs = Lists.newArrayList();
+    pairs.add(Pair.of(s, t));
+    for (int i = 0; i < more.length; i += 2) {
+      pairs.add(Pair.of((S) more[i], (T) more[i + 1]));
+    }
+    return new MemTable<S, T>(pairs);
+  }
+  
+  public static <S, T> PTable<S, T> typedTableOf(PTableType<S, T> ptype, S s, T t, Object... more) {
+    List<Pair<S, T>> pairs = Lists.newArrayList();
+    pairs.add(Pair.of(s, t));
+    for (int i = 0; i < more.length; i += 2) {
+      pairs.add(Pair.of((S) more[i], (T) more[i + 1]));
+    }
+    return new MemTable<S, T>(pairs, ptype, null);
+  }
+  
+  public static <S, T> PTable<S, T> tableOf(Iterable<Pair<S, T>> pairs) {
+    return new MemTable<S, T>(pairs);
+  }
+  
+  public static <S, T> PTable<S, T> typedTableOf(PTableType<S, T> ptype, Iterable<Pair<S, T>> pairs) {
+    return new MemTable<S, T>(pairs, ptype, null);
+  }
+  
+  private Configuration conf = new Configuration();
+
+  private MemPipeline() {
+  }
+  
+  @Override
+  public void setConfiguration(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  @Override
+  public <T> PCollection<T> read(Source<T> source) {
+    if (source instanceof ReadableSource) {
+      try {
+        Iterable<T> iterable = ((ReadableSource<T>) source).read(conf);
+        return new MemCollection<T>(iterable, source.getType(), source.toString());
+      } catch (IOException e) {
+        LOG.error("Exception reading source: " + source.toString(), e);
+        throw new IllegalStateException(e);
+      }
+    }
+    LOG.error("Source " + source + " is not readable");
+    throw new IllegalStateException("Source " + source + " is not readable");
+  }
+
+  @Override
+  public <K, V> PTable<K, V> read(TableSource<K, V> source) {
+    if (source instanceof ReadableSource) {
+      try {
+        Iterable<Pair<K, V>> iterable = ((ReadableSource<Pair<K, V>>) source).read(conf);
+        return new MemTable<K, V>(iterable, source.getTableType(), source.toString());
+      } catch (IOException e) {
+        LOG.error("Exception reading source: " + source.toString(), e);
+        throw new IllegalStateException(e);
+      }
+    }
+    LOG.error("Source " + source + " is not readable");
+    throw new IllegalStateException("Source " + source + " is not readable");
+  }
+
+  @Override
+  public void write(PCollection<?> collection, Target target) {
+    if (target instanceof PathTarget) {
+      Path path = ((PathTarget) target).getPath();
+      try {
+        FileSystem fs = FileSystem.get(conf);
+        FSDataOutputStream os = fs.create(new Path(path, "out"));
+        if (collection instanceof PTable) {
+          for (Object o : collection.materialize()) {
+            Pair p = (Pair) o;
+            os.writeBytes(p.first().toString());
+            os.writeBytes("\t");
+            os.writeBytes(p.second().toString());
+            os.writeBytes("\r\n");
+          }
+        } else {
+          for (Object o : collection.materialize()) {
+            os.writeBytes(o.toString() + "\r\n");
+          }
+        }
+        os.close();
+      } catch (IOException e) {
+        LOG.error("Exception writing target: " + target, e);
+      }
+    } else {
+      LOG.error("Target " + target + " is not a PathTarget instance");
+    }
+  }
+
+  @Override
+  public PCollection<String> readTextFile(String pathName) {
+    return read(At.textFile(pathName));
+  }
+
+  @Override
+  public <T> void writeTextFile(PCollection<T> collection, String pathName) {
+    write(collection, At.textFile(pathName));
+  }
+
+  @Override
+  public <T> Iterable<T> materialize(PCollection<T> pcollection) {
+    return pcollection.materialize();
+  }
+
+  @Override
+  public PipelineResult run() {
+    return PipelineResult.EMPTY;
+  }
+
+  @Override
+  public PipelineResult done() {
+    return PipelineResult.EMPTY;
+  }
+
+  @Override
+  public void enableDebug() {
+	LOG.info("Note: in-memory pipelines do not have debug logging");
+  }
+  
+  @Override
+  public String getName() {
+	  return "Memory Pipeline";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mem/collect/MemCollection.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mem/collect/MemCollection.java b/src/main/java/org/apache/crunch/impl/mem/collect/MemCollection.java
new file mode 100644
index 0000000..7291e50
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mem/collect/MemCollection.java
@@ -0,0 +1,205 @@
+/**
+ * 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.crunch.impl.mem.collect;
+
+import java.util.Collection;
+
+import org.apache.crunch.DoFn;
+import org.apache.crunch.FilterFn;
+import org.apache.crunch.MapFn;
+import org.apache.crunch.PCollection;
+import org.apache.crunch.PTable;
+import org.apache.crunch.Pair;
+import org.apache.crunch.Pipeline;
+import org.apache.crunch.Target;
+import org.apache.crunch.fn.ExtractKeyFn;
+import org.apache.crunch.impl.mem.MemPipeline;
+import org.apache.crunch.lib.Aggregate;
+import org.apache.crunch.lib.Sample;
+import org.apache.crunch.lib.Sort;
+import org.apache.crunch.test.InMemoryEmitter;
+import org.apache.crunch.types.PTableType;
+import org.apache.crunch.types.PType;
+import org.apache.crunch.types.PTypeFamily;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+
+public class MemCollection<S> implements PCollection<S> {
+
+  private final Collection<S> collect;
+  private final PType<S> ptype;
+  private String name;
+  
+  public MemCollection(Iterable<S> collect) {
+    this(collect, null, null);
+  }
+  
+  public MemCollection(Iterable<S> collect, PType<S> ptype) {
+    this(collect, ptype, null);
+  }
+  
+  public MemCollection(Iterable<S> collect, PType<S> ptype, String name) {
+    this.collect = ImmutableList.copyOf(collect);
+    this.ptype = ptype;
+    this.name = name;
+  }
+  
+  @Override
+  public Pipeline getPipeline() {
+    return MemPipeline.getInstance();
+  }
+
+  @Override
+  public PCollection<S> union(PCollection<S>... collections) {
+    Collection<S> output = Lists.newArrayList();    
+    for (PCollection<S> pcollect : collections) {
+      for (S s : pcollect.materialize()) {
+        output.add(s);
+      }
+    }
+    output.addAll(collect);
+    return new MemCollection<S>(output, collections[0].getPType());
+  }
+
+  @Override
+  public <T> PCollection<T> parallelDo(DoFn<S, T> doFn, PType<T> type) {
+    return parallelDo(null, doFn, type);
+  }
+
+  @Override
+  public <T> PCollection<T> parallelDo(String name, DoFn<S, T> doFn, PType<T> type) {
+    InMemoryEmitter<T> emitter = new InMemoryEmitter<T>();
+    doFn.initialize();
+    for (S s : collect) {
+      doFn.process(s, emitter);
+    }
+    doFn.cleanup(emitter);
+    return new MemCollection<T>(emitter.getOutput(), type, name);
+  }
+
+  @Override
+  public <K, V> PTable<K, V> parallelDo(DoFn<S, Pair<K, V>> doFn, PTableType<K, V> type) {
+    return parallelDo(null, doFn, type);
+  }
+
+  @Override
+  public <K, V> PTable<K, V> parallelDo(String name, DoFn<S, Pair<K, V>> doFn,
+      PTableType<K, V> type) {
+    InMemoryEmitter<Pair<K, V>> emitter = new InMemoryEmitter<Pair<K, V>>();
+    doFn.initialize();
+    for (S s : collect) {
+      doFn.process(s, emitter);
+    }
+    doFn.cleanup(emitter);
+    return new MemTable<K, V>(emitter.getOutput(), type, name);
+  }
+
+  @Override
+  public PCollection<S> write(Target target) {
+    getPipeline().write(this, target);
+    return this;
+  }
+
+  @Override
+  public Iterable<S> materialize() {
+    return collect;
+  }
+
+  public Collection<S> getCollection() {
+    return collect;
+  }
+  
+  @Override
+  public PType<S> getPType() {
+    return ptype;
+  }
+
+  @Override
+  public PTypeFamily getTypeFamily() {
+    if (ptype != null) {
+      return ptype.getFamily();
+    }
+    return null;
+  }
+
+  @Override
+  public long getSize() {
+    return collect.size();
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+  
+  @Override
+  public String toString() {
+    return collect.toString();
+  }
+
+  @Override
+  public PTable<S, Long> count() {
+	return Aggregate.count(this);
+  }
+
+  @Override
+  public PCollection<S> sample(double acceptanceProbability) {
+	return Sample.sample(this, acceptanceProbability);
+  }
+
+  @Override
+  public PCollection<S> sample(double acceptanceProbability, long seed) {
+	return Sample.sample(this, seed, acceptanceProbability);
+  }
+
+  @Override
+  public PCollection<S> max() {
+	return Aggregate.max(this);
+  }
+
+  @Override
+  public PCollection<S> min() {
+	return Aggregate.min(this);
+  }
+
+  @Override
+  public PCollection<S> sort(boolean ascending) {
+	return Sort.sort(this, ascending ? Sort.Order.ASCENDING : Sort.Order.DESCENDING);
+  }
+
+  @Override
+  public PCollection<S> filter(FilterFn<S> filterFn) {
+    return parallelDo(filterFn, getPType());
+  }
+  
+  @Override
+  public PCollection<S> filter(String name, FilterFn<S> filterFn) {
+    return parallelDo(name, filterFn, getPType());
+  }
+  
+  @Override
+  public <K> PTable<K, S> by(MapFn<S, K> mapFn, PType<K> keyType) {
+    return parallelDo(new ExtractKeyFn<K, S>(mapFn), getTypeFamily().tableOf(keyType, getPType()));
+  }
+
+  @Override
+  public <K> PTable<K, S> by(String name, MapFn<S, K> mapFn, PType<K> keyType) {
+    return parallelDo(name, new ExtractKeyFn<K, S>(mapFn), getTypeFamily().tableOf(keyType, getPType()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mem/collect/MemGroupedTable.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mem/collect/MemGroupedTable.java b/src/main/java/org/apache/crunch/impl/mem/collect/MemGroupedTable.java
new file mode 100644
index 0000000..4e114ab
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mem/collect/MemGroupedTable.java
@@ -0,0 +1,126 @@
+/**
+ * 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.crunch.impl.mem.collect;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import org.apache.crunch.CombineFn;
+import org.apache.crunch.GroupingOptions;
+import org.apache.crunch.PCollection;
+import org.apache.crunch.PGroupedTable;
+import org.apache.crunch.PTable;
+import org.apache.crunch.Pair;
+import org.apache.crunch.Pipeline;
+import org.apache.crunch.Target;
+import org.apache.crunch.types.PTableType;
+import org.apache.crunch.types.PType;
+import org.apache.crunch.types.PTypeFamily;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+class MemGroupedTable<K, V> extends MemCollection<Pair<K, Iterable<V>>> implements PGroupedTable<K, V> {
+
+  private final MemTable<K, V> parent;
+  
+  private static <S, T> Map<S, Collection<T>> createMapFor(PType<S> keyType, GroupingOptions options, Pipeline pipeline) {
+    if (options != null && options.getSortComparatorClass() != null) {
+      RawComparator<S> rc = ReflectionUtils.newInstance(options.getSortComparatorClass(),
+          pipeline.getConfiguration());
+      return new TreeMap<S, Collection<T>>(rc);
+    } else if (keyType != null && Comparable.class.isAssignableFrom(keyType.getTypeClass())) {
+      return new TreeMap<S, Collection<T>>();
+    }
+    return Maps.newHashMap();
+  }
+  
+  private static <S, T> Iterable<Pair<S, Iterable<T>>> buildMap(MemTable<S, T> parent, GroupingOptions options) {
+    PType<S> keyType = parent.getKeyType();
+    Map<S, Collection<T>> map = createMapFor(keyType, options, parent.getPipeline());
+    
+    for (Pair<S, T> pair : parent.materialize()) {
+      S key = pair.first();
+      if (!map.containsKey(key)) {
+        map.put(key, Lists.<T>newArrayList());
+      }
+      map.get(key).add(pair.second());
+    }
+    
+    List<Pair<S, Iterable<T>>> values = Lists.newArrayList();
+    for (Map.Entry<S, Collection<T>> e : map.entrySet()) {
+      values.add(Pair.of(e.getKey(), (Iterable<T>) e.getValue()));
+    }
+    return values;
+  }
+  
+  public MemGroupedTable(MemTable<K, V> parent, GroupingOptions options) {
+	super(buildMap(parent, options));
+    this.parent = parent;
+  }
+
+  @Override
+  public PCollection<Pair<K, Iterable<V>>> union(
+      PCollection<Pair<K, Iterable<V>>>... collections) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public PCollection<Pair<K, Iterable<V>>> write(Target target) {
+    getPipeline().write(this.ungroup(), target);
+    return this;
+  }
+
+  @Override
+  public PType<Pair<K, Iterable<V>>> getPType() {
+    PTableType<K, V> parentType = parent.getPTableType();
+    if (parentType != null) {
+      return parentType.getGroupedTableType();
+    }
+    return null;
+  }
+
+  @Override
+  public PTypeFamily getTypeFamily() {
+    return parent.getTypeFamily();
+  }
+
+  @Override
+  public long getSize() {
+    return parent.getSize();
+  }
+
+  @Override
+  public String getName() {
+    return "MemGrouped(" + parent.getName() + ")";
+  }
+
+  @Override
+  public PTable<K, V> combineValues(CombineFn<K, V> combineFn) {
+    return parallelDo(combineFn, parent.getPTableType());
+  }
+
+  @Override
+  public PTable<K, V> ungroup() {
+    return parent;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mem/collect/MemTable.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mem/collect/MemTable.java b/src/main/java/org/apache/crunch/impl/mem/collect/MemTable.java
new file mode 100644
index 0000000..53e7526
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mem/collect/MemTable.java
@@ -0,0 +1,146 @@
+/**
+ * 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.crunch.impl.mem.collect;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.crunch.GroupingOptions;
+import org.apache.crunch.PCollection;
+import org.apache.crunch.PGroupedTable;
+import org.apache.crunch.PTable;
+import org.apache.crunch.Pair;
+import org.apache.crunch.Target;
+import org.apache.crunch.lib.Aggregate;
+import org.apache.crunch.lib.Cogroup;
+import org.apache.crunch.lib.Join;
+import org.apache.crunch.lib.PTables;
+import org.apache.crunch.materialize.MaterializableMap;
+import org.apache.crunch.types.PTableType;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.Lists;
+
+public class MemTable<K, V> extends MemCollection<Pair<K, V>> implements PTable<K, V> {
+
+  private PTableType<K, V> ptype;
+  
+  public MemTable(Iterable<Pair<K, V>> collect) {
+    this(collect, null, null);
+  }
+  
+  public MemTable(Iterable<Pair<K, V>> collect, PTableType<K, V> ptype, String name) {
+    super(collect, ptype, name);
+    this.ptype = ptype;
+  }
+  
+  @Override
+  public PTable<K, V> union(PTable<K, V>... others) {
+    List<Pair<K, V>> values = Lists.newArrayList();
+    values.addAll(getCollection());
+    for (PTable<K, V> ptable : others) {
+      for (Pair<K, V> p : ptable.materialize()) {
+        values.add(p);
+      }
+    }
+    return new MemTable<K, V>(values, others[0].getPTableType(), null);
+  }
+
+  @Override
+  public PGroupedTable<K, V> groupByKey() {
+    return groupByKey(null);
+  }
+
+  @Override
+  public PGroupedTable<K, V> groupByKey(int numPartitions) {
+    return groupByKey(null);
+  }
+
+  @Override
+  public PGroupedTable<K, V> groupByKey(GroupingOptions options) {
+    return new MemGroupedTable<K, V>(this, options);
+  }
+
+  @Override
+  public PTable<K, V> write(Target target) {
+    super.write(target);
+    return this;
+  }
+  
+  @Override
+  public PTableType<K, V> getPTableType() {
+    return ptype;
+  }
+
+  @Override
+  public PType<K> getKeyType() {
+    if (ptype != null) {
+      return ptype.getKeyType();
+    }
+    return null;
+  }
+
+  @Override
+  public PType<V> getValueType() {
+    if (ptype != null) {
+      return ptype.getValueType();
+    }
+    return null;
+  }
+
+  @Override
+  public PTable<K, V> top(int count) {
+	return Aggregate.top(this, count, true);
+  }
+
+  @Override
+  public PTable<K, V> bottom(int count) {
+	return Aggregate.top(this, count, false);
+  }
+
+  @Override
+  public PTable<K, Collection<V>> collectValues() {
+	return Aggregate.collectValues(this);
+  }
+
+  @Override
+  public <U> PTable<K, Pair<V, U>> join(PTable<K, U> other) {
+	return Join.join(this, other);
+  }
+  
+  @Override
+  public <U> PTable<K, Pair<Collection<V>, Collection<U>>> cogroup(PTable<K, U> other) {
+	return Cogroup.cogroup(this, other);
+  }
+  
+  @Override
+  public PCollection<K> keys() {
+	return PTables.keys(this);
+  }
+
+  @Override
+  public PCollection<V> values() {
+    return PTables.values(this);
+  }
+
+  @Override
+  public Map<K, V> materializeToMap() {
+    return new MaterializableMap<K, V>(this.materialize());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/MRPipeline.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/MRPipeline.java b/src/main/java/org/apache/crunch/impl/mr/MRPipeline.java
new file mode 100644
index 0000000..0b7d8d7
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/MRPipeline.java
@@ -0,0 +1,322 @@
+/**
+ * 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.crunch.impl.mr;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Appender;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import org.apache.crunch.MapFn;
+import org.apache.crunch.PCollection;
+import org.apache.crunch.PTable;
+import org.apache.crunch.Pipeline;
+import org.apache.crunch.PipelineResult;
+import org.apache.crunch.Source;
+import org.apache.crunch.SourceTarget;
+import org.apache.crunch.TableSource;
+import org.apache.crunch.Target;
+import org.apache.crunch.fn.IdentityFn;
+import org.apache.crunch.impl.mr.collect.InputCollection;
+import org.apache.crunch.impl.mr.collect.InputTable;
+import org.apache.crunch.impl.mr.collect.PCollectionImpl;
+import org.apache.crunch.impl.mr.collect.PGroupedTableImpl;
+import org.apache.crunch.impl.mr.collect.UnionCollection;
+import org.apache.crunch.impl.mr.collect.UnionTable;
+import org.apache.crunch.impl.mr.plan.MSCRPlanner;
+import org.apache.crunch.impl.mr.run.RuntimeParameters;
+import org.apache.crunch.io.At;
+import org.apache.crunch.io.ReadableSourceTarget;
+import org.apache.crunch.materialize.MaterializableIterable;
+import org.apache.crunch.types.PType;
+import org.apache.crunch.types.writable.WritableTypeFamily;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+public class MRPipeline implements Pipeline {
+
+  private static final Log LOG = LogFactory.getLog(MRPipeline.class);
+
+  private static final Random RANDOM = new Random();
+
+  private final Class<?> jarClass;
+  private final String name;
+  private final Map<PCollectionImpl<?>, Set<Target>> outputTargets;
+  private final Map<PCollectionImpl<?>, MaterializableIterable<?>> outputTargetsToMaterialize;
+  private final Path tempDirectory;
+  private int tempFileIndex;
+  private int nextAnonymousStageId;
+
+  private Configuration conf;
+
+  public MRPipeline(Class<?> jarClass) throws IOException {
+    this(jarClass, new Configuration());
+  }
+
+  public MRPipeline(Class<?> jarClass, String name) {
+    this(jarClass, name, new Configuration());
+  }
+
+  public MRPipeline(Class<?> jarClass, Configuration conf) {
+    this(jarClass, jarClass.getName(), conf);
+  }
+
+  public MRPipeline(Class<?> jarClass, String name, Configuration conf) {
+    this.jarClass = jarClass;
+    this.name = name;
+    this.outputTargets = Maps.newHashMap();
+    this.outputTargetsToMaterialize = Maps.newHashMap();
+    this.conf = conf;
+    this.tempDirectory = createTempDirectory(conf);
+    this.tempFileIndex = 0;
+    this.nextAnonymousStageId = 0;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  @Override
+  public void setConfiguration(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public PipelineResult run() {
+    MSCRPlanner planner = new MSCRPlanner(this, outputTargets);
+    PipelineResult res = null;
+    try {
+      res = planner.plan(jarClass, conf).execute();
+    } catch (IOException e) {
+      LOG.error(e);
+      return PipelineResult.EMPTY;
+    }
+    for (PCollectionImpl<?> c : outputTargets.keySet()) {
+      if (outputTargetsToMaterialize.containsKey(c)) {
+        MaterializableIterable iter = outputTargetsToMaterialize.get(c);
+        iter.materialize();
+        c.materializeAt(iter.getSourceTarget());
+        outputTargetsToMaterialize.remove(c);
+      } else {
+        boolean materialized = false;
+        for (Target t : outputTargets.get(c)) {
+          if (!materialized && t instanceof Source) {
+            c.materializeAt((SourceTarget) t);
+            materialized = true;
+          }
+        }
+      }
+    }
+    outputTargets.clear();
+    return res;
+  }
+
+  @Override
+  public PipelineResult done() {
+    PipelineResult res = null;
+    if (!outputTargets.isEmpty()) {
+      res = run();
+    }
+    cleanup();
+    return res;
+  }
+
+  public <S> PCollection<S> read(Source<S> source) {
+    return new InputCollection<S>(source, this);
+  }
+
+  public <K, V> PTable<K, V> read(TableSource<K, V> source) {
+    return new InputTable<K, V>(source, this);
+  }
+
+  public PCollection<String> readTextFile(String pathName) {
+    return read(At.textFile(pathName));
+  }
+
+  @SuppressWarnings("unchecked")
+  public void write(PCollection<?> pcollection, Target target) {
+    if (pcollection instanceof PGroupedTableImpl) {
+      pcollection = ((PGroupedTableImpl<?, ?>) pcollection).ungroup();
+    } else if (pcollection instanceof UnionCollection || pcollection instanceof UnionTable) {
+      pcollection = pcollection.parallelDo("UnionCollectionWrapper",
+          (MapFn) IdentityFn.<Object> getInstance(), pcollection.getPType());
+    }
+    addOutput((PCollectionImpl<?>) pcollection, target);
+  }
+
+  private void addOutput(PCollectionImpl<?> impl, Target target) {
+    if (!outputTargets.containsKey(impl)) {
+      outputTargets.put(impl, Sets.<Target> newHashSet());
+    }
+    outputTargets.get(impl).add(target);
+  }
+
+  @Override
+  public <T> Iterable<T> materialize(PCollection<T> pcollection) {
+
+    PCollectionImpl<T> pcollectionImpl = toPcollectionImpl(pcollection);
+    ReadableSourceTarget<T> srcTarget = getMaterializeSourceTarget(pcollectionImpl);
+
+    MaterializableIterable<T> c = new MaterializableIterable<T>(this, srcTarget);
+    if (!outputTargetsToMaterialize.containsKey(pcollectionImpl)) {
+      outputTargetsToMaterialize.put(pcollectionImpl, c);
+    }
+    return c;
+  }
+
+  /**
+   * Retrieve a ReadableSourceTarget that provides access to the contents of a
+   * {@link PCollection}. This is primarily intended as a helper method to
+   * {@link #materialize(PCollection)}. The underlying data of the
+   * ReadableSourceTarget may not be actually present until the pipeline is run.
+   * 
+   * @param pcollection
+   *          The collection for which the ReadableSourceTarget is to be
+   *          retrieved
+   * @return The ReadableSourceTarget
+   * @throws IllegalArgumentException
+   *           If no ReadableSourceTarget can be retrieved for the given
+   *           PCollection
+   */
+  public <T> ReadableSourceTarget<T> getMaterializeSourceTarget(PCollection<T> pcollection) {
+    PCollectionImpl<T> impl = toPcollectionImpl(pcollection);
+    SourceTarget<T> matTarget = impl.getMaterializedAt();
+    if (matTarget != null && matTarget instanceof ReadableSourceTarget) {
+      return (ReadableSourceTarget<T>) matTarget;
+    }
+
+    ReadableSourceTarget<T> srcTarget = null;
+    if (outputTargets.containsKey(pcollection)) {
+      for (Target target : outputTargets.get(impl)) {
+        if (target instanceof ReadableSourceTarget) {
+          srcTarget = (ReadableSourceTarget<T>) target;
+          break;
+        }
+      }
+    }
+
+    if (srcTarget == null) {
+      SourceTarget<T> st = createIntermediateOutput(pcollection.getPType());
+      if (!(st instanceof ReadableSourceTarget)) {
+        throw new IllegalArgumentException("The PType for the given PCollection is not readable"
+            + " and cannot be materialized");
+      } else {
+        srcTarget = (ReadableSourceTarget<T>) st;
+        addOutput(impl, srcTarget);
+      }
+    }
+
+    return srcTarget;
+  }
+
+  /**
+   * Safely cast a PCollection into a PCollectionImpl, including handling the case of UnionCollections.
+   * @param pcollection The PCollection to be cast/transformed
+   * @return The PCollectionImpl representation
+   */
+  private <T> PCollectionImpl<T> toPcollectionImpl(PCollection<T> pcollection) {
+    PCollectionImpl<T> pcollectionImpl = null;
+    if (pcollection instanceof UnionCollection) {
+      pcollectionImpl = (PCollectionImpl<T>) pcollection.parallelDo("UnionCollectionWrapper",
+          (MapFn) IdentityFn.<Object> getInstance(), pcollection.getPType());
+    } else {
+      pcollectionImpl = (PCollectionImpl<T>) pcollection;
+    }
+    return pcollectionImpl;
+  }
+
+  public <T> SourceTarget<T> createIntermediateOutput(PType<T> ptype) {
+    return ptype.getDefaultFileSource(createTempPath());
+  }
+
+  public Path createTempPath() {
+    tempFileIndex++;
+    return new Path(tempDirectory, "p" + tempFileIndex);
+  }
+
+  private static Path createTempDirectory(Configuration conf) {
+    Path dir = new Path("/tmp/crunch" + RANDOM.nextInt());
+    try {
+      FileSystem.get(conf).mkdirs(dir);
+    } catch (IOException e) {
+      LOG.error("Exception creating job output directory", e);
+      throw new RuntimeException(e);
+    }
+    return dir;
+  }
+
+  @Override
+  public <T> void writeTextFile(PCollection<T> pcollection, String pathName) {
+    // Ensure that this is a writable pcollection instance.
+    pcollection = pcollection.parallelDo("asText", IdentityFn.<T> getInstance(), WritableTypeFamily
+        .getInstance().as(pcollection.getPType()));
+    write(pcollection, At.textFile(pathName));
+  }
+
+  private void cleanup() {
+    if (!outputTargets.isEmpty()) {
+      LOG.warn("Not running cleanup while output targets remain");
+      return;
+    }
+    try {
+      FileSystem fs = FileSystem.get(conf);
+      if (fs.exists(tempDirectory)) {
+        fs.delete(tempDirectory, true);
+      }
+    } catch (IOException e) {
+      LOG.info("Exception during cleanup", e);
+    }
+  }
+
+  public int getNextAnonymousStageId() {
+    return nextAnonymousStageId++;
+  }
+
+  @Override
+  public void enableDebug() {
+    // Turn on Crunch runtime error catching.
+    getConfiguration().setBoolean(RuntimeParameters.DEBUG, true);
+
+    // Write Hadoop's WARN logs to the console.
+    Logger crunchInfoLogger = LogManager.getLogger("org.apache.crunch");
+    Appender console = crunchInfoLogger.getAppender("A");
+    if (console != null) {
+      Logger hadoopLogger = LogManager.getLogger("org.apache.hadoop");
+      hadoopLogger.setLevel(Level.WARN);
+      hadoopLogger.addAppender(console);
+    } else {
+      LOG.warn("Could not find console appender named 'A' for writing Hadoop warning logs");
+    }
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/collect/DoCollectionImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/collect/DoCollectionImpl.java b/src/main/java/org/apache/crunch/impl/mr/collect/DoCollectionImpl.java
new file mode 100644
index 0000000..3c9d522
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/collect/DoCollectionImpl.java
@@ -0,0 +1,65 @@
+/**
+ * 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.crunch.impl.mr.collect;
+
+import java.util.List;
+
+import org.apache.crunch.DoFn;
+import org.apache.crunch.impl.mr.plan.DoNode;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.ImmutableList;
+
+public class DoCollectionImpl<S> extends PCollectionImpl<S> {
+
+  private final PCollectionImpl<Object> parent;
+  private final DoFn<Object, S> fn;
+  private final PType<S> ntype;
+
+  <T> DoCollectionImpl(String name, PCollectionImpl<T> parent, DoFn<T, S> fn,
+      PType<S> ntype) {
+    super(name);
+    this.parent = (PCollectionImpl<Object>) parent;
+    this.fn = (DoFn<Object, S>) fn;
+    this.ntype = ntype;
+  }
+
+  @Override
+  protected long getSizeInternal() {
+    return (long) (fn.scaleFactor() * parent.getSize());
+  }
+  
+  @Override
+  public PType<S> getPType() {
+    return ntype;
+  }
+
+  @Override
+  protected void acceptInternal(PCollectionImpl.Visitor visitor) {
+    visitor.visitDoFnCollection(this);
+  }
+
+  @Override
+  public List<PCollectionImpl<?>> getParents() {
+    return ImmutableList.<PCollectionImpl<?>> of(parent);
+  }
+
+  @Override
+  public DoNode createDoNode() {
+    return DoNode.createFnNode(getName(), fn, ntype);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/collect/DoTableImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/collect/DoTableImpl.java b/src/main/java/org/apache/crunch/impl/mr/collect/DoTableImpl.java
new file mode 100644
index 0000000..3e5a275
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/collect/DoTableImpl.java
@@ -0,0 +1,79 @@
+/**
+ * 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.crunch.impl.mr.collect;
+
+import java.util.List;
+
+import org.apache.crunch.CombineFn;
+import org.apache.crunch.DoFn;
+import org.apache.crunch.PTable;
+import org.apache.crunch.Pair;
+import org.apache.crunch.impl.mr.plan.DoNode;
+import org.apache.crunch.types.PTableType;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.ImmutableList;
+
+public class DoTableImpl<K, V> extends PTableBase<K, V> implements
+    PTable<K, V> {
+
+  private final PCollectionImpl<?> parent;
+  private final DoFn<?, Pair<K, V>> fn;
+  private final PTableType<K, V> type;
+
+  <S> DoTableImpl(String name, PCollectionImpl<S> parent,
+      DoFn<S, Pair<K, V>> fn, PTableType<K, V> ntype) {
+    super(name);
+    this.parent = parent;
+    this.fn = fn;
+    this.type = ntype;
+  }
+
+  @Override
+  protected long getSizeInternal() {
+    return (long) (fn.scaleFactor() * parent.getSize());
+  }
+
+  @Override
+  public PTableType<K, V> getPTableType() {
+    return type;
+  }
+
+  @Override
+  protected void acceptInternal(PCollectionImpl.Visitor visitor) {
+    visitor.visitDoTable(this);
+  }
+
+  @Override
+  public PType<Pair<K, V>> getPType() {
+    return type;
+  }
+
+  @Override
+  public List<PCollectionImpl<?>> getParents() {
+    return ImmutableList.<PCollectionImpl<?>> of(parent);
+  }
+
+  @Override
+  public DoNode createDoNode() {
+    return DoNode.createFnNode(getName(), fn, type);
+  }
+  
+  public boolean hasCombineFn() {
+    return fn instanceof CombineFn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/collect/InputCollection.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/collect/InputCollection.java b/src/main/java/org/apache/crunch/impl/mr/collect/InputCollection.java
new file mode 100644
index 0000000..5c9b93e
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/collect/InputCollection.java
@@ -0,0 +1,85 @@
+/**
+ * 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.crunch.impl.mr.collect;
+
+import java.util.List;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import org.apache.crunch.Source;
+import org.apache.crunch.impl.mr.MRPipeline;
+import org.apache.crunch.impl.mr.plan.DoNode;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.ImmutableList;
+
+public class InputCollection<S> extends PCollectionImpl<S> {
+
+  private final Source<S> source;
+
+  public InputCollection(Source<S> source, MRPipeline pipeline) {
+    super(source.toString());
+    this.source = source;
+    this.pipeline = pipeline;
+  }
+
+  @Override
+  public PType<S> getPType() {
+    return source.getType();
+  }
+
+  public Source<S> getSource() {
+    return source;
+  }
+
+  @Override
+  protected long getSizeInternal() {
+    long sz = source.getSize(pipeline.getConfiguration());
+    if (sz < 0) {
+      throw new IllegalStateException("Input source " + source + " does not exist!");
+    }
+    return sz;
+  }
+  
+  @Override
+  protected void acceptInternal(PCollectionImpl.Visitor visitor) {
+    visitor.visitInputCollection(this);
+  }
+
+  @Override
+  public List<PCollectionImpl<?>> getParents() {
+    return ImmutableList.of();
+  }
+
+  @Override
+  public DoNode createDoNode() {
+    return DoNode.createInputNode(source);
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null || !(obj instanceof InputCollection)) {
+      return false;
+    }
+    return source.equals(((InputCollection) obj).source);
+  }
+  
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().append(source).toHashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/collect/InputTable.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/collect/InputTable.java b/src/main/java/org/apache/crunch/impl/mr/collect/InputTable.java
new file mode 100644
index 0000000..617d768
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/collect/InputTable.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.crunch.impl.mr.collect;
+
+import java.util.List;
+
+import org.apache.crunch.Pair;
+import org.apache.crunch.TableSource;
+import org.apache.crunch.impl.mr.MRPipeline;
+import org.apache.crunch.impl.mr.plan.DoNode;
+import org.apache.crunch.types.PTableType;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.ImmutableList;
+
+public class InputTable<K, V> extends PTableBase<K, V> {
+
+  private final TableSource<K, V> source;
+  private final InputCollection<Pair<K, V>> asCollection;
+  
+  public InputTable(TableSource<K, V> source, MRPipeline pipeline) {
+    super(source.toString());
+    this.source = source;
+    this.pipeline = pipeline;
+    this.asCollection = new InputCollection<Pair<K, V>>(source, pipeline);
+  }
+
+  @Override
+  protected long getSizeInternal() {
+    return asCollection.getSizeInternal();
+  }
+
+  @Override
+  public PTableType<K, V> getPTableType() {
+    return source.getTableType();
+  }
+
+  @Override
+  public PType<Pair<K, V>> getPType() {
+    return source.getType();
+  }
+
+  @Override
+  public List<PCollectionImpl<?>> getParents() {
+    return ImmutableList.of();
+  }
+
+  @Override
+  protected void acceptInternal(PCollectionImpl.Visitor visitor) {
+    visitor.visitInputCollection(asCollection);
+  }
+
+  @Override
+  public DoNode createDoNode() {
+    return DoNode.createInputNode(source);
+  }
+  
+  @Override
+  public int hashCode() {
+    return asCollection.hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    return asCollection.equals(other);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/collect/PCollectionImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/collect/PCollectionImpl.java b/src/main/java/org/apache/crunch/impl/mr/collect/PCollectionImpl.java
new file mode 100644
index 0000000..a9e8401
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/collect/PCollectionImpl.java
@@ -0,0 +1,241 @@
+/**
+ * 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.crunch.impl.mr.collect;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.crunch.DoFn;
+import org.apache.crunch.FilterFn;
+import org.apache.crunch.MapFn;
+import org.apache.crunch.PCollection;
+import org.apache.crunch.PTable;
+import org.apache.crunch.Pair;
+import org.apache.crunch.Pipeline;
+import org.apache.crunch.SourceTarget;
+import org.apache.crunch.Target;
+import org.apache.crunch.fn.ExtractKeyFn;
+import org.apache.crunch.impl.mr.MRPipeline;
+import org.apache.crunch.impl.mr.plan.DoNode;
+import org.apache.crunch.lib.Aggregate;
+import org.apache.crunch.lib.Sample;
+import org.apache.crunch.lib.Sort;
+import org.apache.crunch.types.PTableType;
+import org.apache.crunch.types.PType;
+import org.apache.crunch.types.PTypeFamily;
+import com.google.common.collect.Lists;
+
+public abstract class PCollectionImpl<S> implements PCollection<S> {
+
+  private static final Log LOG = LogFactory.getLog(PCollectionImpl.class);
+
+  private final String name;
+  protected MRPipeline pipeline;
+  private SourceTarget<S> materializedAt;
+
+  public PCollectionImpl(String name) {
+    this.name = name;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String toString() {
+    return getName();
+  }
+
+  @Override
+  public PCollection<S> union(PCollection<S>... collections) {
+    List<PCollectionImpl<S>> internal = Lists.newArrayList();
+    internal.add(this);
+    for (PCollection<S> collection : collections) {
+      internal.add((PCollectionImpl<S>) collection);
+    }
+    return new UnionCollection<S>(internal);
+  }
+
+  @Override
+  public <T> PCollection<T> parallelDo(DoFn<S, T> fn, PType<T> type) {
+    MRPipeline pipeline = (MRPipeline) getPipeline();
+    return parallelDo("S" + pipeline.getNextAnonymousStageId(), fn, type);
+  }
+
+  @Override
+  public <T> PCollection<T> parallelDo(String name, DoFn<S, T> fn, PType<T> type) {
+    return new DoCollectionImpl<T>(name, this, fn, type);
+  }
+
+  @Override
+  public <K, V> PTable<K, V> parallelDo(DoFn<S, Pair<K, V>> fn, PTableType<K, V> type) {
+    MRPipeline pipeline = (MRPipeline) getPipeline();
+    return parallelDo("S" + pipeline.getNextAnonymousStageId(), fn, type);
+  }
+
+  @Override
+  public <K, V> PTable<K, V> parallelDo(String name, DoFn<S, Pair<K, V>> fn, PTableType<K, V> type) {
+    return new DoTableImpl<K, V>(name, this, fn, type);
+  }
+
+  @Override
+  public PCollection<S> write(Target target) {
+    getPipeline().write(this, target);
+    return this;
+  }
+
+  @Override
+  public Iterable<S> materialize() {
+    if (getSize() == 0) {
+      LOG.warn("Materializing an empty PCollection: " + this.getName());
+      return Collections.emptyList();
+    }
+    return getPipeline().materialize(this);
+  }
+
+  public SourceTarget<S> getMaterializedAt() {
+    return materializedAt;
+  }
+
+  public void materializeAt(SourceTarget<S> sourceTarget) {
+    this.materializedAt = sourceTarget;
+  }
+
+  @Override
+  public PCollection<S> filter(FilterFn<S> filterFn) {
+    return parallelDo(filterFn, getPType());
+  }
+
+  @Override
+  public PCollection<S> filter(String name, FilterFn<S> filterFn) {
+    return parallelDo(name, filterFn, getPType());
+  }
+
+  @Override
+  public <K> PTable<K, S> by(MapFn<S, K> mapFn, PType<K> keyType) {
+    return parallelDo(new ExtractKeyFn<K, S>(mapFn), getTypeFamily().tableOf(keyType, getPType()));
+  }
+
+  @Override
+  public <K> PTable<K, S> by(String name, MapFn<S, K> mapFn, PType<K> keyType) {
+    return parallelDo(name, new ExtractKeyFn<K, S>(mapFn), getTypeFamily().tableOf(keyType, getPType()));
+  }
+
+  @Override
+  public PCollection<S> sort(boolean ascending) {
+    return Sort.sort(this, ascending ? Sort.Order.ASCENDING : Sort.Order.DESCENDING);
+  }
+
+  @Override
+  public PTable<S, Long> count() {
+    return Aggregate.count(this);
+  }
+
+  @Override
+  public PCollection<S> max() {
+    return Aggregate.max(this);
+  }
+
+  @Override
+  public PCollection<S> min() {
+    return Aggregate.min(this);
+  }
+
+  @Override
+  public PCollection<S> sample(double acceptanceProbability) {
+    return Sample.sample(this, acceptanceProbability);
+  }
+
+  @Override
+  public PCollection<S> sample(double acceptanceProbability, long seed) {
+    return Sample.sample(this, seed, acceptanceProbability);
+  }
+
+  @Override
+  public PTypeFamily getTypeFamily() {
+    return getPType().getFamily();
+  }
+
+  public abstract DoNode createDoNode();
+
+  public abstract List<PCollectionImpl<?>> getParents();
+
+  public PCollectionImpl<?> getOnlyParent() {
+    List<PCollectionImpl<?>> parents = getParents();
+    if (parents.size() != 1) {
+      throw new IllegalArgumentException("Expected exactly one parent PCollection");
+    }
+    return parents.get(0);
+  }
+
+  @Override
+  public Pipeline getPipeline() {
+    if (pipeline == null) {
+      pipeline = (MRPipeline) getParents().get(0).getPipeline();
+    }
+    return pipeline;
+  }
+
+  public int getDepth() {
+    int parentMax = 0;
+    for (PCollectionImpl parent : getParents()) {
+      parentMax = Math.max(parent.getDepth(), parentMax);
+    }
+    return 1 + parentMax;
+  }
+
+  public interface Visitor {
+    void visitInputCollection(InputCollection<?> collection);
+
+    void visitUnionCollection(UnionCollection<?> collection);
+
+    void visitDoFnCollection(DoCollectionImpl<?> collection);
+
+    void visitDoTable(DoTableImpl<?, ?> collection);
+
+    void visitGroupedTable(PGroupedTableImpl<?, ?> collection);
+  }
+
+  public void accept(Visitor visitor) {
+    if (materializedAt != null) {
+      visitor.visitInputCollection(new InputCollection<S>(materializedAt,
+          (MRPipeline) getPipeline()));
+    } else {
+      acceptInternal(visitor);
+    }
+  }
+
+  protected abstract void acceptInternal(Visitor visitor);
+
+  @Override
+  public long getSize() {
+    if (materializedAt != null) {
+      long sz = materializedAt.getSize(getPipeline().getConfiguration());
+      if (sz > 0) {
+        return sz;
+      }
+    }
+    return getSizeInternal();
+  }
+
+  protected abstract long getSizeInternal();
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/collect/PGroupedTableImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/collect/PGroupedTableImpl.java b/src/main/java/org/apache/crunch/impl/mr/collect/PGroupedTableImpl.java
new file mode 100644
index 0000000..13e5567
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/collect/PGroupedTableImpl.java
@@ -0,0 +1,120 @@
+/**
+ * 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.crunch.impl.mr.collect;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapreduce.Job;
+
+import org.apache.crunch.CombineFn;
+import org.apache.crunch.DoFn;
+import org.apache.crunch.Emitter;
+import org.apache.crunch.GroupingOptions;
+import org.apache.crunch.PGroupedTable;
+import org.apache.crunch.PTable;
+import org.apache.crunch.Pair;
+import org.apache.crunch.impl.mr.plan.DoNode;
+import org.apache.crunch.types.PGroupedTableType;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.ImmutableList;
+
+public class PGroupedTableImpl<K, V> extends
+    PCollectionImpl<Pair<K, Iterable<V>>> implements PGroupedTable<K, V> {
+
+  private static final Log LOG = LogFactory.getLog(PGroupedTableImpl.class);
+  
+  private final PTableBase<K, V> parent;
+  private final GroupingOptions groupingOptions;
+  private final PGroupedTableType<K, V> ptype;
+
+  PGroupedTableImpl(PTableBase<K, V> parent) {
+    this(parent, null);
+  }
+
+  PGroupedTableImpl(PTableBase<K, V> parent, GroupingOptions groupingOptions) {
+    super("GBK");
+    this.parent = parent;
+    this.groupingOptions = groupingOptions;
+    this.ptype = parent.getPTableType().getGroupedTableType();
+  }
+
+  public void configureShuffle(Job job) {
+    ptype.configureShuffle(job, groupingOptions);
+    if (groupingOptions == null || groupingOptions.getNumReducers() <= 0) {
+      long bytesPerTask = job.getConfiguration().getLong("crunch.bytes.per.reduce.task",
+          (1000L * 1000L * 1000L));
+      int numReduceTasks = 1 + (int) (getSize() / bytesPerTask);
+      if (numReduceTasks > 0) {
+        job.setNumReduceTasks(numReduceTasks);
+        LOG.info(String.format("Setting num reduce tasks to %d", numReduceTasks));
+      } else {
+        LOG.warn("Attempted to set a negative number of reduce tasks");
+      }
+    }
+  }
+  
+  @Override
+  protected long getSizeInternal() {
+    return parent.getSizeInternal();
+  }
+  
+  @Override
+  public PType<Pair<K, Iterable<V>>> getPType() {
+    return ptype;
+  }
+
+  public PTable<K, V> combineValues(CombineFn<K, V> combineFn) {
+    return new DoTableImpl<K, V>("combine", this, combineFn,
+        parent.getPTableType());
+  }
+
+  private static class Ungroup<K, V> extends DoFn<Pair<K, Iterable<V>>, Pair<K, V>> {
+    @Override
+    public void process(Pair<K, Iterable<V>> input, Emitter<Pair<K, V>> emitter) {
+      for (V v : input.second()) {
+        emitter.emit(Pair.of(input.first(), v));
+      }
+    }
+  }
+
+  public PTable<K, V> ungroup() {
+    return parallelDo("ungroup", new Ungroup<K, V>(), parent.getPTableType());
+  }
+
+  @Override
+  protected void acceptInternal(PCollectionImpl.Visitor visitor) {
+    visitor.visitGroupedTable(this);
+  }
+
+  @Override
+  public List<PCollectionImpl<?>> getParents() {
+    return ImmutableList.<PCollectionImpl<?>> of(parent);
+  }
+
+  @Override
+  public DoNode createDoNode() {
+    return DoNode.createFnNode(getName(),
+        ptype.getInputMapFn(), ptype);
+  }
+
+  public DoNode getGroupingNode() {
+    return DoNode.createGroupingNode("", ptype);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/collect/PTableBase.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/collect/PTableBase.java b/src/main/java/org/apache/crunch/impl/mr/collect/PTableBase.java
new file mode 100644
index 0000000..f480001
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/collect/PTableBase.java
@@ -0,0 +1,123 @@
+/**
+ * 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.crunch.impl.mr.collect;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.crunch.GroupingOptions;
+import org.apache.crunch.PCollection;
+import org.apache.crunch.PTable;
+import org.apache.crunch.Pair;
+import org.apache.crunch.Target;
+import org.apache.crunch.lib.Aggregate;
+import org.apache.crunch.lib.Cogroup;
+import org.apache.crunch.lib.Join;
+import org.apache.crunch.lib.PTables;
+import org.apache.crunch.materialize.MaterializableMap;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.Lists;
+
+public abstract class PTableBase<K, V> extends PCollectionImpl<Pair<K, V>>
+    implements PTable<K, V> {
+
+  public PTableBase(String name) {
+    super(name);
+  }
+
+  public PType<K> getKeyType() {
+    return getPTableType().getKeyType();
+  }
+  
+  public PType<V> getValueType() {
+    return getPTableType().getValueType();
+  }
+  
+  public PGroupedTableImpl<K, V> groupByKey() {
+    return new PGroupedTableImpl<K, V>(this);
+  }
+
+  public PGroupedTableImpl<K, V> groupByKey(int numReduceTasks) {
+    return new PGroupedTableImpl<K, V>(this,
+        GroupingOptions.builder().numReducers(numReduceTasks).build());
+  }
+  
+  public PGroupedTableImpl<K, V> groupByKey(GroupingOptions groupingOptions) {
+    return new PGroupedTableImpl<K, V>(this, groupingOptions);
+  }
+
+  @Override
+  public PTable<K, V> union(PTable<K, V>... others) {
+    List<PTableBase<K, V>> internal = Lists.newArrayList();
+    internal.add(this);
+    for (PTable<K, V> table : others) {
+      internal.add((PTableBase<K, V>) table);
+    }
+    return new UnionTable<K, V>(internal);
+  }
+  
+  @Override
+  public PTable<K, V> write(Target target) {
+    getPipeline().write(this, target);
+    return this;
+  }
+  
+  @Override
+  public PTable<K, V> top(int count) {
+	return Aggregate.top(this, count, true);
+  }
+
+  @Override
+  public PTable<K, V> bottom(int count) {
+	return Aggregate.top(this, count, false);
+  }
+  
+  @Override
+  public PTable<K, Collection<V>> collectValues() {
+	return Aggregate.collectValues(this);
+  }
+  
+  @Override
+  public <U> PTable<K, Pair<V, U>> join(PTable<K, U> other) {
+	return Join.join(this, other);
+  }
+  
+  @Override
+  public <U> PTable<K, Pair<Collection<V>, Collection<U>>> cogroup(PTable<K, U> other) {
+	return Cogroup.cogroup(this, other);
+  }
+  
+  @Override
+  public PCollection<K> keys() {
+	return PTables.keys(this);
+  }
+ 
+  @Override
+  public PCollection<V> values() {
+    return PTables.values(this);
+  }
+
+  /**
+   * Returns a Map<K, V> made up of the keys and values in this PTable.
+   */
+  public Map<K, V> materializeToMap() {
+    return new MaterializableMap<K, V>(this.materialize());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/collect/UnionCollection.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/collect/UnionCollection.java b/src/main/java/org/apache/crunch/impl/mr/collect/UnionCollection.java
new file mode 100644
index 0000000..19a1161
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/collect/UnionCollection.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.crunch.impl.mr.collect;
+
+import java.util.List;
+
+import org.apache.crunch.impl.mr.MRPipeline;
+import org.apache.crunch.impl.mr.plan.DoNode;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.ImmutableList;
+
+public class UnionCollection<S> extends PCollectionImpl<S> {
+
+  private List<PCollectionImpl<S>> parents;
+  private long size = 0;
+  
+  private static <S> String flatName(List<PCollectionImpl<S>> collections) {
+    StringBuilder sb = new StringBuilder("union(");
+    for (int i = 0; i < collections.size(); i++) {
+      if (i != 0) {
+        sb.append(',');
+      }
+      sb.append(collections.get(i).getName());
+    }
+    return sb.append(')').toString();
+  }
+  
+  UnionCollection(List<PCollectionImpl<S>> collections) {
+    super(flatName(collections));
+    this.parents = ImmutableList.copyOf(collections);
+    this.pipeline = (MRPipeline) parents.get(0).getPipeline();
+    for (PCollectionImpl<S> parent : parents) {
+      if (this.pipeline != parent.getPipeline()) {
+        throw new IllegalStateException(
+            "Cannot union PCollections from different Pipeline instances");
+      }
+      size += parent.getSize();
+    }
+  }
+
+  @Override
+  protected long getSizeInternal() {
+    return size;
+  }
+  
+  @Override
+  protected void acceptInternal(PCollectionImpl.Visitor visitor) {
+    visitor.visitUnionCollection(this);
+  }
+
+  @Override
+  public PType<S> getPType() {
+    return parents.get(0).getPType();
+  }
+
+  @Override
+  public List<PCollectionImpl<?>> getParents() {
+    return ImmutableList.<PCollectionImpl<?>> copyOf(parents);
+  }
+
+  @Override
+  public DoNode createDoNode() {
+    throw new UnsupportedOperationException(
+        "Unioned collection does not support DoNodes");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/collect/UnionTable.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/collect/UnionTable.java b/src/main/java/org/apache/crunch/impl/mr/collect/UnionTable.java
new file mode 100644
index 0000000..f713912
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/collect/UnionTable.java
@@ -0,0 +1,94 @@
+/**
+ * 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.crunch.impl.mr.collect;
+
+import java.util.List;
+
+import org.apache.crunch.Pair;
+import org.apache.crunch.impl.mr.MRPipeline;
+import org.apache.crunch.impl.mr.plan.DoNode;
+import org.apache.crunch.types.PTableType;
+import org.apache.crunch.types.PType;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public class UnionTable<K, V> extends PTableBase<K, V> {
+
+  private PTableType<K, V> ptype;
+  private List<PCollectionImpl<Pair<K, V>>> parents;
+  private long size;
+  
+  private static <K, V> String flatName(List<PTableBase<K, V>> tables) {
+    StringBuilder sb = new StringBuilder("union(");
+    for (int i = 0; i < tables.size(); i++) {
+      if (i != 0) {
+        sb.append(',');
+      }
+      sb.append(tables.get(i).getName());
+    }
+    return sb.append(')').toString();
+  }
+  
+  public UnionTable(List<PTableBase<K, V>> tables) {
+    super(flatName(tables));
+    this.ptype = tables.get(0).getPTableType();
+    this.pipeline = (MRPipeline) tables.get(0).getPipeline();
+    this.parents = Lists.newArrayList();
+    for (PTableBase<K, V> parent : tables) {
+      if (pipeline != parent.getPipeline()) {
+        throw new IllegalStateException(
+            "Cannot union PTables from different Pipeline instances");
+      }
+      this.parents.add(parent);
+      size += parent.getSize();
+    }
+  }
+
+  @Override
+  protected long getSizeInternal() {
+    return size;
+  }
+  
+  @Override
+  public PTableType<K, V> getPTableType() {
+    return ptype;
+  }
+
+  @Override
+  public PType<Pair<K, V>> getPType() {
+    return ptype;
+  }
+
+  @Override
+  public List<PCollectionImpl<?>> getParents() {
+    return ImmutableList.<PCollectionImpl<?>> copyOf(parents);
+  }
+
+  @Override
+  protected void acceptInternal(PCollectionImpl.Visitor visitor) {
+    visitor.visitUnionCollection(new UnionCollection<Pair<K, V>>(
+        parents));
+  }
+
+  @Override
+  public DoNode createDoNode() {
+    throw new UnsupportedOperationException(
+        "Unioned table does not support do nodes");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/emit/IntermediateEmitter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/emit/IntermediateEmitter.java b/src/main/java/org/apache/crunch/impl/mr/emit/IntermediateEmitter.java
new file mode 100644
index 0000000..242aa4d
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/emit/IntermediateEmitter.java
@@ -0,0 +1,49 @@
+/**
+ * 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.crunch.impl.mr.emit;
+
+import java.util.List;
+
+import org.apache.crunch.DoFn;
+import org.apache.crunch.Emitter;
+import org.apache.crunch.impl.mr.run.RTNode;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * An {@link Emitter} implementation that links the output of one {@link DoFn}
+ * to the input of another {@code DoFn}.
+ * 
+ */
+public class IntermediateEmitter implements Emitter<Object> {
+
+  private final List<RTNode> children;
+
+  public IntermediateEmitter(List<RTNode> children) {
+    this.children = ImmutableList.copyOf(children);
+  }
+
+  public void emit(Object emitted) {
+    for (RTNode child : children) {
+      child.process(emitted);
+    }
+  }
+
+  public void flush() {
+    // No-op
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/emit/MultipleOutputEmitter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/emit/MultipleOutputEmitter.java b/src/main/java/org/apache/crunch/impl/mr/emit/MultipleOutputEmitter.java
new file mode 100644
index 0000000..5f52f41
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/emit/MultipleOutputEmitter.java
@@ -0,0 +1,58 @@
+/**
+ * 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.crunch.impl.mr.emit;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.lib.output.CrunchMultipleOutputs;
+
+import org.apache.crunch.Emitter;
+import org.apache.crunch.impl.mr.run.CrunchRuntimeException;
+import org.apache.crunch.types.Converter;
+
+public class MultipleOutputEmitter<T, K, V> implements Emitter<T> {
+
+  private final Converter converter;
+  private final CrunchMultipleOutputs<K, V> outputs;
+  private final String outputName;
+
+  public MultipleOutputEmitter(Converter converter,
+      CrunchMultipleOutputs<K, V> outputs, String outputName) {
+    this.converter = converter;
+    this.outputs = outputs;
+    this.outputName = outputName;
+  }
+
+  @Override
+  public void emit(T emitted) {
+    try {
+      this.outputs.write(outputName, converter.outputKey(emitted),
+          converter.outputValue(emitted));
+    } catch (IOException e) {
+      throw new CrunchRuntimeException(e);
+    } catch (InterruptedException e) {
+      throw new CrunchRuntimeException(e);
+    }
+  }
+
+  @Override
+  public void flush() {
+    // No-op
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/emit/OutputEmitter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/emit/OutputEmitter.java b/src/main/java/org/apache/crunch/impl/mr/emit/OutputEmitter.java
new file mode 100644
index 0000000..deb090c
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/emit/OutputEmitter.java
@@ -0,0 +1,54 @@
+/**
+ * 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.crunch.impl.mr.emit;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.TaskInputOutputContext;
+
+import org.apache.crunch.Emitter;
+import org.apache.crunch.impl.mr.run.CrunchRuntimeException;
+import org.apache.crunch.types.Converter;
+
+public class OutputEmitter<T, K, V> implements Emitter<T> {
+
+  private final Converter<K, V, Object, Object> converter;
+  private final TaskInputOutputContext<?, ?, K, V> context;
+
+  public OutputEmitter(Converter<K, V, Object, Object> converter,
+      TaskInputOutputContext<?, ?, K, V> context) {
+    this.converter = converter;
+    this.context = context;
+  }
+
+  public void emit(T emitted) {
+    try {
+      K key = converter.outputKey(emitted);
+      V value = converter.outputValue(emitted);
+      this.context.write(key, value);
+    } catch (IOException e) {
+      throw new CrunchRuntimeException(e);
+    } catch (InterruptedException e) {
+      throw new CrunchRuntimeException(e);
+    }
+  }
+
+  public void flush() {
+    // No-op
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/exec/CrunchJob.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/exec/CrunchJob.java b/src/main/java/org/apache/crunch/impl/mr/exec/CrunchJob.java
new file mode 100644
index 0000000..c790c1d
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/exec/CrunchJob.java
@@ -0,0 +1,120 @@
+/**
+ * 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.crunch.impl.mr.exec;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.jobcontrol.CrunchControlledJob;
+import org.apache.hadoop.util.StringUtils;
+
+import org.apache.crunch.impl.mr.plan.MSCROutputHandler;
+import org.apache.crunch.impl.mr.plan.PlanningParameters;
+import com.google.common.collect.Lists;
+
+public class CrunchJob extends CrunchControlledJob {
+
+  private final Log log = LogFactory.getLog(CrunchJob.class);
+  
+  private final Path workingPath;
+  private final List<Path> multiPaths;
+  private final boolean mapOnlyJob;
+  
+  public CrunchJob(Job job, Path workingPath, MSCROutputHandler handler) throws IOException {
+    super(job, Lists.<CrunchControlledJob>newArrayList());
+    this.workingPath = workingPath;
+    this.multiPaths = handler.getMultiPaths();
+    this.mapOnlyJob = handler.isMapOnlyJob();
+  }  
+  
+  private synchronized void handleMultiPaths() throws IOException {
+    if (!multiPaths.isEmpty()) {
+      // Need to handle moving the data from the output directory of the
+      // job to the output locations specified in the paths.
+      FileSystem fs = FileSystem.get(job.getConfiguration());
+      for (int i = 0; i < multiPaths.size(); i++) {
+        Path src = new Path(workingPath,
+            PlanningParameters.MULTI_OUTPUT_PREFIX + i + "-*");
+        Path[] srcs = FileUtil.stat2Paths(fs.globStatus(src), src);
+        Path dst = multiPaths.get(i);
+        if (!fs.exists(dst)) {
+          fs.mkdirs(dst);
+        }
+        int minPartIndex = getMinPartIndex(dst, fs);
+        for (Path s : srcs) {
+          fs.rename(s, getDestFile(s, dst, minPartIndex++));
+        }
+      }
+    }
+  }
+  
+  private Path getDestFile(Path src, Path dir, int index) {
+    String form = "part-%s-%05d";
+    if (src.getName().endsWith(org.apache.avro.mapred.AvroOutputFormat.EXT)) {
+      form = form + org.apache.avro.mapred.AvroOutputFormat.EXT;
+    }
+    return new Path(dir, String.format(form, mapOnlyJob ? "m" : "r", index));
+  }
+  
+  private int getMinPartIndex(Path path, FileSystem fs) throws IOException {
+    // Quick and dirty way to ensure unique naming in the directory
+    return fs.listStatus(path).length;
+  }
+  
+  @Override
+  protected void checkRunningState() throws IOException, InterruptedException {
+    try {
+      if (job.isComplete()) {
+        if (job.isSuccessful()) {
+          handleMultiPaths();
+          this.state = State.SUCCESS;
+        } else {
+          this.state = State.FAILED;
+          this.message = "Job failed!";
+        }
+      }
+    } catch (IOException ioe) {
+      this.state = State.FAILED;
+      this.message = StringUtils.stringifyException(ioe);
+      try {
+        if (job != null) {
+          job.killJob();
+        }
+      } catch (IOException e) {
+      }
+    }
+  }
+
+  @Override
+  protected synchronized void submit() {
+    super.submit();
+    if (this.state == State.RUNNING) {
+      log.info("Running job \"" + getJobName() + "\"");
+      log.info("Job status available at: " + job.getTrackingURL());
+    } else {
+      log.info("Error occurred starting job \"" + getJobName() + "\":");
+      log.info(getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/5accc9ac/src/main/java/org/apache/crunch/impl/mr/exec/MRExecutor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/crunch/impl/mr/exec/MRExecutor.java b/src/main/java/org/apache/crunch/impl/mr/exec/MRExecutor.java
new file mode 100644
index 0000000..b678187
--- /dev/null
+++ b/src/main/java/org/apache/crunch/impl/mr/exec/MRExecutor.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.crunch.impl.mr.exec;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapreduce.lib.jobcontrol.CrunchControlledJob;
+import org.apache.hadoop.mapreduce.lib.jobcontrol.CrunchJobControl;
+
+import org.apache.crunch.PipelineResult;
+import com.google.common.collect.Lists;
+
+/**
+ *
+ *
+ */
+public class MRExecutor {
+
+  private static final Log LOG = LogFactory.getLog(MRExecutor.class);
+
+  private final CrunchJobControl control;
+
+  public MRExecutor(Class<?> jarClass) {
+    this.control = new CrunchJobControl(jarClass.toString());
+  }
+
+  public void addJob(CrunchJob job) {
+    this.control.addJob(job);
+  }
+
+  public PipelineResult execute() {
+    try {
+      Thread controlThread = new Thread(control);
+      controlThread.start();
+      while (!control.allFinished()) {
+        Thread.sleep(1000);
+      }
+      control.stop();
+    } catch (InterruptedException e) {
+      LOG.info(e);
+    }
+    List<CrunchControlledJob> failures = control.getFailedJobList();
+    if (!failures.isEmpty()) {
+      System.err.println(failures.size() + " job failure(s) occurred:");
+      for (CrunchControlledJob job : failures) {
+        System.err.println(job.getJobName() + "(" + job.getJobID() + "): " + job.getMessage());
+      }
+    }
+    List<PipelineResult.StageResult> stages = Lists.newArrayList();
+    for (CrunchControlledJob job : control.getSuccessfulJobList()) {
+      try {
+        stages.add(new PipelineResult.StageResult(job.getJobName(), job.getJob().getCounters()));
+      } catch (Exception e) {
+        LOG.error("Exception thrown fetching job counters for stage: " + job.getJobName(), e);
+      }
+    }
+    return new PipelineResult(stages);
+  }
+}


Mime
View raw message