incubator-crunch-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jwi...@apache.org
Subject git commit: CRUNCH-25: Follow incubator rules.
Date Thu, 02 Aug 2012 22:09:40 GMT
Updated Branches:
  refs/heads/master 0e663d406 -> fbc741f10


CRUNCH-25: Follow incubator rules.

Make site compliant with incubator website and branding rules.
Split documentation into smaller parts.
Disable module sites and reports until we have more content.
Change version number according to incubator rules.

Signed-off-by: jwills <jwills@apache.org>


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

Branch: refs/heads/master
Commit: fbc741f102c1103fca3bcb3756eb096d5d4bcd67
Parents: 0e663d4
Author: Matthias Friedrich <matt@mafr.de>
Authored: Thu Aug 2 18:58:53 2012 +0200
Committer: jwills <jwills@apache.org>
Committed: Thu Aug 2 14:25:09 2012 -0700

----------------------------------------------------------------------
 crunch-test/pom.xml                      |    2 +-
 crunch/pom.xml                           |    2 +-
 crunch/src/site/markdown/index.md        |  289 +-----------------------
 crunch/src/site/resources/css/site.css   |   34 ---
 examples/pom.xml                         |    2 +-
 examples/src/site/markdown/index.md      |    2 +
 examples/src/site/resources/css/site.css |   34 ---
 pom.xml                                  |   22 ++-
 scrunch/pom.xml                          |    2 +-
 scrunch/src/site/markdown/index.md       |   73 +------
 scrunch/src/site/resources/css/site.css  |   34 ---
 src/site/markdown/future-work.md         |   29 +++
 src/site/markdown/index.md               |  304 ++-----------------------
 src/site/markdown/intro.md               |  214 +++++++++++++++++
 src/site/markdown/pipelines.md           |  103 +++++++++
 src/site/markdown/scrunch.md             |   90 ++++++++
 src/site/resources/css/site.css          |   34 ---
 src/site/site.xml                        |   29 +--
 18 files changed, 503 insertions(+), 796 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/crunch-test/pom.xml
----------------------------------------------------------------------
diff --git a/crunch-test/pom.xml b/crunch-test/pom.xml
index 45384bc..5874af8 100644
--- a/crunch-test/pom.xml
+++ b/crunch-test/pom.xml
@@ -23,7 +23,7 @@ under the License.
   <parent>
     <groupId>org.apache.crunch</groupId>
     <artifactId>crunch-parent</artifactId>
-    <version>0.3.0-SNAPSHOT</version>
+    <version>0.3.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>crunch-test</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/crunch/pom.xml
----------------------------------------------------------------------
diff --git a/crunch/pom.xml b/crunch/pom.xml
index 8b7b5e7..6368673 100644
--- a/crunch/pom.xml
+++ b/crunch/pom.xml
@@ -23,7 +23,7 @@ under the License.
   <parent>
     <groupId>org.apache.crunch</groupId>
     <artifactId>crunch-parent</artifactId>
-    <version>0.3.0-SNAPSHOT</version>
+    <version>0.3.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>crunch</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/crunch/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/crunch/src/site/markdown/index.md b/crunch/src/site/markdown/index.md
index ad1d529..a89c404 100644
--- a/crunch/src/site/markdown/index.md
+++ b/crunch/src/site/markdown/index.md
@@ -16,290 +16,5 @@ KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
 -->
-# Crunch - Simple and Efficient Java Library for MapReduce Pipelines
-
-## Introduction
-
-Crunch is a Java library for writing, testing, and running MapReduce pipelines, based on
-Google\'s FlumeJava. Its goal is to make pipelines that are composed of many user-defined
-functions simple to write, easy to test, and efficient to run.
-
-## Build and Installation
-
-Crunch uses Maven for dependency management. The code in the examples/ subdirectory relies
-on the top-level crunch libraries. In order to execute the included WordCount application, run:
-
-    mvn clean install
-    hadoop jar examples/target/crunch-examples-*-job.jar org.apache.crunch.examples.WordCount <inputfile> <outputdir>
-
-## High Level Concepts
-
-### Data Model and Operators
-
-Crunch is centered around three interfaces that represent distributed datasets: `PCollection<T>`, `PTable<K, V>`, and `PGroupedTable<K, V>`.
-
-A `PCollection<T>` represents a distributed, unordered collection of elements of type T. For example, we represent a text file in Crunch as a
-`PCollection<String>` object. PCollection provides a method, `parallelDo`, that applies a function to each element in a PCollection in parallel,
-and returns a new PCollection as its result.
-
-A `PTable<K, V>` is a sub-interface of PCollection that represents a distributed, unordered multimap of its key type K to its value type V.
-In addition to the parallelDo operation, PTable provides a `groupByKey` operation that aggregates all of the values in the PTable that
-have the same key into a single record. It is the groupByKey operation that triggers the sort phase of a MapReduce job.
-
-The result of a groupByKey operation is a `PGroupedTable<K, V>` object, which is a distributed, sorted map of keys of type K to an Iterable
-collection of values of type V. In addition to parallelDo, the PGroupedTable provides a `combineValues` operation, which allows for
-a commutative and associative aggregation operator to be applied to the values of the PGroupedTable instance on both the map side and the
-reduce side of a MapReduce job.
-
-Finally, PCollection, PTable, and PGroupedTable all support a `union` operation, which takes a series of distinct PCollections and treats
-them as a single, virtual PCollection. The union operator is required for operations that combine multiple inputs, such as cogroups and
-joins.
-
-### Pipeline Building and Execution
-
-Every Crunch pipeline starts with a `Pipeline` object that is used to coordinate building the pipeline and executing the underlying MapReduce
-jobs. For efficiency, Crunch uses lazy evaluation, so it will only construct MapReduce jobs from the different stages of the pipelines when
-the Pipeline object\'s `run` or `done` methods are called.
-
-## A Detailed Example
-
-Here is the classic WordCount application using Crunch:
-
-    import org.apache.crunch.DoFn;
-    import org.apache.crunch.Emitter;
-    import org.apache.crunch.PCollection;
-    import org.apache.crunch.PTable;
-    import org.apache.crunch.Pipeline;
-    import org.apache.crunch.impl.mr.MRPipeline;
-    import org.apache.crunch.lib.Aggregate;
-    import org.apache.crunch.types.writable.Writables;
-
-    public class WordCount {
-      public static void main(String[] args) throws Exception {
-        Pipeline pipeline = new MRPipeline(WordCount.class);    
-        PCollection<String> lines = pipeline.readTextFile(args[0]);
-    
-        PCollection<String> words = lines.parallelDo("my splitter", new DoFn<String, String>() {
-          public void process(String line, Emitter<String> emitter) {
-            for (String word : line.split("\\s+")) {
-              emitter.emit(word);
-            }
-          }
-        }, Writables.strings());
-
-        PTable<String, Long> counts = Aggregate.count(words);
-    
-        pipeline.writeTextFile(counts, args[1]);
-        pipeline.run();
-      }
-    }
-
-Let\'s walk through the example line by line.
-
-### Step 1: Creating a Pipeline and referencing a text file
-
-The `MRPipeline` implementation of the Pipeline interface compiles the individual stages of a
-pipeline into a series of MapReduce jobs. The MRPipeline constructor takes a class argument
-that is used to tell Hadoop where to find the code that is used in the pipeline execution.
-
-We now need to tell the Pipeline about the inputs it will be consuming. The Pipeline interface
-defines a `readTextFile` method that takes in a String and returns a PCollection of Strings.
-In addition to text files, Crunch supports reading data from SequenceFiles and Avro container files,
-via the `SequenceFileSource` and `AvroFileSource` classes defined in the org.apache.crunch.io package.
-
-Note that each PCollection is a _reference_ to a source of data- no data is actually loaded into a
-PCollection on the client machine.
-
-### Step 2: Splitting the lines of text into words
-
-Crunch defines a small set of primitive operations that can be composed in order to build complex data
-pipelines. The first of these primitives is the `parallelDo` function, which applies a function (defined
-by a subclass of `DoFn`) to every record in a PCollection, and returns a new PCollection that contains
-the results.
-
-The first argument to parallelDo is a string that is used to identify this step in the pipeline. When
-a pipeline is composed into a series of MapReduce jobs, it is often the case that multiple stages will
-run within the same Mapper or Reducer. Having a string that identifies each processing step is useful
-for debugging errors that occur in a running pipeline.
-
-The second argument to parallelDo is an anonymous subclass of DoFn. Each DoFn subclass must override
-the `process` method, which takes in a record from the input PCollection and an `Emitter` object that
-may have any number of output values written to it. In this case, our DoFn splits each lines up into
-words, using a blank space as a separator, and emits the words from the split to the output PCollection.
-
-The last argument to parallelDo is an instance of the `PType` interface, which specifies how the data
-in the output PCollection is serialized. While Crunch takes advantage of Java Generics to provide
-compile-time type safety, the generic type information is not available at runtime. Crunch needs to know
-how to map the records stored in each PCollection into a Hadoop-supported serialization format in order
-to read and write data to disk. Two serialization implementations are supported in crunch via the
-`PTypeFamily` interface: a Writable-based system that is defined in the org.apache.crunch.types.writable
-package, and an Avro-based system that is defined in the org.apache.crunch.types.avro package. Each
-implementation provides convenience methods for working with the common PTypes (Strings, longs, bytes, etc.)
-as well as utility methods for creating PTypes from existing Writable classes or Avro schemas.
-
-### Step 3: Counting the words
-
-Out of Crunch\'s simple primitive operations, we can build arbitrarily complex chains of operations in order
-to perform higher-level operations, like aggregations and joins, that can work on any type of input data.
-Let\'s look at the implementation of the `Aggregate.count` function:
-
-    package org.apache.crunch.lib;
-
-    import org.apache.crunch.CombineFn;
-    import org.apache.crunch.MapFn;
-    import org.apache.crunch.PCollection;
-    import org.apache.crunch.PGroupedTable;
-    import org.apache.crunch.PTable;
-    import org.apache.crunch.Pair;
-    import org.apache.crunch.types.PTypeFamily;
-    
-    public class Aggregate {
-    
-      private static class Counter<S> extends MapFn<S, Pair<S, Long>> {
-        public Pair<S, Long> map(S input) {
-              return Pair.of(input, 1L);
-        }
-      }
-    
-      public static <S> PTable<S, Long> count(PCollection<S> collect) {
-        PTypeFamily tf = collect.getTypeFamily();
-    
-        // Create a PTable from the PCollection by mapping each element
-        // to a key of the PTable with the value equal to 1L
-        PTable<S, Long> withCounts = collect.parallelDo("count:" + collect.getName(),
-            new Counter<S>(), tf.tableOf(collect.getPType(), tf.longs()));
-    
-        // Group the records of the PTable based on their key.
-        PGroupedTable<S, Long> grouped = withCounts.groupByKey();
-    
-        // Sum the 1L values associated with the keys to get the
-        // count of each element in this PCollection, and return it
-        // as a PTable so that it may be processed further or written
-        // out for storage.
-        return grouped.combineValues(CombineFn.<S>SUM_LONGS());
-      }
-    }
-
-First, we get the PTypeFamily that is associated with the PType for the collection. The
-call to parallelDo converts each record in this PCollection into a Pair of the input record
-and the number one by extending the `MapFn` convenience subclass of DoFn, and uses the
-`tableOf` method of the PTypeFamily to specify that the returned PCollection should be a
-PTable instance, with the key being the PType of the PCollection and the value being the Long
-implementation for this PTypeFamily.
-
-The next line features the second of Crunch\'s four operations, `groupByKey`. The groupByKey
-operation may only be applied to a PTable, and returns an instance of the `PGroupedTable`
-interface, which references the grouping of all of the values in the PTable that have the same key.
-The groupByKey operation is what triggers the reduce phase of a MapReduce within Crunch.
-
-The last line in the function returns the output of the third of Crunch\'s four operations,
-`combineValues`. The combineValues operator takes a `CombineFn` as an argument, which is a
-specialized subclass of DoFn that operates on an implementation of Java\'s Iterable interface. The
-use of combineValues (as opposed to parallelDo) signals to Crunch that the CombineFn may be used to
-aggregate values for the same key on the map side of a MapReduce job as well as the reduce side.
-
-### Step 4: Writing the output and running the pipeline
-
-The Pipeline object also provides a `writeTextFile` convenience method for indicating that a
-PCollection should be written to a text file. There are also output targets for SequenceFiles and
-Avro container files, available in the org.apache.crunch.io package.
-
-After you are finished constructing a pipeline and specifying the output destinations, call the
-pipeline\'s blocking `run` method in order to compile the pipeline into one or more MapReduce
-jobs and execute them.
-
-## Writing Your Own Pipelines
-
-This section discusses the different steps of creating your own Crunch pipelines in more detail.
-
-### Writing a DoFn
-
-The DoFn class is designed to keep the complexity of the MapReduce APIs out of your way when you
-don\'t need them while still keeping them accessible when you do.
-
-#### Serialization
-
-First, all DoFn instances are required to be `java.io.Serializable`. This is a key aspect of Crunch\'s design:
-once a particular DoFn is assigned to the Map or Reduce stage of a MapReduce job, all of the state
-of that DoFn is serialized so that it may be distributed to all of the nodes in the Hadoop cluster that
-will be running that task. There are two important implications of this for developers:
-
-1. All member values of a DoFn must be either serializable or marked as `transient`.
-2. All anonymous DoFn instances must be defined in a static method or in a class that is itself serializable.
-
-Because sometimes you will need to work with non-serializable objects inside of a DoFn, every DoFn provides an
-`initialize` method that is called before the `process` method is ever called so that any initialization tasks,
-such as creating a non-serializable member variable, can be performed before processing begins. Similarly, all
-DoFn instances have a `cleanup` method that may be called after processing has finished to perform any required
-cleanup tasks.
-
-#### Scale Factor
-
-The DoFn class defines a `scaleFactor` method that can be used to signal to the MapReduce compiler that a particular
-DoFn implementation will yield an output PCollection that is larger (scaleFactor > 1) or smaller (0 < scaleFactor < 1)
-than the input PCollection it is applied to. The compiler may use this information to determine how to optimally
-split processing tasks between the Map and Reduce phases of dependent MapReduce jobs.
-
-#### Other Utilities
-
-The DoFn base class provides convenience methods for accessing the `Configuration` and `Counter` objects that
-are associated with a MapReduce stage, so that they may be accessed during initialization, processing, and cleanup.
-
-### Performing Cogroups and Joins
-
-In Crunch, cogroups and joins are performed on PTable instances that have the same key type. This section walks through
-the basic flow of a cogroup operation, explaining how this higher-level operation is composed of Crunch\'s four primitives.
-In general, these common operations are provided as part of the core Crunch library or in extensions, you do not need
-to write them yourself. But it can be useful to understand how they work under the covers.
-
-Assume we have a `PTable<K, U>` named "a" and a different `PTable<K, V>` named "b" that we would like to combine into a
-single `PTable<K, Pair<Collection<U>, Collection<V>>>`. First, we need to apply parallelDo operations to a and b that
-convert them into the same Crunch type, `PTable<K, Pair<U, V>>`:
-
-    // Perform the "tagging" operation as a parallelDo on PTable a
-    PTable<K, Pair<U, V>> aPrime = a.parallelDo("taga", new MapFn<Pair<K, U>, Pair<K, Pair<U, V>>>() {
-      public Pair<K, Pair<U, V>> map(Pair<K, U> input) {
-        return Pair.of(input.first(), Pair.of(input.second(), null));
-      }
-    }, tableOf(a.getKeyType(), pair(a.getValueType(), b.getValueType())));
-    
-    // Perform the "tagging" operation as a parallelDo on PTable b
-    PTable<K, Pair<U, V>> bPrime = b.parallelDo("tagb", new MapFn<Pair<K, V>, Pair<K, Pair<U, V>>>() {
-      public Pair<K, Pair<U, V>> map(Pair<K, V> input) {
-        return Pair.of(input.first(), Pair.of(null, input.second()));
-      }
-    }, tableOf(a.getKeyType(), pair(a.getValueType(), b.getValueType())));
-
-Once the input PTables are tagged into a single type, we can apply the union operation to create a single PTable
-reference that includes both of the tagged PTables and then group the unioned PTable by the common key:
-
-    PTable<K, Pair<U, V>> both = aPrime.union(bPrime);
-    PGroupedTable<K, Pair<U, V>> grouped = both.groupByKey();
-
-The grouping operation will create an `Iterable<Pair<U, V>>` which we can then convert to a `Pair<Collection<U>, Collection<V>>`:
-
-    grouped.parallelDo("cogroup", new MapFn<Pair<K, Iterable<Pair<U, V>>>, Pair<K, Pair<Collection<U>, Collection<V>>>>() {
-      public Pair<K, Pair<Collection<U>, Collection<V>>> map(Pair<K, Iterable<Pair<U, V>>> input) {
-        Collection<U> uValues = new ArrayList<U>();
-        Collection<V> vValues = new ArrayList<V>();
-        for (Pair<U, V> pair : input.second()) {
-          if (pair.first() != null) {
-            uValues.add(pair.first());
-          } else {
-            vValues.add(pair.second());
-          }
-        }
-        return Pair.of(input.first(), Pair.of(uValues, vValues));
-      },
-    }, tableOf(grouped.getKeyType(), pair(collections(a.getValueType()), collections(b.getValueType()))));
-
-## Current Limitations and Future Work
-
-This section contains an almost certainly incomplete list of known limitations of Crunch and plans for future work.
-
-* We would like to have easy support for reading and writing data from/to HCatalog.
-* The decision of how to split up processing tasks between dependent MapReduce jobs is very naiive right now- we simply
-delegate all of the work to the reduce stage of the predecessor job. We should take advantage of information about the
-expected size of different PCollections to optimize this processing.
-* The Crunch optimizer does not yet merge different groupByKey operations that run over the same input data into a single
-MapReduce job. Implementing this optimization will provide a major performance benefit for a number of problems.
+# Apache Crunch - Core
+---

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/crunch/src/site/resources/css/site.css
----------------------------------------------------------------------
diff --git a/crunch/src/site/resources/css/site.css b/crunch/src/site/resources/css/site.css
deleted file mode 100644
index 8ee551e..0000000
--- a/crunch/src/site/resources/css/site.css
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-/* Borrowed from Apache Whirr */
-
-#banner {
-  height: 93px;
-  background: none;
-}
-
-#bannerLeft img {
-  height: 90px;
-  margin-left: 30px;
-  margin-top: 4px;
-}
-
-#bannerRight img {
-  margin: 17px;
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index d5fa2a0..4b3113f 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -23,7 +23,7 @@ under the License.
   <parent>
     <groupId>org.apache.crunch</groupId>
     <artifactId>crunch-parent</artifactId>
-    <version>0.3.0-SNAPSHOT</version>
+    <version>0.3.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>crunch-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/examples/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/examples/src/site/markdown/index.md b/examples/src/site/markdown/index.md
index 2011a0f..838e3ae 100644
--- a/examples/src/site/markdown/index.md
+++ b/examples/src/site/markdown/index.md
@@ -16,3 +16,5 @@ KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
 -->
+# Apache Crunch - Examples
+---

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/examples/src/site/resources/css/site.css
----------------------------------------------------------------------
diff --git a/examples/src/site/resources/css/site.css b/examples/src/site/resources/css/site.css
deleted file mode 100644
index 8ee551e..0000000
--- a/examples/src/site/resources/css/site.css
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-/* Borrowed from Apache Whirr */
-
-#banner {
-  height: 93px;
-  background: none;
-}
-
-#bannerLeft img {
-  height: 90px;
-  margin-left: 30px;
-  margin-top: 4px;
-}
-
-#bannerRight img {
-  margin: 17px;
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 70b1569..498fc6c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -22,7 +22,7 @@ under the License.
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.crunch</groupId>
   <artifactId>crunch-parent</artifactId>
-  <version>0.3.0-SNAPSHOT</version>
+  <version>0.3.0-incubating-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <name>Apache Crunch</name>
@@ -366,6 +366,24 @@ under the License.
           </reportPlugins>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-javadoc-plugin</artifactId>
+        <version>2.8.1</version>
+        <configuration>
+          <show>public</show>
+          <nohelp>true</nohelp>
+        </configuration>
+        <executions>
+          <execution>
+            <id>aggregate</id>
+            <goals>
+              <goal>aggregate</goal>
+            </goals>
+            <phase>site</phase>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
 
     <pluginManagement>
@@ -576,7 +594,7 @@ under the License.
                  <reportSets><reportSet>
                     <id>generate-javadoc</id>
                     <reports>
-                       <report>javadoc</report>
+                       <report>aggregate</report>
                   </reports>
                 </reportSet></reportSets>
             </plugin>

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/scrunch/pom.xml
----------------------------------------------------------------------
diff --git a/scrunch/pom.xml b/scrunch/pom.xml
index 33e4119..532b62a 100644
--- a/scrunch/pom.xml
+++ b/scrunch/pom.xml
@@ -23,7 +23,7 @@ under the License.
   <parent>
     <groupId>org.apache.crunch</groupId>
     <artifactId>crunch-parent</artifactId>
-    <version>0.3.0-SNAPSHOT</version>
+    <version>0.3.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>scrunch</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/scrunch/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/scrunch/src/site/markdown/index.md b/scrunch/src/site/markdown/index.md
index 704dfcc..32a9279 100644
--- a/scrunch/src/site/markdown/index.md
+++ b/scrunch/src/site/markdown/index.md
@@ -16,74 +16,5 @@ KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
 -->
-## Scrunch - A Scala Wrapper for Crunch
-
-## Introduction
-
-Scrunch is an experimental Scala wrapper for Crunch, based on the same ideas as the
-[Cascade](http://days2011.scala-lang.org/node/138/282) project at Google, which created
-a Scala wrapper for FlumeJava.
-
-## Why Scala?
-
-In many ways, Scala is the perfect language for writing Crunch pipelines. Scala supports
-a mixture of functional and object-oriented programming styles and has powerful type-inference
-capabilities, allowing us to create complex pipelines using very few keystrokes. Here is
-the Scrunch analogue of the classic WordCount problem:
-
-	import com.cloudera.crunch.io.{From => from}
-	import com.cloudera.scrunch._
-	import com.cloudera.scrunch.Conversions_  # For implicit type conversions
-
-	class WordCountExample {
-	  val pipeline = new Pipeline[WordCountExample]
-
-	  def wordCount(fileName: String) = {
-	    pipeline.read(from.textFile(fileName))
-	      .flatMap(_.toLowerCase.split("\\W+"))
-	      .filter(!_.isEmpty())
-	      .count
-	  }
-	}
-
-The Scala compiler can infer the return type of the flatMap function as an Array[String], and
-the Scrunch wrapper uses the type inference mechanism to figure out how to serialize the
-data between the Map and Reduce stages. Here's a slightly more complex example, in which we
-get the word counts for two different files and compute the deltas of how often different
-words occur, and then only returns the words where the first file had more occurrences then
-the second:
-
-	class WordCountExample {
-	  def wordGt(firstFile: String, secondFile: String) = {
-	    wordCount(firstFile).cogroup(wordCount(secondFile))
-	      .map((k, v) => (k, (v._1.sum - v._2.sum)))
-	      .filter((k, v) => v > 0).map((k, v) => k)
-	  }
-	}
-
-Note that all of the functions are using Scala Tuples, not Crunch Tuples. Under the covers,
-Scrunch uses Scala's implicit type conversion mechanism to transparently convert data from the
-Crunch format to the Scala format and back again.
-
-## Materializing Job Outputs
-
-Scrunch also incorporates Crunch's materialize functionality, which allows us to easily read
-the output of a Crunch pipeline into the client:
-
-	class WordCountExample {
-	  def hasHamlet = wordGt("shakespeare.txt", "maugham.txt").materialize.exists(_ == "hamlet")
-	}
-
-## Notes and Thanks
-
-Scrunch is alpha-quality code, written by someone who was learning Scala on the fly. There will be bugs,
-rough edges, and non-idiomatic Scala usage all over the place. This will improve with time, and we welcome
-contributions from Scala experts who are interested in helping us make Scrunch into a first-class project.
-The Crunch developers mailing list is [here](https://groups.google.com/a/cloudera.org/group/crunch-dev/topics).
-
-Scrunch emerged out of conversations with [Dmitriy Ryaboy](http://twitter.com/#!/squarecog),
-[Oscar Boykin](http://twitter.com/#!/posco), and [Avi Bryant](http://twitter.com/#!/avibryant) from Twitter.
-Many thanks to them for their feedback, guidance, and encouragement. We are also grateful to
-[Matei Zaharia](http://twitter.com/#!/matei_zaharia), whose [Spark Project](http://www.spark-project.org/)
-inspired much of our implementation and was kind enough to loan us the ClosureCleaner implementation
-Spark developed for use in Scrunch.
+# Scrunch - A Scala Wrapper for Crunch
+---

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/scrunch/src/site/resources/css/site.css
----------------------------------------------------------------------
diff --git a/scrunch/src/site/resources/css/site.css b/scrunch/src/site/resources/css/site.css
deleted file mode 100644
index 8ee551e..0000000
--- a/scrunch/src/site/resources/css/site.css
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-/* Borrowed from Apache Whirr */
-
-#banner {
-  height: 93px;
-  background: none;
-}
-
-#bannerLeft img {
-  height: 90px;
-  margin-left: 30px;
-  margin-top: 4px;
-}
-
-#bannerRight img {
-  margin: 17px;
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/src/site/markdown/future-work.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/future-work.md b/src/site/markdown/future-work.md
new file mode 100644
index 0000000..bf1e16f
--- /dev/null
+++ b/src/site/markdown/future-work.md
@@ -0,0 +1,29 @@
+<!--
+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.
+-->
+# Current Limitations and Future Work
+---
+
+This section contains an almost certainly incomplete list of known limitations of Crunch and plans for future work.
+
+* We would like to have easy support for reading and writing data from/to HCatalog.
+* The decision of how to split up processing tasks between dependent MapReduce jobs is very naiive right now- we simply
+delegate all of the work to the reduce stage of the predecessor job. We should take advantage of information about the
+expected size of different PCollections to optimize this processing.
+* The Crunch optimizer does not yet merge different groupByKey operations that run over the same input data into a single
+MapReduce job. Implementing this optimization will provide a major performance benefit for a number of problems.

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/index.md b/src/site/markdown/index.md
index ad1d529..f5862ce 100644
--- a/src/site/markdown/index.md
+++ b/src/site/markdown/index.md
@@ -16,290 +16,34 @@ KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
 -->
-# Crunch - Simple and Efficient Java Library for MapReduce Pipelines
+# Apache Crunch - Simple and Efficient MapReduce Pipelines
+---
 
-## Introduction
+> *Apache Crunch (incubating)* is a Java library for writing, testing, and
+> running MapReduce pipelines, based on Google\'s FlumeJava. Its goal is to make
+> pipelines that are composed of many user-defined functions simple to write,
+> easy to test, and efficient to run.
 
-Crunch is a Java library for writing, testing, and running MapReduce pipelines, based on
-Google\'s FlumeJava. Its goal is to make pipelines that are composed of many user-defined
-functions simple to write, easy to test, and efficient to run.
+---
 
-## Build and Installation
+Running on top of [Hadoop MapReduce](http://hadoop.apache.org/mapreduce/), Apache
+Crunch provides a simple Java API for tasks like joining and data aggregation
+that are tedious to implement on plain MapReduce. For Scala users, there is also
+Scrunch, an idiomatic Scala API to Crunch.
 
-Crunch uses Maven for dependency management. The code in the examples/ subdirectory relies
-on the top-level crunch libraries. In order to execute the included WordCount application, run:
+## Documentation
 
-    mvn clean install
-    hadoop jar examples/target/crunch-examples-*-job.jar org.apache.crunch.examples.WordCount <inputfile> <outputdir>
+  * [Introduction to Apache Crunch](intro.html)
+  * [Introduction to Scrunch](scrunch.html)
+  * [Current Limitations and Future Work](future-work.html)
 
-## High Level Concepts
+## Disclaimer
 
-### Data Model and Operators
-
-Crunch is centered around three interfaces that represent distributed datasets: `PCollection<T>`, `PTable<K, V>`, and `PGroupedTable<K, V>`.
-
-A `PCollection<T>` represents a distributed, unordered collection of elements of type T. For example, we represent a text file in Crunch as a
-`PCollection<String>` object. PCollection provides a method, `parallelDo`, that applies a function to each element in a PCollection in parallel,
-and returns a new PCollection as its result.
-
-A `PTable<K, V>` is a sub-interface of PCollection that represents a distributed, unordered multimap of its key type K to its value type V.
-In addition to the parallelDo operation, PTable provides a `groupByKey` operation that aggregates all of the values in the PTable that
-have the same key into a single record. It is the groupByKey operation that triggers the sort phase of a MapReduce job.
-
-The result of a groupByKey operation is a `PGroupedTable<K, V>` object, which is a distributed, sorted map of keys of type K to an Iterable
-collection of values of type V. In addition to parallelDo, the PGroupedTable provides a `combineValues` operation, which allows for
-a commutative and associative aggregation operator to be applied to the values of the PGroupedTable instance on both the map side and the
-reduce side of a MapReduce job.
-
-Finally, PCollection, PTable, and PGroupedTable all support a `union` operation, which takes a series of distinct PCollections and treats
-them as a single, virtual PCollection. The union operator is required for operations that combine multiple inputs, such as cogroups and
-joins.
-
-### Pipeline Building and Execution
-
-Every Crunch pipeline starts with a `Pipeline` object that is used to coordinate building the pipeline and executing the underlying MapReduce
-jobs. For efficiency, Crunch uses lazy evaluation, so it will only construct MapReduce jobs from the different stages of the pipelines when
-the Pipeline object\'s `run` or `done` methods are called.
-
-## A Detailed Example
-
-Here is the classic WordCount application using Crunch:
-
-    import org.apache.crunch.DoFn;
-    import org.apache.crunch.Emitter;
-    import org.apache.crunch.PCollection;
-    import org.apache.crunch.PTable;
-    import org.apache.crunch.Pipeline;
-    import org.apache.crunch.impl.mr.MRPipeline;
-    import org.apache.crunch.lib.Aggregate;
-    import org.apache.crunch.types.writable.Writables;
-
-    public class WordCount {
-      public static void main(String[] args) throws Exception {
-        Pipeline pipeline = new MRPipeline(WordCount.class);    
-        PCollection<String> lines = pipeline.readTextFile(args[0]);
-    
-        PCollection<String> words = lines.parallelDo("my splitter", new DoFn<String, String>() {
-          public void process(String line, Emitter<String> emitter) {
-            for (String word : line.split("\\s+")) {
-              emitter.emit(word);
-            }
-          }
-        }, Writables.strings());
-
-        PTable<String, Long> counts = Aggregate.count(words);
-    
-        pipeline.writeTextFile(counts, args[1]);
-        pipeline.run();
-      }
-    }
-
-Let\'s walk through the example line by line.
-
-### Step 1: Creating a Pipeline and referencing a text file
-
-The `MRPipeline` implementation of the Pipeline interface compiles the individual stages of a
-pipeline into a series of MapReduce jobs. The MRPipeline constructor takes a class argument
-that is used to tell Hadoop where to find the code that is used in the pipeline execution.
-
-We now need to tell the Pipeline about the inputs it will be consuming. The Pipeline interface
-defines a `readTextFile` method that takes in a String and returns a PCollection of Strings.
-In addition to text files, Crunch supports reading data from SequenceFiles and Avro container files,
-via the `SequenceFileSource` and `AvroFileSource` classes defined in the org.apache.crunch.io package.
-
-Note that each PCollection is a _reference_ to a source of data- no data is actually loaded into a
-PCollection on the client machine.
-
-### Step 2: Splitting the lines of text into words
-
-Crunch defines a small set of primitive operations that can be composed in order to build complex data
-pipelines. The first of these primitives is the `parallelDo` function, which applies a function (defined
-by a subclass of `DoFn`) to every record in a PCollection, and returns a new PCollection that contains
-the results.
-
-The first argument to parallelDo is a string that is used to identify this step in the pipeline. When
-a pipeline is composed into a series of MapReduce jobs, it is often the case that multiple stages will
-run within the same Mapper or Reducer. Having a string that identifies each processing step is useful
-for debugging errors that occur in a running pipeline.
-
-The second argument to parallelDo is an anonymous subclass of DoFn. Each DoFn subclass must override
-the `process` method, which takes in a record from the input PCollection and an `Emitter` object that
-may have any number of output values written to it. In this case, our DoFn splits each lines up into
-words, using a blank space as a separator, and emits the words from the split to the output PCollection.
-
-The last argument to parallelDo is an instance of the `PType` interface, which specifies how the data
-in the output PCollection is serialized. While Crunch takes advantage of Java Generics to provide
-compile-time type safety, the generic type information is not available at runtime. Crunch needs to know
-how to map the records stored in each PCollection into a Hadoop-supported serialization format in order
-to read and write data to disk. Two serialization implementations are supported in crunch via the
-`PTypeFamily` interface: a Writable-based system that is defined in the org.apache.crunch.types.writable
-package, and an Avro-based system that is defined in the org.apache.crunch.types.avro package. Each
-implementation provides convenience methods for working with the common PTypes (Strings, longs, bytes, etc.)
-as well as utility methods for creating PTypes from existing Writable classes or Avro schemas.
-
-### Step 3: Counting the words
-
-Out of Crunch\'s simple primitive operations, we can build arbitrarily complex chains of operations in order
-to perform higher-level operations, like aggregations and joins, that can work on any type of input data.
-Let\'s look at the implementation of the `Aggregate.count` function:
-
-    package org.apache.crunch.lib;
-
-    import org.apache.crunch.CombineFn;
-    import org.apache.crunch.MapFn;
-    import org.apache.crunch.PCollection;
-    import org.apache.crunch.PGroupedTable;
-    import org.apache.crunch.PTable;
-    import org.apache.crunch.Pair;
-    import org.apache.crunch.types.PTypeFamily;
-    
-    public class Aggregate {
-    
-      private static class Counter<S> extends MapFn<S, Pair<S, Long>> {
-        public Pair<S, Long> map(S input) {
-              return Pair.of(input, 1L);
-        }
-      }
-    
-      public static <S> PTable<S, Long> count(PCollection<S> collect) {
-        PTypeFamily tf = collect.getTypeFamily();
-    
-        // Create a PTable from the PCollection by mapping each element
-        // to a key of the PTable with the value equal to 1L
-        PTable<S, Long> withCounts = collect.parallelDo("count:" + collect.getName(),
-            new Counter<S>(), tf.tableOf(collect.getPType(), tf.longs()));
-    
-        // Group the records of the PTable based on their key.
-        PGroupedTable<S, Long> grouped = withCounts.groupByKey();
-    
-        // Sum the 1L values associated with the keys to get the
-        // count of each element in this PCollection, and return it
-        // as a PTable so that it may be processed further or written
-        // out for storage.
-        return grouped.combineValues(CombineFn.<S>SUM_LONGS());
-      }
-    }
-
-First, we get the PTypeFamily that is associated with the PType for the collection. The
-call to parallelDo converts each record in this PCollection into a Pair of the input record
-and the number one by extending the `MapFn` convenience subclass of DoFn, and uses the
-`tableOf` method of the PTypeFamily to specify that the returned PCollection should be a
-PTable instance, with the key being the PType of the PCollection and the value being the Long
-implementation for this PTypeFamily.
-
-The next line features the second of Crunch\'s four operations, `groupByKey`. The groupByKey
-operation may only be applied to a PTable, and returns an instance of the `PGroupedTable`
-interface, which references the grouping of all of the values in the PTable that have the same key.
-The groupByKey operation is what triggers the reduce phase of a MapReduce within Crunch.
-
-The last line in the function returns the output of the third of Crunch\'s four operations,
-`combineValues`. The combineValues operator takes a `CombineFn` as an argument, which is a
-specialized subclass of DoFn that operates on an implementation of Java\'s Iterable interface. The
-use of combineValues (as opposed to parallelDo) signals to Crunch that the CombineFn may be used to
-aggregate values for the same key on the map side of a MapReduce job as well as the reduce side.
-
-### Step 4: Writing the output and running the pipeline
-
-The Pipeline object also provides a `writeTextFile` convenience method for indicating that a
-PCollection should be written to a text file. There are also output targets for SequenceFiles and
-Avro container files, available in the org.apache.crunch.io package.
-
-After you are finished constructing a pipeline and specifying the output destinations, call the
-pipeline\'s blocking `run` method in order to compile the pipeline into one or more MapReduce
-jobs and execute them.
-
-## Writing Your Own Pipelines
-
-This section discusses the different steps of creating your own Crunch pipelines in more detail.
-
-### Writing a DoFn
-
-The DoFn class is designed to keep the complexity of the MapReduce APIs out of your way when you
-don\'t need them while still keeping them accessible when you do.
-
-#### Serialization
-
-First, all DoFn instances are required to be `java.io.Serializable`. This is a key aspect of Crunch\'s design:
-once a particular DoFn is assigned to the Map or Reduce stage of a MapReduce job, all of the state
-of that DoFn is serialized so that it may be distributed to all of the nodes in the Hadoop cluster that
-will be running that task. There are two important implications of this for developers:
-
-1. All member values of a DoFn must be either serializable or marked as `transient`.
-2. All anonymous DoFn instances must be defined in a static method or in a class that is itself serializable.
-
-Because sometimes you will need to work with non-serializable objects inside of a DoFn, every DoFn provides an
-`initialize` method that is called before the `process` method is ever called so that any initialization tasks,
-such as creating a non-serializable member variable, can be performed before processing begins. Similarly, all
-DoFn instances have a `cleanup` method that may be called after processing has finished to perform any required
-cleanup tasks.
-
-#### Scale Factor
-
-The DoFn class defines a `scaleFactor` method that can be used to signal to the MapReduce compiler that a particular
-DoFn implementation will yield an output PCollection that is larger (scaleFactor > 1) or smaller (0 < scaleFactor < 1)
-than the input PCollection it is applied to. The compiler may use this information to determine how to optimally
-split processing tasks between the Map and Reduce phases of dependent MapReduce jobs.
-
-#### Other Utilities
-
-The DoFn base class provides convenience methods for accessing the `Configuration` and `Counter` objects that
-are associated with a MapReduce stage, so that they may be accessed during initialization, processing, and cleanup.
-
-### Performing Cogroups and Joins
-
-In Crunch, cogroups and joins are performed on PTable instances that have the same key type. This section walks through
-the basic flow of a cogroup operation, explaining how this higher-level operation is composed of Crunch\'s four primitives.
-In general, these common operations are provided as part of the core Crunch library or in extensions, you do not need
-to write them yourself. But it can be useful to understand how they work under the covers.
-
-Assume we have a `PTable<K, U>` named "a" and a different `PTable<K, V>` named "b" that we would like to combine into a
-single `PTable<K, Pair<Collection<U>, Collection<V>>>`. First, we need to apply parallelDo operations to a and b that
-convert them into the same Crunch type, `PTable<K, Pair<U, V>>`:
-
-    // Perform the "tagging" operation as a parallelDo on PTable a
-    PTable<K, Pair<U, V>> aPrime = a.parallelDo("taga", new MapFn<Pair<K, U>, Pair<K, Pair<U, V>>>() {
-      public Pair<K, Pair<U, V>> map(Pair<K, U> input) {
-        return Pair.of(input.first(), Pair.of(input.second(), null));
-      }
-    }, tableOf(a.getKeyType(), pair(a.getValueType(), b.getValueType())));
-    
-    // Perform the "tagging" operation as a parallelDo on PTable b
-    PTable<K, Pair<U, V>> bPrime = b.parallelDo("tagb", new MapFn<Pair<K, V>, Pair<K, Pair<U, V>>>() {
-      public Pair<K, Pair<U, V>> map(Pair<K, V> input) {
-        return Pair.of(input.first(), Pair.of(null, input.second()));
-      }
-    }, tableOf(a.getKeyType(), pair(a.getValueType(), b.getValueType())));
-
-Once the input PTables are tagged into a single type, we can apply the union operation to create a single PTable
-reference that includes both of the tagged PTables and then group the unioned PTable by the common key:
-
-    PTable<K, Pair<U, V>> both = aPrime.union(bPrime);
-    PGroupedTable<K, Pair<U, V>> grouped = both.groupByKey();
-
-The grouping operation will create an `Iterable<Pair<U, V>>` which we can then convert to a `Pair<Collection<U>, Collection<V>>`:
-
-    grouped.parallelDo("cogroup", new MapFn<Pair<K, Iterable<Pair<U, V>>>, Pair<K, Pair<Collection<U>, Collection<V>>>>() {
-      public Pair<K, Pair<Collection<U>, Collection<V>>> map(Pair<K, Iterable<Pair<U, V>>> input) {
-        Collection<U> uValues = new ArrayList<U>();
-        Collection<V> vValues = new ArrayList<V>();
-        for (Pair<U, V> pair : input.second()) {
-          if (pair.first() != null) {
-            uValues.add(pair.first());
-          } else {
-            vValues.add(pair.second());
-          }
-        }
-        return Pair.of(input.first(), Pair.of(uValues, vValues));
-      },
-    }, tableOf(grouped.getKeyType(), pair(collections(a.getValueType()), collections(b.getValueType()))));
-
-## Current Limitations and Future Work
-
-This section contains an almost certainly incomplete list of known limitations of Crunch and plans for future work.
-
-* We would like to have easy support for reading and writing data from/to HCatalog.
-* The decision of how to split up processing tasks between dependent MapReduce jobs is very naiive right now- we simply
-delegate all of the work to the reduce stage of the predecessor job. We should take advantage of information about the
-expected size of different PCollections to optimize this processing.
-* The Crunch optimizer does not yet merge different groupByKey operations that run over the same input data into a single
-MapReduce job. Implementing this optimization will provide a major performance benefit for a number of problems.
+Apache Crunch is an effort undergoing incubation at [The Apache Software Foundation
+(ASF)](http://apache.org/) sponsored by the [Apache Incubator PMC](http://incubator.apache.org/).
+Incubation is required of all newly accepted projects until a further review
+indicates that the infrastructure, communications, and decision making process
+have stabilized in a manner consistent with other successful ASF projects.
+While incubation status is not necessarily a reflection of the completeness or
+stability of the code, it does indicate that the project has yet to be fully
+endorsed by the ASF.

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/src/site/markdown/intro.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/intro.md b/src/site/markdown/intro.md
new file mode 100644
index 0000000..0a8089b
--- /dev/null
+++ b/src/site/markdown/intro.md
@@ -0,0 +1,214 @@
+<!--
+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.
+-->
+# Introduction to Apache Crunch
+---
+
+## Build and Installation
+
+To use Crunch you first have to build the source code using Maven and install
+it in your local repository:
+
+    mvn clean install
+
+This also runs the integration test suite which will take a while. Afterwards
+you can run the bundled example applications:
+
+    hadoop jar examples/target/crunch-examples-*-job.jar org.apache.crunch.examples.WordCount <inputfile> <outputdir>
+
+
+## High Level Concepts
+
+### Data Model and Operators
+
+Crunch is centered around three interfaces that represent distributed datasets: `PCollection<T>`, `PTable<K, V>`, and `PGroupedTable<K, V>`.
+
+A `PCollection<T>` represents a distributed, unordered collection of elements of type T. For example, we represent a text file in Crunch as a
+`PCollection<String>` object. PCollection provides a method, `parallelDo`, that applies a function to each element in a PCollection in parallel,
+and returns a new PCollection as its result.
+
+A `PTable<K, V>` is a sub-interface of PCollection that represents a distributed, unordered multimap of its key type K to its value type V.
+In addition to the parallelDo operation, PTable provides a `groupByKey` operation that aggregates all of the values in the PTable that
+have the same key into a single record. It is the groupByKey operation that triggers the sort phase of a MapReduce job.
+
+The result of a groupByKey operation is a `PGroupedTable<K, V>` object, which is a distributed, sorted map of keys of type K to an Iterable
+collection of values of type V. In addition to parallelDo, the PGroupedTable provides a `combineValues` operation, which allows for
+a commutative and associative aggregation operator to be applied to the values of the PGroupedTable instance on both the map side and the
+reduce side of a MapReduce job.
+
+Finally, PCollection, PTable, and PGroupedTable all support a `union` operation, which takes a series of distinct PCollections and treats
+them as a single, virtual PCollection. The union operator is required for operations that combine multiple inputs, such as cogroups and
+joins.
+
+### Pipeline Building and Execution
+
+Every Crunch pipeline starts with a `Pipeline` object that is used to coordinate building the pipeline and executing the underlying MapReduce
+jobs. For efficiency, Crunch uses lazy evaluation, so it will only construct MapReduce jobs from the different stages of the pipelines when
+the Pipeline object\'s `run` or `done` methods are called.
+
+## A Detailed Example
+
+Here is the classic WordCount application using Crunch:
+
+    import org.apache.crunch.DoFn;
+    import org.apache.crunch.Emitter;
+    import org.apache.crunch.PCollection;
+    import org.apache.crunch.PTable;
+    import org.apache.crunch.Pipeline;
+    import org.apache.crunch.impl.mr.MRPipeline;
+    import org.apache.crunch.lib.Aggregate;
+    import org.apache.crunch.types.writable.Writables;
+
+    public class WordCount {
+      public static void main(String[] args) throws Exception {
+        Pipeline pipeline = new MRPipeline(WordCount.class);
+        PCollection<String> lines = pipeline.readTextFile(args[0]);
+
+        PCollection<String> words = lines.parallelDo("my splitter", new DoFn<String, String>() {
+          public void process(String line, Emitter<String> emitter) {
+            for (String word : line.split("\\s+")) {
+              emitter.emit(word);
+            }
+          }
+        }, Writables.strings());
+
+        PTable<String, Long> counts = Aggregate.count(words);
+
+        pipeline.writeTextFile(counts, args[1]);
+        pipeline.run();
+      }
+    }
+
+Let\'s walk through the example line by line.
+
+### Step 1: Creating a Pipeline and referencing a text file
+
+The `MRPipeline` implementation of the Pipeline interface compiles the individual stages of a
+pipeline into a series of MapReduce jobs. The MRPipeline constructor takes a class argument
+that is used to tell Hadoop where to find the code that is used in the pipeline execution.
+
+We now need to tell the Pipeline about the inputs it will be consuming. The Pipeline interface
+defines a `readTextFile` method that takes in a String and returns a PCollection of Strings.
+In addition to text files, Crunch supports reading data from SequenceFiles and Avro container files,
+via the `SequenceFileSource` and `AvroFileSource` classes defined in the org.apache.crunch.io package.
+
+Note that each PCollection is a _reference_ to a source of data- no data is actually loaded into a
+PCollection on the client machine.
+
+### Step 2: Splitting the lines of text into words
+
+Crunch defines a small set of primitive operations that can be composed in order to build complex data
+pipelines. The first of these primitives is the `parallelDo` function, which applies a function (defined
+by a subclass of `DoFn`) to every record in a PCollection, and returns a new PCollection that contains
+the results.
+
+The first argument to parallelDo is a string that is used to identify this step in the pipeline. When
+a pipeline is composed into a series of MapReduce jobs, it is often the case that multiple stages will
+run within the same Mapper or Reducer. Having a string that identifies each processing step is useful
+for debugging errors that occur in a running pipeline.
+
+The second argument to parallelDo is an anonymous subclass of DoFn. Each DoFn subclass must override
+the `process` method, which takes in a record from the input PCollection and an `Emitter` object that
+may have any number of output values written to it. In this case, our DoFn splits each lines up into
+words, using a blank space as a separator, and emits the words from the split to the output PCollection.
+
+The last argument to parallelDo is an instance of the `PType` interface, which specifies how the data
+in the output PCollection is serialized. While Crunch takes advantage of Java Generics to provide
+compile-time type safety, the generic type information is not available at runtime. Crunch needs to know
+how to map the records stored in each PCollection into a Hadoop-supported serialization format in order
+to read and write data to disk. Two serialization implementations are supported in crunch via the
+`PTypeFamily` interface: a Writable-based system that is defined in the org.apache.crunch.types.writable
+package, and an Avro-based system that is defined in the org.apache.crunch.types.avro package. Each
+implementation provides convenience methods for working with the common PTypes (Strings, longs, bytes, etc.)
+as well as utility methods for creating PTypes from existing Writable classes or Avro schemas.
+
+### Step 3: Counting the words
+
+Out of Crunch\'s simple primitive operations, we can build arbitrarily complex chains of operations in order
+to perform higher-level operations, like aggregations and joins, that can work on any type of input data.
+Let\'s look at the implementation of the `Aggregate.count` function:
+
+    package org.apache.crunch.lib;
+
+    import org.apache.crunch.CombineFn;
+    import org.apache.crunch.MapFn;
+    import org.apache.crunch.PCollection;
+    import org.apache.crunch.PGroupedTable;
+    import org.apache.crunch.PTable;
+    import org.apache.crunch.Pair;
+    import org.apache.crunch.types.PTypeFamily;
+
+    public class Aggregate {
+
+      private static class Counter<S> extends MapFn<S, Pair<S, Long>> {
+        public Pair<S, Long> map(S input) {
+              return Pair.of(input, 1L);
+        }
+      }
+
+      public static <S> PTable<S, Long> count(PCollection<S> collect) {
+        PTypeFamily tf = collect.getTypeFamily();
+
+        // Create a PTable from the PCollection by mapping each element
+        // to a key of the PTable with the value equal to 1L
+        PTable<S, Long> withCounts = collect.parallelDo("count:" + collect.getName(),
+            new Counter<S>(), tf.tableOf(collect.getPType(), tf.longs()));
+
+        // Group the records of the PTable based on their key.
+        PGroupedTable<S, Long> grouped = withCounts.groupByKey();
+
+        // Sum the 1L values associated with the keys to get the
+        // count of each element in this PCollection, and return it
+        // as a PTable so that it may be processed further or written
+        // out for storage.
+        return grouped.combineValues(CombineFn.<S>SUM_LONGS());
+      }
+    }
+
+First, we get the PTypeFamily that is associated with the PType for the collection. The
+call to parallelDo converts each record in this PCollection into a Pair of the input record
+and the number one by extending the `MapFn` convenience subclass of DoFn, and uses the
+`tableOf` method of the PTypeFamily to specify that the returned PCollection should be a
+PTable instance, with the key being the PType of the PCollection and the value being the Long
+implementation for this PTypeFamily.
+
+The next line features the second of Crunch\'s four operations, `groupByKey`. The groupByKey
+operation may only be applied to a PTable, and returns an instance of the `PGroupedTable`
+interface, which references the grouping of all of the values in the PTable that have the same key.
+The groupByKey operation is what triggers the reduce phase of a MapReduce within Crunch.
+
+The last line in the function returns the output of the third of Crunch\'s four operations,
+`combineValues`. The combineValues operator takes a `CombineFn` as an argument, which is a
+specialized subclass of DoFn that operates on an implementation of Java\'s Iterable interface. The
+use of combineValues (as opposed to parallelDo) signals to Crunch that the CombineFn may be used to
+aggregate values for the same key on the map side of a MapReduce job as well as the reduce side.
+
+### Step 4: Writing the output and running the pipeline
+
+The Pipeline object also provides a `writeTextFile` convenience method for indicating that a
+PCollection should be written to a text file. There are also output targets for SequenceFiles and
+Avro container files, available in the org.apache.crunch.io package.
+
+After you are finished constructing a pipeline and specifying the output destinations, call the
+pipeline\'s blocking `run` method in order to compile the pipeline into one or more MapReduce
+jobs and execute them.
+
+
+## More Information
+
+[Writing Your Own Pipelines](pipelines.html)

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/src/site/markdown/pipelines.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/pipelines.md b/src/site/markdown/pipelines.md
new file mode 100644
index 0000000..6fea519
--- /dev/null
+++ b/src/site/markdown/pipelines.md
@@ -0,0 +1,103 @@
+<!--
+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.
+-->
+# Writing Your Own Pipelines
+---
+
+This section discusses the different steps of creating your own Crunch pipelines in more detail.
+
+## Writing a DoFn
+
+The DoFn class is designed to keep the complexity of the MapReduce APIs out of your way when you
+don\'t need them while still keeping them accessible when you do.
+
+### Serialization
+
+First, all DoFn instances are required to be `java.io.Serializable`. This is a key aspect of Crunch\'s design:
+once a particular DoFn is assigned to the Map or Reduce stage of a MapReduce job, all of the state
+of that DoFn is serialized so that it may be distributed to all of the nodes in the Hadoop cluster that
+will be running that task. There are two important implications of this for developers:
+
+1. All member values of a DoFn must be either serializable or marked as `transient`.
+2. All anonymous DoFn instances must be defined in a static method or in a class that is itself serializable.
+
+Because sometimes you will need to work with non-serializable objects inside of a DoFn, every DoFn provides an
+`initialize` method that is called before the `process` method is ever called so that any initialization tasks,
+such as creating a non-serializable member variable, can be performed before processing begins. Similarly, all
+DoFn instances have a `cleanup` method that may be called after processing has finished to perform any required
+cleanup tasks.
+
+### Scale Factor
+
+The DoFn class defines a `scaleFactor` method that can be used to signal to the MapReduce compiler that a particular
+DoFn implementation will yield an output PCollection that is larger (scaleFactor > 1) or smaller (0 < scaleFactor < 1)
+than the input PCollection it is applied to. The compiler may use this information to determine how to optimally
+split processing tasks between the Map and Reduce phases of dependent MapReduce jobs.
+
+### Other Utilities
+
+The DoFn base class provides convenience methods for accessing the `Configuration` and `Counter` objects that
+are associated with a MapReduce stage, so that they may be accessed during initialization, processing, and cleanup.
+
+### Performing Cogroups and Joins
+
+In Crunch, cogroups and joins are performed on PTable instances that have the same key type. This section walks through
+the basic flow of a cogroup operation, explaining how this higher-level operation is composed of Crunch\'s four primitives.
+In general, these common operations are provided as part of the core Crunch library or in extensions, you do not need
+to write them yourself. But it can be useful to understand how they work under the covers.
+
+Assume we have a `PTable<K, U>` named "a" and a different `PTable<K, V>` named "b" that we would like to combine into a
+single `PTable<K, Pair<Collection<U>, Collection<V>>>`. First, we need to apply parallelDo operations to a and b that
+convert them into the same Crunch type, `PTable<K, Pair<U, V>>`:
+
+    // Perform the "tagging" operation as a parallelDo on PTable a
+    PTable<K, Pair<U, V>> aPrime = a.parallelDo("taga", new MapFn<Pair<K, U>, Pair<K, Pair<U, V>>>() {
+      public Pair<K, Pair<U, V>> map(Pair<K, U> input) {
+        return Pair.of(input.first(), Pair.of(input.second(), null));
+      }
+    }, tableOf(a.getKeyType(), pair(a.getValueType(), b.getValueType())));
+
+    // Perform the "tagging" operation as a parallelDo on PTable b
+    PTable<K, Pair<U, V>> bPrime = b.parallelDo("tagb", new MapFn<Pair<K, V>, Pair<K, Pair<U, V>>>() {
+      public Pair<K, Pair<U, V>> map(Pair<K, V> input) {
+        return Pair.of(input.first(), Pair.of(null, input.second()));
+      }
+    }, tableOf(a.getKeyType(), pair(a.getValueType(), b.getValueType())));
+
+Once the input PTables are tagged into a single type, we can apply the union operation to create a single PTable
+reference that includes both of the tagged PTables and then group the unioned PTable by the common key:
+
+    PTable<K, Pair<U, V>> both = aPrime.union(bPrime);
+    PGroupedTable<K, Pair<U, V>> grouped = both.groupByKey();
+
+The grouping operation will create an `Iterable<Pair<U, V>>` which we can then convert to a `Pair<Collection<U>, Collection<V>>`:
+
+    grouped.parallelDo("cogroup", new MapFn<Pair<K, Iterable<Pair<U, V>>>, Pair<K, Pair<Collection<U>, Collection<V>>>>() {
+      public Pair<K, Pair<Collection<U>, Collection<V>>> map(Pair<K, Iterable<Pair<U, V>>> input) {
+        Collection<U> uValues = new ArrayList<U>();
+        Collection<V> vValues = new ArrayList<V>();
+        for (Pair<U, V> pair : input.second()) {
+          if (pair.first() != null) {
+            uValues.add(pair.first());
+          } else {
+            vValues.add(pair.second());
+          }
+        }
+        return Pair.of(input.first(), Pair.of(uValues, vValues));
+      },
+    }, tableOf(grouped.getKeyType(), pair(collections(a.getValueType()), collections(b.getValueType()))));

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/src/site/markdown/scrunch.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/scrunch.md b/src/site/markdown/scrunch.md
new file mode 100644
index 0000000..7d0585a
--- /dev/null
+++ b/src/site/markdown/scrunch.md
@@ -0,0 +1,90 @@
+<!--
+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.
+-->
+# Scrunch - A Scala Wrapper for Apache Crunch
+---
+
+## Introduction
+
+Scrunch is an experimental Scala wrapper for Crunch, based on the same ideas as the
+[Cascade](http://days2011.scala-lang.org/node/138/282) project at Google, which created
+a Scala wrapper for FlumeJava.
+
+## Why Scala?
+
+In many ways, Scala is the perfect language for writing Crunch pipelines. Scala supports
+a mixture of functional and object-oriented programming styles and has powerful type-inference
+capabilities, allowing us to create complex pipelines using very few keystrokes. Here is
+the Scrunch analogue of the classic WordCount problem:
+
+	import com.cloudera.crunch.io.{From => from}
+	import com.cloudera.scrunch._
+	import com.cloudera.scrunch.Conversions_  # For implicit type conversions
+
+	class WordCountExample {
+	  val pipeline = new Pipeline[WordCountExample]
+
+	  def wordCount(fileName: String) = {
+	    pipeline.read(from.textFile(fileName))
+	      .flatMap(_.toLowerCase.split("\\W+"))
+	      .filter(!_.isEmpty())
+	      .count
+	  }
+	}
+
+The Scala compiler can infer the return type of the flatMap function as an Array[String], and
+the Scrunch wrapper uses the type inference mechanism to figure out how to serialize the
+data between the Map and Reduce stages. Here\'s a slightly more complex example, in which we
+get the word counts for two different files and compute the deltas of how often different
+words occur, and then only returns the words where the first file had more occurrences then
+the second:
+
+	class WordCountExample {
+	  def wordGt(firstFile: String, secondFile: String) = {
+	    wordCount(firstFile).cogroup(wordCount(secondFile))
+	      .map((k, v) => (k, (v._1.sum - v._2.sum)))
+	      .filter((k, v) => v > 0).map((k, v) => k)
+	  }
+	}
+
+Note that all of the functions are using Scala Tuples, not Crunch Tuples. Under the covers,
+Scrunch uses Scala\'s implicit type conversion mechanism to transparently convert data from the
+Crunch format to the Scala format and back again.
+
+## Materializing Job Outputs
+
+Scrunch also incorporates Crunch\'s materialize functionality, which allows us to easily read
+the output of a Crunch pipeline into the client:
+
+	class WordCountExample {
+	  def hasHamlet = wordGt("shakespeare.txt", "maugham.txt").materialize.exists(_ == "hamlet")
+	}
+
+## Notes and Thanks
+
+Scrunch is alpha-quality code, written by someone who was learning Scala on the fly. There will be bugs,
+rough edges, and non-idiomatic Scala usage all over the place. This will improve with time, and we welcome
+contributions from Scala experts who are interested in helping us make Scrunch into a first-class project.
+The Crunch developers mailing list is [here](https://groups.google.com/a/cloudera.org/group/crunch-dev/topics).
+
+Scrunch emerged out of conversations with [Dmitriy Ryaboy](http://twitter.com/#!/squarecog),
+[Oscar Boykin](http://twitter.com/#!/posco), and [Avi Bryant](http://twitter.com/#!/avibryant) from Twitter.
+Many thanks to them for their feedback, guidance, and encouragement. We are also grateful to
+[Matei Zaharia](http://twitter.com/#!/matei_zaharia), whose [Spark Project](http://www.spark-project.org/)
+inspired much of our implementation and was kind enough to loan us the ClosureCleaner implementation
+Spark developed for use in Scrunch.

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/src/site/resources/css/site.css
----------------------------------------------------------------------
diff --git a/src/site/resources/css/site.css b/src/site/resources/css/site.css
deleted file mode 100644
index 8ee551e..0000000
--- a/src/site/resources/css/site.css
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-/* Borrowed from Apache Whirr */
-
-#banner {
-  height: 93px;
-  background: none;
-}
-
-#bannerLeft img {
-  height: 90px;
-  margin-left: 30px;
-  margin-top: 4px;
-}
-
-#bannerRight img {
-  margin: 17px;
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-crunch/blob/fbc741f1/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index 252123e..87e20e3 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -23,17 +23,17 @@
 
   <skin>
     <groupId>org.apache.maven.skins</groupId>
-    <artifactId>maven-stylus-skin</artifactId>
-    <version>1.4</version>
+    <artifactId>maven-fluido-skin</artifactId>
+    <version>1.2.2</version>
   </skin>
 
   <bannerLeft>
-    <name>${project.name}</name>
+    <name></name> <!-- Leave this empty until we have a logo -->
   </bannerLeft>
 
   <bannerRight>
-    <src>http://www.apache.org/images/asf_logo_wide.png</src>
-    <href>http://www.apache.org/</href>
+    <src>http://incubator.apache.org/images/apache-incubator-logo.png</src>
+    <href>http://incubator.apache.org/</href>
   </bannerRight>
 
   <publishDate position="right"/>
@@ -43,26 +43,23 @@
     <!-- Note: Breadcrumbs for Doxia's Markdown parser are currently broken,
                see https://jira.codehaus.org/browse/DOXIA-472 -->
     <breadcrumbs>
-      <item name="Apache" href="http://www.apache.org/index.html" />
+      <item name="Apache Incubator" href="http://incubator.apache.org/" />
     </breadcrumbs>
 
-    <menu name="Crunch" inherit="top">
-      <item name="About" href="http://incubator.apache.org/crunch/"/>
-      <item name="Wiki" href="https://cwiki.apache.org/confluence/display/CRUNCH/" />
-    </menu>
-
-    <!--menu name="Project Information">
+    <menu name="Apache Crunch" inherit="top">
+      <item name="About" href="index.html" />
+      <item name="API" href="apidocs/" />
       <item name="Mailing Lists" href="mail-lists.html" />
       <item name="Source Code" href="source-repository.html" />
       <item name="Issue Tracking" href="issue-tracking.html" />
-      <item name="Continuous Integration" href="integration.html" />
-      <item name="Team" href="team-list.html" />
       <item name="License" href="license.html" />
-    </menu-->
+      <item name="Wiki" href="https://cwiki.apache.org/confluence/display/CRUNCH/" />
+    </menu>
     
+    <!--
     <menu ref="modules" />
     <menu ref="reports" inherit="bottom" />
-
+    -->
   </body>
 
 </project>


Mime
View raw message