You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by ja...@apache.org on 2014/11/20 00:43:57 UTC

[1/5] bigtop git commit: Add BPS Spark driver for new data generator. Re-organize BPS into MapReduce and Spark versions.

Repository: bigtop
Updated Branches:
  refs/heads/master 801bade70 -> 6ec6cebfc


http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/test/scala/org/apache/bigtop/bigpetstore/ScalaTestSample.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/test/scala/org/apache/bigtop/bigpetstore/ScalaTestSample.scala b/bigtop-bigpetstore/src/test/scala/org/apache/bigtop/bigpetstore/ScalaTestSample.scala
deleted file mode 100644
index a393b4b..0000000
--- a/bigtop-bigpetstore/src/test/scala/org/apache/bigtop/bigpetstore/ScalaTestSample.scala
+++ /dev/null
@@ -1,35 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore
-
-import org.junit.Test
-import org.junit.runner.RunWith
-import org.scalatest.junit.JUnitRunner
-import org.scalatest._
-import scala.collection.mutable.Stack
-
-@RunWith(classOf[JUnitRunner])
-class ScalaTestSample extends FlatSpec with Matchers {
-	"This test" should "show an example of what we can do with the scala-test library" in {
-		val stack = new Stack[Int]
-		stack.push(1)
-		stack.push(2)
-		stack.pop() should be(2)
-		stack.pop() should be(1)
-	}
-}


[4/5] bigtop git commit: Add BPS Spark driver for new data generator. Re-organize BPS into MapReduce and Spark versions.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/CrunchETL.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/CrunchETL.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/CrunchETL.java
new file mode 100755
index 0000000..271083d
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/CrunchETL.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bigtop.bigpetstore.etl;
+
+import java.util.Map;
+
+import org.apache.bigtop.bigpetstore.contract.PetStoreStatistics;
+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.impl.mem.MemPipeline;
+import org.apache.crunch.impl.mr.MRPipeline;
+import org.apache.crunch.io.From;
+import org.apache.crunch.types.avro.Avros;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class CrunchETL extends PetStoreStatistics {
+
+    public static MapFn<LineItem, String> COUNT_BY_PRODUCT = new MapFn<LineItem, String>() {
+        public String map(LineItem lineItem) {
+            try {
+                return lineItem.getDescription();
+            } catch (Throwable t) {
+                throw new RuntimeException(t);
+            }
+        }
+    };
+    public static MapFn<LineItem, String> COUNT_BY_STATE = new MapFn<LineItem, String>() {
+        public String map(LineItem lineItem) {
+            try {
+                return lineItem.getDescription();
+            } catch (Throwable t) {
+                throw new RuntimeException(t);
+            }
+        }
+    };
+
+    PCollection<LineItem> lineItems;
+
+    public CrunchETL(Path input, Path output) throws Exception {
+        Pipeline pipeline = MemPipeline.getInstance();
+        PCollection<String> lines = pipeline.read(From.textFile(new Path(input,
+                "part-r-00000")));
+        System.out.println("crunch : " + lines.getName() + "  "
+                + lines.getSize());
+        lineItems = lines.parallelDo(ETL, Avros.reflects(LineItem.class));
+
+    }
+
+    public static MapFn ETL = new MapFn<String, LineItem>() {
+        @Override
+        public LineItem map(String input) {
+            String[] fields = input.split(",");
+            LineItem li = new LineItem();
+            li.setAppName(fields[1]);
+            li.setFirstName(fields[3]);
+            // ...
+            li.setDescription(fields[fields.length - 1]);
+            return li;
+        }
+    };
+
+    @Override
+    public Map<String, ? extends Number> numberOfTransactionsByState()
+            throws Exception {
+        PTable<String, Long> counts = lineItems.parallelDo(COUNT_BY_STATE,
+                Avros.strings()).count();
+        Map m = counts.materializeToMap();
+
+        System.out.println("Crunch:::  " + m);
+        return m;
+    }
+
+    @Override
+    public Map<String, ? extends Number> numberOfProductsByProduct()
+            throws Exception {
+        PTable<String, Long> counts = lineItems.parallelDo(COUNT_BY_PRODUCT,
+                Avros.strings()).count();
+        Map m = counts.materializeToMap();
+        //CrunchETL. System.out.println("Crunch:::  " + m);
+        return m;
+    }
+
+    public static void main(String... args) throws Exception {
+        /**
+         * PCollection<String> lines = MemPipeline .collectionOf(
+         *  "BigPetStore,storeCode_AK,1  lindsay,franco,Sat Jan 10 00:11:10 EST 1970,10.5,dog-food"
+         *  "BigPetStore,storeCode_AZ,1  tom,giles,Sun Dec 28 23:08:45 EST 1969,10.5,dog-food"
+         *  "BigPetStore,storeCode_CA,1  brandon,ewing,Mon Dec 08 20:23:57 EST 1969,16.5,organic-dog-food"
+         *  "BigPetStore,storeCode_CA,2  angie,coleman,Thu Dec 11 07:00:31 EST 1969,10.5,dog-food"
+         *  "BigPetStore,storeCode_CA,3  angie,coleman,Tue Jan 20 06:24:23 EST 1970,7.5,cat-food"
+         *  "BigPetStore,storeCode_CO,1  sharon,trevino,Mon Jan 12 07:52:10 EST 1970,30.1,antelope snacks"
+         *  "BigPetStore,storeCode_CT,1  kevin,fitzpatrick,Wed Dec 10 05:24:13 EST 1969,10.5,dog-food"
+         *  "BigPetStore,storeCode_NY,1  dale,holden,Mon Jan 12 23:02:13 EST 1970,19.75,fish-food"
+         *  "BigPetStore,storeCode_NY,2  dale,holden,Tue Dec 30 12:29:52 EST 1969,10.5,dog-food"
+         *  "BigPetStore,storeCode_OK,1  donnie,tucker,Sun Jan 18 04:50:26 EST 1970,7.5,cat-food"
+         * );
+         **/
+        // FAILS
+        Pipeline pipeline = new MRPipeline(CrunchETL.class);
+
+        PCollection<String> lines = pipeline.read(From.textFile(new Path(
+                "/tmp/BigPetStore1388719888255/generated/part-r-00000")));
+
+
+        PCollection<LineItem> lineItems = lines.parallelDo(
+                new MapFn<String, LineItem>() {
+                    @Override
+                    public LineItem map(String input) {
+
+                        System.out.println("proc1 " + input);
+                        String[] fields = input.split(",");
+                        LineItem li = new LineItem();
+                        li.setAppName("" + fields[1]);
+                        li.setFirstName("" + fields[3]);
+                        li.setDescription("" + fields[fields.length - 1]);
+                        return li;
+                    }
+                }, Avros.reflects(LineItem.class));
+
+        for (LineItem i : lineItems.materialize())
+            System.out.println(i);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/LineItem.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/LineItem.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/LineItem.java
new file mode 100755
index 0000000..a415cf4
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/LineItem.java
@@ -0,0 +1,112 @@
+/**
+ * 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.bigtop.bigpetstore.etl;
+
+import java.io.Serializable;
+
+public class LineItem implements Serializable{
+
+    public LineItem(String appName, String storeCode, Integer lineId, String firstName, String lastName, String timestamp, Double price, String description){
+        super();
+        this.appName=appName;
+        this.storeCode=storeCode;
+        this.lineId=lineId;
+        this.firstName=firstName;
+        this.lastName=lastName;
+        this.timestamp=timestamp;
+        this.price=price;
+        this.description=description;
+    }
+
+    String appName;
+    String storeCode;
+    Integer lineId;
+    String firstName;
+    String lastName;
+    String timestamp;
+    Double price;
+    String description;
+
+    public LineItem(){
+        super();
+    }
+
+    public String getAppName(){
+        return appName;
+    }
+
+    public void setAppName(String appName){
+        this.appName=appName;
+    }
+
+    public String getStoreCode(){
+        return storeCode;
+    }
+
+    public void setStoreCode(String storeCode){
+        this.storeCode=storeCode;
+    }
+
+    public int getLineId(){
+        return lineId;
+    }
+
+    public void setLineId(int lineId){
+        this.lineId=lineId;
+    }
+
+    public String getFirstName(){
+        return firstName;
+    }
+
+    public void setFirstName(String firstName){
+        this.firstName=firstName;
+    }
+
+    public String getLastName(){
+        return lastName;
+    }
+
+    public void setLastName(String lastName){
+        this.lastName=lastName;
+    }
+
+    public String getTimestamp(){
+        return timestamp;
+    }
+
+    public void setTimestamp(String timestamp){
+        this.timestamp=timestamp;
+    }
+
+    public double getPrice(){
+        return price;
+    }
+
+    public void setPrice(double price){
+        this.price=price;
+    }
+
+    public String getDescription(){
+        return description;
+    }
+
+    public void setDescription(String description){
+        this.description=description;
+    }
+
+    // other constructors, parsers, etc.
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/PigCSVCleaner.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/PigCSVCleaner.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/PigCSVCleaner.java
new file mode 100644
index 0000000..0ca7444
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/etl/PigCSVCleaner.java
@@ -0,0 +1,156 @@
+/**
+ * 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.bigtop.bigpetstore.etl;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants.OUTPUTS;
+import org.apache.bigtop.bigpetstore.util.DeveloperTools;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+
+/**
+ * This class operates by ETL'ing the data-set into pig.
+ * The pigServer is persisted through the life of the class, so that the
+ * intermediate data sets created in the constructor can be reused.
+ */
+public class PigCSVCleaner  {
+
+    PigServer pigServer;
+
+    private static Path getCleanedTsvPath(Path outputPath) {
+      return new Path(outputPath, OUTPUTS.tsv.name());
+    }
+
+    public PigCSVCleaner(Path inputPath, Path outputPath, ExecType ex, File... scripts)
+            throws Exception {
+        FileSystem fs = FileSystem.get(inputPath.toUri(), new Configuration());
+
+        if(! fs.exists(inputPath)){
+            throw new RuntimeException("INPUT path DOES NOT exist : " + inputPath);
+        }
+
+        if(fs.exists(outputPath)){
+            throw new RuntimeException("OUTPUT already exists : " + outputPath);
+        }
+        // run pig in local mode
+        pigServer = new PigServer(ex);
+
+        /**
+         * First, split the tabs up.
+         *
+         * BigPetStore,storeCode_OK,2 1,yang,jay,3,flea collar,69.56,Mon Dec 15 23:33:49 EST 1969
+         *
+         * ("BigPetStore,storeCode_OK,2", "1,yang,jay,3,flea collar,69.56,Mon Dec 15 23:33:49 EST 1969")
+         */
+        pigServer.registerQuery("csvdata = LOAD '<i>' AS (ID,DETAILS);".replaceAll("<i>", inputPath.toString()));
+
+        // currentCustomerId, firstName, lastName, product.id, product.name.toLowerCase, product.price, date
+        /**
+         * Now, we want to split the two tab delimited fields into uniform
+         * fields of comma separated values. To do this, we 1) Internally split
+         * the FIRST and SECOND fields by commas "a,b,c" --> (a,b,c) 2) FLATTEN
+         * the FIRST and SECOND fields. (d,e) (a,b,c) -> d e a b c
+         */
+        pigServer.registerQuery(
+              "id_details = FOREACH csvdata GENERATE "
+              + "FLATTEN(STRSPLIT(ID, ',', 3)) AS " +
+			"(drop, code, transaction) ,"
+
+              + "FLATTEN(STRSPLIT(DETAILS, ',', 7)) AS " +
+                  "(custId, fname, lname, productId, product:chararray, price, date);");
+        pigServer.registerQuery("mahout_records = FOREACH id_details GENERATE custId, productId, 1;");
+        pigServer.store("id_details", getCleanedTsvPath(outputPath).toString());
+        pigServer.store("mahout_records", new Path(outputPath, OUTPUTS.MahoutPaths.Mahout.name()).toString());
+        /**
+         * Now we run scripts... this is where you can add some
+         * arbitrary analytics.
+         *
+         * We add "input" and "output" parameters so that each
+         * script can read them and use them if they want.
+         *
+         * Otherwise, just hardcode your inputs into your pig scripts.
+         */
+        int i = 0;
+        for(File script : scripts) {
+            Map<String,String> parameters = new HashMap<>();
+            parameters.put("input", getCleanedTsvPath(outputPath).toString());
+
+            Path dir = outputPath.getParent();
+            Path adHocOut = new Path(dir, OUTPUTS.pig_ad_hoc_script.name() + (i++));
+            System.out.println("Setting default output to " + adHocOut);
+            parameters.put("output", adHocOut.toString());
+            pigServer.registerScript(script.getAbsolutePath(), parameters);
+        }
+    }
+
+    private static File[] files(String[] args,int startIndex) {
+        List<File> files = new ArrayList<File>();
+        for(int i = startIndex ; i < args.length ; i++) {
+            File f = new File(args[i]);
+            if(! f.exists()) {
+                throw new RuntimeException("Pig script arg " + i + " " + f.getAbsolutePath() + " not found. ");
+            }
+            files.add(f);
+        }
+        System.out.println(
+                "Ad-hoc analytics:"+
+                "Added  " + files.size() + " pig scripts to post process.  "+
+                "Each one will be given $input and $output arguments.");
+        return files.toArray(new File[]{});
+    }
+
+    public static void main(final String[] args) throws Exception {
+        System.out.println("Starting pig etl " + args.length);
+        Configuration c = new Configuration();
+        int res = ToolRunner.run(c, new Tool() {
+                    Configuration conf;
+                    @Override
+                    public void setConf(Configuration conf) {
+                        this.conf=conf;
+                    }
+
+                    @Override
+                    public Configuration getConf() {
+                        return this.conf;
+                    }
+
+                    @Override
+                    public int run(String[] args) throws Exception {
+                        DeveloperTools.validate(
+                                args,
+                                "generated data directory",
+                                "pig output directory");
+                        new PigCSVCleaner(
+                                new Path(args[0]),
+                                new Path(args[1]),
+                                ExecType.MAPREDUCE,
+                                files(args,2));
+                        return 0;
+                    }
+                }, args);
+        System.exit(res);
+      }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/BPSGenerator.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/BPSGenerator.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/BPSGenerator.java
new file mode 100755
index 0000000..6c8beef
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/BPSGenerator.java
@@ -0,0 +1,108 @@
+/**
+ * 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.bigtop.bigpetstore.generator;
+
+import java.io.IOException;
+import java.util.Date;
+
+import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants;
+import org.apache.bigtop.bigpetstore.util.DeveloperTools;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.lib.MultipleOutputs;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.apache.bigtop.bigpetstore.generator.PetStoreTransactionsInputFormat.props;
+
+/**
+ * This is a mapreduce implementation of a generator of a large sentiment
+ * analysis data set. The scenario is as follows:
+ *
+ * The number of records will (roughly) correspond to the output size - each
+ * record is about 80 bytes.
+ *
+ * 1KB set bigpetstore_records=10 1MB set bigpetstore_records=10,000 1GB set
+ * bigpetstore_records=10,000,000 1TB set bigpetstore_records=10,000,000,000
+ */
+public class BPSGenerator {
+
+  public static final int DEFAULT_NUM_RECORDS = 100;
+
+  final static Logger log = LoggerFactory.getLogger(BPSGenerator.class);
+
+  public enum props {
+    bigpetstore_records
+  }
+
+  public static Job createJob(Path output, int records) throws IOException {
+    Configuration c = new Configuration();
+    c.setInt(props.bigpetstore_records.name(), DEFAULT_NUM_RECORDS);
+    return getCreateTransactionRecordsJob(output, c);
+  }
+
+  public static Job getCreateTransactionRecordsJob(Path outputDir, Configuration conf)
+          throws IOException {
+    Job job = new Job(conf, "PetStoreTransaction_ETL_" + System.currentTimeMillis());
+    // recursively delete the data set if it exists.
+    FileSystem.get(outputDir.toUri(), conf).delete(outputDir, true);
+    job.setJarByClass(BPSGenerator.class);
+    job.setMapperClass(MyMapper.class);
+    // use the default reducer
+    // job.setReducerClass(PetStoreTransactionGeneratorJob.Red.class);
+    job.setOutputKeyClass(Text.class);
+    job.setOutputValueClass(Text.class);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(Text.class);
+    job.setInputFormatClass(PetStoreTransactionsInputFormat.class);
+    job.setOutputFormatClass(TextOutputFormat.class);
+    FileOutputFormat.setOutputPath(job, outputDir);
+    return job;
+  }
+
+  public static class MyMapper extends Mapper<Text, Text, Text, Text> {
+    @Override
+    protected void setup(Context context) throws IOException,
+    InterruptedException {
+      super.setup(context);
+    }
+
+    protected void map(Text key, Text value, Context context)
+            throws java.io.IOException, InterruptedException {
+      context.write(key, value);
+    }
+  }
+
+  public static void main(String args[]) throws Exception {
+    if (args.length != 2) {
+      System.err.println("USAGE : [number of records] [output path]");
+      System.exit(0);
+    } else {
+      Configuration conf = new Configuration();
+      DeveloperTools.validate(args, "# of records", "output path");
+      conf.setInt(PetStoreTransactionsInputFormat.props.bigpetstore_records.name(),
+              Integer.parseInt(args[0]));
+      getCreateTransactionRecordsJob(new Path(args[1]), conf).waitForCompletion(true);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/CustomerGenerator.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/CustomerGenerator.scala b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/CustomerGenerator.scala
new file mode 100644
index 0000000..0223c8d
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/CustomerGenerator.scala
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bigtop.bigpetstore.generator
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileSystem
+import org.apache.bigtop.bigpetstore.generator.util.State
+import org.apache.hadoop.fs.Path
+import parquet.org.codehaus.jackson.format.DataFormatDetector
+import org.slf4j.LoggerFactory
+import java.util.{Collection => JavaCollection}
+import scala.collection.JavaConversions.asJavaCollection
+import java.util.Random
+import scala.collection.mutable.{HashMap, Set, MultiMap}
+import scala.collection.immutable.NumericRange
+
+/**
+ * This class generates random customer data. The generated customer
+ * ids will be consecutive. The client code that generates the transactions
+ * records needs to know the available customer ids. If we keep the customer
+ * ids consecutive here. we don't have to store those ids in memory, or perform
+ * costly lookups. Once we introduce something that allows efficient lookup
+ * of data, we can do something else as well.
+ *
+ * The generated customer ids will start from 1. So, if we have 100 customers,
+ * the ids will be [1, 100].
+ */
+class CustomerGenerator(val desiredCustomerCount: Int, val outputPath: Path) {
+  private val logger = LoggerFactory.getLogger(getClass)
+  private val random = new Random;
+  private val assertion = "The generateCustomerRecords() hasn't been called yet";
+  private var customerFileGenerated = false
+  private val _stateToCustomerIds = new HashMap[State, NumericRange[Long]]
+
+  def isCustomerFileGenrated = customerFileGenerated
+
+  def customerIds(state: State) = {
+    assert(customerFileGenerated, assertion)
+    _stateToCustomerIds(state)
+  }
+
+  def generateCustomerRecords() = {
+    val config = new Configuration
+    val fs = FileSystem.getLocal(config)
+
+    assert(!fs.exists(outputPath))
+
+    val outputStream = fs.create(outputPath)
+
+    var currentId: Long = 1
+    logger.info("Generating customer records at: {}", fs.pathToFile(outputPath))
+    for (state <- State.values();
+            stateCustomerCount = (state.probability * desiredCustomerCount) toLong;
+            random = new Random(state.hashCode);
+            i <- 1L to stateCustomerCount) {
+      val customerRecord = CustomerGenerator.createRecord(currentId, state, random);
+      logger.info("generated customer: {}", customerRecord)
+      outputStream.writeBytes(customerRecord)
+
+      if(i == 1) {
+        val stateCustomerIdRange = currentId until (currentId + stateCustomerCount);
+        _stateToCustomerIds += (state -> stateCustomerIdRange)
+      }
+      currentId += 1
+    }
+
+    println(_stateToCustomerIds)
+    outputStream.flush
+    outputStream.close
+    customerFileGenerated = true
+  }
+}
+
+object CustomerGenerator {
+  val OUTPUT_FILE_NAME = "customers"
+
+  private def createRecord(id: Long, state: State, r: Random) = {
+    val firstName = DataForger.firstName
+    val lastName = DataForger.lastName
+    s"$id\t${DataForger.firstName(r)}\t${DataForger.lastName(r)}\t${state.name}\n"
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransaction.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransaction.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransaction.java
new file mode 100755
index 0000000..27a3407
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransaction.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.bigtop.bigpetstore.generator;
+
+import java.util.Date;
+
+public interface PetStoreTransaction {
+
+    public String getFirstName();
+
+    public String getLastName();
+
+    public String getProduct();
+
+    public Date getDate();
+
+    public Integer getPrice();
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionInputSplit.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionInputSplit.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionInputSplit.java
new file mode 100755
index 0000000..d350cc8
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionInputSplit.java
@@ -0,0 +1,73 @@
+/**
+ * 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.bigtop.bigpetstore.generator;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.bigtop.bigpetstore.generator.util.State;
+import org.apache.commons.lang3.Range;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+/**
+ * What does an `InputSplit` actually do? From the Javadocs, it looks like ...
+ * absolutely nothing.
+ *
+ * Note: for some reason, you *have* to implement Writable, even if your methods
+ * do nothing, or you will got strange and un-debuggable null pointer
+ * exceptions.
+ */
+public class PetStoreTransactionInputSplit extends InputSplit implements
+        Writable {
+
+    public PetStoreTransactionInputSplit() {
+    }
+
+    public int records;
+    public State state;
+    public Range<Long> customerIdRange;
+
+    public PetStoreTransactionInputSplit(int records, Range<Long> customerIdRange, State state) {
+        this.records = records;
+        this.state = state;
+        this.customerIdRange = customerIdRange;
+    }
+
+    public void readFields(DataInput dataInputStream) throws IOException {
+        records = dataInputStream.readInt();
+        state = State.valueOf(dataInputStream.readUTF());
+        customerIdRange = Range.between(dataInputStream.readLong(), dataInputStream.readLong());
+    }
+
+    public void write(DataOutput dataOutputStream) throws IOException {
+        dataOutputStream.writeInt(records);
+        dataOutputStream.writeUTF(state.name());
+        dataOutputStream.writeLong(customerIdRange.getMinimum());
+        dataOutputStream.writeLong(customerIdRange.getMaximum());
+    }
+
+    @Override
+    public String[] getLocations() throws IOException, InterruptedException {
+        return new String[] {};
+    }
+
+    @Override
+    public long getLength() throws IOException, InterruptedException {
+        return records;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionsInputFormat.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionsInputFormat.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionsInputFormat.java
new file mode 100755
index 0000000..4c22e36
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionsInputFormat.java
@@ -0,0 +1,139 @@
+/**
+ * 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.bigtop.bigpetstore.generator;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.bigtop.bigpetstore.generator.TransactionIteratorFactory.KeyVal;
+import org.apache.bigtop.bigpetstore.generator.util.State;
+import org.apache.commons.lang3.Range;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+/**
+ * A simple input split that fakes input.
+ */
+public class PetStoreTransactionsInputFormat extends
+    FileInputFormat<Text, Text> {
+
+  @Override
+  public RecordReader<Text, Text> createRecordReader(
+          final InputSplit inputSplit, TaskAttemptContext arg1)
+                  throws IOException, InterruptedException {
+    return new RecordReader<Text, Text>() {
+
+      @Override
+      public void close() throws IOException {
+
+      }
+
+      /**
+       * We need the "state" information to generate records. - Each state
+       * has a probability associated with it, so that our data set can be
+       * realistic (i.e. Colorado should have more transactions than rhode
+       * island).
+       *
+       * - Each state also will its name as part of the key.
+       *
+       * - This task would be distributed, for example, into 50 nodes on a
+       * real cluster, each creating the data for a given state.
+       */
+
+      PetStoreTransactionInputSplit bpsInputplit = (PetStoreTransactionInputSplit) inputSplit;
+      int records = bpsInputplit.records;
+      // TODO why not send the whole InputSplit there?
+      Iterator<KeyVal<String, String>> data =
+              (new TransactionIteratorFactory(records, bpsInputplit.customerIdRange, bpsInputplit.state)).data();
+      KeyVal<String, String> currentRecord;
+
+      @Override
+      public Text getCurrentKey() throws IOException,
+      InterruptedException {
+        return new Text(currentRecord.key());
+      }
+
+      @Override
+      public Text getCurrentValue() throws IOException,
+      InterruptedException {
+        return new Text(currentRecord.value());
+      }
+
+      @Override
+      public void initialize(InputSplit arg0, TaskAttemptContext arg1)
+              throws IOException, InterruptedException {
+      }
+
+      @Override
+      public boolean nextKeyValue() throws IOException,
+      InterruptedException {
+        if (data.hasNext()) {
+          currentRecord = data.next();
+          return true;
+        }
+        return false;
+      }
+
+      @Override
+      public float getProgress() throws IOException, InterruptedException {
+        return 0f;
+      }
+
+    };
+  }
+
+  public enum props {
+    bigpetstore_records
+  }
+
+  @Override
+  public List<InputSplit> getSplits(JobContext arg) throws IOException {
+    int numRecordsDesired = arg
+            .getConfiguration()
+            .getInt(PetStoreTransactionsInputFormat.props.bigpetstore_records
+                    .name(), -1);
+    if (numRecordsDesired == -1) {
+      throw new RuntimeException(
+              "# of total records not set in configuration object: "
+                      + arg.getConfiguration());
+    }
+
+    List<InputSplit> list = new ArrayList<InputSplit>();
+    long customerIdStart = 1;
+    for (State s : State.values()) {
+      int numRecords = numRecords(numRecordsDesired, s.probability);
+      // each state is assigned a range of customer-ids from which it can choose.
+      // The number of customers can be as many as the number of transactions.
+      Range<Long> customerIdRange = Range.between(customerIdStart, customerIdStart + numRecords - 1);
+      PetStoreTransactionInputSplit split =
+              new PetStoreTransactionInputSplit(numRecords, customerIdRange, s);
+      System.out.println(s + " _ " + split.records);
+      list.add(split);
+      customerIdStart += numRecords;
+    }
+    return list;
+  }
+
+  private int numRecords(int numRecordsDesired, float probability) {
+    return (int) (Math.ceil(numRecordsDesired * probability));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/Product.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/Product.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/Product.java
new file mode 100644
index 0000000..54ae8fe
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/Product.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bigtop.bigpetstore.generator.util;
+
+import java.math.BigDecimal;
+import static org.apache.bigtop.bigpetstore.generator.util.ProductType.*;
+
+public enum Product {
+  DOG_FOOD(DOG, 10.50),
+  ORGANIC_DOG_FOOD(DOG, 16.99),
+  STEEL_LEASH(DOG, 19.99),
+  FUZZY_COLLAR(DOG, 24.90),
+  LEATHER_COLLAR(DOG, 18.90),
+  CHOKE_COLLAR(DOG, 15.50),
+  DOG_HOUSE(DOG, 109.99),
+  CHEWY_BONE(DOG, 20.10),
+  DOG_VEST(DOG, 19.99),
+  DOG_SOAP(DOG, 5.45),
+
+  CAT_FOOD(CAT, 7.50),
+  FEEDER_BOWL(CAT, 10.99),
+  LITTER_BOX(CAT, 24.95),
+  CAT_COLLAR(CAT, 7.95),
+  CAT_BLANKET(CAT, 14.49),
+
+  TURTLE_PELLETS(TURTLE, 4.95),
+  TURTLE_FOOD(TURTLE, 10.90),
+  TURTLE_TUB(TURTLE, 40.45),
+
+  FISH_FOOD(FISH, 12.50),
+  SALMON_BAIT(FISH, 29.95),
+  FISH_BOWL(FISH, 20.99),
+  AIR_PUMP(FISH, 13.95),
+  FILTER(FISH, 34.95),
+
+  DUCK_COLLAR(DUCK, 13.25),
+  DUCK_FOOD(DUCK, 20.25),
+  WADING_POOL(DUCK, 45.90);
+
+  /*
+  ANTELOPE_COLLAR(OTHER, 19.90),
+  ANTELOPE_SNACKS(OTHER, 29.25),
+  RODENT_CAGE(OTHER, 39.95),
+  HAY_BALE(OTHER, 4.95),
+  COW_DUNG(OTHER, 1.95),
+  SEAL_SPRAY(OTHER, 24.50),
+  SNAKE_BITE_OINTMENT(OTHER, 29.90);
+  */
+  private final BigDecimal price;
+  public final ProductType productType;
+  private Product(ProductType productType, double price) {
+    this.price = BigDecimal.valueOf(price);
+    this.productType = productType;
+  }
+
+  public int id() {
+    return this.ordinal();
+  }
+
+  public BigDecimal price() {
+    return this.price;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/ProductType.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/ProductType.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/ProductType.java
new file mode 100644
index 0000000..af9ea7f
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/ProductType.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bigtop.bigpetstore.generator.util;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public enum ProductType {
+  DOG, CAT, TURTLE, FISH, DUCK;
+
+  private List<Product> products;
+
+  public List<Product> getProducts() {
+    if(products == null) {
+      generateProductList();
+    }
+    return products;
+  }
+
+  private void generateProductList() {
+    List<Product> products = new ArrayList<>();
+    for(Product p : Product.values()) {
+      if(p.productType == this) {
+        products.add(p);
+      }
+    }
+    this.products = Collections.unmodifiableList(products);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/State.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/State.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/State.java
new file mode 100644
index 0000000..2c729a7
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/generator/util/State.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bigtop.bigpetstore.generator.util;
+
+import java.util.Random;
+
+
+/**
+ * Each "state" has a pet store , with a certain "proportion" of the
+ * transactions.
+ */
+public enum State {
+  // Each state is associated with a relative probability.
+  AZ(.1f),
+  AK(.1f),
+  CT(.1f),
+  OK(.1f),
+  CO(.1f),
+  CA(.3f),
+  NY(.2f);
+
+  public static Random rand = new Random();
+  public float probability;
+
+  private State(float probability) {
+    this.probability = probability;
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/recommend/ItemRecommender.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/recommend/ItemRecommender.scala b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/recommend/ItemRecommender.scala
new file mode 100644
index 0000000..10acd5a
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/recommend/ItemRecommender.scala
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.bigtop.bigpetstore.recommend
+
+import org.apache.mahout.cf.taste.hadoop.als.RecommenderJob
+import org.apache.mahout.cf.taste.hadoop.als.ParallelALSFactorizationJob
+import java.io.File
+import parquet.org.codehaus.jackson.map.DeserializerFactory.Config
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.conf.Configurable
+import org.apache.hadoop.util.ToolRunner
+import org.apache.mahout.cf.taste.hadoop.als.SharingMapper
+import org.apache.hadoop.util.Tool
+import org.apache.bigtop.bigpetstore.util.DeveloperTools
+
+// We don't need to wrap these two jobs in ToolRunner.run calls since the only
+// thing that we are doing right now is calling the run() methods of RecommenderJob
+// and ParallelALSFactorizationJob. Both of these classes have a main() method that
+// internally calls ToolRunner.run with all the command line args passed. So, if
+// we want to run this code from the command line, we can easily do so by running
+// the main methods of the ParallelALSFactorizationJob, followed by running the
+// main method of RecommenderJob. That would also take care of the multiple-jvm
+// instance issue metioned in the comments below, so the call to
+class ItemRecommender(private val inputDir: String,
+        private val factorizationOutputDir: String,
+        private val recommendationsOutputDir: String) {
+  private val recommenderJob = new RecommenderJob
+  private val factorizationJob = new ParallelALSFactorizationJob
+
+  private def tempDir = "/tmp/mahout_" + System.currentTimeMillis
+
+  private def performAlsFactorization() = {
+    ToolRunner.run(factorizationJob, Array(
+        "--input", inputDir,
+        "--output", factorizationOutputDir,
+        "--lambda", "0.1",
+        "--tempDir", tempDir,
+        "--implicitFeedback", "false",
+        "--alpha", "0.8",
+        "--numFeatures", "2",
+        "--numIterations", "5",
+        "--numThreadsPerSolver", "1"))
+  }
+
+  private def generateRecommendations() = {
+    ToolRunner.run(recommenderJob, (Array(
+        "--input", factorizationOutputDir + "/userRatings/",
+        "--userFeatures", factorizationOutputDir + "/U/",
+        "--itemFeatures", factorizationOutputDir + "/M/",
+        "--numRecommendations", "1",
+        "--output", recommendationsOutputDir,
+        "--maxRating", "1")))
+  }
+
+  // At this point, the performAlsFactorization generateRecommendations
+  // and this method can not be run from the same VM instance. These two jobs
+  // share a common static variable which is not being handled correctly.
+  // This, unfortunately, results in a class-cast exception being thrown. That's
+  // why the resetFlagInSharedAlsMapper is required. See the comments on
+  // resetFlagInSharedAlsMapper() method.
+  def recommend = {
+    performAlsFactorization
+    resetFlagInSharedAlsMapper
+    generateRecommendations
+  }
+
+  // necessary for local execution in the same JVM only. If the performAlsFactorization()
+  // and generateRecommendations() calls are performed in separate JVM instances, this
+  // would be taken care of automatically. However, if we want to run this two methods
+  // as one task, we need to clean up the static state set by these methods, and we don't
+  // have any legitimate way of doing this directly. This clean-up should have been
+  // performed by ParallelALSFactorizationJob class after the job is finished.
+  // TODO: remove this when a better way comes along, or ParallelALSFactorizationJob
+  // takes responsibility.
+  private def resetFlagInSharedAlsMapper {
+    val m = classOf[SharingMapper[_, _, _, _, _]].getDeclaredMethod("reset");
+    m setAccessible true
+    m.invoke(null)
+  }
+}
+
+object ItemRecommender {
+  def main(args: Array[String]) {
+      val res = ToolRunner.run(new Configuration(), new Tool() {
+      var conf: Configuration = _;
+
+      override def setConf(conf: Configuration) {
+        this.conf=conf;
+      }
+
+
+      override def getConf() = {
+        this.conf;
+      }
+
+
+      override def run(toolArgs: Array[String]) = {
+        val ir = new ItemRecommender(toolArgs(0), toolArgs(1), toolArgs(2))
+        ir.recommend
+        0;
+      }
+    }, args);
+    System.exit(res);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/BigPetStoreConstants.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/BigPetStoreConstants.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/BigPetStoreConstants.java
new file mode 100755
index 0000000..01a6b95
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/BigPetStoreConstants.java
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ *
+ * Static final constants
+ *
+ * is useful to have the basic sql here as the HIVE SQL can vary between hive
+ * versions if updated here will update everywhere
+ */
+
+package org.apache.bigtop.bigpetstore.util;
+
+public class BigPetStoreConstants {
+
+   //Files should be stored in graphviz arch.dot
+   public static enum OUTPUTS {
+        generated,//generator
+        cleaned,//pig
+        tsv,
+        pig_ad_hoc_script,
+        CUSTOMER_PAGE; //crunchhh
+
+        public static enum MahoutPaths {
+          Mahout,
+          AlsFactorization,
+          AlsRecommendations
+        }
+    };
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/DeveloperTools.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/DeveloperTools.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/DeveloperTools.java
new file mode 100755
index 0000000..06671b9
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/DeveloperTools.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.bigtop.bigpetstore.util;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.mapreduce.Job;
+
+/**
+ * Dev utilities for testing arguments etc...
+ */
+public class DeveloperTools {
+
+    /**
+     * Validates that the expected args are present in the "args" array.
+     * Just some syntactic sugar for good arg error handling.
+     * @param args
+     * @param expected arguments.
+     */
+    public static void validate(String[] args, String... expected) {
+        int i=-1;
+        try{
+            for(i = 0 ; i < expected.length ; i++) {
+                System.out.println("VALUE OF " + expected[i] + " = " + args[i]);
+            }
+        }
+        catch(Throwable t) {
+            System.out.println("Argument " + i + " not available.");
+            System.out.println("We expected " + expected.length + " arguments for this phase");
+        }
+
+
+    }
+    public static void main(String[] args) throws Exception {
+        Log LOG = LogFactory.getLog(Job.class);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/NumericalIdUtils.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/NumericalIdUtils.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/NumericalIdUtils.java
new file mode 100644
index 0000000..c652beb
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/NumericalIdUtils.java
@@ -0,0 +1,48 @@
+/**
+ * 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.bigtop.bigpetstore.util;
+
+import org.apache.bigtop.bigpetstore.generator.util.State;
+
+/**
+ * User and Product IDs need numerical
+ * identifiers for recommender algorithms
+ * which attempt to interpolate new
+ * products.
+ *
+ * TODO: Delete this class. Its not necessarily required: We might just use HIVE HASH() as our
+ * standard for this.
+ */
+public class NumericalIdUtils {
+
+    /**
+     * People: Leading with ordinal code for state.
+     */
+    public static long toId(State state, String name){
+        String fromRawData =
+                state==null?
+                        name:
+                         (state.name()+"_"+name);
+        return fromRawData.hashCode();
+    }
+    /**
+     * People: Leading with ordinal code for state.
+     */
+    public static long toId(String name){
+        return toId(null,name);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/PetStoreParseFunctions.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/PetStoreParseFunctions.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/PetStoreParseFunctions.java
new file mode 100755
index 0000000..056dfc3
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/PetStoreParseFunctions.java
@@ -0,0 +1,55 @@
+/**
+ * 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.bigtop.bigpetstore.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * TODO: This might be dead code.
+ */
+public class PetStoreParseFunctions {
+
+    String[] headers = { "code", "city", "country", "lat", "lon" };
+
+    public Map<String, Object> parse(String line) {
+
+        Map<String, Object> resultMap = new HashMap<String, Object>();
+
+        List<String> csvObj = null;
+
+        String[] temp = line.split(",");
+        csvObj = new ArrayList<String>(Arrays.asList(temp));
+
+        if (csvObj.isEmpty()) {
+            return resultMap;
+        }
+
+        int k = 0;
+
+        for (String valueStr : csvObj) {
+
+            resultMap.put(headers[k++], valueStr);
+
+        }
+
+        return resultMap;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/StringUtils.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/StringUtils.java
new file mode 100644
index 0000000..e4e012e
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/util/StringUtils.java
@@ -0,0 +1,53 @@
+/**
+ * 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.bigtop.bigpetstore.util;
+
+import java.util.ArrayList;
+
+/**
+********************************************************************
+* Borrowed from apache-commons-lang StringUtils, overtime we might
+* add more elements here .
+* To maintain minor dependencies on a cluster sometimes this is easier
+* jar's manually in the hadoop classpath or via DistributedCache.
+********************************************************************/
+
+public class StringUtils {
+
+     public static String substringBefore(String str, String separator) {
+         int pos = str.indexOf(separator);
+         if (pos == -1) {
+             return str;
+         }
+         return str.substring(0, pos);
+     }
+
+
+     public static String substringAfter(String str, String separator) {
+         if (str.length()==0) {
+             return str;
+         }
+         if (separator == null) {
+             return "";
+         }
+         int pos = str.indexOf(separator);
+         if (pos == -1) {
+             return "";
+         }
+         return str.substring(pos + separator.length());
+     }
+ }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/scala/org/apache/bigtop/bigpetstore/generator/DataForger.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/scala/org/apache/bigtop/bigpetstore/generator/DataForger.scala b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/scala/org/apache/bigtop/bigpetstore/generator/DataForger.scala
new file mode 100644
index 0000000..c5e6513
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/scala/org/apache/bigtop/bigpetstore/generator/DataForger.scala
@@ -0,0 +1,280 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bigtop.bigpetstore.generator
+
+import java.util.Random
+import org.jfairy.Fairy
+import java.util.Date
+
+
+/**
+ * Generic class for generating random data. This class was created so
+ * that we can provide a uniform API for getting random data. If we want,
+ * we can replace the underlying data-generation implementation using
+ * existing libraries.
+ */
+object DataForger {
+  private val random = new Random
+  private val fairy = Fairy.create()
+
+  // TODO: Jay / Bhashit : refactor to use a random data generator?
+  def firstName(random: Random) = firstNames(random.nextInt(firstNames.length))
+  def firstName: String = firstName(random)
+
+  // TODO: Jay / Bhashit : refactor to use a random data generator?
+  def lastName(random: Random) = lastNames(random.nextInt(lastNames.length))
+  def lastName: String = lastName(random)
+
+  def randomDateInPastYears(maxYearsEarlier: Int) = fairy.dateProducer().randomDateInThePast(maxYearsEarlier).toDate()
+
+  private val firstNames =  IndexedSeq("Aaron", "Abby", "Abigail", "Adam",
+          "Alan", "Albert", "Alex", "Alexandra", "Alexis", "Alice", "Alicia",
+          "Alisha", "Alissa", "Allen", "Allison", "Alyssa", "Amanda", "Amber",
+          "Amy", "Andrea", "Andrew", "Andy", "Angel", "Angela", "Angie",
+          "Anita", "Ann", "Anna", "Annette", "Anthony", "Antonio", "April",
+          "Arthur", "Ashley", "Audrey", "Austin", "Autumn", "Baby", "Barb",
+          "Barbara", "Becky", "Benjamin", "Beth", "Bethany", "Betty",
+          "Beverly", "Bill", "Billie", "Billy", "Blake", "Bob", "Bobbie",
+          "Bobby", "Bonnie", "Brad", "Bradley", "Brady", "Brandi", "Brandon",
+          "Brandy", "Breanna", "Brenda", "Brent", "Brett", "Brian", "Brianna",
+          "Brittany", "Brooke", "Brooklyn", "Bruce", "Bryan", "Caleb",
+          "Cameron", "Candy", "Carl", "Carla", "Carmen", "Carol", "Carolyn",
+          "Carrie", "Casey", "Cassandra", "Catherine", "Cathy", "Chad",
+          "Charlene", "Charles", "Charlie", "Charlotte", "Chase", "Chasity",
+          "Chastity", "Chelsea", "Cheryl", "Chester", "Cheyenne", "Chris",
+          "Christian", "Christina", "Christine", "Christoph", "Christopher",
+          "Christy", "Chuck", "Cindy", "Clara", "Clarence", "Clayton",
+          "Clifford", "Clint", "Cody", "Colton", "Connie", "Corey", "Cory",
+          "Courtney", "Craig", "Crystal", "Curtis", "Cynthia", "Dakota",
+          "Dale", "Dallas", "Dalton", "Dan", "Dana", "Daniel", "Danielle",
+          "Danny", "Darla", "Darlene", "Darrell", "Darren", "Dave", "David",
+          "Dawn", "Dean", "Deanna", "Debbie", "Deborah", "Debra", "Denise",
+          "Dennis", "Derek", "Derrick", "Destiny", "Devin", "Diana", "Diane",
+          "Dillon", "Dixie", "Dominic", "Don", "Donald", "Donna", "Donnie",
+          "Doris", "Dorothy", "Doug", "Douglas", "Drew", "Duane", "Dustin",
+          "Dusty", "Dylan", "Earl", "Ed", "Eddie", "Edward", "Elaine",
+          "Elizabeth", "Ellen", "Emily", "Eric", "Erica", "Erika", "Erin",
+          "Ernest", "Ethan", "Eugene", "Eva", "Evelyn", "Everett", "Faith",
+          "Father", "Felicia", "Floyd", "Francis", "Frank", "Fred", "Gabriel",
+          "Gage", "Gail", "Gary", "Gene", "George", "Gerald", "Gina", "Ginger",
+          "Glen", "Glenn", "Gloria", "Grace", "Greg", "Gregory", "Haley",
+          "Hannah", "Harley", "Harold", "Harry", "Heath", "Heather", "Heidi",
+          "Helen", "Herbert", "Holly", "Hope", "Howard", "Hunter", "Ian",
+          "Isaac", "Jack", "Jackie", "Jacob", "Jade", "Jake", "James", "Jamie",
+          "Jan", "Jane", "Janet", "Janice", "Jared", "Jasmine", "Jason", "Jay",
+          "Jean", "Jeannie", "Jeff", "Jeffery", "Jeffrey", "Jenna", "Jennifer",
+          "Jenny", "Jeremiah", "Jeremy", "Jerry", "Jesse", "Jessica", "Jessie",
+          "Jill", "Jim", "Jimmy", "Joann", "Joanne", "Jodi", "Jody", "Joe",
+          "Joel", "Joey", "John", "Johnathan", "Johnny", "Jon", "Jonathan",
+          "Jonathon", "Jordan", "Joseph", "Josh", "Joshua", "Joyce", "Juanita",
+          "Judy", "Julia", "Julie", "Justin", "Kaitlyn", "Karen", "Katelyn",
+          "Katherine", "Kathleen", "Kathryn", "Kathy", "Katie", "Katrina",
+          "Kay", "Kayla", "Kaylee", "Keith", "Kelly", "Kelsey", "Ken",
+          "Kendra", "Kenneth", "Kenny", "Kevin", "Kim", "Kimberly", "Kris",
+          "Krista", "Kristen", "Kristin", "Kristina", "Kristy", "Kyle",
+          "Kylie", "Lacey", "Laken", "Lance", "Larry", "Laura", "Lawrence",
+          "Leah", "Lee", "Leonard", "Leroy", "Leslie", "Levi", "Lewis",
+          "Linda", "Lindsay", "Lindsey", "Lisa", "Lloyd", "Logan", "Lois",
+          "Loretta", "Lori", "Louis", "Lynn", "Madison", "Mandy", "Marcus",
+          "Margaret", "Maria", "Mariah", "Marie", "Marilyn", "Marion", "Mark",
+          "Marlene", "Marsha", "Martha", "Martin", "Marty", "Marvin", "Mary",
+          "Mary ann", "Mason", "Matt", "Matthew", "Max", "Megan", "Melanie",
+          "Melinda", "Melissa", "Melody", "Michael", "Michelle", "Mickey",
+          "Mike", "Mindy", "Miranda", "Misty", "Mitchell", "Molly", "Monica",
+          "Morgan", "Mother", "Myron", "Nancy", "Natasha", "Nathan",
+          "Nicholas", "Nick", "Nicole", "Nina", "Noah", "Norma", "Norman",
+          "Olivia", "Paige", "Pam", "Pamela", "Pat", "Patricia", "Patrick",
+          "Patty", "Paul", "Paula", "Peggy", "Penny", "Pete", "Phillip",
+          "Phyllis", "Rachael", "Rachel", "Ralph", "Randall", "Randi", "Randy",
+          "Ray", "Raymond", "Rebecca", "Regina", "Renee", "Rex", "Rhonda",
+          "Richard", "Rick", "Ricky", "Rita", "Rob", "Robbie", "Robert",
+          "Roberta", "Robin", "Rochelle", "Rocky", "Rod", "Rodney", "Roger",
+          "Ron", "Ronald", "Ronda", "Ronnie", "Rose", "Roxanne", "Roy", "Russ",
+          "Russell", "Rusty", "Ruth", "Ryan", "Sabrina", "Sally", "Sam",
+          "Samantha", "Samuel", "Sandra", "Sandy", "Sara", "Sarah", "Savannah",
+          "Scott", "Sean", "Seth", "Shanda", "Shane", "Shanna", "Shannon",
+          "Sharon", "Shaun", "Shawn", "Shawna", "Sheila", "Shelly", "Sher",
+          "Sherri", "Sherry", "Shirley", "Sierra", "Skyler", "Stacey", "Stacy",
+          "Stanley", "Stephanie", "Stephen", "Steve", "Steven", "Sue",
+          "Summer", "Susan", "Sydney", "Tabatha", "Tabitha", "Tamara", "Tammy",
+          "Tara", "Tasha", "Tashia", "Taylor", "Ted", "Teresa", "Terri",
+          "Terry", "Tessa", "Thelma", "Theresa", "Thomas", "Tia", "Tiffany",
+          "Tim", "Timmy", "Timothy", "Tina", "Todd", "Tom", "Tommy", "Toni",
+          "Tony", "Tonya", "Tracey", "Tracie", "Tracy", "Travis", "Trent",
+          "Trevor", "Trey", "Trisha", "Tristan", "Troy", "Tyler", "Tyrone",
+          "Unborn", "Valerie", "Vanessa", "Vernon", "Veronica", "Vicki",
+          "Vickie", "Vicky", "Victor", "Victoria", "Vincent", "Virginia",
+          "Vivian", "Walter", "Wanda", "Wayne", "Wendy", "Wesley", "Whitney",
+          "William", "Willie", "Wyatt", "Zachary")
+
+  private val lastNames = IndexedSeq("Abbott", "Acevedo", "Acosta", "Adams",
+          "Adkins", "Aguilar", "Aguirre", "Albert", "Alexander", "Alford",
+          "Allen", "Allison", "Alston", "Alvarado", "Alvarez", "Anderson",
+          "Andrews", "Anthony", "Armstrong", "Arnold", "Ashley", "Atkins",
+          "Atkinson", "Austin", "Avery", "Avila", "Ayala", "Ayers", "Bailey",
+          "Baird", "Baker", "Baldwin", "Ball", "Ballard", "Banks", "Barber",
+          "Smith", "Johnson", "Williams", "Jones", "Brown", "Davis", "Miller",
+          "Wilson", "Moore", "Taylor", "Thomas", "Jackson", "Barker", "Barlow",
+          "Barnes", "Barnett", "Barr", "Barrera", "Barrett", "Barron", "Barry",
+          "Bartlett", "Barton", "Bass", "Bates", "Battle", "Bauer", "Baxter",
+          "Beach", "Bean", "Beard", "Beasley", "Beck", "Becker", "Bell",
+          "Bender", "Benjamin", "Bennett", "Benson", "Bentley", "Benton",
+          "Berg", "Berger", "Bernard", "Berry", "Best", "Bird", "Bishop",
+          "Black", "Blackburn", "Blackwell", "Blair", "Blake", "Blanchard",
+          "Blankenship", "Blevins", "Bolton", "Bond", "Bonner", "Booker",
+          "Boone", "Booth", "Bowen", "Bowers", "Bowman", "Boyd", "Boyer",
+          "Boyle", "Bradford", "Bradley", "Bradshaw", "Brady", "Branch",
+          "Bray", "Brennan", "Brewer", "Bridges", "Briggs", "Bright", "Britt",
+          "Brock", "Brooks", "Browning", "Bruce", "Bryan", "Bryant",
+          "Buchanan", "Buck", "Buckley", "Buckner", "Bullock", "Burch",
+          "Burgess", "Burke", "Burks", "Burnett", "Burns", "Burris", "Burt",
+          "Burton", "Bush", "Butler", "Byers", "Byrd", "Cabrera", "Cain",
+          "Calderon", "Caldwell", "Calhoun", "Callahan", "Camacho", "Cameron",
+          "Campbell", "Campos", "Cannon", "Cantrell", "Cantu", "Cardenas",
+          "Carey", "Carlson", "Carney", "Carpenter", "Carr", "Carrillo",
+          "Carroll", "Carson", "Carter", "Carver", "Case", "Casey", "Cash",
+          "Castaneda", "Castillo", "Castro", "Cervantes", "Chambers", "Chan",
+          "Chandler", "Chaney", "Chang", "Chapman", "Charles", "Chase",
+          "Chavez", "Chen", "Cherry", "Christensen", "Christian", "Church",
+          "Clark", "Clarke", "Clay", "Clayton", "Clements", "Clemons",
+          "Cleveland", "Cline", "Cobb", "Cochran", "Coffey", "Cohen", "Cole",
+          "Coleman", "Collier", "Collins", "Colon", "Combs", "Compton",
+          "Conley", "Conner", "Conrad", "Contreras", "Conway", "Cook", "Cooke",
+          "Cooley", "Cooper", "Copeland", "Cortez", "Cote", "Cotton", "Cox",
+          "Craft", "Craig", "Crane", "Crawford", "Crosby", "Cross", "Cruz",
+          "Cummings", "Cunningham", "Curry", "Curtis", "Dale", "Dalton",
+          "Daniel", "Daniels", "Daugherty", "Davenport", "David", "Davidson",
+          "Dawson", "Day", "Dean", "Decker", "Dejesus", "Delacruz", "Delaney",
+          "Deleon", "Delgado", "Dennis", "Diaz", "Dickerson", "Dickinson",
+          "Dillard", "Dillon", "Dixon", "Dodson", "Dominguez", "Donaldson",
+          "Donovan", "Dorsey", "Dotson", "Douglas", "Downs", "Doyle", "Drake",
+          "Dudley", "Duffy", "Duke", "Duncan", "Dunlap", "Dunn", "Duran",
+          "Durham", "Dyer", "Eaton", "Edwards", "Elliott", "Ellis", "Ellison",
+          "Emerson", "England", "English", "Erickson", "Espinoza", "Estes",
+          "Estrada", "Evans", "Everett", "Ewing", "Farley", "Farmer",
+          "Farrell", "Faulkner", "Ferguson", "Fernandez", "Ferrell", "Fields",
+          "Figueroa", "Finch", "Finley", "Fischer", "Fisher", "Fitzgerald",
+          "Fitzpatrick", "Fleming", "Fletcher", "Flores", "Flowers", "Floyd",
+          "Flynn", "Foley", "Forbes", "Ford", "Foreman", "Foster", "Fowler",
+          "Fox", "Francis", "Franco", "Frank", "Franklin", "Franks", "Frazier",
+          "Frederick", "Freeman", "French", "Frost", "Fry", "Frye", "Fuentes",
+          "Fuller", "Fulton", "Gaines", "Gallagher", "Gallegos", "Galloway",
+          "Gamble", "Garcia", "Gardner", "Garner", "Garrett", "Garrison",
+          "Garza", "Gates", "Gay", "Gentry", "George", "Gibbs", "Gibson",
+          "Gilbert", "Giles", "Gill", "Gillespie", "Gilliam", "Gilmore",
+          "Glass", "Glenn", "Glover", "Goff", "Golden", "Gomez", "Gonzales",
+          "Gonzalez", "Good", "Goodman", "Goodwin", "Gordon", "Gould",
+          "Graham", "Grant", "Graves", "Gray", "Green", "Greene", "Greer",
+          "Gregory", "Griffin", "Griffith", "Grimes", "Gross", "Guerra",
+          "Guerrero", "Guthrie", "Gutierrez", "Guy", "Guzman", "Hahn", "Hale",
+          "Haley", "Hall", "Hamilton", "Hammond", "Hampton", "Hancock",
+          "Haney", "Hansen", "Hanson", "Hardin", "Harding", "Hardy", "Harmon",
+          "Harper", "Harris", "Harrington", "Harrison", "Hart", "Hartman",
+          "Harvey", "Hatfield", "Hawkins", "Hayden", "Hayes", "Haynes", "Hays",
+          "Head", "Heath", "Hebert", "Henderson", "Hendricks", "Hendrix",
+          "Henry", "Hensley", "Henson", "Herman", "Hernandez", "Herrera",
+          "Herring", "Hess", "Hester", "Hewitt", "Hickman", "Hicks", "Higgins",
+          "Hill", "Hines", "Hinton", "Hobbs", "Hodge", "Hodges", "Hoffman",
+          "Hogan", "Holcomb", "Holden", "Holder", "Holland", "Holloway",
+          "Holman", "Holmes", "Holt", "Hood", "Hooper", "Hoover", "Hopkins",
+          "Hopper", "Horn", "Horne", "Horton", "House", "Houston", "Howard",
+          "Howe", "Howell", "Hubbard", "Huber", "Hudson", "Huff", "Huffman",
+          "Hughes", "Hull", "Humphrey", "Hunt", "Hunter", "Hurley", "Hurst",
+          "Hutchinson", "Hyde", "Ingram", "Irwin", "Jacobs", "Jacobson",
+          "James", "Jarvis", "Jefferson", "Jenkins", "Jennings", "Jensen",
+          "Jimenez", "Johns", "Johnston", "Jordan", "Joseph", "Joyce",
+          "Joyner", "Juarez", "Justice", "Kane", "Kaufman", "Keith", "Keller",
+          "Kelley", "Kelly", "Kemp", "Kennedy", "Kent", "Kerr", "Key", "Kidd",
+          "Kim", "King", "Kinney", "Kirby", "Kirk", "Kirkland", "Klein",
+          "Kline", "Knapp", "Knight", "Knowles", "Knox", "Koch", "Kramer",
+          "Lamb", "Lambert", "Lancaster", "Landry", "Lane", "Lang", "Langley",
+          "Lara", "Larsen", "Larson", "Lawrence", "Lawson", "Le", "Leach",
+          "Leblanc", "Lee", "Leon", "Leonard", "Lester", "Levine", "Levy",
+          "Lewis", "Lindsay", "Lindsey", "Little", "Livingston", "Lloyd",
+          "Logan", "Long", "Lopez", "Lott", "Love", "Lowe", "Lowery", "Lucas",
+          "Luna", "Lynch", "Lynn", "Lyons", "Macdonald", "Macias", "Mack",
+          "Madden", "Maddox", "Maldonado", "Malone", "Mann", "Manning",
+          "Marks", "Marquez", "Marsh", "Marshall", "Martin", "Martinez",
+          "Mason", "Massey", "Mathews", "Mathis", "Matthews", "Maxwell", "May",
+          "Mayer", "Maynard", "Mayo", "Mays", "McBride", "McCall", "McCarthy",
+          "McCarty", "McClain", "McClure", "McConnell", "McCormick", "McCoy",
+          "McCray", "McCullough", "McDaniel", "McDonald", "McDowell",
+          "McFadden", "McFarland", "McGee", "McGowan", "McGuire", "McIntosh",
+          "McIntyre", "McKay", "McKee", "McKenzie", "McKinney", "McKnight",
+          "McLaughlin", "McLean", "McLeod", "McMahon", "McMillan", "McNeil",
+          "McPherson", "Meadows", "Medina", "Mejia", "Melendez", "Melton",
+          "Mendez", "Mendoza", "Mercado", "Mercer", "Merrill", "Merritt",
+          "Meyer", "Meyers", "Michael", "Middleton", "Miles", "Mills",
+          "Miranda", "Mitchell", "Molina", "Monroe", "Montgomery", "Montoya",
+          "Moody", "Moon", "Mooney", "Morales", "Moran", "Moreno", "Morgan",
+          "Morin", "Morris", "Morrison", "Morrow", "Morse", "Morton", "Moses",
+          "Mosley", "Moss", "Mueller", "Mullen", "Mullins", "Munoz", "Murphy",
+          "Murray", "Myers", "Nash", "Navarro", "Neal", "Nelson", "Newman",
+          "Newton", "Nguyen", "Nichols", "Nicholson", "Nielsen", "Nieves",
+          "Nixon", "Noble", "Noel", "Nolan", "Norman", "Norris", "Norton",
+          "Nunez", "Obrien", "Ochoa", "Oconnor", "Odom", "Odonnell", "Oliver",
+          "Olsen", "Olson", "O'neal", "O'neil", "O'neill", "Orr", "Ortega",
+          "Ortiz", "Osborn", "Osborne", "Owen", "Owens", "Pace", "Pacheco",
+          "Padilla", "Page", "Palmer", "Park", "Parker", "Parks", "Parrish",
+          "Parsons", "Pate", "Patel", "Patrick", "Patterson", "Patton", "Paul",
+          "Payne", "Pearson", "Peck", "Pena", "Pennington", "Perez", "Perkins",
+          "Perry", "Peters", "Petersen", "Peterson", "Petty", "Phelps",
+          "Phillips", "Pickett", "Pierce", "Pittman", "Pitts", "Pollard",
+          "Poole", "Pope", "Porter", "Potter", "Potts", "Powell", "Powers",
+          "Pratt", "Preston", "Price", "Prince", "Pruitt", "Puckett", "Pugh",
+          "Quinn", "Ramirez", "Ramos", "Ramsey", "Randall", "Randolph",
+          "Rasmussen", "Ratliff", "Ray", "Raymond", "Reed", "Reese", "Reeves",
+          "Reid", "Reilly", "Reyes", "Reynolds", "Rhodes", "Rice", "Rich",
+          "Richard", "Richards", "Richardson", "Richmond", "Riddle", "Riggs",
+          "Riley", "Rios", "Rivas", "Rivera", "Rivers", "Roach", "Robbins",
+          "Roberson", "Roberts", "Robertson", "Robinson", "Robles", "Rocha",
+          "Rodgers", "Rodriguez", "Rodriquez", "Rogers", "Rojas", "Rollins",
+          "Roman", "Romero", "Rosa", "Rosales", "Rosario", "Rose", "Ross",
+          "Roth", "Rowe", "Rowland", "Roy", "Ruiz", "Rush", "Russell", "Russo",
+          "Rutledge", "Ryan", "Salas", "Salazar", "Salinas", "Sampson",
+          "Sanchez", "Sanders", "Sandoval", "Sanford", "Santana", "Santiago",
+          "Santos", "Sargent", "Saunders", "Savage", "Sawyer", "Schmidt",
+          "Schneider", "Schroeder", "Schultz", "Schwartz", "Scott", "Sears",
+          "Sellers", "Serrano", "Sexton", "Shaffer", "Shannon", "Sharp",
+          "Sharpe", "Shaw", "Shelton", "Shepard", "Shepherd", "Sheppard",
+          "Sherman", "Shields", "Short", "Silva", "Simmons", "Simon",
+          "Simpson", "Sims", "Singleton", "Skinner", "Slater", "Sloan",
+          "Small", "Snider", "Snow", "Snyder", "Solis", "Solomon", "Sosa",
+          "Soto", "Sparks", "Spears", "Spence", "Spencer", "Stafford",
+          "Stanley", "Stanton", "Stark", "Steele", "Stein", "Stephens",
+          "Stephenson", "Stevens", "Stevenson", "Stewart", "Stokes", "Stone",
+          "Stout", "Strickland", "Strong", "Stuart", "Suarez", "Sullivan",
+          "Summers", "Sutton", "Swanson", "Sweeney", "Sweet", "Sykes",
+          "Talley", "Tanner", "Tate", "Terrell", "Terry", "Thompson",
+          "Thornton", "Tillman", "Todd", "Torres", "Townsend", "Tran",
+          "Travis", "Trevino", "Trujillo", "Tucker", "Turner", "Tyler",
+          "Tyson", "Underwood", "Valdez", "Valencia", "Valentine",
+          "Valenzuela", "Vance", "Vang", "Vargas", "Vasquez", "Vaughan",
+          "Vaughn", "Vazquez", "Vega", "Velasquez", "Velazquez", "Velez",
+          "Van halen", "Vincent", "Vinson", "Wade", "Wagner", "Walker", "Wall",
+          "Wallace", "Waller", "Walls", "Walsh", "Walter", "Walters", "Walton",
+          "Ward", "Ware", "Warner", "Warren", "Washington", "Waters",
+          "Watkins", "Watson", "Watts", "Weaver", "Webb", "Weber", "Webster",
+          "Weeks", "Weiss", "Welch", "Wells", "West", "Wheeler", "Whitaker",
+          "White", "Whitehead", "Whitfield", "Whitley", "Whitney", "Wiggins",
+          "Wilcox", "Wilder", "Wiley", "Wilkerson", "Wilkins", "Wilkinson",
+          "William", "Williamson", "Willis", "Winters", "Wise", "Witt", "Wolf",
+          "Wolfe", "Wong", "Wood", "Woodard", "Woods", "Woodward", "Wooten",
+          "Workman", "Wright", "Wyatt", "Wynn", "Yang", "Yates", "York",
+          "Young", "Zamora", "Zimmerman")
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/scala/org/apache/bigtop/bigpetstore/generator/TransactionIteratorFactory.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/scala/org/apache/bigtop/bigpetstore/generator/TransactionIteratorFactory.scala b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/scala/org/apache/bigtop/bigpetstore/generator/TransactionIteratorFactory.scala
new file mode 100644
index 0000000..534c606
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/scala/org/apache/bigtop/bigpetstore/generator/TransactionIteratorFactory.scala
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bigtop.bigpetstore.generator;
+
+import java.util.Date
+import org.apache.bigtop.bigpetstore.generator.util.State
+import org.apache.commons.lang3.StringUtils
+import java.util.Arrays.asList
+import java.util.Random
+import scala.collection.Iterator
+import com.sun.org.apache.xml.internal.serializer.ToStream
+import java.util.{Iterator => JavaIterator}
+import scala.collection.JavaConversions.asJavaIterator
+import org.apache.bigtop.bigpetstore.generator.util.Product
+import org.apache.commons.lang3.Range;
+import org.apache.bigtop.bigpetstore.generator.util.ProductType
+
+/**
+ * This class generates our data. Over time we will use it to embed bias which
+ * can then be teased out, i.e. by clustering/classifiers. For example:
+ *
+ * certain products <--> certain years or days
+ */
+class TransactionIteratorFactory(private val records: Int,
+        private val customerIdRange: Range[java.lang.Long],
+        private val state: State) {
+  assert(records > 0, "Number of records must be greater than 0 to generate a data iterator!")
+  private val random = new Random(state.hashCode)
+
+  def data: JavaIterator[TransactionIteratorFactory.KeyVal[String, String]] = {
+    new TransactionIteratorFactory.DataIterator(records, customerIdRange, state, random)
+  }
+}
+
+object TransactionIteratorFactory {
+  class KeyVal[K, V](val key: K, val value: V)
+
+  private class DataIterator(records: Int,
+          customerIdRange: Range[java.lang.Long],
+          state: State,
+          r: Random) extends Iterator[KeyVal[String, String]] {
+    private var firstName: String = null
+    private var lastName: String = null
+    private var elementsProcducedCount = 0
+    private var repeatCount = 0
+    private var currentCustomerId = customerIdRange.getMinimum
+    private var currentProductType = selectRandomProductType;
+
+    def hasNext =
+      elementsProcducedCount < records && currentCustomerId <= customerIdRange.getMaximum
+
+
+    def next(): TransactionIteratorFactory.KeyVal[String,String] = {
+      val date = DataForger.randomDateInPastYears(50);
+      setIteratorState();
+
+      val product = randomProductOfCurrentlySelectedType
+      val key = StringUtils.join(asList("BigPetStore", "storeCode_" + state.name(),
+              elementsProcducedCount.toString), ",")
+      val value = StringUtils.join(asList(currentCustomerId, firstName, lastName, product.id,
+              product.name.toLowerCase, product.price, date), ",")
+
+      elementsProcducedCount += 1
+      new TransactionIteratorFactory.KeyVal(key, value)
+    }
+
+    private def setIteratorState() = {
+      /** Some customers come back for more :) We repeat a customer up to ten times */
+      if (repeatCount > 0) {
+        repeatCount -= 1
+      } else {
+        firstName = DataForger.firstName(r)
+        lastName = DataForger.lastName(r)
+        // this sometimes generates numbers much larger than 10. We don't really need Gaussian
+        // distribution since number of transactions per customer can be truly arbitrary.
+        repeatCount = (r.nextGaussian * 4f) toInt;
+        println("####Repeat: " + repeatCount)
+        currentCustomerId += 1
+        currentProductType = selectRandomProductType;
+      }
+    }
+
+    private def selectRandomProductType = {
+      ProductType.values.apply(r.nextInt(ProductType.values.length))
+    }
+
+    private def randomProductOfCurrentlySelectedType = {
+      currentProductType.getProducts.get(r.nextInt(currentProductType.getProducts.size))
+    }
+  }
+}
\ No newline at end of file


[5/5] bigtop git commit: Add BPS Spark driver for new data generator. Re-organize BPS into MapReduce and Spark versions.

Posted by ja...@apache.org.
Add BPS Spark driver for new data generator. Re-organize BPS into MapReduce and Spark versions.

[BIGTOP-1366][BigPetStore] Fix bigpetstore-mapreduce/build.gradle pom.xml path

[BIGTOP-1366][BigPetStore] Refactor SparkDriver into more easily tested functions

[BIGTOP-1366][BigPetStore] Add unit test for Spark Driver

[BIGTOP-1366][BigPetStore] Update BPS Spark README to mention tests

Signed-off-by: jayunit100 <ja...@apache.org>


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

Branch: refs/heads/master
Commit: 6ec6cebfc56f05520e477768441eaecdc505ede8
Parents: 801bade
Author: RJ Nowling <rn...@gmail.com>
Authored: Fri Nov 14 14:42:13 2014 -0500
Committer: jayunit100 <ja...@apache.org>
Committed: Wed Nov 19 18:43:04 2014 -0500

----------------------------------------------------------------------
 bigtop-bigpetstore/BPS_analytics.pig            |  79 ---
 bigtop-bigpetstore/README.md                    | 214 +------
 bigtop-bigpetstore/arch.dot                     |  41 --
 .../bigpetstore-mapreduce/BPS_analytics.pig     |  79 +++
 .../bigpetstore-mapreduce/README.md             | 201 +++++++
 .../bigpetstore-mapreduce/arch.dot              |  41 ++
 .../bigpetstore-mapreduce/build.gradle          | 292 ++++++++++
 .../bigpetstore-mapreduce/pom.xml               | 584 +++++++++++++++++++
 .../bigpetstore-mapreduce/settings.gradle       |  18 +
 .../bigtop/bigpetstore/BigPetStoreMahoutIT.java |  73 +++
 .../bigtop/bigpetstore/BigPetStorePigIT.java    | 100 ++++
 .../org/apache/bigtop/bigpetstore/ITUtils.java  | 168 ++++++
 .../contract/PetStoreStatistics.java            |  34 ++
 .../bigtop/bigpetstore/etl/CrunchETL.java       | 142 +++++
 .../apache/bigtop/bigpetstore/etl/LineItem.java | 112 ++++
 .../bigtop/bigpetstore/etl/PigCSVCleaner.java   | 156 +++++
 .../bigpetstore/generator/BPSGenerator.java     | 108 ++++
 .../generator/CustomerGenerator.scala           |  97 +++
 .../generator/PetStoreTransaction.java          |  32 +
 .../PetStoreTransactionInputSplit.java          |  73 +++
 .../PetStoreTransactionsInputFormat.java        | 139 +++++
 .../bigpetstore/generator/util/Product.java     |  80 +++
 .../bigpetstore/generator/util/ProductType.java |  46 ++
 .../bigpetstore/generator/util/State.java       |  43 ++
 .../bigpetstore/recommend/ItemRecommender.scala | 121 ++++
 .../bigpetstore/util/BigPetStoreConstants.java  |  41 ++
 .../bigtop/bigpetstore/util/DeveloperTools.java |  58 ++
 .../bigpetstore/util/NumericalIdUtils.java      |  48 ++
 .../util/PetStoreParseFunctions.java            |  55 ++
 .../bigtop/bigpetstore/util/StringUtils.java    |  53 ++
 .../bigpetstore/generator/DataForger.scala      | 280 +++++++++
 .../generator/TransactionIteratorFactory.scala  | 106 ++++
 .../bigtop/bigpetstore/docs/TestDocs.java       |  37 ++
 .../generator/TestNumericalIdUtils.java         |  35 ++
 .../TestPetStoreTransactionGeneratorJob.java    | 104 ++++
 .../src/test/resources/log4j.properties         |  47 ++
 .../bigtop/bigpetstore/ScalaTestSample.scala    |  35 ++
 bigtop-bigpetstore/bigpetstore-spark/README.md  |  43 ++
 .../bigpetstore-spark/build.gradle              | 137 +++++
 .../spark/generator/SparkDriver.scala           | 244 ++++++++
 .../spark/generator/SparkDriverSuite.scala      |  60 ++
 bigtop-bigpetstore/build.gradle                 | 292 ----------
 bigtop-bigpetstore/pom.xml                      | 584 -------------------
 bigtop-bigpetstore/settings.gradle              |  18 -
 .../bigtop/bigpetstore/BigPetStoreMahoutIT.java |  73 ---
 .../bigtop/bigpetstore/BigPetStorePigIT.java    | 100 ----
 .../org/apache/bigtop/bigpetstore/ITUtils.java  | 168 ------
 .../contract/PetStoreStatistics.java            |  34 --
 .../bigtop/bigpetstore/etl/CrunchETL.java       | 142 -----
 .../apache/bigtop/bigpetstore/etl/LineItem.java | 112 ----
 .../bigtop/bigpetstore/etl/PigCSVCleaner.java   | 156 -----
 .../bigpetstore/generator/BPSGenerator.java     | 108 ----
 .../generator/CustomerGenerator.scala           |  97 ---
 .../generator/PetStoreTransaction.java          |  32 -
 .../PetStoreTransactionInputSplit.java          |  73 ---
 .../PetStoreTransactionsInputFormat.java        | 139 -----
 .../bigpetstore/generator/util/Product.java     |  80 ---
 .../bigpetstore/generator/util/ProductType.java |  46 --
 .../bigpetstore/generator/util/State.java       |  43 --
 .../bigpetstore/recommend/ItemRecommender.scala | 121 ----
 .../bigpetstore/util/BigPetStoreConstants.java  |  41 --
 .../bigtop/bigpetstore/util/DeveloperTools.java |  58 --
 .../bigpetstore/util/NumericalIdUtils.java      |  48 --
 .../util/PetStoreParseFunctions.java            |  55 --
 .../bigtop/bigpetstore/util/StringUtils.java    |  53 --
 .../bigpetstore/generator/DataForger.scala      | 280 ---------
 .../generator/TransactionIteratorFactory.scala  | 106 ----
 .../bigtop/bigpetstore/docs/TestDocs.java       |  37 --
 .../generator/TestNumericalIdUtils.java         |  35 --
 .../TestPetStoreTransactionGeneratorJob.java    | 104 ----
 .../src/test/resources/log4j.properties         |  47 --
 .../bigtop/bigpetstore/ScalaTestSample.scala    |  35 --
 72 files changed, 4145 insertions(+), 3628 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/BPS_analytics.pig
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/BPS_analytics.pig b/bigtop-bigpetstore/BPS_analytics.pig
deleted file mode 100755
index 23e3749..0000000
--- a/bigtop-bigpetstore/BPS_analytics.pig
+++ /dev/null
@@ -1,79 +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.
------------------------------------------------------------------------------
-
--- This is the analytics script that BigPetStore uses as an example for 
--- demos of how to do ad-hoc analytics on the cleaned transaction data.
--- It is used in conjunction with the big pet store web app, soon to be 
--- added to apache bigtop (As of 4/12/2014, the
--- corresponding web app to consume this scripts output is 
--- in jayunit100.github.io/bigpetstore).
-
--- invoke with two arguments, the input file , and the output file. -input /bps/gen -output /bps/analytics
-
--- FYI...
--- If you run into errors, you can see them in
--- ./target/failsafe-reports/TEST-org.bigtop.bigpetstore.integration.BigPetStorePigIT.xml
-
--- First , we load data in from a file, as tuples.
--- in pig, relations like tables in a relational database
--- so each relation is just a bunch of tuples.
--- in this case csvdata will be a relation,
--- where each tuple is a single petstore transaction.
-csvdata =
-    LOAD '$input' using PigStorage()
-        AS (
-          dump:chararray,
-          state:chararray,
-          transaction:int,
-          custId:long,
-          fname:chararray,
-          lname:chararray,
-          productId:int,
-          product:chararray,
-          price:float,
-          date:chararray);
-
--- RESULT:
--- (BigPetStore,storeCode_AK,1,11,jay,guy,3,dog-food,10.5,Thu Dec 18 12:17:10 EST 1969)
--- ...
-
--- Okay! Now lets group our data so we can do some stats.
--- lets create a new relation,
--- where each tuple will contain all transactions for a product in a state.
-
-state_product = group csvdata by ( state, product ) ;
-
--- RESULT
--- ((storeCode_AK,dog-food) , {(BigPetStore,storeCode_AK,1,11,jay,guy,3,dog-food,10.5,Thu Dec 18 12:17:10 EST 1969)}) --
--- ...
-
-
--- Okay now lets make some summary stats so that the boss man can
--- decide which products are hottest in which states.
-
--- Note that for the "groups", we tease out each individual field here for formatting with
--- the BigPetStore visualization app.
-summary1 = FOREACH state_product generate STRSPLIT(group.state,'_').$1 as sp, group.product, COUNT($1);
-
-
--- Okay, the stats look like this.  Lets clean them up.
--- (storeCode_AK,cat-food)      2530
--- (storeCode_AK,dog-food)      2540
--- (storeCode_AK,fuzzy-collar)     2495
-
-dump summary1;
-
-store summary1 into '$output';

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/README.md
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/README.md b/bigtop-bigpetstore/README.md
index c806d57..6f04e8f 100644
--- a/bigtop-bigpetstore/README.md
+++ b/bigtop-bigpetstore/README.md
@@ -1,201 +1,33 @@
-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.
-
-(See accompanying source code for licensing information)
-
 BigPetStore
 ============
 
-Apache Bigtop/Hadoop Ecosystem Demo
------------------------------------
-This software is created to demonstrate Apache Bigtop for processing
-big data sets.
-
-Architecture
-------------
-The application consists of the following modules
-
-* generator: generates raw data on the dfs
-* recommendations: Apache Mahout demo code for generating recommendations by anaylyzing the transaction records. This feature can be tracked at this [`JIRA` issue](https://issues.apache.org/jira/browse/BIGTOP-1272)
-* Pig: demo code for processing the data using Apache Pig
-* Hive: demo code for processing the data using Apache Hive. This part is not complete yet. We are working on it. You can track it using this [`JIRA` issue](https://issues.apache.org/jira/browse/BIGTOP-1270)
-* Crunch: demo code for processing the data using Apache Crunch
-
-Build Instructions
-------------------
-
-You'll need to have version 2.0 of  [`gradle`](http://www.gradle.org/downloads) installed and set-up correctly in order to follow along these instructions.
-We could have used the [`gradle-wrapper`](http://www.gradle.org/docs/current/userguide/gradle_wrapper.html) to avoid having to install `gradle`, but the `bigtop` project includes all `gradle*` directories in `.gitignore`. So, that's not going to work.
-
-### Build the JAR
-
-`gradle clean build` will build the bigpetstore `jar`. The `jar` will be located in the `build\libs` directory.
-
-### Run Intergration Tests With
-  * Pig profile: `gradle clean integrationTest -P ITProfile=pig`
-  * Mahout Profile: `gradle clean integrationTest -P ITProfile=mahout`
-  * Crunch profile: Not Implemented Yet
-  * Hive profile: Not implemented yet.
-
-If you don't specify any profile-name, or if you specify an invalid-name for the `integrationTest` task, no integration tests will be run.
-
-*Note:* At this stage, only the `Pig` and `Mahout` profiles are working. Will continue to update this area as further work is completed.
-
-For Eclipse Users
------------------
-
-1. Run `gradle eclipse` to create an eclipse project.
-2. Import the project into eclipse.
-
-*Note* whenever you modify the dependencies, you will need to run the `gradle eclipse` again. Refresh the project after doing so. You'd also need to have the `scala` plugin installed. Also, having a `gradle` plugin would be quite useful as well, for ex. when you want to update dependencies.
-
-High level summary
-------------------
-
-The bigpetstore project exemplifies the hadoop ecosystem for newcomers, and also for benchmarking and
-comparing functional space of tools.
-
-The end goal is to run many different implementations of each phase
-using different tools, thus exemplifying overlap of tools in the hadoop ecosystem, and allowing people to benchmark/compare tools
-using a common framework and easily understood use case
-
-
-How it works (To Do)
---------------------
-
-### Phase 1: Generating pet store data:
-
-The first step is to generate a raw data set.  This is done by the "GeneratePetStoreTransactionsInputFormat":
-
-The first MapReduce job in the pipeline runs a simple job which takes this input format and forwards
-its output.  The result is a list of "transactions".  Each transaction is a tuple of the format
-
-  *{state,name,date,price,product}.*
-
-### Phase 2: Processing the data
-
-The next phase of the application processes the data to create basic aggregations. For example with both pig and hive these could easily include
-
-- *Number of transactions by state* or
-- *Most valuable customer by state* or
-- *Most popular items by state*
-
-
-### Phase 3: Clustering the states by all fields
-
-  Now, say we want to cluster the states, so as to put different states into different buying categories
-  for our marketing team to deal with differently.
-
-### Phase 4: Visualizing the Data in D3.
-
- - try it [on the gh-pages branch](http://jayunit100.github.io/bigpetstore/)
-
-
-Running on a hadoop cluster
----------------------------
-
-*Note:* For running the code using the `hadoop jar` command instead of the `gradle` tasks, you will need to set the classpath appropriately. The discussion after [this comment][jira-mahout] in JIRA could also be useful apart from these instructions.
-
-### Build the fat-jar
-
-We are going to use a fat-jar in order to avoid specifying the entire classpath ourselves.
-
-The fat-jar is required when we are running the application on a hadoop cluster. The other way would be to specify all the dependencies (including the transitive ones) manually while running the hadoop job. Fat-jars make it easier to bundle almost all the dependencies inside the distribution jar itself.
-
-```
-gradle clean shadowJar -Pfor-cluster
-```
-
-This command will build the fat-jar with all the dependencies bundled in except the hadoop, mahout and pig dependencies, which we'll specify using `-libjars` option while running the hadoop job. These dependencies are excluded to avoid conflicts with the jars provided by hadoop itself.
-
-The generated jar will be inside the `build/libs` dir, with name like `BigPetStore-x.x.x-SNAPSHOT-all.jar`. For the remainig discussion I'll refer to this jar by `bps.jar`.
-
-### Get the mahout and pig jars
-
-You'll need both mahout and pig jars with the hadoop classes excluded. Commonly, you can find both of these in their respective distros. The required pig jar is generally named like `pig-x.x.x-withouthadoop.jar` and the mahout jar would be named like `mahout-core-job.jar`. If you want, you can build those yourself by following the instructions in [this JIRA comment][jira-mahout]]. For the remaining discussion, I am going to refer to these two jars by `pig-withouthadoop.jar` and `mahout-core-job.jar`.
-
-### Setup the classpath for hadoop nodes in the cluster
-
-```
-export JARS="/usr/lib/pig/pig-withouthadoop.jar,/usr/lib/mahout/mahout-core-job.jar"
-```
-
-We also need these jars to be present on the client side to kick-off the jobs. Reusing the `JARS` variable to put the same jars on the client classpath.
-
-```
-export HADOOP_CLASSPATH=`echo $JARS | sed s/,/:/g`
-```
-
-### Generate the data
-
-```
-hadoop jar bps.jar org.apache.bigtop.bigpetstore.generator.BPSGenerator 1000000 bigpetstore/gen
-```
-
-### Clean with pig
-
-```
-hadoop jar bps.jar org.apache.bigtop.bigpetstore.etl.PigCSVCleaner -libjars $JARS bigpetstore/gen/ bigpetstore/ custom_pigscript.pig
-```
-
-### Analyze and generate recommendations with mahout
-
-```
-hadoop jar bps.jar org.apache.bigtop.bigpetstore.recommend.ItemRecommender -libjars $JARS  bigpetstore/pig/Mahout bigpetstore/Mahout/AlsFactorization bigpetstore/Mahout/AlsRecommendations
-```
-
-
-... (will add more steps as we add more phases to the workflow) ...
-
-
-Example of running in EMR
---------------------------
-- Put the jar in s3.  Right now there is a copy of it at the url below.
-
-- Download the elastic-mapreduce ruby shell script.
-create your "credentials.json" file.
-
-Now run this to generate 1,000,000 pet store transactions:
-
-./elastic-mapreduce --create --jar s3://bigpetstore/bigpetstore.jar \
---main-class org.apache.bigtop.bigpetstore.generator.BPSGenerator \
---num-instances 10  \
---arg 1000000 \
---arg s3://bigpetstore/data/generated \
---hadoop-version "2.2.0"  \
---master-instance-type m1.medium \
---slave-instance-type m1.medium
+BigPetStore is a family of example applications for the Hadoop/Spark
+ecosystems. BigPetStore generates and analyzes synthetic transaction data for
+a fictional chain of petstores.
 
-...Now lets clean the data with pig...
+BigPetStore has the following aims:
 
-Replace the above "main-class", and "--arg" options with
---main-class org.apache.bigtop.bigpetstore.etl.PigCSVCleaner
---arg s3://bigpetstore/data/generated
---arg s3://bigpetstore/data/pig_out
-(optional, you can send a script referencing the cleaned $input path to do some
-custom analytics, see the BPS_Analytics.pig script and companion
-http://jayunit100.github.io/bigpetstore) as an example).
---arg s3://path_to_custom_analytics_script.pig
+* Serve as a demo application to showcase capabilities of the BigTop distribution
+* Perform integration testing for BigTop's components
+* Server as a template for building / packaging Hadoop/Spark applications
+* Provide scalable generation of complex synthetic data
+* Examples for using and integrating components such as Pig, Hive, Spark SQL, etc.
+* Examples of how to perform popular analytics tasks
 
-(note about pig: We support custom pig scripts.... for EMR, custom pig scripts will need to point to a
-local path, so youll have to put that script on the machine as part
-of EMR setup w/ a custom script).
+BigPetStore has the following components to date:
 
-...
+* Gradle build systems supporting Java, Scala, and Groovy
+* Data generators
+* Analytics
+  * ETL
+  * Item Recommenders
 
-And so on.
+The BigPetStore application was originally developed for MapReduce and associated
+components such as Pig, Hive, Mahout, Crunch, etc. With the increasing popularity
+and importance of Spark, BigPetStore has been expanded to support Spark.  To support
+the use case of deploying to pure MapReduce or Spark environments, we've elected to
+separate the MapReduce and Spark support into separate applications. You can find the
+two applications, along with futher documentation, under `bigpetstore-mapreduce` and
+`bigpetstore-spark`, respectively.
 
 
-[jira-mahout]: https://issues.apache.org/jira/browse/BIGTOP-1272?focusedCommentId=14076023&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-1407602

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/arch.dot
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/arch.dot b/bigtop-bigpetstore/arch.dot
deleted file mode 100644
index 7d17c5a..0000000
--- a/bigtop-bigpetstore/arch.dot
+++ /dev/null
@@ -1,41 +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.
-*/
-digraph bigpetstore {
-
-   node [shape=record];
-
-
-   BPSAnalytics [label="BPSAnalytics.pig" ,style="rounded, filled", shape=diamond];
-   CUSTOMER_PAGE [label="CUSTOMER_PAGE|json|CUSTOMER_PAGE/part*"];
-   DIRTY_CSV [label="DIRTY_CSV|fname   lname -prod , price ,prod,..|generated/part*"];
-   CSV [label="CSV|fname,lname,prod,price,date,xcoord,ycoord,...|cleaned/part*"];
-   MAHOUT_VIEW_INPUT [label="MAHOUT_VIEW  |  (user-id) 10001  (product-id) 203  (implicit-rating) 1 |  cleaned/Mahout/part*" ];
-   MAHOUT_ALS [label="Parallel ALS Recommender output  | (user-id) 10001  [(product-id) 201: (recommendation-strength 0-1)0.546] | Mahout/AlsRecommendations/part*" ];
-
-   Generate -> DIRTY_CSV [label="hadoop jar bigpetstore.jar org.bigtop.bigpetstore.generator.BPSGenerator 100 bps/generated/"] ;
-   DIRTY_CSV -> pig [label="hadoop jar bigpetstore.jar org.bigtop.bigpetstore.etl.PigCSVCleaner bps/generated/ bps/cleaned/ "];
-
-   pig -> CSV [label="pig query to clean up generated transaction records"];
-   pig -> MAHOUT_VIEW_INPUT [label="pig query to produce mahout input format"];
-
-   MAHOUT_VIEW_INPUT -> ParallelALSFactorizationJob [label="hadoop jar bigpetstore.jar org.apache.bigtop.bigpetstore.recommend.ItemRecommender cleaned/Mahout Mahout/AlsFactorization Mahout/AlsRecommendations"];
-   ParallelALSFactorizationJob -> "Mahout RecommenderJob"
-   "Mahout RecommenderJob" -> MAHOUT_ALS
-
-   CSV -> BPSAnalytics;
-   BPSAnalytics  -> pig_job2;
-   pig_job2  -> CUSTOMER_PAGE [label=""];
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/BPS_analytics.pig
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/BPS_analytics.pig b/bigtop-bigpetstore/bigpetstore-mapreduce/BPS_analytics.pig
new file mode 100755
index 0000000..8516a7d
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/BPS_analytics.pig
@@ -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.
+-----------------------------------------------------------------------------
+
+-- This is the analytics script that BigPetStore uses as an example for
+-- demos of how to do ad-hoc analytics on the cleaned transaction data.
+-- It is used in conjunction with the big pet store web app, soon to be
+-- added to apache bigtop (As of 4/12/2014, the
+-- corresponding web app to consume this scripts output is
+-- in jayunit100.github.io/bigpetstore).
+
+-- invoke with two arguments, the input file , and the output file. -input /bps/gen -output /bps/analytics
+
+-- FYI...
+-- If you run into errors, you can see them in
+-- ./target/failsafe-reports/TEST-org.bigtop.bigpetstore.integration.BigPetStorePigIT.xml
+
+-- First , we load data in from a file, as tuples.
+-- in pig, relations like tables in a relational database
+-- so each relation is just a bunch of tuples.
+-- in this case csvdata will be a relation,
+-- where each tuple is a single petstore transaction.
+csvdata =
+    LOAD '$input' using PigStorage()
+        AS (
+          dump:chararray,
+          state:chararray,
+          transaction:int,
+          custId:long,
+          fname:chararray,
+          lname:chararray,
+          productId:int,
+          product:chararray,
+          price:float,
+          date:chararray);
+
+-- RESULT:
+-- (BigPetStore,storeCode_AK,1,11,jay,guy,3,dog-food,10.5,Thu Dec 18 12:17:10 EST 1969)
+-- ...
+
+-- Okay! Now lets group our data so we can do some stats.
+-- lets create a new relation,
+-- where each tuple will contain all transactions for a product in a state.
+
+state_product = group csvdata by ( state, product ) ;
+
+-- RESULT
+-- ((storeCode_AK,dog-food) , {(BigPetStore,storeCode_AK,1,11,jay,guy,3,dog-food,10.5,Thu Dec 18 12:17:10 EST 1969)}) --
+-- ...
+
+
+-- Okay now lets make some summary stats so that the boss man can
+-- decide which products are hottest in which states.
+
+-- Note that for the "groups", we tease out each individual field here for formatting with
+-- the BigPetStore visualization app.
+summary1 = FOREACH state_product generate STRSPLIT(group.state,'_').$1 as sp, group.product, COUNT($1);
+
+
+-- Okay, the stats look like this.  Lets clean them up.
+-- (storeCode_AK,cat-food)      2530
+-- (storeCode_AK,dog-food)      2540
+-- (storeCode_AK,fuzzy-collar)     2495
+
+dump summary1;
+
+store summary1 into '$output';

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/README.md
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/README.md b/bigtop-bigpetstore/bigpetstore-mapreduce/README.md
new file mode 100644
index 0000000..c806d57
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/README.md
@@ -0,0 +1,201 @@
+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.
+
+(See accompanying source code for licensing information)
+
+BigPetStore
+============
+
+Apache Bigtop/Hadoop Ecosystem Demo
+-----------------------------------
+This software is created to demonstrate Apache Bigtop for processing
+big data sets.
+
+Architecture
+------------
+The application consists of the following modules
+
+* generator: generates raw data on the dfs
+* recommendations: Apache Mahout demo code for generating recommendations by anaylyzing the transaction records. This feature can be tracked at this [`JIRA` issue](https://issues.apache.org/jira/browse/BIGTOP-1272)
+* Pig: demo code for processing the data using Apache Pig
+* Hive: demo code for processing the data using Apache Hive. This part is not complete yet. We are working on it. You can track it using this [`JIRA` issue](https://issues.apache.org/jira/browse/BIGTOP-1270)
+* Crunch: demo code for processing the data using Apache Crunch
+
+Build Instructions
+------------------
+
+You'll need to have version 2.0 of  [`gradle`](http://www.gradle.org/downloads) installed and set-up correctly in order to follow along these instructions.
+We could have used the [`gradle-wrapper`](http://www.gradle.org/docs/current/userguide/gradle_wrapper.html) to avoid having to install `gradle`, but the `bigtop` project includes all `gradle*` directories in `.gitignore`. So, that's not going to work.
+
+### Build the JAR
+
+`gradle clean build` will build the bigpetstore `jar`. The `jar` will be located in the `build\libs` directory.
+
+### Run Intergration Tests With
+  * Pig profile: `gradle clean integrationTest -P ITProfile=pig`
+  * Mahout Profile: `gradle clean integrationTest -P ITProfile=mahout`
+  * Crunch profile: Not Implemented Yet
+  * Hive profile: Not implemented yet.
+
+If you don't specify any profile-name, or if you specify an invalid-name for the `integrationTest` task, no integration tests will be run.
+
+*Note:* At this stage, only the `Pig` and `Mahout` profiles are working. Will continue to update this area as further work is completed.
+
+For Eclipse Users
+-----------------
+
+1. Run `gradle eclipse` to create an eclipse project.
+2. Import the project into eclipse.
+
+*Note* whenever you modify the dependencies, you will need to run the `gradle eclipse` again. Refresh the project after doing so. You'd also need to have the `scala` plugin installed. Also, having a `gradle` plugin would be quite useful as well, for ex. when you want to update dependencies.
+
+High level summary
+------------------
+
+The bigpetstore project exemplifies the hadoop ecosystem for newcomers, and also for benchmarking and
+comparing functional space of tools.
+
+The end goal is to run many different implementations of each phase
+using different tools, thus exemplifying overlap of tools in the hadoop ecosystem, and allowing people to benchmark/compare tools
+using a common framework and easily understood use case
+
+
+How it works (To Do)
+--------------------
+
+### Phase 1: Generating pet store data:
+
+The first step is to generate a raw data set.  This is done by the "GeneratePetStoreTransactionsInputFormat":
+
+The first MapReduce job in the pipeline runs a simple job which takes this input format and forwards
+its output.  The result is a list of "transactions".  Each transaction is a tuple of the format
+
+  *{state,name,date,price,product}.*
+
+### Phase 2: Processing the data
+
+The next phase of the application processes the data to create basic aggregations. For example with both pig and hive these could easily include
+
+- *Number of transactions by state* or
+- *Most valuable customer by state* or
+- *Most popular items by state*
+
+
+### Phase 3: Clustering the states by all fields
+
+  Now, say we want to cluster the states, so as to put different states into different buying categories
+  for our marketing team to deal with differently.
+
+### Phase 4: Visualizing the Data in D3.
+
+ - try it [on the gh-pages branch](http://jayunit100.github.io/bigpetstore/)
+
+
+Running on a hadoop cluster
+---------------------------
+
+*Note:* For running the code using the `hadoop jar` command instead of the `gradle` tasks, you will need to set the classpath appropriately. The discussion after [this comment][jira-mahout] in JIRA could also be useful apart from these instructions.
+
+### Build the fat-jar
+
+We are going to use a fat-jar in order to avoid specifying the entire classpath ourselves.
+
+The fat-jar is required when we are running the application on a hadoop cluster. The other way would be to specify all the dependencies (including the transitive ones) manually while running the hadoop job. Fat-jars make it easier to bundle almost all the dependencies inside the distribution jar itself.
+
+```
+gradle clean shadowJar -Pfor-cluster
+```
+
+This command will build the fat-jar with all the dependencies bundled in except the hadoop, mahout and pig dependencies, which we'll specify using `-libjars` option while running the hadoop job. These dependencies are excluded to avoid conflicts with the jars provided by hadoop itself.
+
+The generated jar will be inside the `build/libs` dir, with name like `BigPetStore-x.x.x-SNAPSHOT-all.jar`. For the remainig discussion I'll refer to this jar by `bps.jar`.
+
+### Get the mahout and pig jars
+
+You'll need both mahout and pig jars with the hadoop classes excluded. Commonly, you can find both of these in their respective distros. The required pig jar is generally named like `pig-x.x.x-withouthadoop.jar` and the mahout jar would be named like `mahout-core-job.jar`. If you want, you can build those yourself by following the instructions in [this JIRA comment][jira-mahout]]. For the remaining discussion, I am going to refer to these two jars by `pig-withouthadoop.jar` and `mahout-core-job.jar`.
+
+### Setup the classpath for hadoop nodes in the cluster
+
+```
+export JARS="/usr/lib/pig/pig-withouthadoop.jar,/usr/lib/mahout/mahout-core-job.jar"
+```
+
+We also need these jars to be present on the client side to kick-off the jobs. Reusing the `JARS` variable to put the same jars on the client classpath.
+
+```
+export HADOOP_CLASSPATH=`echo $JARS | sed s/,/:/g`
+```
+
+### Generate the data
+
+```
+hadoop jar bps.jar org.apache.bigtop.bigpetstore.generator.BPSGenerator 1000000 bigpetstore/gen
+```
+
+### Clean with pig
+
+```
+hadoop jar bps.jar org.apache.bigtop.bigpetstore.etl.PigCSVCleaner -libjars $JARS bigpetstore/gen/ bigpetstore/ custom_pigscript.pig
+```
+
+### Analyze and generate recommendations with mahout
+
+```
+hadoop jar bps.jar org.apache.bigtop.bigpetstore.recommend.ItemRecommender -libjars $JARS  bigpetstore/pig/Mahout bigpetstore/Mahout/AlsFactorization bigpetstore/Mahout/AlsRecommendations
+```
+
+
+... (will add more steps as we add more phases to the workflow) ...
+
+
+Example of running in EMR
+--------------------------
+- Put the jar in s3.  Right now there is a copy of it at the url below.
+
+- Download the elastic-mapreduce ruby shell script.
+create your "credentials.json" file.
+
+Now run this to generate 1,000,000 pet store transactions:
+
+./elastic-mapreduce --create --jar s3://bigpetstore/bigpetstore.jar \
+--main-class org.apache.bigtop.bigpetstore.generator.BPSGenerator \
+--num-instances 10  \
+--arg 1000000 \
+--arg s3://bigpetstore/data/generated \
+--hadoop-version "2.2.0"  \
+--master-instance-type m1.medium \
+--slave-instance-type m1.medium
+
+...Now lets clean the data with pig...
+
+Replace the above "main-class", and "--arg" options with
+--main-class org.apache.bigtop.bigpetstore.etl.PigCSVCleaner
+--arg s3://bigpetstore/data/generated
+--arg s3://bigpetstore/data/pig_out
+(optional, you can send a script referencing the cleaned $input path to do some
+custom analytics, see the BPS_Analytics.pig script and companion
+http://jayunit100.github.io/bigpetstore) as an example).
+--arg s3://path_to_custom_analytics_script.pig
+
+(note about pig: We support custom pig scripts.... for EMR, custom pig scripts will need to point to a
+local path, so youll have to put that script on the machine as part
+of EMR setup w/ a custom script).
+
+...
+
+And so on.
+
+
+[jira-mahout]: https://issues.apache.org/jira/browse/BIGTOP-1272?focusedCommentId=14076023&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-1407602

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/arch.dot
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/arch.dot b/bigtop-bigpetstore/bigpetstore-mapreduce/arch.dot
new file mode 100644
index 0000000..7d17c5a
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/arch.dot
@@ -0,0 +1,41 @@
+/**
+* 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.
+*/
+digraph bigpetstore {
+
+   node [shape=record];
+
+
+   BPSAnalytics [label="BPSAnalytics.pig" ,style="rounded, filled", shape=diamond];
+   CUSTOMER_PAGE [label="CUSTOMER_PAGE|json|CUSTOMER_PAGE/part*"];
+   DIRTY_CSV [label="DIRTY_CSV|fname   lname -prod , price ,prod,..|generated/part*"];
+   CSV [label="CSV|fname,lname,prod,price,date,xcoord,ycoord,...|cleaned/part*"];
+   MAHOUT_VIEW_INPUT [label="MAHOUT_VIEW  |  (user-id) 10001  (product-id) 203  (implicit-rating) 1 |  cleaned/Mahout/part*" ];
+   MAHOUT_ALS [label="Parallel ALS Recommender output  | (user-id) 10001  [(product-id) 201: (recommendation-strength 0-1)0.546] | Mahout/AlsRecommendations/part*" ];
+
+   Generate -> DIRTY_CSV [label="hadoop jar bigpetstore.jar org.bigtop.bigpetstore.generator.BPSGenerator 100 bps/generated/"] ;
+   DIRTY_CSV -> pig [label="hadoop jar bigpetstore.jar org.bigtop.bigpetstore.etl.PigCSVCleaner bps/generated/ bps/cleaned/ "];
+
+   pig -> CSV [label="pig query to clean up generated transaction records"];
+   pig -> MAHOUT_VIEW_INPUT [label="pig query to produce mahout input format"];
+
+   MAHOUT_VIEW_INPUT -> ParallelALSFactorizationJob [label="hadoop jar bigpetstore.jar org.apache.bigtop.bigpetstore.recommend.ItemRecommender cleaned/Mahout Mahout/AlsFactorization Mahout/AlsRecommendations"];
+   ParallelALSFactorizationJob -> "Mahout RecommenderJob"
+   "Mahout RecommenderJob" -> MAHOUT_ALS
+
+   CSV -> BPSAnalytics;
+   BPSAnalytics  -> pig_job2;
+   pig_job2  -> CUSTOMER_PAGE [label=""];
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/build.gradle b/bigtop-bigpetstore/bigpetstore-mapreduce/build.gradle
new file mode 100644
index 0000000..c80672c
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/build.gradle
@@ -0,0 +1,292 @@
+/*
+ * 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.
+ */
+apply plugin: "java"
+apply plugin: "eclipse"
+// TODO add idea module config.
+apply plugin: "idea"
+apply plugin: "scala"
+apply plugin: 'com.github.johnrengelman.shadow'
+
+buildscript {
+  repositories { jcenter() }
+  dependencies {
+    classpath 'com.github.jengelman.gradle.plugins:shadow:1.0.2'
+  }
+}
+
+// Read the groupId and version properties from the "parent" bigtop project.
+// It would be better if there was some better way of doing this. Howvever,
+// at this point, we have to do this (or some variation thereof) since gradle
+// projects can't have maven projects as parents (AFAIK. If there is a way to do it,
+// it doesn't seem to be well-documented).
+def setProjectProperties() {
+    Node xml = new XmlParser().parse("../../pom.xml")
+    group = xml.groupId.first().value().first()
+    version = xml.version.first().value().first()
+}
+
+setProjectProperties()
+description = """"""
+
+// We are using 1.7 as gradle can't play well when java 8 and scala are combined.
+// There is an open issue here: http://issues.gradle.org/browse/GRADLE-3023
+// There is talk of this being resolved in the next version of gradle. Till then,
+// we are stuck with java 7. But we do have scala if we want more syntactic sugar.
+sourceCompatibility = 1.7
+targetCompatibility = 1.7
+
+// Specify any additional project properties.
+ext {
+    slf4jVersion = "1.7.5"
+    guavaVersion = "15.0"
+    datanucleusVersion = "3.2.2"
+    datanucleusJpaVersion = "3.2.1"
+    bonecpVersion = "0.8.0.RELEASE"
+    derbyVersion = "10.10.1.1"
+
+    // from horton-works repo. They compile mahout-core against hadoop2.x. These
+    // mahout is compiled against 2.4.0
+    hadoopVersion = "2.4.0.2.1.2.0-402"
+    mahoutVersion = "0.9.0.2.1.2.0-402"
+}
+
+repositories {
+    mavenCentral()
+    maven {
+        url "http://repo.hortonworks.com/content/repositories/releases/"
+    }
+}
+
+tasks.withType(AbstractCompile) {
+    options.encoding = 'UTF-8'
+    options.compilerArgs << "-Xlint:all"
+}
+
+tasks.withType(ScalaCompile) {
+    // Enables incremental compilation.
+    // http://www.gradle.org/docs/current/userguide/userguide_single.html#N12F78
+    scalaCompileOptions.useAnt = false
+}
+
+tasks.withType(Test) {
+    testLogging {
+        // Uncomment this if you want to see the console output from the tests.
+        // showStandardStreams = true
+        events "passed", "skipped", "failed"
+        // show standard out and standard error of the test JVM(s) on the console
+        //showStandardStreams = true
+    }
+}
+
+test {
+    exclude "**/*TestPig.java", "**/*TestHiveEmbedded.java", "**/*TestCrunch.java", "**/*TestPetStoreTransactionGeneratorJob.java"
+}
+
+// Create a separate source-set for the src/integrationTest set of classes. The convention here
+// is that gradle will look for a directory with the same name as that of the specified source-set
+// under the 'src' directory. So, in this case, it will look for a directory named 'src/integrationTest'
+// since the name of the source-set is 'integrationTest'
+sourceSets {
+    main {
+        java.srcDirs = [];
+        scala.srcDirs = ["src/main/scala", "src/main/java"]
+    }
+    // The main and test source-sets are configured by both java and scala plugins. They contain
+    // all the src/main and src/test classes. The following statements make all of those classes
+    // available on the classpath for the integration-tests, for both java and scala.
+    integrationTest {
+        java {
+            compileClasspath += main.output + test.output
+            runtimeClasspath += main.output + test.output
+        }
+        scala {
+            compileClasspath += main.output + test.output
+            runtimeClasspath += main.output + test.output
+        }
+    }
+}
+
+// Creating a source-set automatically add a couple of corresponding configurations (when java/scala
+// plugins are applied). The convention for these configurations is <sourceSetName>Compile and
+// <sourceSetName>Runtime. The following statements declare that all the dependencies from the
+// testCompile configuration will now be available for integrationTestCompile, and all the
+// dependencies (and other configuration that we might have provided) for testRuntime will be
+// available for integrationTestRuntime. For ex. the testCompile configuration has a dependency on
+// jUnit and scalatest. This makes them available for the integration tests as well.
+configurations {
+    integrationTestCompile {
+        extendsFrom testCompile
+    }
+
+    integrationTestRuntime {
+        extendsFrom integrationTestCompile, testRuntime
+    }
+}
+
+// To see the API that is being used here, consult the following docs
+// http://www.gradle.org/docs/current/dsl/org.gradle.api.artifacts.ResolutionStrategy.html
+def updateDependencyVersion(dependencyDetails, dependencyString) {
+    def parts = dependencyString.split(':')
+    def group = parts[0]
+    def name = parts[1]
+    def version = parts[2]
+    if (dependencyDetails.requested.group == group
+            && dependencyDetails.requested.name == name) {
+        dependencyDetails.useVersion version
+    }
+}
+
+def setupPigIntegrationTestDependencyVersions(dependencyResolveDetails) {
+    // This is the way we override the dependencies.
+    updateDependencyVersion dependencyResolveDetails, "joda-time:joda-time:2.2"
+}
+
+def setupCrunchIntegrationTestDependencyVersions(dependencyResolveDetails) {
+    // Specify any dependencies that you want to override for crunch integration tests.
+}
+
+def setupMahoutIntegrationTestDependencyVersions(dependencyResolveDetails) {
+    // Specify any dependencies that you want to override for mahout integration tests.
+}
+
+
+task integrationTest(type: Test, dependsOn: test) {
+
+    testClassesDir = sourceSets.integrationTest.output.classesDir
+    classpath = sourceSets.integrationTest.runtimeClasspath
+
+    if(!project.hasProperty('ITProfile')) {
+        // skip integration-tests if no profile has been specified.
+        integrationTest.onlyIf { false }
+        return;
+    }
+
+    def patternsToInclude
+    def dependencyConfigClosure
+    def skipDependencyUpdates = false
+    // Select the pattern for test classes that should be executed, and the dependency
+    // configuration function to be called based on the profile name specified at the command line.
+    switch (project.ITProfile) {
+        case "pig":
+            patternsToInclude = "*PigIT*"
+            dependencyConfigClosure = { setupPigIntegrationTestDependencyVersions(it) }
+            break
+        case "crunch":
+            patternsToInclude = "*CrunchIT*"
+            dependencyConfigClosure = { setupCrunchIntegrationTestDependencyVersions(it) }
+            break
+        case "mahout":
+            patternsToInclude = "*MahoutIT*"
+            dependencyConfigClosure = { setupMahoutIntegrationTestDependencyVersions(it) }
+            break
+        // skip integration-tests if the passed in profile-name is not valid
+        default: integrationTest.onlyIf { false }; return
+    }
+
+
+    filter { includeTestsMatching patternsToInclude }
+
+    // This is the standard way gradle allows overriding each specific dependency.
+    // see: http://www.gradle.org/docs/current/dsl/org.gradle.api.artifacts.ResolutionStrategy.html
+    project.configurations.all {
+        resolutionStrategy {
+            eachDependency {
+                dependencyConfigClosure(it)
+            }
+        }
+    }
+}
+
+dependencies {
+    compile "org.kohsuke:graphviz-api:1.0"
+    compile "org.apache.crunch:crunch-core:0.9.0-hadoop2"
+    compile "com.jolbox:bonecp:${project.bonecpVersion}"
+    compile "org.apache.derby:derby:${project.derbyVersion}"
+    compile "com.google.guava:guava:${project.guavaVersion}"
+    compile "commons-lang:commons-lang:2.6"
+    compile "joda-time:joda-time:2.3"
+    compile "org.apache.commons:commons-lang3:3.1"
+    compile "com.google.protobuf:protobuf-java:2.5.0"
+    compile "commons-logging:commons-logging:1.1.3"
+    compile "com.thoughtworks.xstream:xstream:+"
+    compile "org.apache.lucene:lucene-core:+"
+    compile "org.apache.lucene:lucene-analyzers-common:+"
+    compile "org.apache.solr:solr-commons-csv:3.5.0"
+
+    compile group: "org.apache.pig", name: "pig", version: "0.12.0", classifier:"h2"
+    compile "org.slf4j:slf4j-api:${project.slf4jVersion}"
+    compile "log4j:log4j:1.2.12"
+    compile "org.slf4j:slf4j-log4j12:${project.slf4jVersion}"
+    compile "org.datanucleus:datanucleus-core:${project.datanucleusVersion}"
+    compile "org.datanucleus:datanucleus-rdbms:${project.datanucleusJpaVersion}"
+    compile "org.datanucleus:datanucleus-api-jdo:${project.datanucleusJpaVersion}"
+    compile "org.datanucleus:datanucleus-accessplatform-jdo-rdbms:${project.datanucleusJpaVersion}"
+    compile group: "org.apache.mrunit", name: "mrunit", version: "1.0.0", classifier:"hadoop2"
+
+    compile "org.jfairy:jfairy:0.2.4"
+
+    // from horton-works repo. They compile mahout-core against hadoop2.x
+    compile "org.apache.hadoop:hadoop-client:${hadoopVersion}"
+    compile "org.apache.mahout:mahout-core:${mahoutVersion}"
+
+    compile 'org.scala-lang:scala-library:2.11.0'
+
+    testCompile "junit:junit:4.11"
+    testCompile "org.hamcrest:hamcrest-all:1.3"
+    testCompile "org.scalatest:scalatest_2.11:2.1.7"
+}
+
+configurations {
+    hadoopClusterRuntime {
+	    // extendsFrom integrationTestRuntime
+	    if(project.hasProperty('for-cluster')) {
+		    excludeRules += [getGroup: { 'org.apache.crunch' }, getModule: { 'crunch-core' } ] as ExcludeRule
+		    excludeRules += [getGroup: { 'org.apache.pig' }, getModule: { 'pig' } ] as ExcludeRule
+		    excludeRules += [getGroup: { 'org.apache.mahout' }, getModule: { 'mahout-core' } ] as ExcludeRule
+		    excludeRules += [getGroup: { 'org.apache.hadoop' }, getModule: { 'hadoop-client' } ] as ExcludeRule
+		}
+    }
+}
+
+task listJars << {
+    configurations.shadow.each { println it.name }
+}
+
+def copyDependencyJarsForHadoopCluster() {
+    copy {
+        from configurations.hadoopClusterRuntime
+        into 'build/libs'
+    }
+}
+
+build {
+    doLast {
+        copyDependencyJarsForHadoopCluster()
+    }
+}
+
+eclipse {
+    classpath {
+        // Add the dependencies and the src dirs for the integrationTest source-set to the
+        // .classpath file that will be generated by the eclipse plugin.
+        plusConfigurations += [configurations.integrationTestCompile]
+        // Comment out the following two lines if you want to generate an eclipse project quickly.
+        downloadSources = true
+        downloadJavadoc = false
+    }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/pom.xml b/bigtop-bigpetstore/bigpetstore-mapreduce/pom.xml
new file mode 100644
index 0000000..a5fc979
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/pom.xml
@@ -0,0 +1,584 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+	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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<groupId>org.apache.bigtop</groupId>
+	<artifactId>BigPetStore</artifactId>
+	<version>0.9.0-SNAPSHOT</version>
+	<properties>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+		<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+		<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+		<slf4j.version>1.7.5</slf4j.version>
+		<guava.version>15.0</guava.version>
+		<hadoop.version>2.2.0</hadoop.version>
+		<hive.version>0.12.0</hive.version>
+		<datanucleus.version>3.2.2</datanucleus.version>
+		<datanucleus.jpa.version>3.2.1</datanucleus.jpa.version>
+		<bonecp.version>0.9.0-SNAPSHOT.RELEASE</bonecp.version>
+		<derby.version>10.10.1.1</derby.version>
+		<plugin.surefire.version>2.17</plugin.surefire.version>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.kohsuke</groupId>
+			<artifactId>graphviz-api</artifactId>
+			<version>1.0</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.crunch</groupId>
+			<artifactId>crunch-core</artifactId>
+			<version>0.9.0-hadoop2</version>
+		</dependency>
+
+		<!-- misc deps -->
+		<dependency>
+			<groupId>com.jolbox</groupId>
+			<artifactId>bonecp</artifactId>
+			<version>${bonecp.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.derby</groupId>
+			<artifactId>derby</artifactId>
+			<version>${derby.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+
+		<!-- From pig profile -->
+		<dependency>
+			<groupId>commons-lang</groupId>
+			<artifactId>commons-lang</artifactId>
+			<version>2.6</version>
+		</dependency>
+
+		<dependency>
+			<groupId>joda-time</groupId>
+			<artifactId>joda-time</artifactId>
+			<version>2.3</version>
+		</dependency>
+		<!-- end pig profile -->
+		<!-- From hive profile -->
+		<dependency>
+			<groupId>org.apache.commons</groupId>
+			<artifactId>commons-lang3</artifactId>
+			<version>3.1</version>
+		</dependency>
+		<!-- end hive profile -->
+		<!-- From Crunch profile -->
+		<dependency>
+			<groupId>com.google.protobuf</groupId>
+			<artifactId>protobuf-java</artifactId>
+			<version>2.5.0</version>
+		</dependency>
+		<!-- end crunch profile -->
+		<!-- From Mahout profile -->
+		<dependency>
+			<groupId>commons-logging</groupId>
+			<artifactId>commons-logging</artifactId>
+			<version>1.1.3</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.mahout</groupId>
+			<artifactId>mahout-math</artifactId>
+			<version>0.9</version>
+		</dependency>
+		<dependency>
+			<groupId>com.thoughtworks.xstream</groupId>
+			<artifactId>xstream</artifactId>
+			<version>LATEST</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-core</artifactId>
+			<version>LATEST</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-analyzers-common</artifactId>
+			<version>LATEST</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.mahout.commons</groupId>
+			<artifactId>commons-cli</artifactId>
+			<version>LATEST</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.commons</groupId>
+			<artifactId>commons-math3</artifactId>
+			<version>LATEST</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.solr</groupId>
+			<artifactId>solr-commons-csv</artifactId>
+			<version>3.5.0</version>
+		</dependency>
+		<!-- end Mahout profile -->
+
+		<!-- TODO ask question about this comment -->
+		<!-- We keep this at top level so that mvn eclipse:eclipse creates a nice
+			tidy project, but its a little messy. later we'll create a profile for eclipse
+			and move this (and other deps) into profiles as needed. Important: Remove
+			this dependency when running hive integration tests... -->
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-client</artifactId>
+			<version>${hadoop.version}</version>
+		</dependency>
+		<!-- TODO ask question about this comment -->
+		<!-- mahout deps : may need to turn these on/off when testing mahout locally -->
+		<!-- For testing on my machine, I created a bigpetstore mahout jar which
+			is compiled for 2.2.0 . Or substitute this with the standard apache mahout-core
+			but not sure if it will work. -->
+		<dependency>
+			<groupId>org.apache.mahout</groupId>
+			<artifactId>mahout-core</artifactId>
+			<version>0.8</version>
+		</dependency>
+		<!-- pig deps -->
+		<dependency>
+			<groupId>org.apache.pig</groupId>
+			<artifactId>pig</artifactId>
+			<classifier>h2</classifier>
+			<version>0.12.0</version>
+		</dependency>
+
+		<!--logging -->
+
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<version>${slf4j.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>log4j</groupId>
+			<artifactId>log4j</artifactId>
+			<version>1.2.12</version>
+		</dependency>
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-log4j12</artifactId>
+			<version>${slf4j.version}</version>
+		</dependency>
+		<!-- hive -->
+		<dependency>
+			<groupId>org.apache.hive</groupId>
+			<artifactId>hive-common</artifactId>
+			<version>${hive.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hive</groupId>
+			<artifactId>hive-serde</artifactId>
+			<version>${hive.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hive</groupId>
+			<artifactId>hive-jdbc</artifactId>
+			<version>${hive.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hive</groupId>
+			<artifactId>hive-contrib</artifactId>
+			<version>${hive.version}</version>
+		</dependency>
+
+		<!-- datanucleus -->
+		<dependency>
+			<groupId>org.datanucleus</groupId>
+			<artifactId>datanucleus-core</artifactId>
+			<version>${datanucleus.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.datanucleus</groupId>
+			<artifactId>datanucleus-rdbms</artifactId>
+			<version>${datanucleus.jpa.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.datanucleus</groupId>
+			<artifactId>datanucleus-api-jdo</artifactId>
+			<version>${datanucleus.jpa.version}</version>
+		</dependency>
+
+		<!-- TODO eliminate this pom dependency -->
+		<dependency>
+			<groupId>org.datanucleus</groupId>
+			<artifactId>datanucleus-accessplatform-jdo-rdbms</artifactId>
+			<version>${datanucleus.jpa.version}</version>
+			<type>pom</type>
+		</dependency>
+
+		<!-- Unit test artifacts -->
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>4.11</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.hamcrest</groupId>
+			<artifactId>hamcrest-all</artifactId>
+			<version>1.3</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.mrunit</groupId>
+			<artifactId>mrunit</artifactId>
+			<version>1.0.0</version>
+			<classifier>hadoop2</classifier>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<extensions>
+			<extension>
+				<groupId>org.springframework.build.aws</groupId>
+				<artifactId>org.springframework.build.aws.maven</artifactId>
+				<version>3.0.0.RELEASE</version>
+			</extension>
+		</extensions>
+		<finalName>bigpetstore-${project.version}</finalName>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-release-plugin</artifactId>
+				<version>2.5</version>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-eclipse-plugin</artifactId>
+				<version>2.9</version>
+				<configuration>
+					<downloadSources>true</downloadSources>
+					<downloadJavadocs>true</downloadJavadocs>
+				</configuration>
+			</plugin>
+
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>${maven-compiler-plugin.version}</version>
+				<configuration>
+					<source>1.8</source>
+					<target>1.8</target>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<version>2.4</version>
+				<configuration>
+					<outputDirectory>${basedir}/target</outputDirectory>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<version>${plugin.surefire.version}</version>
+				<configuration>
+					<excludes>
+						<exclude>**/*TestPig.java</exclude>
+						<exclude>**/*TestHiveEmbedded.java</exclude>
+						<exclude>**/*TestCrunch.java</exclude>
+					</excludes>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+	<profiles>
+		<profile>
+			<id>pig</id>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<version>${plugin.surefire.version}</version>
+						<configuration>
+							<excludes>
+								<exclude>**/*TestPig.java</exclude>
+								<exclude>**/*TestHiveEmbedded.java</exclude>
+								<exclude>**/*TestCrunch.java</exclude>
+								<exclude>**/*TestPetStoreTransactionGeneratorJob.java</exclude>
+							</excludes>
+
+						</configuration>
+					</plugin>
+					<plugin>
+						<groupId>org.codehaus.mojo</groupId>
+						<artifactId>build-helper-maven-plugin</artifactId>
+						<version>1.5</version>
+						<executions>
+							<execution>
+								<id>add-test-source</id>
+								<phase>generate-test-sources</phase>
+								<goals>
+									<goal>add-test-source</goal>
+								</goals>
+								<configuration>
+									<sources>
+										<source>src/integration/java</source>
+									</sources>
+								</configuration>
+							</execution>
+						</executions>
+					</plugin>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-failsafe-plugin</artifactId>
+						<version>2.12</version>
+
+						<configuration>
+							<argLine>-Xmx1g</argLine>
+							<excludes>
+								<exclude>**/*BigPetStoreMahoutIT.java</exclude>
+								<exclude>**/*BigPetStoreHiveIT.java</exclude>
+								<exclude>**/*BigPetStoreCrunchIT.java</exclude>
+							</excludes>
+						</configuration>
+						<executions>
+							<!-- States that both integration-test and verify goals of the Failsafe
+								Maven plugin are executed. -->
+							<execution>
+								<id>integration-tests</id>
+								<goals>
+									<goal>integration-test</goal>
+									<goal>verify</goal>
+								</goals>
+							</execution>
+						</executions>
+					</plugin>
+				</plugins>
+			</build>
+		</profile>
+
+		<profile>
+			<id>hive</id>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<version>${plugin.surefire.version}</version>
+						<configuration>
+							<excludes>
+								<exclude>**/*TestPig.java</exclude>
+								<exclude>**/*TestHiveEmbedded.java</exclude>
+								<exclude>**/*TestCrunch.java</exclude>
+								<exclude>**/*TestPetStoreTransactionGeneratorJob.java</exclude>
+							</excludes>
+						</configuration>
+					</plugin>
+					<plugin>
+						<groupId>org.codehaus.mojo</groupId>
+						<artifactId>build-helper-maven-plugin</artifactId>
+						<version>1.5</version>
+						<executions>
+							<execution>
+								<id>add-test-source</id>
+								<phase>generate-test-sources</phase>
+								<goals>
+									<goal>add-test-source</goal>
+								</goals>
+								<configuration>
+									<sources>
+										<source>src/integration/java</source>
+									</sources>
+								</configuration>
+							</execution>
+						</executions>
+					</plugin>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-failsafe-plugin</artifactId>
+						<version>2.12</version>
+						<configuration>
+							<excludes>
+								<exclude>**/*BigPetStoreMahoutIT.java</exclude>
+								<exclude>**/*BigPetStorePigIT.java</exclude>
+								<exclude>**/*BigPetStoreCrunchIT.java</exclude>
+							</excludes>
+						</configuration>
+						<executions>
+							<!-- States that both integration-test and verify goals of the Failsafe
+								Maven plugin are executed. -->
+							<execution>
+								<id>integration-tests</id>
+								<goals>
+									<goal>integration-test</goal>
+									<goal>verify</goal>
+								</goals>
+							</execution>
+						</executions>
+					</plugin>
+				</plugins>
+			</build>
+			<dependencies>
+				<!-- hadoop -->
+				<!-- TODO is this version change required? Version 2.2.0 is provided
+					by hadoop-client dependency. Shouldn't we have the same versions for the
+					related dependencies? -->
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-mapreduce-client-app</artifactId>
+					<version>2.3.0</version>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<id>crunch</id>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<version>${plugin.surefire.version}</version>
+						<configuration>
+							<excludes>
+								<exclude>**/*TestPig.java</exclude>
+								<exclude>**/*TestHiveEmbedded.java</exclude>
+								<exclude>**/*TestCrunch.java</exclude>
+								<exclude>**/*TestPetStoreTransactionGeneratorJob.java</exclude>
+							</excludes>
+						</configuration>
+					</plugin>
+					<plugin>
+						<groupId>org.codehaus.mojo</groupId>
+						<artifactId>build-helper-maven-plugin</artifactId>
+						<version>1.5</version>
+						<executions>
+							<execution>
+								<id>add-test-source</id>
+								<phase>generate-test-sources</phase>
+								<goals>
+									<goal>add-test-source</goal>
+								</goals>
+								<configuration>
+									<sources>
+										<source>src/integration/java</source>
+									</sources>
+								</configuration>
+							</execution>
+						</executions>
+					</plugin>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-failsafe-plugin</artifactId>
+						<version>2.12</version>
+						<configuration>
+							<excludes>
+								<exclude>**/*BigPetStorePigIT.java</exclude>
+								<exclude>**/*BigPetStoreHiveIT.java</exclude>
+								<exclude>**/*BigPetStoreMahoutIT.java</exclude>
+							</excludes>
+						</configuration>
+						<executions>
+							<!-- States that both integration-test and verify goals of the Failsafe
+								Maven plugin are executed. -->
+							<execution>
+								<id>integration-tests</id>
+								<goals>
+									<goal>integration-test</goal>
+									<goal>verify</goal>
+								</goals>
+							</execution>
+						</executions>
+					</plugin>
+				</plugins>
+			</build>
+		</profile>
+		<profile>
+			<id>mahout</id>
+			<!-- TODO this property is not being used anywhere. It's not even automatically
+				detectable. Remove? Or do something that the name suggests? -->
+			<properties>
+				<skip.unit.tests>true</skip.unit.tests>
+			</properties>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<version>${plugin.surefire.version}</version>
+						<configuration>
+							<excludes>
+								<exclude>**/*TestPig.java</exclude>
+								<exclude>**/*TestHiveEmbedded.java</exclude>
+								<exclude>**/*TestCrunch.java</exclude>
+								<exclude>**/*TestPetStoreTransactionGeneratorJob.java</exclude>
+							</excludes>
+						</configuration>
+					</plugin>
+					<plugin>
+						<groupId>org.codehaus.mojo</groupId>
+						<artifactId>build-helper-maven-plugin</artifactId>
+						<version>1.5</version>
+						<executions>
+							<execution>
+								<id>add-test-source</id>
+								<phase>generate-test-sources</phase>
+								<goals>
+									<goal>add-test-source</goal>
+								</goals>
+								<configuration>
+									<sources>
+										<source>src/integration/java</source>
+									</sources>
+								</configuration>
+							</execution>
+						</executions>
+					</plugin>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-failsafe-plugin</artifactId>
+						<version>2.12</version>
+						<configuration>
+							<excludes>
+								<exclude>**/*BigPetStorePigIT.java</exclude>
+								<exclude>**/*BigPetStoreCrunchIT.java</exclude>
+								<exclude>**/*BigPetStoreHiveIT.java</exclude>
+							</excludes>
+						</configuration>
+						<executions>
+							<!-- States that both integration-test and verify goals of the Failsafe
+								Maven plugin are executed. -->
+							<execution>
+								<id>integration-tests</id>
+								<goals>
+									<goal>integration-test</goal>
+									<goal>verify</goal>
+								</goals>
+							</execution>
+						</executions>
+					</plugin>
+				</plugins>
+			</build>
+		</profile>
+	</profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/settings.gradle
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/settings.gradle b/bigtop-bigpetstore/bigpetstore-mapreduce/settings.gradle
new file mode 100644
index 0000000..53d74f2
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/settings.gradle
@@ -0,0 +1,18 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+rootProject.name = 'BigPetStore'

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStoreMahoutIT.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStoreMahoutIT.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStoreMahoutIT.java
new file mode 100644
index 0000000..b07c5a0
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStoreMahoutIT.java
@@ -0,0 +1,73 @@
+/**
+ * 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.bigtop.bigpetstore;
+
+import static org.apache.bigtop.bigpetstore.ITUtils.createTestOutputPath;
+import static org.apache.bigtop.bigpetstore.ITUtils.setup;
+
+import java.util.regex.Pattern;
+
+import org.apache.bigtop.bigpetstore.recommend.ItemRecommender;
+import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants.OUTPUTS.MahoutPaths;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.base.Predicate;
+
+public class BigPetStoreMahoutIT {
+
+  public static final Path INPUT_DIR_PATH =
+          new Path(ITUtils.BPS_TEST_PIG_CLEANED, MahoutPaths.Mahout.name());
+  public static final String INPUT_DIR_PATH_STR = INPUT_DIR_PATH.toString();
+  private static final Path MAHOUT_OUTPUT_DIR = createTestOutputPath(MahoutPaths.Mahout.name());
+  private static final Path ALS_FACTORIZATION_OUTPUT_DIR =
+          createTestOutputPath(MahoutPaths.Mahout.name(), MahoutPaths.AlsFactorization.name());
+  private static final Path ALS_RECOMMENDATIONS_DIR =
+          createTestOutputPath(MahoutPaths.Mahout.name(), MahoutPaths.AlsRecommendations.name());
+
+  private ItemRecommender itemRecommender;
+
+  @Before
+  public void setupTest() throws Throwable {
+    setup();
+    try {
+      FileSystem fs = FileSystem.get(new Configuration());
+      fs.delete(MAHOUT_OUTPUT_DIR, true);
+      itemRecommender = new ItemRecommender(INPUT_DIR_PATH_STR, ALS_FACTORIZATION_OUTPUT_DIR.toString(),
+              ALS_RECOMMENDATIONS_DIR.toString());
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static final Predicate<String> TEST_OUTPUT_FORMAT = new Predicate<String>() {
+    private final Pattern p = Pattern.compile("^\\d+\\s\\[\\d+:\\d+\\.\\d+\\]$");
+    @Override
+    public boolean apply(String input) {
+      return p.matcher(input).matches();
+    }
+  };
+
+  @Test
+  public void testPetStorePipeline() throws Exception {
+    itemRecommender.recommend();
+    ITUtils.assertOutput(ALS_RECOMMENDATIONS_DIR, TEST_OUTPUT_FORMAT);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStorePigIT.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStorePigIT.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStorePigIT.java
new file mode 100644
index 0000000..78d5c6b
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStorePigIT.java
@@ -0,0 +1,100 @@
+/**
+ * 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.bigtop.bigpetstore;
+
+import static org.apache.bigtop.bigpetstore.ITUtils.BPS_TEST_GENERATED;
+import static org.apache.bigtop.bigpetstore.ITUtils.BPS_TEST_PIG_CLEANED;
+import static org.apache.bigtop.bigpetstore.ITUtils.fs;
+
+import java.io.File;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.bigtop.bigpetstore.etl.PigCSVCleaner;
+import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.pig.ExecType;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * This is the main integration test for pig. Like all BPS integration tests, it
+ * is designed to simulate exactly what will happen on the actual cluster,
+ * except with a small amount of records.
+ *
+ * In addition to cleaning the dataset, it also runs the BPS_analytics.pig
+ * script which BigPetStore ships with.
+ */
+public class BigPetStorePigIT {
+
+	final static Logger log = LoggerFactory.getLogger(BigPetStorePigIT.class);
+
+	/**
+	 * An extra unsupported code path that we have so people can do ad hoc
+	 * analytics on pig data after it is cleaned.
+	 */
+	public static final Path BPS_TEST_PIG_COUNT_PRODUCTS = fs
+			.makeQualified(new Path("bps_integration_",
+					BigPetStoreConstants.OUTPUTS.pig_ad_hoc_script.name() + "0"));
+
+	static final File PIG_SCRIPT = new File("BPS_analytics.pig");
+
+	static {
+		if (!PIG_SCRIPT.exists()) {
+			throw new RuntimeException("Couldnt find pig script at " + PIG_SCRIPT.getAbsolutePath());
+		}
+	}
+
+	@Before
+	public void setupTest() throws Throwable {
+		ITUtils.setup();
+		try {
+			FileSystem.get(new Configuration()).delete(BPS_TEST_PIG_CLEANED, true);
+			FileSystem.get(new Configuration()).delete(BPS_TEST_PIG_COUNT_PRODUCTS, true);
+		} catch (Exception e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+	static Map<Path, Predicate<String>> TESTS = ImmutableMap.of(
+		/** Test of the main output */
+		BPS_TEST_PIG_CLEANED, ITUtils.VERIFICATION_PERDICATE,
+		// Example of how to count products after doing basic pig data cleanup
+		BPS_TEST_PIG_COUNT_PRODUCTS, ITUtils.VERIFICATION_PERDICATE,
+		// Test the output that is to be used as an input for Mahout.
+		BigPetStoreMahoutIT.INPUT_DIR_PATH, ITUtils.VERIFICATION_PERDICATE
+	);
+
+	@Test
+	public void testPetStoreCorePipeline() throws Exception {
+		runPig(BPS_TEST_GENERATED, BPS_TEST_PIG_CLEANED, PIG_SCRIPT);
+		for (Entry<Path, Predicate<String>> e : TESTS.entrySet()) {
+			ITUtils.assertOutput(e.getKey(), e.getValue());
+		}
+	}
+
+	private void runPig(Path input, Path output, File pigscript)
+			throws Exception {
+		new PigCSVCleaner(input, output, ExecType.LOCAL, pigscript);
+	}
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/ITUtils.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/ITUtils.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/ITUtils.java
new file mode 100644
index 0000000..fd53dc1
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/integrationTest/java/org/apache/bigtop/bigpetstore/ITUtils.java
@@ -0,0 +1,168 @@
+/**
+ * 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.bigtop.bigpetstore;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.net.InetAddress;
+import java.nio.charset.Charset;
+import java.util.List;
+
+import org.apache.bigtop.bigpetstore.generator.BPSGenerator;
+import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Predicate;
+import com.google.common.io.Files;
+
+public class ITUtils {
+  public static final Path TEST_OUTPUT_DIR = new Path("bps_integration_");
+
+  public static Predicate<String> VERIFICATION_PERDICATE = new Predicate<String>() {
+    @Override
+    public boolean apply(String input) {
+      return true;
+    }
+  };
+
+	static final Logger log = LoggerFactory.getLogger(ITUtils.class);
+
+	static FileSystem fs;
+	static {
+		try {
+			fs = FileSystem.getLocal(new Configuration());
+		} catch (Throwable e) {
+			String cpath = (String) System.getProperties().get("java.class.path");
+			String msg = "";
+			for (String cp : cpath.split(":")) {
+				if (cp.contains("hadoop")) {
+					msg += cp.replaceAll("hadoop", "**HADOOP**") + "\n";
+				}
+			}
+			throw new RuntimeException("Major error:  Probably issue.   "
+			        + "Check hadoop version?  " + e.getMessage()
+			        + " .... check these classpath elements:" + msg);
+		}
+	}
+
+	public static final Path BPS_TEST_GENERATED =
+	        createTestOutputPath(BigPetStoreConstants.OUTPUTS.generated.name());
+	public static final Path BPS_TEST_PIG_CLEANED =
+	        createTestOutputPath (BigPetStoreConstants.OUTPUTS.cleaned.name());
+
+	public static Path createTestOutputPath(String... pathParts) {
+	  Path path = TEST_OUTPUT_DIR;
+	  for(String pathPart: pathParts) {
+	    path = new Path(path, pathPart);
+	  }
+	  return path;
+	}
+
+	/**
+	 * Some simple checks to make sure that unit tests in local FS. these arent
+	 * designed to be run against a distribtued system.
+	 */
+	public static void checkConf(Configuration conf) throws Exception {
+		if (conf.get("mapreduce.jobtracker.address") == null) {
+			log.warn("Missing mapreduce.jobtracker.address???????!!!! " + "This can be the case in hive tests which use special "
+					+ "configurations, but we should fix it sometime.");
+			return;
+		}
+		if (!conf.get("mapreduce.jobtracker.address").equals("local")) {
+			throw new RuntimeException("ERROR: bad conf : " + "mapreduce.jobtracker.address");
+		}
+		if (!conf.get("fs.AbstractFileSystem.file.impl").contains("Local")) {
+			throw new RuntimeException("ERROR: bad conf : " + "mapreduce.jobtracker.address");
+		}
+		try {
+			InetAddress addr = java.net.InetAddress.getLocalHost();
+			System.out.println("Localhost = hn=" + addr.getHostName() + " / ha=" + addr.getHostAddress());
+		} catch (Throwable e) {
+			throw new RuntimeException(" ERROR : Hadoop wont work at all  on this machine yet"
+					+ "...I can't get / resolve localhost ! Check java version/ " + "/etc/hosts / DNS or other networking related issues on your box"
+					+ e.getMessage());
+		}
+	}
+
+	/**
+	 * Creates a generated input data set in
+	 *
+	 * test_data_directory/generated. i.e.
+	 * test_data_directory/generated/part-r-00000
+	 */
+	public static void setup() throws Throwable {
+		Configuration conf = new Configuration();
+
+		// debugging for Jeff and others in local fs that won't build
+		checkConf(conf);
+
+		conf.setInt(BPSGenerator.props.bigpetstore_records.name(), BPSGenerator.DEFAULT_NUM_RECORDS);
+
+		if (FileSystem.getLocal(conf).exists(BPS_TEST_GENERATED)) {
+			return;
+		}
+
+		Job createInput = BPSGenerator.getCreateTransactionRecordsJob(BPS_TEST_GENERATED, conf);
+		createInput.waitForCompletion(true);
+
+		Path outputfile = new Path(BPS_TEST_GENERATED, "part-r-00000");
+		List<String> lines = Files.readLines(FileSystem.getLocal(conf).pathToFile(outputfile), Charset.defaultCharset());
+		log.info("output : " + FileSystem.getLocal(conf).pathToFile(outputfile));
+		for (String l : lines) {
+			System.out.println(l);
+		}
+	}
+
+
+	// A functions that logs the output file as a verification test
+	public static void assertOutput(Path base, Predicate<String> validator) throws Exception {
+	  FileSystem fs = FileSystem.getLocal(new Configuration());
+
+	  FileStatus[] files = fs.listStatus(base);
+	  // print out all the files.
+	  for (FileStatus stat : files) {
+	    System.out.println(stat.getPath() + "  " + stat.getLen());
+	  }
+
+	  /**
+	   * Support map OR reduce outputs
+	   */
+	  Path partm = new Path(base, "part-m-00000");
+	  Path partr = new Path(base, "part-r-00000");
+	  Path p = fs.exists(partm) ? partm : partr;
+
+	  /**
+	   * Now we read through the file and validate its contents.
+	   */
+	  BufferedReader r = new BufferedReader(new InputStreamReader(fs.open(p)));
+
+	  // line:{"product":"big chew toy","count":3}
+	  while (r.ready()) {
+	    String line = r.readLine();
+	    log.info("line:" + line);
+	    // System.out.println("line:"+line);
+	    Assert.assertTrue("validationg line : " + line, validator.apply(line));
+	  }
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/contract/PetStoreStatistics.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/contract/PetStoreStatistics.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/contract/PetStoreStatistics.java
new file mode 100755
index 0000000..158f875
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/main/java/org/apache/bigtop/bigpetstore/contract/PetStoreStatistics.java
@@ -0,0 +1,34 @@
+/**
+ * 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.bigtop.bigpetstore.contract;
+
+import java.util.Map;
+
+/**
+ * This is the contract for the web site. This object is created by each ETL
+ * tool : Summary stats.
+ */
+public abstract class PetStoreStatistics {
+
+    public abstract Map<String, ? extends Number> numberOfTransactionsByState()
+            throws Exception;
+
+    public abstract Map<String, ? extends Number> numberOfProductsByProduct()
+            throws Exception;
+
+}
\ No newline at end of file


[2/5] bigtop git commit: Add BPS Spark driver for new data generator. Re-organize BPS into MapReduce and Spark versions.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/PigCSVCleaner.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/PigCSVCleaner.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/PigCSVCleaner.java
deleted file mode 100644
index 0ca7444..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/PigCSVCleaner.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.etl;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants.OUTPUTS;
-import org.apache.bigtop.bigpetstore.util.DeveloperTools;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.pig.ExecType;
-import org.apache.pig.PigServer;
-
-/**
- * This class operates by ETL'ing the data-set into pig.
- * The pigServer is persisted through the life of the class, so that the
- * intermediate data sets created in the constructor can be reused.
- */
-public class PigCSVCleaner  {
-
-    PigServer pigServer;
-
-    private static Path getCleanedTsvPath(Path outputPath) {
-      return new Path(outputPath, OUTPUTS.tsv.name());
-    }
-
-    public PigCSVCleaner(Path inputPath, Path outputPath, ExecType ex, File... scripts)
-            throws Exception {
-        FileSystem fs = FileSystem.get(inputPath.toUri(), new Configuration());
-
-        if(! fs.exists(inputPath)){
-            throw new RuntimeException("INPUT path DOES NOT exist : " + inputPath);
-        }
-
-        if(fs.exists(outputPath)){
-            throw new RuntimeException("OUTPUT already exists : " + outputPath);
-        }
-        // run pig in local mode
-        pigServer = new PigServer(ex);
-
-        /**
-         * First, split the tabs up.
-         *
-         * BigPetStore,storeCode_OK,2 1,yang,jay,3,flea collar,69.56,Mon Dec 15 23:33:49 EST 1969
-         *
-         * ("BigPetStore,storeCode_OK,2", "1,yang,jay,3,flea collar,69.56,Mon Dec 15 23:33:49 EST 1969")
-         */
-        pigServer.registerQuery("csvdata = LOAD '<i>' AS (ID,DETAILS);".replaceAll("<i>", inputPath.toString()));
-
-        // currentCustomerId, firstName, lastName, product.id, product.name.toLowerCase, product.price, date
-        /**
-         * Now, we want to split the two tab delimited fields into uniform
-         * fields of comma separated values. To do this, we 1) Internally split
-         * the FIRST and SECOND fields by commas "a,b,c" --> (a,b,c) 2) FLATTEN
-         * the FIRST and SECOND fields. (d,e) (a,b,c) -> d e a b c
-         */
-        pigServer.registerQuery(
-              "id_details = FOREACH csvdata GENERATE "
-              + "FLATTEN(STRSPLIT(ID, ',', 3)) AS " +
-			"(drop, code, transaction) ,"
-
-              + "FLATTEN(STRSPLIT(DETAILS, ',', 7)) AS " +
-                  "(custId, fname, lname, productId, product:chararray, price, date);");
-        pigServer.registerQuery("mahout_records = FOREACH id_details GENERATE custId, productId, 1;");
-        pigServer.store("id_details", getCleanedTsvPath(outputPath).toString());
-        pigServer.store("mahout_records", new Path(outputPath, OUTPUTS.MahoutPaths.Mahout.name()).toString());
-        /**
-         * Now we run scripts... this is where you can add some
-         * arbitrary analytics.
-         *
-         * We add "input" and "output" parameters so that each
-         * script can read them and use them if they want.
-         *
-         * Otherwise, just hardcode your inputs into your pig scripts.
-         */
-        int i = 0;
-        for(File script : scripts) {
-            Map<String,String> parameters = new HashMap<>();
-            parameters.put("input", getCleanedTsvPath(outputPath).toString());
-
-            Path dir = outputPath.getParent();
-            Path adHocOut = new Path(dir, OUTPUTS.pig_ad_hoc_script.name() + (i++));
-            System.out.println("Setting default output to " + adHocOut);
-            parameters.put("output", adHocOut.toString());
-            pigServer.registerScript(script.getAbsolutePath(), parameters);
-        }
-    }
-
-    private static File[] files(String[] args,int startIndex) {
-        List<File> files = new ArrayList<File>();
-        for(int i = startIndex ; i < args.length ; i++) {
-            File f = new File(args[i]);
-            if(! f.exists()) {
-                throw new RuntimeException("Pig script arg " + i + " " + f.getAbsolutePath() + " not found. ");
-            }
-            files.add(f);
-        }
-        System.out.println(
-                "Ad-hoc analytics:"+
-                "Added  " + files.size() + " pig scripts to post process.  "+
-                "Each one will be given $input and $output arguments.");
-        return files.toArray(new File[]{});
-    }
-
-    public static void main(final String[] args) throws Exception {
-        System.out.println("Starting pig etl " + args.length);
-        Configuration c = new Configuration();
-        int res = ToolRunner.run(c, new Tool() {
-                    Configuration conf;
-                    @Override
-                    public void setConf(Configuration conf) {
-                        this.conf=conf;
-                    }
-
-                    @Override
-                    public Configuration getConf() {
-                        return this.conf;
-                    }
-
-                    @Override
-                    public int run(String[] args) throws Exception {
-                        DeveloperTools.validate(
-                                args,
-                                "generated data directory",
-                                "pig output directory");
-                        new PigCSVCleaner(
-                                new Path(args[0]),
-                                new Path(args[1]),
-                                ExecType.MAPREDUCE,
-                                files(args,2));
-                        return 0;
-                    }
-                }, args);
-        System.exit(res);
-      }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/BPSGenerator.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/BPSGenerator.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/BPSGenerator.java
deleted file mode 100755
index 6c8beef..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/BPSGenerator.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.bigtop.bigpetstore.generator;
-
-import java.io.IOException;
-import java.util.Date;
-
-import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants;
-import org.apache.bigtop.bigpetstore.util.DeveloperTools;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.lib.MultipleOutputs;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Mapper.Context;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import static org.apache.bigtop.bigpetstore.generator.PetStoreTransactionsInputFormat.props;
-
-/**
- * This is a mapreduce implementation of a generator of a large sentiment
- * analysis data set. The scenario is as follows:
- *
- * The number of records will (roughly) correspond to the output size - each
- * record is about 80 bytes.
- *
- * 1KB set bigpetstore_records=10 1MB set bigpetstore_records=10,000 1GB set
- * bigpetstore_records=10,000,000 1TB set bigpetstore_records=10,000,000,000
- */
-public class BPSGenerator {
-
-  public static final int DEFAULT_NUM_RECORDS = 100;
-
-  final static Logger log = LoggerFactory.getLogger(BPSGenerator.class);
-
-  public enum props {
-    bigpetstore_records
-  }
-
-  public static Job createJob(Path output, int records) throws IOException {
-    Configuration c = new Configuration();
-    c.setInt(props.bigpetstore_records.name(), DEFAULT_NUM_RECORDS);
-    return getCreateTransactionRecordsJob(output, c);
-  }
-
-  public static Job getCreateTransactionRecordsJob(Path outputDir, Configuration conf)
-          throws IOException {
-    Job job = new Job(conf, "PetStoreTransaction_ETL_" + System.currentTimeMillis());
-    // recursively delete the data set if it exists.
-    FileSystem.get(outputDir.toUri(), conf).delete(outputDir, true);
-    job.setJarByClass(BPSGenerator.class);
-    job.setMapperClass(MyMapper.class);
-    // use the default reducer
-    // job.setReducerClass(PetStoreTransactionGeneratorJob.Red.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setMapOutputKeyClass(Text.class);
-    job.setMapOutputValueClass(Text.class);
-    job.setInputFormatClass(PetStoreTransactionsInputFormat.class);
-    job.setOutputFormatClass(TextOutputFormat.class);
-    FileOutputFormat.setOutputPath(job, outputDir);
-    return job;
-  }
-
-  public static class MyMapper extends Mapper<Text, Text, Text, Text> {
-    @Override
-    protected void setup(Context context) throws IOException,
-    InterruptedException {
-      super.setup(context);
-    }
-
-    protected void map(Text key, Text value, Context context)
-            throws java.io.IOException, InterruptedException {
-      context.write(key, value);
-    }
-  }
-
-  public static void main(String args[]) throws Exception {
-    if (args.length != 2) {
-      System.err.println("USAGE : [number of records] [output path]");
-      System.exit(0);
-    } else {
-      Configuration conf = new Configuration();
-      DeveloperTools.validate(args, "# of records", "output path");
-      conf.setInt(PetStoreTransactionsInputFormat.props.bigpetstore_records.name(),
-              Integer.parseInt(args[0]));
-      getCreateTransactionRecordsJob(new Path(args[1]), conf).waitForCompletion(true);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/CustomerGenerator.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/CustomerGenerator.scala b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/CustomerGenerator.scala
deleted file mode 100644
index 0223c8d..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/CustomerGenerator.scala
+++ /dev/null
@@ -1,97 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileSystem
-import org.apache.bigtop.bigpetstore.generator.util.State
-import org.apache.hadoop.fs.Path
-import parquet.org.codehaus.jackson.format.DataFormatDetector
-import org.slf4j.LoggerFactory
-import java.util.{Collection => JavaCollection}
-import scala.collection.JavaConversions.asJavaCollection
-import java.util.Random
-import scala.collection.mutable.{HashMap, Set, MultiMap}
-import scala.collection.immutable.NumericRange
-
-/**
- * This class generates random customer data. The generated customer
- * ids will be consecutive. The client code that generates the transactions
- * records needs to know the available customer ids. If we keep the customer
- * ids consecutive here. we don't have to store those ids in memory, or perform
- * costly lookups. Once we introduce something that allows efficient lookup
- * of data, we can do something else as well.
- *
- * The generated customer ids will start from 1. So, if we have 100 customers,
- * the ids will be [1, 100].
- */
-class CustomerGenerator(val desiredCustomerCount: Int, val outputPath: Path) {
-  private val logger = LoggerFactory.getLogger(getClass)
-  private val random = new Random;
-  private val assertion = "The generateCustomerRecords() hasn't been called yet";
-  private var customerFileGenerated = false
-  private val _stateToCustomerIds = new HashMap[State, NumericRange[Long]]
-
-  def isCustomerFileGenrated = customerFileGenerated
-
-  def customerIds(state: State) = {
-    assert(customerFileGenerated, assertion)
-    _stateToCustomerIds(state)
-  }
-
-  def generateCustomerRecords() = {
-    val config = new Configuration
-    val fs = FileSystem.getLocal(config)
-
-    assert(!fs.exists(outputPath))
-
-    val outputStream = fs.create(outputPath)
-
-    var currentId: Long = 1
-    logger.info("Generating customer records at: {}", fs.pathToFile(outputPath))
-    for (state <- State.values();
-            stateCustomerCount = (state.probability * desiredCustomerCount) toLong;
-            random = new Random(state.hashCode);
-            i <- 1L to stateCustomerCount) {
-      val customerRecord = CustomerGenerator.createRecord(currentId, state, random);
-      logger.info("generated customer: {}", customerRecord)
-      outputStream.writeBytes(customerRecord)
-
-      if(i == 1) {
-        val stateCustomerIdRange = currentId until (currentId + stateCustomerCount);
-        _stateToCustomerIds += (state -> stateCustomerIdRange)
-      }
-      currentId += 1
-    }
-
-    println(_stateToCustomerIds)
-    outputStream.flush
-    outputStream.close
-    customerFileGenerated = true
-  }
-}
-
-object CustomerGenerator {
-  val OUTPUT_FILE_NAME = "customers"
-
-  private def createRecord(id: Long, state: State, r: Random) = {
-    val firstName = DataForger.firstName
-    val lastName = DataForger.lastName
-    s"$id\t${DataForger.firstName(r)}\t${DataForger.lastName(r)}\t${state.name}\n"
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransaction.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransaction.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransaction.java
deleted file mode 100755
index 71aa6d6..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransaction.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator;
-
-import java.util.Date;
-
-public interface PetStoreTransaction {
-
-    public String getFirstName();
-
-    public String getLastName();
-
-    public String getProduct();
-
-    public Date getDate();
-
-    public Integer getPrice();
-
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionInputSplit.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionInputSplit.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionInputSplit.java
deleted file mode 100755
index d350cc8..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionInputSplit.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.bigtop.bigpetstore.generator.util.State;
-import org.apache.commons.lang3.Range;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputSplit;
-
-/**
- * What does an `InputSplit` actually do? From the Javadocs, it looks like ...
- * absolutely nothing.
- *
- * Note: for some reason, you *have* to implement Writable, even if your methods
- * do nothing, or you will got strange and un-debuggable null pointer
- * exceptions.
- */
-public class PetStoreTransactionInputSplit extends InputSplit implements
-        Writable {
-
-    public PetStoreTransactionInputSplit() {
-    }
-
-    public int records;
-    public State state;
-    public Range<Long> customerIdRange;
-
-    public PetStoreTransactionInputSplit(int records, Range<Long> customerIdRange, State state) {
-        this.records = records;
-        this.state = state;
-        this.customerIdRange = customerIdRange;
-    }
-
-    public void readFields(DataInput dataInputStream) throws IOException {
-        records = dataInputStream.readInt();
-        state = State.valueOf(dataInputStream.readUTF());
-        customerIdRange = Range.between(dataInputStream.readLong(), dataInputStream.readLong());
-    }
-
-    public void write(DataOutput dataOutputStream) throws IOException {
-        dataOutputStream.writeInt(records);
-        dataOutputStream.writeUTF(state.name());
-        dataOutputStream.writeLong(customerIdRange.getMinimum());
-        dataOutputStream.writeLong(customerIdRange.getMaximum());
-    }
-
-    @Override
-    public String[] getLocations() throws IOException, InterruptedException {
-        return new String[] {};
-    }
-
-    @Override
-    public long getLength() throws IOException, InterruptedException {
-        return records;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionsInputFormat.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionsInputFormat.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionsInputFormat.java
deleted file mode 100755
index 4c22e36..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/PetStoreTransactionsInputFormat.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.bigtop.bigpetstore.generator.TransactionIteratorFactory.KeyVal;
-import org.apache.bigtop.bigpetstore.generator.util.State;
-import org.apache.commons.lang3.Range;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-
-/**
- * A simple input split that fakes input.
- */
-public class PetStoreTransactionsInputFormat extends
-    FileInputFormat<Text, Text> {
-
-  @Override
-  public RecordReader<Text, Text> createRecordReader(
-          final InputSplit inputSplit, TaskAttemptContext arg1)
-                  throws IOException, InterruptedException {
-    return new RecordReader<Text, Text>() {
-
-      @Override
-      public void close() throws IOException {
-
-      }
-
-      /**
-       * We need the "state" information to generate records. - Each state
-       * has a probability associated with it, so that our data set can be
-       * realistic (i.e. Colorado should have more transactions than rhode
-       * island).
-       *
-       * - Each state also will its name as part of the key.
-       *
-       * - This task would be distributed, for example, into 50 nodes on a
-       * real cluster, each creating the data for a given state.
-       */
-
-      PetStoreTransactionInputSplit bpsInputplit = (PetStoreTransactionInputSplit) inputSplit;
-      int records = bpsInputplit.records;
-      // TODO why not send the whole InputSplit there?
-      Iterator<KeyVal<String, String>> data =
-              (new TransactionIteratorFactory(records, bpsInputplit.customerIdRange, bpsInputplit.state)).data();
-      KeyVal<String, String> currentRecord;
-
-      @Override
-      public Text getCurrentKey() throws IOException,
-      InterruptedException {
-        return new Text(currentRecord.key());
-      }
-
-      @Override
-      public Text getCurrentValue() throws IOException,
-      InterruptedException {
-        return new Text(currentRecord.value());
-      }
-
-      @Override
-      public void initialize(InputSplit arg0, TaskAttemptContext arg1)
-              throws IOException, InterruptedException {
-      }
-
-      @Override
-      public boolean nextKeyValue() throws IOException,
-      InterruptedException {
-        if (data.hasNext()) {
-          currentRecord = data.next();
-          return true;
-        }
-        return false;
-      }
-
-      @Override
-      public float getProgress() throws IOException, InterruptedException {
-        return 0f;
-      }
-
-    };
-  }
-
-  public enum props {
-    bigpetstore_records
-  }
-
-  @Override
-  public List<InputSplit> getSplits(JobContext arg) throws IOException {
-    int numRecordsDesired = arg
-            .getConfiguration()
-            .getInt(PetStoreTransactionsInputFormat.props.bigpetstore_records
-                    .name(), -1);
-    if (numRecordsDesired == -1) {
-      throw new RuntimeException(
-              "# of total records not set in configuration object: "
-                      + arg.getConfiguration());
-    }
-
-    List<InputSplit> list = new ArrayList<InputSplit>();
-    long customerIdStart = 1;
-    for (State s : State.values()) {
-      int numRecords = numRecords(numRecordsDesired, s.probability);
-      // each state is assigned a range of customer-ids from which it can choose.
-      // The number of customers can be as many as the number of transactions.
-      Range<Long> customerIdRange = Range.between(customerIdStart, customerIdStart + numRecords - 1);
-      PetStoreTransactionInputSplit split =
-              new PetStoreTransactionInputSplit(numRecords, customerIdRange, s);
-      System.out.println(s + " _ " + split.records);
-      list.add(split);
-      customerIdStart += numRecords;
-    }
-    return list;
-  }
-
-  private int numRecords(int numRecordsDesired, float probability) {
-    return (int) (Math.ceil(numRecordsDesired * probability));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/Product.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/Product.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/Product.java
deleted file mode 100644
index 54ae8fe..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/Product.java
+++ /dev/null
@@ -1,80 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator.util;
-
-import java.math.BigDecimal;
-import static org.apache.bigtop.bigpetstore.generator.util.ProductType.*;
-
-public enum Product {
-  DOG_FOOD(DOG, 10.50),
-  ORGANIC_DOG_FOOD(DOG, 16.99),
-  STEEL_LEASH(DOG, 19.99),
-  FUZZY_COLLAR(DOG, 24.90),
-  LEATHER_COLLAR(DOG, 18.90),
-  CHOKE_COLLAR(DOG, 15.50),
-  DOG_HOUSE(DOG, 109.99),
-  CHEWY_BONE(DOG, 20.10),
-  DOG_VEST(DOG, 19.99),
-  DOG_SOAP(DOG, 5.45),
-
-  CAT_FOOD(CAT, 7.50),
-  FEEDER_BOWL(CAT, 10.99),
-  LITTER_BOX(CAT, 24.95),
-  CAT_COLLAR(CAT, 7.95),
-  CAT_BLANKET(CAT, 14.49),
-
-  TURTLE_PELLETS(TURTLE, 4.95),
-  TURTLE_FOOD(TURTLE, 10.90),
-  TURTLE_TUB(TURTLE, 40.45),
-
-  FISH_FOOD(FISH, 12.50),
-  SALMON_BAIT(FISH, 29.95),
-  FISH_BOWL(FISH, 20.99),
-  AIR_PUMP(FISH, 13.95),
-  FILTER(FISH, 34.95),
-
-  DUCK_COLLAR(DUCK, 13.25),
-  DUCK_FOOD(DUCK, 20.25),
-  WADING_POOL(DUCK, 45.90);
-
-  /*
-  ANTELOPE_COLLAR(OTHER, 19.90),
-  ANTELOPE_SNACKS(OTHER, 29.25),
-  RODENT_CAGE(OTHER, 39.95),
-  HAY_BALE(OTHER, 4.95),
-  COW_DUNG(OTHER, 1.95),
-  SEAL_SPRAY(OTHER, 24.50),
-  SNAKE_BITE_OINTMENT(OTHER, 29.90);
-  */
-  private final BigDecimal price;
-  public final ProductType productType;
-  private Product(ProductType productType, double price) {
-    this.price = BigDecimal.valueOf(price);
-    this.productType = productType;
-  }
-
-  public int id() {
-    return this.ordinal();
-  }
-
-  public BigDecimal price() {
-    return this.price;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/ProductType.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/ProductType.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/ProductType.java
deleted file mode 100644
index af9ea7f..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/ProductType.java
+++ /dev/null
@@ -1,46 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator.util;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public enum ProductType {
-  DOG, CAT, TURTLE, FISH, DUCK;
-
-  private List<Product> products;
-
-  public List<Product> getProducts() {
-    if(products == null) {
-      generateProductList();
-    }
-    return products;
-  }
-
-  private void generateProductList() {
-    List<Product> products = new ArrayList<>();
-    for(Product p : Product.values()) {
-      if(p.productType == this) {
-        products.add(p);
-      }
-    }
-    this.products = Collections.unmodifiableList(products);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/State.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/State.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/State.java
deleted file mode 100644
index 2c729a7..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/generator/util/State.java
+++ /dev/null
@@ -1,43 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator.util;
-
-import java.util.Random;
-
-
-/**
- * Each "state" has a pet store , with a certain "proportion" of the
- * transactions.
- */
-public enum State {
-  // Each state is associated with a relative probability.
-  AZ(.1f),
-  AK(.1f),
-  CT(.1f),
-  OK(.1f),
-  CO(.1f),
-  CA(.3f),
-  NY(.2f);
-
-  public static Random rand = new Random();
-  public float probability;
-
-  private State(float probability) {
-    this.probability = probability;
-  }
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/recommend/ItemRecommender.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/recommend/ItemRecommender.scala b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/recommend/ItemRecommender.scala
deleted file mode 100644
index 10acd5a..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/recommend/ItemRecommender.scala
+++ /dev/null
@@ -1,121 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.bigtop.bigpetstore.recommend
-
-import org.apache.mahout.cf.taste.hadoop.als.RecommenderJob
-import org.apache.mahout.cf.taste.hadoop.als.ParallelALSFactorizationJob
-import java.io.File
-import parquet.org.codehaus.jackson.map.DeserializerFactory.Config
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.conf.Configurable
-import org.apache.hadoop.util.ToolRunner
-import org.apache.mahout.cf.taste.hadoop.als.SharingMapper
-import org.apache.hadoop.util.Tool
-import org.apache.bigtop.bigpetstore.util.DeveloperTools
-
-// We don't need to wrap these two jobs in ToolRunner.run calls since the only
-// thing that we are doing right now is calling the run() methods of RecommenderJob
-// and ParallelALSFactorizationJob. Both of these classes have a main() method that
-// internally calls ToolRunner.run with all the command line args passed. So, if
-// we want to run this code from the command line, we can easily do so by running
-// the main methods of the ParallelALSFactorizationJob, followed by running the
-// main method of RecommenderJob. That would also take care of the multiple-jvm
-// instance issue metioned in the comments below, so the call to
-class ItemRecommender(private val inputDir: String,
-        private val factorizationOutputDir: String,
-        private val recommendationsOutputDir: String) {
-  private val recommenderJob = new RecommenderJob
-  private val factorizationJob = new ParallelALSFactorizationJob
-
-  private def tempDir = "/tmp/mahout_" + System.currentTimeMillis
-
-  private def performAlsFactorization() = {
-    ToolRunner.run(factorizationJob, Array(
-        "--input", inputDir,
-        "--output", factorizationOutputDir,
-        "--lambda", "0.1",
-        "--tempDir", tempDir,
-        "--implicitFeedback", "false",
-        "--alpha", "0.8",
-        "--numFeatures", "2",
-        "--numIterations", "5",
-        "--numThreadsPerSolver", "1"))
-  }
-
-  private def generateRecommendations() = {
-    ToolRunner.run(recommenderJob, (Array(
-        "--input", factorizationOutputDir + "/userRatings/",
-        "--userFeatures", factorizationOutputDir + "/U/",
-        "--itemFeatures", factorizationOutputDir + "/M/",
-        "--numRecommendations", "1",
-        "--output", recommendationsOutputDir,
-        "--maxRating", "1")))
-  }
-
-  // At this point, the performAlsFactorization generateRecommendations
-  // and this method can not be run from the same VM instance. These two jobs
-  // share a common static variable which is not being handled correctly.
-  // This, unfortunately, results in a class-cast exception being thrown. That's
-  // why the resetFlagInSharedAlsMapper is required. See the comments on
-  // resetFlagInSharedAlsMapper() method.
-  def recommend = {
-    performAlsFactorization
-    resetFlagInSharedAlsMapper
-    generateRecommendations
-  }
-
-  // necessary for local execution in the same JVM only. If the performAlsFactorization()
-  // and generateRecommendations() calls are performed in separate JVM instances, this
-  // would be taken care of automatically. However, if we want to run this two methods
-  // as one task, we need to clean up the static state set by these methods, and we don't
-  // have any legitimate way of doing this directly. This clean-up should have been
-  // performed by ParallelALSFactorizationJob class after the job is finished.
-  // TODO: remove this when a better way comes along, or ParallelALSFactorizationJob
-  // takes responsibility.
-  private def resetFlagInSharedAlsMapper {
-    val m = classOf[SharingMapper[_, _, _, _, _]].getDeclaredMethod("reset");
-    m setAccessible true
-    m.invoke(null)
-  }
-}
-
-object ItemRecommender {
-  def main(args: Array[String]) {
-      val res = ToolRunner.run(new Configuration(), new Tool() {
-      var conf: Configuration = _;
-
-      override def setConf(conf: Configuration) {
-        this.conf=conf;
-      }
-
-
-      override def getConf() = {
-        this.conf;
-      }
-
-
-      override def run(toolArgs: Array[String]) = {
-        val ir = new ItemRecommender(toolArgs(0), toolArgs(1), toolArgs(2))
-        ir.recommend
-        0;
-      }
-    }, args);
-    System.exit(res);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/BigPetStoreConstants.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/BigPetStoreConstants.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/BigPetStoreConstants.java
deleted file mode 100755
index 01a6b95..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/BigPetStoreConstants.java
+++ /dev/null
@@ -1,41 +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.
- *
- * Static final constants
- *
- * is useful to have the basic sql here as the HIVE SQL can vary between hive
- * versions if updated here will update everywhere
- */
-
-package org.apache.bigtop.bigpetstore.util;
-
-public class BigPetStoreConstants {
-
-   //Files should be stored in graphviz arch.dot
-   public static enum OUTPUTS {
-        generated,//generator
-        cleaned,//pig
-        tsv,
-        pig_ad_hoc_script,
-        CUSTOMER_PAGE; //crunchhh
-
-        public static enum MahoutPaths {
-          Mahout,
-          AlsFactorization,
-          AlsRecommendations
-        }
-    };
-
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/DeveloperTools.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/DeveloperTools.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/DeveloperTools.java
deleted file mode 100755
index 9c2d684..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/DeveloperTools.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.util;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.mapreduce.Job;
-
-/**
- * Dev utilities for testing arguments etc...
- */
-public class DeveloperTools {
-
-    /**
-     * Validates that the expected args are present in the "args" array.
-     * Just some syntactic sugar for good arg error handling.
-     * @param args
-     * @param expected arguments.
-     */
-    public static void validate(String[] args, String... expected) {
-        int i=-1;
-        try{
-            for(i = 0 ; i < expected.length ; i++) {
-                System.out.println("VALUE OF " + expected[i] + " = " + args[i]);
-            }
-        }
-        catch(Throwable t) {
-            System.out.println("Argument " + i + " not available.");
-            System.out.println("We expected " + expected.length + " arguments for this phase");
-        }
-
-
-    }
-    public static void main(String[] args) throws Exception {
-        Log LOG = LogFactory.getLog(Job.class);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/NumericalIdUtils.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/NumericalIdUtils.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/NumericalIdUtils.java
deleted file mode 100644
index c652beb..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/NumericalIdUtils.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.bigtop.bigpetstore.util;
-
-import org.apache.bigtop.bigpetstore.generator.util.State;
-
-/**
- * User and Product IDs need numerical
- * identifiers for recommender algorithms
- * which attempt to interpolate new
- * products.
- *
- * TODO: Delete this class. Its not necessarily required: We might just use HIVE HASH() as our
- * standard for this.
- */
-public class NumericalIdUtils {
-
-    /**
-     * People: Leading with ordinal code for state.
-     */
-    public static long toId(State state, String name){
-        String fromRawData =
-                state==null?
-                        name:
-                         (state.name()+"_"+name);
-        return fromRawData.hashCode();
-    }
-    /**
-     * People: Leading with ordinal code for state.
-     */
-    public static long toId(String name){
-        return toId(null,name);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/PetStoreParseFunctions.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/PetStoreParseFunctions.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/PetStoreParseFunctions.java
deleted file mode 100755
index 7b6bede..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/PetStoreParseFunctions.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.bigtop.bigpetstore.util;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * TODO: This might be dead code.
- */
-public class PetStoreParseFunctions {
-
-    String[] headers = { "code", "city", "country", "lat", "lon" };
-
-    public Map<String, Object> parse(String line) {
-
-        Map<String, Object> resultMap = new HashMap<String, Object>();
-
-        List<String> csvObj = null;
-
-        String[] temp = line.split(",");
-        csvObj = new ArrayList<String>(Arrays.asList(temp));
-
-        if (csvObj.isEmpty()) {
-            return resultMap;
-        }
-
-        int k = 0;
-
-        for (String valueStr : csvObj) {
-
-            resultMap.put(headers[k++], valueStr);
-
-        }
-
-        return resultMap;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/StringUtils.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/StringUtils.java
deleted file mode 100644
index 02399bf..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/util/StringUtils.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.util;
-
-import java.util.ArrayList;
-
-/**
-********************************************************************
-* Borrowed from apache-commons-lang StringUtils, overtime we might
-* add more elements here .  
-* To maintain minor dependencies on a cluster sometimes this is easier
-* jar's manually in the hadoop classpath or via DistributedCache. 
-********************************************************************/
-
-public class StringUtils {
-
-     public static String substringBefore(String str, String separator) {
-         int pos = str.indexOf(separator);
-         if (pos == -1) {
-             return str;
-         }
-         return str.substring(0, pos);
-     }
-
-
-     public static String substringAfter(String str, String separator) {
-         if (str.length()==0) {
-             return str;
-         }
-         if (separator == null) {
-             return "";
-         }
-         int pos = str.indexOf(separator);
-         if (pos == -1) {
-             return "";
-         }
-         return str.substring(pos + separator.length());
-     }
- }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/scala/org/apache/bigtop/bigpetstore/generator/DataForger.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/scala/org/apache/bigtop/bigpetstore/generator/DataForger.scala b/bigtop-bigpetstore/src/main/scala/org/apache/bigtop/bigpetstore/generator/DataForger.scala
deleted file mode 100644
index c5e6513..0000000
--- a/bigtop-bigpetstore/src/main/scala/org/apache/bigtop/bigpetstore/generator/DataForger.scala
+++ /dev/null
@@ -1,280 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator
-
-import java.util.Random
-import org.jfairy.Fairy
-import java.util.Date
-
-
-/**
- * Generic class for generating random data. This class was created so
- * that we can provide a uniform API for getting random data. If we want,
- * we can replace the underlying data-generation implementation using
- * existing libraries.
- */
-object DataForger {
-  private val random = new Random
-  private val fairy = Fairy.create()
-
-  // TODO: Jay / Bhashit : refactor to use a random data generator?
-  def firstName(random: Random) = firstNames(random.nextInt(firstNames.length))
-  def firstName: String = firstName(random)
-
-  // TODO: Jay / Bhashit : refactor to use a random data generator?
-  def lastName(random: Random) = lastNames(random.nextInt(lastNames.length))
-  def lastName: String = lastName(random)
-
-  def randomDateInPastYears(maxYearsEarlier: Int) = fairy.dateProducer().randomDateInThePast(maxYearsEarlier).toDate()
-
-  private val firstNames =  IndexedSeq("Aaron", "Abby", "Abigail", "Adam",
-          "Alan", "Albert", "Alex", "Alexandra", "Alexis", "Alice", "Alicia",
-          "Alisha", "Alissa", "Allen", "Allison", "Alyssa", "Amanda", "Amber",
-          "Amy", "Andrea", "Andrew", "Andy", "Angel", "Angela", "Angie",
-          "Anita", "Ann", "Anna", "Annette", "Anthony", "Antonio", "April",
-          "Arthur", "Ashley", "Audrey", "Austin", "Autumn", "Baby", "Barb",
-          "Barbara", "Becky", "Benjamin", "Beth", "Bethany", "Betty",
-          "Beverly", "Bill", "Billie", "Billy", "Blake", "Bob", "Bobbie",
-          "Bobby", "Bonnie", "Brad", "Bradley", "Brady", "Brandi", "Brandon",
-          "Brandy", "Breanna", "Brenda", "Brent", "Brett", "Brian", "Brianna",
-          "Brittany", "Brooke", "Brooklyn", "Bruce", "Bryan", "Caleb",
-          "Cameron", "Candy", "Carl", "Carla", "Carmen", "Carol", "Carolyn",
-          "Carrie", "Casey", "Cassandra", "Catherine", "Cathy", "Chad",
-          "Charlene", "Charles", "Charlie", "Charlotte", "Chase", "Chasity",
-          "Chastity", "Chelsea", "Cheryl", "Chester", "Cheyenne", "Chris",
-          "Christian", "Christina", "Christine", "Christoph", "Christopher",
-          "Christy", "Chuck", "Cindy", "Clara", "Clarence", "Clayton",
-          "Clifford", "Clint", "Cody", "Colton", "Connie", "Corey", "Cory",
-          "Courtney", "Craig", "Crystal", "Curtis", "Cynthia", "Dakota",
-          "Dale", "Dallas", "Dalton", "Dan", "Dana", "Daniel", "Danielle",
-          "Danny", "Darla", "Darlene", "Darrell", "Darren", "Dave", "David",
-          "Dawn", "Dean", "Deanna", "Debbie", "Deborah", "Debra", "Denise",
-          "Dennis", "Derek", "Derrick", "Destiny", "Devin", "Diana", "Diane",
-          "Dillon", "Dixie", "Dominic", "Don", "Donald", "Donna", "Donnie",
-          "Doris", "Dorothy", "Doug", "Douglas", "Drew", "Duane", "Dustin",
-          "Dusty", "Dylan", "Earl", "Ed", "Eddie", "Edward", "Elaine",
-          "Elizabeth", "Ellen", "Emily", "Eric", "Erica", "Erika", "Erin",
-          "Ernest", "Ethan", "Eugene", "Eva", "Evelyn", "Everett", "Faith",
-          "Father", "Felicia", "Floyd", "Francis", "Frank", "Fred", "Gabriel",
-          "Gage", "Gail", "Gary", "Gene", "George", "Gerald", "Gina", "Ginger",
-          "Glen", "Glenn", "Gloria", "Grace", "Greg", "Gregory", "Haley",
-          "Hannah", "Harley", "Harold", "Harry", "Heath", "Heather", "Heidi",
-          "Helen", "Herbert", "Holly", "Hope", "Howard", "Hunter", "Ian",
-          "Isaac", "Jack", "Jackie", "Jacob", "Jade", "Jake", "James", "Jamie",
-          "Jan", "Jane", "Janet", "Janice", "Jared", "Jasmine", "Jason", "Jay",
-          "Jean", "Jeannie", "Jeff", "Jeffery", "Jeffrey", "Jenna", "Jennifer",
-          "Jenny", "Jeremiah", "Jeremy", "Jerry", "Jesse", "Jessica", "Jessie",
-          "Jill", "Jim", "Jimmy", "Joann", "Joanne", "Jodi", "Jody", "Joe",
-          "Joel", "Joey", "John", "Johnathan", "Johnny", "Jon", "Jonathan",
-          "Jonathon", "Jordan", "Joseph", "Josh", "Joshua", "Joyce", "Juanita",
-          "Judy", "Julia", "Julie", "Justin", "Kaitlyn", "Karen", "Katelyn",
-          "Katherine", "Kathleen", "Kathryn", "Kathy", "Katie", "Katrina",
-          "Kay", "Kayla", "Kaylee", "Keith", "Kelly", "Kelsey", "Ken",
-          "Kendra", "Kenneth", "Kenny", "Kevin", "Kim", "Kimberly", "Kris",
-          "Krista", "Kristen", "Kristin", "Kristina", "Kristy", "Kyle",
-          "Kylie", "Lacey", "Laken", "Lance", "Larry", "Laura", "Lawrence",
-          "Leah", "Lee", "Leonard", "Leroy", "Leslie", "Levi", "Lewis",
-          "Linda", "Lindsay", "Lindsey", "Lisa", "Lloyd", "Logan", "Lois",
-          "Loretta", "Lori", "Louis", "Lynn", "Madison", "Mandy", "Marcus",
-          "Margaret", "Maria", "Mariah", "Marie", "Marilyn", "Marion", "Mark",
-          "Marlene", "Marsha", "Martha", "Martin", "Marty", "Marvin", "Mary",
-          "Mary ann", "Mason", "Matt", "Matthew", "Max", "Megan", "Melanie",
-          "Melinda", "Melissa", "Melody", "Michael", "Michelle", "Mickey",
-          "Mike", "Mindy", "Miranda", "Misty", "Mitchell", "Molly", "Monica",
-          "Morgan", "Mother", "Myron", "Nancy", "Natasha", "Nathan",
-          "Nicholas", "Nick", "Nicole", "Nina", "Noah", "Norma", "Norman",
-          "Olivia", "Paige", "Pam", "Pamela", "Pat", "Patricia", "Patrick",
-          "Patty", "Paul", "Paula", "Peggy", "Penny", "Pete", "Phillip",
-          "Phyllis", "Rachael", "Rachel", "Ralph", "Randall", "Randi", "Randy",
-          "Ray", "Raymond", "Rebecca", "Regina", "Renee", "Rex", "Rhonda",
-          "Richard", "Rick", "Ricky", "Rita", "Rob", "Robbie", "Robert",
-          "Roberta", "Robin", "Rochelle", "Rocky", "Rod", "Rodney", "Roger",
-          "Ron", "Ronald", "Ronda", "Ronnie", "Rose", "Roxanne", "Roy", "Russ",
-          "Russell", "Rusty", "Ruth", "Ryan", "Sabrina", "Sally", "Sam",
-          "Samantha", "Samuel", "Sandra", "Sandy", "Sara", "Sarah", "Savannah",
-          "Scott", "Sean", "Seth", "Shanda", "Shane", "Shanna", "Shannon",
-          "Sharon", "Shaun", "Shawn", "Shawna", "Sheila", "Shelly", "Sher",
-          "Sherri", "Sherry", "Shirley", "Sierra", "Skyler", "Stacey", "Stacy",
-          "Stanley", "Stephanie", "Stephen", "Steve", "Steven", "Sue",
-          "Summer", "Susan", "Sydney", "Tabatha", "Tabitha", "Tamara", "Tammy",
-          "Tara", "Tasha", "Tashia", "Taylor", "Ted", "Teresa", "Terri",
-          "Terry", "Tessa", "Thelma", "Theresa", "Thomas", "Tia", "Tiffany",
-          "Tim", "Timmy", "Timothy", "Tina", "Todd", "Tom", "Tommy", "Toni",
-          "Tony", "Tonya", "Tracey", "Tracie", "Tracy", "Travis", "Trent",
-          "Trevor", "Trey", "Trisha", "Tristan", "Troy", "Tyler", "Tyrone",
-          "Unborn", "Valerie", "Vanessa", "Vernon", "Veronica", "Vicki",
-          "Vickie", "Vicky", "Victor", "Victoria", "Vincent", "Virginia",
-          "Vivian", "Walter", "Wanda", "Wayne", "Wendy", "Wesley", "Whitney",
-          "William", "Willie", "Wyatt", "Zachary")
-
-  private val lastNames = IndexedSeq("Abbott", "Acevedo", "Acosta", "Adams",
-          "Adkins", "Aguilar", "Aguirre", "Albert", "Alexander", "Alford",
-          "Allen", "Allison", "Alston", "Alvarado", "Alvarez", "Anderson",
-          "Andrews", "Anthony", "Armstrong", "Arnold", "Ashley", "Atkins",
-          "Atkinson", "Austin", "Avery", "Avila", "Ayala", "Ayers", "Bailey",
-          "Baird", "Baker", "Baldwin", "Ball", "Ballard", "Banks", "Barber",
-          "Smith", "Johnson", "Williams", "Jones", "Brown", "Davis", "Miller",
-          "Wilson", "Moore", "Taylor", "Thomas", "Jackson", "Barker", "Barlow",
-          "Barnes", "Barnett", "Barr", "Barrera", "Barrett", "Barron", "Barry",
-          "Bartlett", "Barton", "Bass", "Bates", "Battle", "Bauer", "Baxter",
-          "Beach", "Bean", "Beard", "Beasley", "Beck", "Becker", "Bell",
-          "Bender", "Benjamin", "Bennett", "Benson", "Bentley", "Benton",
-          "Berg", "Berger", "Bernard", "Berry", "Best", "Bird", "Bishop",
-          "Black", "Blackburn", "Blackwell", "Blair", "Blake", "Blanchard",
-          "Blankenship", "Blevins", "Bolton", "Bond", "Bonner", "Booker",
-          "Boone", "Booth", "Bowen", "Bowers", "Bowman", "Boyd", "Boyer",
-          "Boyle", "Bradford", "Bradley", "Bradshaw", "Brady", "Branch",
-          "Bray", "Brennan", "Brewer", "Bridges", "Briggs", "Bright", "Britt",
-          "Brock", "Brooks", "Browning", "Bruce", "Bryan", "Bryant",
-          "Buchanan", "Buck", "Buckley", "Buckner", "Bullock", "Burch",
-          "Burgess", "Burke", "Burks", "Burnett", "Burns", "Burris", "Burt",
-          "Burton", "Bush", "Butler", "Byers", "Byrd", "Cabrera", "Cain",
-          "Calderon", "Caldwell", "Calhoun", "Callahan", "Camacho", "Cameron",
-          "Campbell", "Campos", "Cannon", "Cantrell", "Cantu", "Cardenas",
-          "Carey", "Carlson", "Carney", "Carpenter", "Carr", "Carrillo",
-          "Carroll", "Carson", "Carter", "Carver", "Case", "Casey", "Cash",
-          "Castaneda", "Castillo", "Castro", "Cervantes", "Chambers", "Chan",
-          "Chandler", "Chaney", "Chang", "Chapman", "Charles", "Chase",
-          "Chavez", "Chen", "Cherry", "Christensen", "Christian", "Church",
-          "Clark", "Clarke", "Clay", "Clayton", "Clements", "Clemons",
-          "Cleveland", "Cline", "Cobb", "Cochran", "Coffey", "Cohen", "Cole",
-          "Coleman", "Collier", "Collins", "Colon", "Combs", "Compton",
-          "Conley", "Conner", "Conrad", "Contreras", "Conway", "Cook", "Cooke",
-          "Cooley", "Cooper", "Copeland", "Cortez", "Cote", "Cotton", "Cox",
-          "Craft", "Craig", "Crane", "Crawford", "Crosby", "Cross", "Cruz",
-          "Cummings", "Cunningham", "Curry", "Curtis", "Dale", "Dalton",
-          "Daniel", "Daniels", "Daugherty", "Davenport", "David", "Davidson",
-          "Dawson", "Day", "Dean", "Decker", "Dejesus", "Delacruz", "Delaney",
-          "Deleon", "Delgado", "Dennis", "Diaz", "Dickerson", "Dickinson",
-          "Dillard", "Dillon", "Dixon", "Dodson", "Dominguez", "Donaldson",
-          "Donovan", "Dorsey", "Dotson", "Douglas", "Downs", "Doyle", "Drake",
-          "Dudley", "Duffy", "Duke", "Duncan", "Dunlap", "Dunn", "Duran",
-          "Durham", "Dyer", "Eaton", "Edwards", "Elliott", "Ellis", "Ellison",
-          "Emerson", "England", "English", "Erickson", "Espinoza", "Estes",
-          "Estrada", "Evans", "Everett", "Ewing", "Farley", "Farmer",
-          "Farrell", "Faulkner", "Ferguson", "Fernandez", "Ferrell", "Fields",
-          "Figueroa", "Finch", "Finley", "Fischer", "Fisher", "Fitzgerald",
-          "Fitzpatrick", "Fleming", "Fletcher", "Flores", "Flowers", "Floyd",
-          "Flynn", "Foley", "Forbes", "Ford", "Foreman", "Foster", "Fowler",
-          "Fox", "Francis", "Franco", "Frank", "Franklin", "Franks", "Frazier",
-          "Frederick", "Freeman", "French", "Frost", "Fry", "Frye", "Fuentes",
-          "Fuller", "Fulton", "Gaines", "Gallagher", "Gallegos", "Galloway",
-          "Gamble", "Garcia", "Gardner", "Garner", "Garrett", "Garrison",
-          "Garza", "Gates", "Gay", "Gentry", "George", "Gibbs", "Gibson",
-          "Gilbert", "Giles", "Gill", "Gillespie", "Gilliam", "Gilmore",
-          "Glass", "Glenn", "Glover", "Goff", "Golden", "Gomez", "Gonzales",
-          "Gonzalez", "Good", "Goodman", "Goodwin", "Gordon", "Gould",
-          "Graham", "Grant", "Graves", "Gray", "Green", "Greene", "Greer",
-          "Gregory", "Griffin", "Griffith", "Grimes", "Gross", "Guerra",
-          "Guerrero", "Guthrie", "Gutierrez", "Guy", "Guzman", "Hahn", "Hale",
-          "Haley", "Hall", "Hamilton", "Hammond", "Hampton", "Hancock",
-          "Haney", "Hansen", "Hanson", "Hardin", "Harding", "Hardy", "Harmon",
-          "Harper", "Harris", "Harrington", "Harrison", "Hart", "Hartman",
-          "Harvey", "Hatfield", "Hawkins", "Hayden", "Hayes", "Haynes", "Hays",
-          "Head", "Heath", "Hebert", "Henderson", "Hendricks", "Hendrix",
-          "Henry", "Hensley", "Henson", "Herman", "Hernandez", "Herrera",
-          "Herring", "Hess", "Hester", "Hewitt", "Hickman", "Hicks", "Higgins",
-          "Hill", "Hines", "Hinton", "Hobbs", "Hodge", "Hodges", "Hoffman",
-          "Hogan", "Holcomb", "Holden", "Holder", "Holland", "Holloway",
-          "Holman", "Holmes", "Holt", "Hood", "Hooper", "Hoover", "Hopkins",
-          "Hopper", "Horn", "Horne", "Horton", "House", "Houston", "Howard",
-          "Howe", "Howell", "Hubbard", "Huber", "Hudson", "Huff", "Huffman",
-          "Hughes", "Hull", "Humphrey", "Hunt", "Hunter", "Hurley", "Hurst",
-          "Hutchinson", "Hyde", "Ingram", "Irwin", "Jacobs", "Jacobson",
-          "James", "Jarvis", "Jefferson", "Jenkins", "Jennings", "Jensen",
-          "Jimenez", "Johns", "Johnston", "Jordan", "Joseph", "Joyce",
-          "Joyner", "Juarez", "Justice", "Kane", "Kaufman", "Keith", "Keller",
-          "Kelley", "Kelly", "Kemp", "Kennedy", "Kent", "Kerr", "Key", "Kidd",
-          "Kim", "King", "Kinney", "Kirby", "Kirk", "Kirkland", "Klein",
-          "Kline", "Knapp", "Knight", "Knowles", "Knox", "Koch", "Kramer",
-          "Lamb", "Lambert", "Lancaster", "Landry", "Lane", "Lang", "Langley",
-          "Lara", "Larsen", "Larson", "Lawrence", "Lawson", "Le", "Leach",
-          "Leblanc", "Lee", "Leon", "Leonard", "Lester", "Levine", "Levy",
-          "Lewis", "Lindsay", "Lindsey", "Little", "Livingston", "Lloyd",
-          "Logan", "Long", "Lopez", "Lott", "Love", "Lowe", "Lowery", "Lucas",
-          "Luna", "Lynch", "Lynn", "Lyons", "Macdonald", "Macias", "Mack",
-          "Madden", "Maddox", "Maldonado", "Malone", "Mann", "Manning",
-          "Marks", "Marquez", "Marsh", "Marshall", "Martin", "Martinez",
-          "Mason", "Massey", "Mathews", "Mathis", "Matthews", "Maxwell", "May",
-          "Mayer", "Maynard", "Mayo", "Mays", "McBride", "McCall", "McCarthy",
-          "McCarty", "McClain", "McClure", "McConnell", "McCormick", "McCoy",
-          "McCray", "McCullough", "McDaniel", "McDonald", "McDowell",
-          "McFadden", "McFarland", "McGee", "McGowan", "McGuire", "McIntosh",
-          "McIntyre", "McKay", "McKee", "McKenzie", "McKinney", "McKnight",
-          "McLaughlin", "McLean", "McLeod", "McMahon", "McMillan", "McNeil",
-          "McPherson", "Meadows", "Medina", "Mejia", "Melendez", "Melton",
-          "Mendez", "Mendoza", "Mercado", "Mercer", "Merrill", "Merritt",
-          "Meyer", "Meyers", "Michael", "Middleton", "Miles", "Mills",
-          "Miranda", "Mitchell", "Molina", "Monroe", "Montgomery", "Montoya",
-          "Moody", "Moon", "Mooney", "Morales", "Moran", "Moreno", "Morgan",
-          "Morin", "Morris", "Morrison", "Morrow", "Morse", "Morton", "Moses",
-          "Mosley", "Moss", "Mueller", "Mullen", "Mullins", "Munoz", "Murphy",
-          "Murray", "Myers", "Nash", "Navarro", "Neal", "Nelson", "Newman",
-          "Newton", "Nguyen", "Nichols", "Nicholson", "Nielsen", "Nieves",
-          "Nixon", "Noble", "Noel", "Nolan", "Norman", "Norris", "Norton",
-          "Nunez", "Obrien", "Ochoa", "Oconnor", "Odom", "Odonnell", "Oliver",
-          "Olsen", "Olson", "O'neal", "O'neil", "O'neill", "Orr", "Ortega",
-          "Ortiz", "Osborn", "Osborne", "Owen", "Owens", "Pace", "Pacheco",
-          "Padilla", "Page", "Palmer", "Park", "Parker", "Parks", "Parrish",
-          "Parsons", "Pate", "Patel", "Patrick", "Patterson", "Patton", "Paul",
-          "Payne", "Pearson", "Peck", "Pena", "Pennington", "Perez", "Perkins",
-          "Perry", "Peters", "Petersen", "Peterson", "Petty", "Phelps",
-          "Phillips", "Pickett", "Pierce", "Pittman", "Pitts", "Pollard",
-          "Poole", "Pope", "Porter", "Potter", "Potts", "Powell", "Powers",
-          "Pratt", "Preston", "Price", "Prince", "Pruitt", "Puckett", "Pugh",
-          "Quinn", "Ramirez", "Ramos", "Ramsey", "Randall", "Randolph",
-          "Rasmussen", "Ratliff", "Ray", "Raymond", "Reed", "Reese", "Reeves",
-          "Reid", "Reilly", "Reyes", "Reynolds", "Rhodes", "Rice", "Rich",
-          "Richard", "Richards", "Richardson", "Richmond", "Riddle", "Riggs",
-          "Riley", "Rios", "Rivas", "Rivera", "Rivers", "Roach", "Robbins",
-          "Roberson", "Roberts", "Robertson", "Robinson", "Robles", "Rocha",
-          "Rodgers", "Rodriguez", "Rodriquez", "Rogers", "Rojas", "Rollins",
-          "Roman", "Romero", "Rosa", "Rosales", "Rosario", "Rose", "Ross",
-          "Roth", "Rowe", "Rowland", "Roy", "Ruiz", "Rush", "Russell", "Russo",
-          "Rutledge", "Ryan", "Salas", "Salazar", "Salinas", "Sampson",
-          "Sanchez", "Sanders", "Sandoval", "Sanford", "Santana", "Santiago",
-          "Santos", "Sargent", "Saunders", "Savage", "Sawyer", "Schmidt",
-          "Schneider", "Schroeder", "Schultz", "Schwartz", "Scott", "Sears",
-          "Sellers", "Serrano", "Sexton", "Shaffer", "Shannon", "Sharp",
-          "Sharpe", "Shaw", "Shelton", "Shepard", "Shepherd", "Sheppard",
-          "Sherman", "Shields", "Short", "Silva", "Simmons", "Simon",
-          "Simpson", "Sims", "Singleton", "Skinner", "Slater", "Sloan",
-          "Small", "Snider", "Snow", "Snyder", "Solis", "Solomon", "Sosa",
-          "Soto", "Sparks", "Spears", "Spence", "Spencer", "Stafford",
-          "Stanley", "Stanton", "Stark", "Steele", "Stein", "Stephens",
-          "Stephenson", "Stevens", "Stevenson", "Stewart", "Stokes", "Stone",
-          "Stout", "Strickland", "Strong", "Stuart", "Suarez", "Sullivan",
-          "Summers", "Sutton", "Swanson", "Sweeney", "Sweet", "Sykes",
-          "Talley", "Tanner", "Tate", "Terrell", "Terry", "Thompson",
-          "Thornton", "Tillman", "Todd", "Torres", "Townsend", "Tran",
-          "Travis", "Trevino", "Trujillo", "Tucker", "Turner", "Tyler",
-          "Tyson", "Underwood", "Valdez", "Valencia", "Valentine",
-          "Valenzuela", "Vance", "Vang", "Vargas", "Vasquez", "Vaughan",
-          "Vaughn", "Vazquez", "Vega", "Velasquez", "Velazquez", "Velez",
-          "Van halen", "Vincent", "Vinson", "Wade", "Wagner", "Walker", "Wall",
-          "Wallace", "Waller", "Walls", "Walsh", "Walter", "Walters", "Walton",
-          "Ward", "Ware", "Warner", "Warren", "Washington", "Waters",
-          "Watkins", "Watson", "Watts", "Weaver", "Webb", "Weber", "Webster",
-          "Weeks", "Weiss", "Welch", "Wells", "West", "Wheeler", "Whitaker",
-          "White", "Whitehead", "Whitfield", "Whitley", "Whitney", "Wiggins",
-          "Wilcox", "Wilder", "Wiley", "Wilkerson", "Wilkins", "Wilkinson",
-          "William", "Williamson", "Willis", "Winters", "Wise", "Witt", "Wolf",
-          "Wolfe", "Wong", "Wood", "Woodard", "Woods", "Woodward", "Wooten",
-          "Workman", "Wright", "Wyatt", "Wynn", "Yang", "Yates", "York",
-          "Young", "Zamora", "Zimmerman")
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/scala/org/apache/bigtop/bigpetstore/generator/TransactionIteratorFactory.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/scala/org/apache/bigtop/bigpetstore/generator/TransactionIteratorFactory.scala b/bigtop-bigpetstore/src/main/scala/org/apache/bigtop/bigpetstore/generator/TransactionIteratorFactory.scala
deleted file mode 100644
index 534c606..0000000
--- a/bigtop-bigpetstore/src/main/scala/org/apache/bigtop/bigpetstore/generator/TransactionIteratorFactory.scala
+++ /dev/null
@@ -1,106 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator;
-
-import java.util.Date
-import org.apache.bigtop.bigpetstore.generator.util.State
-import org.apache.commons.lang3.StringUtils
-import java.util.Arrays.asList
-import java.util.Random
-import scala.collection.Iterator
-import com.sun.org.apache.xml.internal.serializer.ToStream
-import java.util.{Iterator => JavaIterator}
-import scala.collection.JavaConversions.asJavaIterator
-import org.apache.bigtop.bigpetstore.generator.util.Product
-import org.apache.commons.lang3.Range;
-import org.apache.bigtop.bigpetstore.generator.util.ProductType
-
-/**
- * This class generates our data. Over time we will use it to embed bias which
- * can then be teased out, i.e. by clustering/classifiers. For example:
- *
- * certain products <--> certain years or days
- */
-class TransactionIteratorFactory(private val records: Int,
-        private val customerIdRange: Range[java.lang.Long],
-        private val state: State) {
-  assert(records > 0, "Number of records must be greater than 0 to generate a data iterator!")
-  private val random = new Random(state.hashCode)
-
-  def data: JavaIterator[TransactionIteratorFactory.KeyVal[String, String]] = {
-    new TransactionIteratorFactory.DataIterator(records, customerIdRange, state, random)
-  }
-}
-
-object TransactionIteratorFactory {
-  class KeyVal[K, V](val key: K, val value: V)
-
-  private class DataIterator(records: Int,
-          customerIdRange: Range[java.lang.Long],
-          state: State,
-          r: Random) extends Iterator[KeyVal[String, String]] {
-    private var firstName: String = null
-    private var lastName: String = null
-    private var elementsProcducedCount = 0
-    private var repeatCount = 0
-    private var currentCustomerId = customerIdRange.getMinimum
-    private var currentProductType = selectRandomProductType;
-
-    def hasNext =
-      elementsProcducedCount < records && currentCustomerId <= customerIdRange.getMaximum
-
-
-    def next(): TransactionIteratorFactory.KeyVal[String,String] = {
-      val date = DataForger.randomDateInPastYears(50);
-      setIteratorState();
-
-      val product = randomProductOfCurrentlySelectedType
-      val key = StringUtils.join(asList("BigPetStore", "storeCode_" + state.name(),
-              elementsProcducedCount.toString), ",")
-      val value = StringUtils.join(asList(currentCustomerId, firstName, lastName, product.id,
-              product.name.toLowerCase, product.price, date), ",")
-
-      elementsProcducedCount += 1
-      new TransactionIteratorFactory.KeyVal(key, value)
-    }
-
-    private def setIteratorState() = {
-      /** Some customers come back for more :) We repeat a customer up to ten times */
-      if (repeatCount > 0) {
-        repeatCount -= 1
-      } else {
-        firstName = DataForger.firstName(r)
-        lastName = DataForger.lastName(r)
-        // this sometimes generates numbers much larger than 10. We don't really need Gaussian
-        // distribution since number of transactions per customer can be truly arbitrary.
-        repeatCount = (r.nextGaussian * 4f) toInt;
-        println("####Repeat: " + repeatCount)
-        currentCustomerId += 1
-        currentProductType = selectRandomProductType;
-      }
-    }
-
-    private def selectRandomProductType = {
-      ProductType.values.apply(r.nextInt(ProductType.values.length))
-    }
-
-    private def randomProductOfCurrentlySelectedType = {
-      currentProductType.getProducts.get(r.nextInt(currentProductType.getProducts.size))
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/docs/TestDocs.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/docs/TestDocs.java b/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/docs/TestDocs.java
deleted file mode 100644
index 3292ba5..0000000
--- a/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/docs/TestDocs.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.docs;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-
-import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants.OUTPUTS;
-import org.apache.commons.io.FileUtils;
-import org.junit.Test;
-
-public class TestDocs {
-
-	@Test
-	public void testGraphViz() throws Exception {
-		// test the graphviz file by grepping out the constants.
-		String graphviz = FileUtils.readFileToString(new File("arch.dot"));
-		System.out.println(graphviz);
-
-		assertTrue(graphviz.contains(OUTPUTS.generated.name()));
-		assertTrue(graphviz.contains(OUTPUTS.cleaned.name()));
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/generator/TestNumericalIdUtils.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/generator/TestNumericalIdUtils.java b/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/generator/TestNumericalIdUtils.java
deleted file mode 100644
index e2f1f25..0000000
--- a/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/generator/TestNumericalIdUtils.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator;
-
-import static org.junit.Assert.assertFalse;
-
-import org.apache.bigtop.bigpetstore.generator.util.State;
-import org.apache.bigtop.bigpetstore.util.NumericalIdUtils;
-import org.junit.Test;
-
-public class TestNumericalIdUtils {
-
-    @Test
-    public void testName() {
-        String strId= State.OK.name()+"_"+ "jay vyas";
-        long id = NumericalIdUtils.toId(strId);
-        String strId2= State.CO.name()+"_"+ "jay vyas";
-        long id2 = NumericalIdUtils.toId(strId2);
-        System.out.println(id + " " + id2);
-        assertFalse(id==id2);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/generator/TestPetStoreTransactionGeneratorJob.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/generator/TestPetStoreTransactionGeneratorJob.java b/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/generator/TestPetStoreTransactionGeneratorJob.java
deleted file mode 100755
index 76de3d0..0000000
--- a/bigtop-bigpetstore/src/test/java/org/apache/bigtop/bigpetstore/generator/TestPetStoreTransactionGeneratorJob.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.generator;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.InputStreamReader;
-import java.util.Date;
-
-import org.apache.bigtop.bigpetstore.generator.BPSGenerator.props;
-import org.apache.bigtop.bigpetstore.generator.util.State;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Job;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * run this test with vm options -XX:MaxPermSize=256m -Xms512m -Xmx1024m
- *
- */
-public class TestPetStoreTransactionGeneratorJob {
-
-    final static Logger log = LoggerFactory
-            .getLogger(TestPetStoreTransactionGeneratorJob.class);
-
-    @Test
-    public void test() throws Exception {
-        System.out.println("memory : " + Runtime.getRuntime().freeMemory()
-                / 1000000);
-        if (Runtime.getRuntime().freeMemory() / 1000000 < 75) {
-            // throw new
-            // RuntimeException("need more memory to run this test !");
-        }
-        int records = 20;
-        /**
-         * Setup configuration with prop.
-         */
-        Configuration c = new Configuration();
-        c.setInt(props.bigpetstore_records.name(), records);
-
-        /**
-         * Run the job
-         */
-        Path output = new Path("petstoredata/" + (new Date()).toString());
-        Job createInput = BPSGenerator.getCreateTransactionRecordsJob(output, c);
-        createInput.submit();
-        System.out.println(createInput);
-        createInput.waitForCompletion(true);
-
-        FileSystem fs = FileSystem.getLocal(new Configuration());
-
-        /**
-         * Read file output into string.
-         */
-        DataInputStream f = fs.open(new Path(output, "part-r-00000"));
-        BufferedReader br = new BufferedReader(new InputStreamReader(f));
-        String s;
-        int recordsSeen = 0;
-        boolean CTseen = false;
-        boolean AZseen = false;
-
-        // confirm that both CT and AZ are seen in the outputs.
-        while (br.ready()) {
-            s = br.readLine();
-            System.out.println("===>" + s);
-            recordsSeen++;
-            if (s.contains(State.CT.name())) {
-                CTseen = true;
-            }
-            if (s.contains(State.AZ.name())) {
-                AZseen = true;
-            }
-        }
-
-        // records seen should = 20
-        assertEquals(records, recordsSeen);
-        // Assert that a couple of the states are seen (todo make it
-        // comprehensive for all states).
-        assertTrue(CTseen);
-        assertTrue(AZseen);
-        log.info("Created " + records + " , file was "
-                + fs.getFileStatus(new Path(output, "part-r-00000")).getLen()
-                + " bytes.");
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/test/resources/log4j.properties b/bigtop-bigpetstore/src/test/resources/log4j.properties
deleted file mode 100644
index 1e33093..0000000
--- a/bigtop-bigpetstore/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,47 +0,0 @@
-#
-# Licensed 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.
-
-hadoop.root.logger=INFO,console
-hadoop.log.dir=.
-hadoop.log.file=hadoop.log
-
-#
-# Job Summary Appender
-#
-# Use following logger to send summary to separate file defined by
-# hadoop.mapreduce.jobsummary.log.file rolled daily:
-# hadoop.mapreduce.jobsummary.logger=INFO,JSA
-#
-hadoop.mapreduce.jobsummary.logger=${hadoop.root.logger}
-hadoop.mapreduce.jobsummary.log.file=hadoop-mapreduce.jobsummary.log
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.EventCounter=org.apache.log4j.ConsoleAppender
-log4j.appender.EventCounter.layout=org.apache.log4j.PatternLayout
-# Define the root logger to the system property "hadoop.root.logger".
-log4j.rootLogger=${hadoop.root.logger}, EventCounter
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-# Logging Threshold
-log4j.threshold=ALL
-
-#
-# Daily Rolling File Appender
-#
-
-log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
-
-# Rollver at midnight
-log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
-
-# 30-day backup
-#log4j.appender.DRFA.MaxBackupIndex=30
-log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout


[3/5] bigtop git commit: Add BPS Spark driver for new data generator. Re-organize BPS into MapReduce and Spark versions.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/docs/TestDocs.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/docs/TestDocs.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/docs/TestDocs.java
new file mode 100644
index 0000000..8d7bf99
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/docs/TestDocs.java
@@ -0,0 +1,37 @@
+/**
+ * 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.bigtop.bigpetstore.docs;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+
+import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants.OUTPUTS;
+import org.apache.commons.io.FileUtils;
+import org.junit.Test;
+
+public class TestDocs {
+
+	@Test
+	public void testGraphViz() throws Exception {
+		// test the graphviz file by grepping out the constants.
+		String graphviz = FileUtils.readFileToString(new File("arch.dot"));
+		System.out.println(graphviz);
+
+		assertTrue(graphviz.contains(OUTPUTS.generated.name()));
+		assertTrue(graphviz.contains(OUTPUTS.cleaned.name()));
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/generator/TestNumericalIdUtils.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/generator/TestNumericalIdUtils.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/generator/TestNumericalIdUtils.java
new file mode 100644
index 0000000..e2f1f25
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/generator/TestNumericalIdUtils.java
@@ -0,0 +1,35 @@
+/**
+ * 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.bigtop.bigpetstore.generator;
+
+import static org.junit.Assert.assertFalse;
+
+import org.apache.bigtop.bigpetstore.generator.util.State;
+import org.apache.bigtop.bigpetstore.util.NumericalIdUtils;
+import org.junit.Test;
+
+public class TestNumericalIdUtils {
+
+    @Test
+    public void testName() {
+        String strId= State.OK.name()+"_"+ "jay vyas";
+        long id = NumericalIdUtils.toId(strId);
+        String strId2= State.CO.name()+"_"+ "jay vyas";
+        long id2 = NumericalIdUtils.toId(strId2);
+        System.out.println(id + " " + id2);
+        assertFalse(id==id2);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/generator/TestPetStoreTransactionGeneratorJob.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/generator/TestPetStoreTransactionGeneratorJob.java b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/generator/TestPetStoreTransactionGeneratorJob.java
new file mode 100755
index 0000000..76de3d0
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/java/org/apache/bigtop/bigpetstore/generator/TestPetStoreTransactionGeneratorJob.java
@@ -0,0 +1,104 @@
+/**
+ * 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.bigtop.bigpetstore.generator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.DataInputStream;
+import java.io.InputStreamReader;
+import java.util.Date;
+
+import org.apache.bigtop.bigpetstore.generator.BPSGenerator.props;
+import org.apache.bigtop.bigpetstore.generator.util.State;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * run this test with vm options -XX:MaxPermSize=256m -Xms512m -Xmx1024m
+ *
+ */
+public class TestPetStoreTransactionGeneratorJob {
+
+    final static Logger log = LoggerFactory
+            .getLogger(TestPetStoreTransactionGeneratorJob.class);
+
+    @Test
+    public void test() throws Exception {
+        System.out.println("memory : " + Runtime.getRuntime().freeMemory()
+                / 1000000);
+        if (Runtime.getRuntime().freeMemory() / 1000000 < 75) {
+            // throw new
+            // RuntimeException("need more memory to run this test !");
+        }
+        int records = 20;
+        /**
+         * Setup configuration with prop.
+         */
+        Configuration c = new Configuration();
+        c.setInt(props.bigpetstore_records.name(), records);
+
+        /**
+         * Run the job
+         */
+        Path output = new Path("petstoredata/" + (new Date()).toString());
+        Job createInput = BPSGenerator.getCreateTransactionRecordsJob(output, c);
+        createInput.submit();
+        System.out.println(createInput);
+        createInput.waitForCompletion(true);
+
+        FileSystem fs = FileSystem.getLocal(new Configuration());
+
+        /**
+         * Read file output into string.
+         */
+        DataInputStream f = fs.open(new Path(output, "part-r-00000"));
+        BufferedReader br = new BufferedReader(new InputStreamReader(f));
+        String s;
+        int recordsSeen = 0;
+        boolean CTseen = false;
+        boolean AZseen = false;
+
+        // confirm that both CT and AZ are seen in the outputs.
+        while (br.ready()) {
+            s = br.readLine();
+            System.out.println("===>" + s);
+            recordsSeen++;
+            if (s.contains(State.CT.name())) {
+                CTseen = true;
+            }
+            if (s.contains(State.AZ.name())) {
+                AZseen = true;
+            }
+        }
+
+        // records seen should = 20
+        assertEquals(records, recordsSeen);
+        // Assert that a couple of the states are seen (todo make it
+        // comprehensive for all states).
+        assertTrue(CTseen);
+        assertTrue(AZseen);
+        log.info("Created " + records + " , file was "
+                + fs.getFileStatus(new Path(output, "part-r-00000")).getLen()
+                + " bytes.");
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/resources/log4j.properties b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/resources/log4j.properties
new file mode 100644
index 0000000..1e33093
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/resources/log4j.properties
@@ -0,0 +1,47 @@
+#
+# Licensed 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.
+
+hadoop.root.logger=INFO,console
+hadoop.log.dir=.
+hadoop.log.file=hadoop.log
+
+#
+# Job Summary Appender
+#
+# Use following logger to send summary to separate file defined by
+# hadoop.mapreduce.jobsummary.log.file rolled daily:
+# hadoop.mapreduce.jobsummary.logger=INFO,JSA
+#
+hadoop.mapreduce.jobsummary.logger=${hadoop.root.logger}
+hadoop.mapreduce.jobsummary.log.file=hadoop-mapreduce.jobsummary.log
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.EventCounter=org.apache.log4j.ConsoleAppender
+log4j.appender.EventCounter.layout=org.apache.log4j.PatternLayout
+# Define the root logger to the system property "hadoop.root.logger".
+log4j.rootLogger=${hadoop.root.logger}, EventCounter
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+# Logging Threshold
+log4j.threshold=ALL
+
+#
+# Daily Rolling File Appender
+#
+
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/scala/org/apache/bigtop/bigpetstore/ScalaTestSample.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/scala/org/apache/bigtop/bigpetstore/ScalaTestSample.scala b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/scala/org/apache/bigtop/bigpetstore/ScalaTestSample.scala
new file mode 100644
index 0000000..a393b4b
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-mapreduce/src/test/scala/org/apache/bigtop/bigpetstore/ScalaTestSample.scala
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bigtop.bigpetstore
+
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.scalatest.junit.JUnitRunner
+import org.scalatest._
+import scala.collection.mutable.Stack
+
+@RunWith(classOf[JUnitRunner])
+class ScalaTestSample extends FlatSpec with Matchers {
+	"This test" should "show an example of what we can do with the scala-test library" in {
+		val stack = new Stack[Int]
+		stack.push(1)
+		stack.push(2)
+		stack.pop() should be(2)
+		stack.pop() should be(1)
+	}
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-spark/README.md
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-spark/README.md b/bigtop-bigpetstore/bigpetstore-spark/README.md
new file mode 100644
index 0000000..4533366
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-spark/README.md
@@ -0,0 +1,43 @@
+BigPetStore -- Spark
+====================
+
+BigPetStore is a family of example applications for the Hadoop and Spark
+ecosystems.  BigPetStore is build around a fictional chain pet stores,
+providing generators for synthetic transaction data and pipelines for
+processing that data.  Each ecosystems has its own version of the
+application.
+
+The Spark application currently builds against Spark 1.1.0.
+
+Architecture
+------------
+The Spark application consists of the following modules so far:
+
+* generator: generates raw data on the dfs
+
+Building and Running with Spark
+-------------------------------
+BigPetStore has a Spark driver for generating data with the new data generator.
+Build a fat jar as follows:
+
+```
+gradle clean shadowJar
+```
+
+This will produce a jar file under `build/libs` (referred to as `bigpetstore-spark-X.jar`).  You can then
+use this jar to run a Spark job as follows:
+
+```
+spark-submit --master local[2] --class org.apache.bigtop.bigpetstore.generator.SparkDriver bigpetstore-spark-X.jar generated_data/ 10 1000 365.0 345
+```
+
+You will need to change the master if you want to run on a cluster.  The last five parameters control the output directory,
+the number of stores, the number of customers, simulation length (in days), and the random seed (which is optional).
+
+Running Tests
+-------------
+BigPetStore Spark includes unit tests that you can run with the following command:
+
+```
+gradle test
+```

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-spark/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-spark/build.gradle b/bigtop-bigpetstore/bigpetstore-spark/build.gradle
new file mode 100644
index 0000000..726a0e5
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-spark/build.gradle
@@ -0,0 +1,137 @@
+/*
+ * 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.
+ */
+
+apply plugin: "java"
+apply plugin: "eclipse"
+// TODO add idea module config.
+apply plugin: "idea"
+apply plugin: "scala"
+apply plugin: 'com.github.johnrengelman.shadow'
+
+buildscript {
+  repositories { jcenter() }
+  dependencies {
+    classpath 'com.github.jengelman.gradle.plugins:shadow:1.0.2'
+  }
+}
+
+
+// Read the groupId and version properties from the "parent" bigtop project.
+// It would be better if there was some better way of doing this. Howvever,
+// at this point, we have to do this (or some variation thereof) since gradle
+// projects can't have maven projects as parents (AFAIK. If there is a way to do it,
+// it doesn't seem to be well-documented).
+def setProjectProperties() {
+    Node xml = new XmlParser().parse("../../pom.xml")
+    group = xml.groupId.first().value().first()
+    version = xml.version.first().value().first()
+}
+
+setProjectProperties()
+description = """"""
+
+// We are using 1.7 as gradle can't play well when java 8 and scala are combined.
+// There is an open issue here: http://issues.gradle.org/browse/GRADLE-3023
+// There is talk of this being resolved in the next version of gradle. Till then,
+// we are stuck with java 7. But we do have scala if we want more syntactic sugar.
+sourceCompatibility = 1.7
+targetCompatibility = 1.7
+
+// Specify any additional project properties.
+ext {
+    sparkVersion = "1.1.0"
+}
+
+shadowJar {
+    zip64 true
+}
+
+repositories {
+    mavenCentral()
+    maven {
+        url "http://dl.bintray.com/rnowling/bigpetstore"
+    }
+}
+
+tasks.withType(AbstractCompile) {
+    options.encoding = 'UTF-8'
+    options.compilerArgs << "-Xlint:all"
+}
+
+tasks.withType(ScalaCompile) {
+    // Enables incremental compilation.
+    // http://www.gradle.org/docs/current/userguide/userguide_single.html#N12F78
+    scalaCompileOptions.useAnt = false
+}
+
+tasks.withType(Test) {
+    testLogging {
+        // Uncomment this if you want to see the console output from the tests.
+        // showStandardStreams = true
+        events "passed", "skipped", "failed"
+        // show standard out and standard error of the test JVM(s) on the console
+        //showStandardStreams = true
+    }
+}
+
+// Create a separate source-set for the src/integrationTest set of classes. The convention here
+// is that gradle will look for a directory with the same name as that of the specified source-set
+// under the 'src' directory.
+sourceSets {
+    main {
+        java.srcDirs = [];
+        scala.srcDirs = ["src/main/scala", "src/main/java"]
+    }
+}
+
+
+// To see the API that is being used here, consult the following docs
+// http://www.gradle.org/docs/current/dsl/org.gradle.api.artifacts.ResolutionStrategy.html
+def updateDependencyVersion(dependencyDetails, dependencyString) {
+    def parts = dependencyString.split(':')
+    def group = parts[0]
+    def name = parts[1]
+    def version = parts[2]
+    if (dependencyDetails.requested.group == group
+            && dependencyDetails.requested.name == name) {
+        dependencyDetails.useVersion version
+    }
+}
+
+
+dependencies {
+    compile "org.apache.spark:spark-assembly_2.10:${sparkVersion}"
+    compile "com.github.rnowling.bigpetstore:bigpetstore-data-generator:0.2"
+
+    testCompile "junit:junit:4.11"
+    testCompile "org.hamcrest:hamcrest-all:1.3"
+    testCompile "org.scalatest:scalatest_2.10:2.2.1"
+}
+
+task listJars << {
+    configurations.shadow.each { println it.name }
+}
+
+
+eclipse {
+    classpath {
+        // Comment out the following two lines if you want to generate an eclipse project quickly.
+        downloadSources = true
+        downloadJavadoc = false
+    }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-spark/src/main/scala/org/apache/bigpetstore/spark/generator/SparkDriver.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-spark/src/main/scala/org/apache/bigpetstore/spark/generator/SparkDriver.scala b/bigtop-bigpetstore/bigpetstore-spark/src/main/scala/org/apache/bigpetstore/spark/generator/SparkDriver.scala
new file mode 100644
index 0000000..1ab1057
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-spark/src/main/scala/org/apache/bigpetstore/spark/generator/SparkDriver.scala
@@ -0,0 +1,244 @@
+/*
+ *  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.bigtop.bigpetstore.spark.generator
+
+import com.github.rnowling.bps.datagenerator.datamodels.{Store,Customer,PurchasingProfile,Transaction}
+import com.github.rnowling.bps.datagenerator.{DataLoader,StoreGenerator,CustomerGenerator => CustGen, PurchasingProfileGenerator,TransactionGenerator}
+import com.github.rnowling.bps.datagenerator.framework.SeedFactory
+
+import org.apache.spark.{SparkContext, SparkConf}
+import org.apache.spark.SparkContext._
+import org.apache.spark.rdd._
+
+import java.util.ArrayList
+import scala.util.Random
+import java.io.File
+import java.util.Date
+
+object SparkDriver {
+  private var nStores: Int = -1
+  private var nCustomers: Int = -1
+  private var simulationLength: Double = -1.0
+  private var seed: Long = -1
+  private var outputDir: File = new File(".")
+
+  private val NPARAMS = 5
+
+  private def printUsage() {
+    val usage: String = "BigPetStore Data Generator\n" +
+      "\n" +
+      "Usage: spark-submit ... outputDir nStores nCustomers simulationLength [seed]\n" +
+      "\n" +
+      "outputDir - (string) directory to write files\n" +
+      "nStores - (int) number of stores to generate\n" +
+      "nCustomers - (int) number of customers to generate\n" +
+      "simulationLength - (float) number of days to simulate\n" +
+      "seed - (long) seed for RNG. If not given, one is reandomly generated.\n"
+
+    println(usage)
+  }
+
+  def parseArgs(args: Array[String]) {
+    if(args.length != NPARAMS && args.length != (NPARAMS - 1)) {
+      printUsage()
+      System.exit(1)
+    }
+
+    var i = 0
+
+    outputDir = new File(args(i))
+    if(! outputDir.exists()) {
+      System.err.println("Given path (" + args(i) + ") does not exist.\n")
+      printUsage()
+      System.exit(1)
+    }
+
+    if(! outputDir.isDirectory()) {
+      System.err.println("Given path (" + args(i) + ") is not a directory.\n")
+      printUsage()
+      System.exit(1)
+    }
+
+    i += 1
+    try {
+      nStores = args(i).toInt
+    }
+    catch {
+      case _ : NumberFormatException =>
+        System.err.println("Unable to parse '" + args(i) + "' as an integer for nStores.\n")
+        printUsage()
+        System.exit(1)
+    }
+
+    i += 1
+    try {
+      nCustomers = args(i).toInt
+    }
+    catch {
+      case _ : NumberFormatException =>
+        System.err.println("Unable to parse '" + args(i) + "' as an integer for nCustomers.\n")
+        printUsage()
+        System.exit(1)
+    }
+
+    i += 1
+    try {
+      simulationLength = args(i).toDouble
+    }
+    catch {
+      case _ : NumberFormatException =>
+        System.err.println("Unable to parse '" + args(i) + "' as a float for simulationLength.\n")
+        printUsage()
+        System.exit(1)
+    }
+
+    if(args.length == NPARAMS) {
+      i += 1
+      try {
+        seed = args(i).toLong
+      }
+      catch {
+        case _ : NumberFormatException =>
+          System.err.println("Unable to parse '" + args(i) + "' as a long for seed.\n")
+          printUsage()
+          System.exit(1)
+      }
+    }
+    else {
+      seed = (new Random()).nextLong
+    }
+  }
+
+  def generateData(sc: SparkContext): RDD[Transaction] = {
+    val inputData = new DataLoader().loadData()
+    val seedFactory = new SeedFactory(seed);
+
+    println("Generating stores...")
+    val stores : ArrayList[Store] = new ArrayList()
+    val storeGenerator = new StoreGenerator(inputData, seedFactory);
+    for(i <- 1 to nStores) {
+      val store = storeGenerator.generate()
+      stores.add(store)
+    }
+    println("Done.")
+
+    println("Generating customers...")
+    var customers: List[Customer] = List()
+    val custGen = new CustGen(inputData, stores, seedFactory)
+    for(i <- 1 to nCustomers) {
+      val customer = custGen.generate()
+      customers = customer :: customers
+    }
+    println("Done.")
+
+    println("Broadcasting stores and products")
+    val storesBC = sc.broadcast(stores)
+    val productBC = sc.broadcast(inputData.getProductCategories())
+    val customerRDD = sc.parallelize(customers)
+    val nextSeed = seedFactory.getNextSeed()
+
+    println("Defining transaction DAG")
+    val transactionRDD = customerRDD.mapPartitionsWithIndex { (index, custIter) =>
+      val seedFactory = new SeedFactory(nextSeed ^ index)
+      val transactionIter = custIter.map{ customer =>
+	val products = productBC.value
+
+        val profileGen = new PurchasingProfileGenerator(products, seedFactory)
+        val profile = profileGen.generate()
+
+        val transGen = new TransactionGenerator(customer, profile, storesBC.value, products,
+          seedFactory)
+
+        var transactions : List[Transaction] = List()
+	var transaction = transGen.generate()
+        while(transaction.getDateTime() < simulationLength) {
+          transactions = transaction :: transactions
+
+          transaction = transGen.generate()
+        }
+
+	transactions
+      }
+      transactionIter
+    }.flatMap( s => s)
+
+    println("Generating transactions...")
+    val nTrans = transactionRDD.count()
+    println(s"Generated $nTrans transactions.")
+
+    transactionRDD
+  }
+
+  def writeData(transactionRDD : RDD[Transaction]) {
+    val initialDate : Long = new Date().getTime()
+
+    val transactionStringsRDD = transactionRDD.map { t =>
+      var records : List[String] = List()
+      val products = t.getProducts()
+      for(i <- 0 until products.size()) {
+        val p = products.get(i)
+	val name = t.getCustomer().getName()
+        val custLocation = t.getCustomer().getLocation()
+        val storeLocation = t.getStore().getLocation()
+
+        // days -> milliseconds = days * 24 h / day * 60 min / hr * 60 sec / min * 1000 ms / sec
+        val dateMS = (t.getDateTime * 24.0 * 60.0 * 60.0 * 1000.0).toLong
+        val date = new Date(initialDate + dateMS)
+
+
+        var record = ""
+        record += t.getStore().getId() + ","
+        record += storeLocation.getZipcode() + ","
+        record += storeLocation.getCity() + ","
+        record += storeLocation.getState() + ","
+
+        record += t.getCustomer().getId() + ","
+	record += name.getFirst() + "," + name.getSecond() + ","
+	record += custLocation.getZipcode() + ","
+	record += custLocation.getCity() + ","
+	record += custLocation.getState() + ","
+
+        record += t.getId() + ","
+        record += date + ","
+	record += p
+
+        records = record :: records
+      }
+
+      records
+    }.flatMap { s => s }
+
+    transactionStringsRDD.saveAsTextFile(outputDir + "/transactions")
+  }
+
+  def main(args: Array[String]) {
+    parseArgs(args)
+
+    println("Creating SparkConf")
+    val conf = new SparkConf().setAppName("BPS Data Generator")
+
+    println("Creating SparkContext")
+    val sc = new SparkContext(conf)
+
+    val transactionRDD = generateData(sc)
+
+    writeData(transactionRDD)
+
+    sc.stop()
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/bigpetstore-spark/src/test/scala/org/apache/bigpetstore/spark/generator/SparkDriverSuite.scala
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/bigpetstore-spark/src/test/scala/org/apache/bigpetstore/spark/generator/SparkDriverSuite.scala b/bigtop-bigpetstore/bigpetstore-spark/src/test/scala/org/apache/bigpetstore/spark/generator/SparkDriverSuite.scala
new file mode 100644
index 0000000..8fbfb71
--- /dev/null
+++ b/bigtop-bigpetstore/bigpetstore-spark/src/test/scala/org/apache/bigpetstore/spark/generator/SparkDriverSuite.scala
@@ -0,0 +1,60 @@
+/*
+ *  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.bigtop.bigpetstore.spark.generator
+
+import Array._
+
+import java.io.File
+import java.nio.file.Files
+
+import org.apache.spark.{SparkContext, SparkConf}
+
+import org.scalatest.FunSuite
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
+
+
+// hack for running tests with Gradle
+@RunWith(classOf[JUnitRunner])
+class SparkDriverSuite extends FunSuite {
+
+  test("Generating data") {
+
+    val tmpDir:File = Files.createTempDirectory("sparkDriverSuiteGeneratedData").toFile()
+    // 10 stores, 1000 customers, 365.0 days
+    val parameters:Array[String] = Array(tmpDir.toString(), "10", "1000", "365.0")
+
+    SparkDriver.parseArgs(parameters)
+
+    val conf = new SparkConf().setAppName("BPS Data Generator Test Suite").setMaster("local[2]")
+    val sc = new SparkContext(conf)
+
+    val transactionRDD = SparkDriver.generateData(sc)
+    val transactionCount = transactionRDD.count()
+    assert(transactionCount > 0)
+
+    SparkDriver.writeData(transactionRDD)
+
+    // check that generator wrote out the  data
+    val transactionDir:File = new File(tmpDir, "transactions")
+    assert(transactionDir.exists())
+    assert(transactionDir.isDirectory())
+
+    sc.stop()
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/build.gradle b/bigtop-bigpetstore/build.gradle
deleted file mode 100644
index 6d42e8d..0000000
--- a/bigtop-bigpetstore/build.gradle
+++ /dev/null
@@ -1,292 +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.
- */
-apply plugin: "java"
-apply plugin: "eclipse"
-// TODO add idea module config.
-apply plugin: "idea"
-apply plugin: "scala"
-apply plugin: 'com.github.johnrengelman.shadow'
-
-buildscript {
-  repositories { jcenter() }
-  dependencies {
-    classpath 'com.github.jengelman.gradle.plugins:shadow:1.0.2'
-  }
-}
-
-// Read the groupId and version properties from the "parent" bigtop project.
-// It would be better if there was some better way of doing this. Howvever,
-// at this point, we have to do this (or some variation thereof) since gradle
-// projects can't have maven projects as parents (AFAIK. If there is a way to do it,
-// it doesn't seem to be well-documented).
-def setProjectProperties() {
-    Node xml = new XmlParser().parse("../pom.xml")
-    group = xml.groupId.first().value().first()
-    version = xml.version.first().value().first()
-}
-
-setProjectProperties()
-description = """"""
-
-// We are using 1.7 as gradle can't play well when java 8 and scala are combined.
-// There is an open issue here: http://issues.gradle.org/browse/GRADLE-3023
-// There is talk of this being resolved in the next version of gradle. Till then,
-// we are stuck with java 7. But we do have scala if we want more syntactic sugar.
-sourceCompatibility = 1.7
-targetCompatibility = 1.7
-
-// Specify any additional project properties.
-ext {
-    slf4jVersion = "1.7.5"
-    guavaVersion = "15.0"
-    datanucleusVersion = "3.2.2"
-    datanucleusJpaVersion = "3.2.1"
-    bonecpVersion = "0.8.0.RELEASE"
-    derbyVersion = "10.10.1.1"
-
-    // from horton-works repo. They compile mahout-core against hadoop2.x. These
-    // mahout is compiled against 2.4.0
-    hadoopVersion = "2.4.0.2.1.2.0-402"
-    mahoutVersion = "0.9.0.2.1.2.0-402"
-}
-
-repositories {
-    mavenCentral()
-    maven {
-        url "http://repo.hortonworks.com/content/repositories/releases/"
-    }
-}
-
-tasks.withType(AbstractCompile) {
-    options.encoding = 'UTF-8'
-    options.compilerArgs << "-Xlint:all"
-}
-
-tasks.withType(ScalaCompile) {
-    // Enables incremental compilation.
-    // http://www.gradle.org/docs/current/userguide/userguide_single.html#N12F78
-    scalaCompileOptions.useAnt = false
-}
-
-tasks.withType(Test) {
-    testLogging {
-        // Uncomment this if you want to see the console output from the tests.
-        // showStandardStreams = true
-        events "passed", "skipped", "failed"
-        // show standard out and standard error of the test JVM(s) on the console
-        //showStandardStreams = true
-    }
-}
-
-test {
-    exclude "**/*TestPig.java", "**/*TestHiveEmbedded.java", "**/*TestCrunch.java", "**/*TestPetStoreTransactionGeneratorJob.java"
-}
-
-// Create a separate source-set for the src/integrationTest set of classes. The convention here
-// is that gradle will look for a directory with the same name as that of the specified source-set
-// under the 'src' directory. So, in this case, it will look for a directory named 'src/integrationTest'
-// since the name of the source-set is 'integrationTest'
-sourceSets {
-    main {
-        java.srcDirs = [];
-        scala.srcDirs = ["src/main/scala", "src/main/java"]
-    }
-    // The main and test source-sets are configured by both java and scala plugins. They contain
-    // all the src/main and src/test classes. The following statements make all of those classes
-    // available on the classpath for the integration-tests, for both java and scala.
-    integrationTest {
-        java {
-            compileClasspath += main.output + test.output
-            runtimeClasspath += main.output + test.output
-        }
-        scala {
-            compileClasspath += main.output + test.output
-            runtimeClasspath += main.output + test.output
-        }
-    }
-}
-
-// Creating a source-set automatically add a couple of corresponding configurations (when java/scala
-// plugins are applied). The convention for these configurations is <sourceSetName>Compile and
-// <sourceSetName>Runtime. The following statements declare that all the dependencies from the
-// testCompile configuration will now be available for integrationTestCompile, and all the
-// dependencies (and other configuration that we might have provided) for testRuntime will be
-// available for integrationTestRuntime. For ex. the testCompile configuration has a dependency on
-// jUnit and scalatest. This makes them available for the integration tests as well.
-configurations {
-    integrationTestCompile {
-        extendsFrom testCompile
-    }
-
-    integrationTestRuntime {
-        extendsFrom integrationTestCompile, testRuntime
-    }
-}
-
-// To see the API that is being used here, consult the following docs
-// http://www.gradle.org/docs/current/dsl/org.gradle.api.artifacts.ResolutionStrategy.html
-def updateDependencyVersion(dependencyDetails, dependencyString) {
-    def parts = dependencyString.split(':')
-    def group = parts[0]
-    def name = parts[1]
-    def version = parts[2]
-    if (dependencyDetails.requested.group == group
-            && dependencyDetails.requested.name == name) {
-        dependencyDetails.useVersion version
-    }
-}
-
-def setupPigIntegrationTestDependencyVersions(dependencyResolveDetails) {
-    // This is the way we override the dependencies.
-    updateDependencyVersion dependencyResolveDetails, "joda-time:joda-time:2.2"
-}
-
-def setupCrunchIntegrationTestDependencyVersions(dependencyResolveDetails) {
-    // Specify any dependencies that you want to override for crunch integration tests.
-}
-
-def setupMahoutIntegrationTestDependencyVersions(dependencyResolveDetails) {
-    // Specify any dependencies that you want to override for mahout integration tests.
-}
-
-
-task integrationTest(type: Test, dependsOn: test) {
-
-    testClassesDir = sourceSets.integrationTest.output.classesDir
-    classpath = sourceSets.integrationTest.runtimeClasspath
-
-    if(!project.hasProperty('ITProfile')) {
-        // skip integration-tests if no profile has been specified.
-        integrationTest.onlyIf { false }
-        return;
-    }
-
-    def patternsToInclude
-    def dependencyConfigClosure
-    def skipDependencyUpdates = false
-    // Select the pattern for test classes that should be executed, and the dependency
-    // configuration function to be called based on the profile name specified at the command line.
-    switch (project.ITProfile) {
-        case "pig":
-            patternsToInclude = "*PigIT*"
-            dependencyConfigClosure = { setupPigIntegrationTestDependencyVersions(it) }
-            break
-        case "crunch":
-            patternsToInclude = "*CrunchIT*"
-            dependencyConfigClosure = { setupCrunchIntegrationTestDependencyVersions(it) }
-            break
-        case "mahout":
-            patternsToInclude = "*MahoutIT*"
-            dependencyConfigClosure = { setupMahoutIntegrationTestDependencyVersions(it) }
-            break
-        // skip integration-tests if the passed in profile-name is not valid
-        default: integrationTest.onlyIf { false }; return
-    }
-
-
-    filter { includeTestsMatching patternsToInclude }
-
-    // This is the standard way gradle allows overriding each specific dependency.
-    // see: http://www.gradle.org/docs/current/dsl/org.gradle.api.artifacts.ResolutionStrategy.html
-    project.configurations.all {
-        resolutionStrategy {
-            eachDependency {
-                dependencyConfigClosure(it)
-            }
-        }
-    }
-}
-
-dependencies {
-    compile "org.kohsuke:graphviz-api:1.0"
-    compile "org.apache.crunch:crunch-core:0.9.0-hadoop2"
-    compile "com.jolbox:bonecp:${project.bonecpVersion}"
-    compile "org.apache.derby:derby:${project.derbyVersion}"
-    compile "com.google.guava:guava:${project.guavaVersion}"
-    compile "commons-lang:commons-lang:2.6"
-    compile "joda-time:joda-time:2.3"
-    compile "org.apache.commons:commons-lang3:3.1"
-    compile "com.google.protobuf:protobuf-java:2.5.0"
-    compile "commons-logging:commons-logging:1.1.3"
-    compile "com.thoughtworks.xstream:xstream:+"
-    compile "org.apache.lucene:lucene-core:+"
-    compile "org.apache.lucene:lucene-analyzers-common:+"
-    compile "org.apache.solr:solr-commons-csv:3.5.0"
-
-    compile group: "org.apache.pig", name: "pig", version: "0.12.0", classifier:"h2"
-    compile "org.slf4j:slf4j-api:${project.slf4jVersion}"
-    compile "log4j:log4j:1.2.12"
-    compile "org.slf4j:slf4j-log4j12:${project.slf4jVersion}"
-    compile "org.datanucleus:datanucleus-core:${project.datanucleusVersion}"
-    compile "org.datanucleus:datanucleus-rdbms:${project.datanucleusJpaVersion}"
-    compile "org.datanucleus:datanucleus-api-jdo:${project.datanucleusJpaVersion}"
-    compile "org.datanucleus:datanucleus-accessplatform-jdo-rdbms:${project.datanucleusJpaVersion}"
-    compile group: "org.apache.mrunit", name: "mrunit", version: "1.0.0", classifier:"hadoop2"
-
-    compile "org.jfairy:jfairy:0.2.4"
-
-    // from horton-works repo. They compile mahout-core against hadoop2.x
-    compile "org.apache.hadoop:hadoop-client:${hadoopVersion}"
-    compile "org.apache.mahout:mahout-core:${mahoutVersion}"
-
-    compile 'org.scala-lang:scala-library:2.11.0'
-
-    testCompile "junit:junit:4.11"
-    testCompile "org.hamcrest:hamcrest-all:1.3"
-    testCompile "org.scalatest:scalatest_2.11:2.1.7"
-}
-
-configurations {
-    hadoopClusterRuntime {
-	    // extendsFrom integrationTestRuntime
-	    if(project.hasProperty('for-cluster')) {
-		    excludeRules += [getGroup: { 'org.apache.crunch' }, getModule: { 'crunch-core' } ] as ExcludeRule
-		    excludeRules += [getGroup: { 'org.apache.pig' }, getModule: { 'pig' } ] as ExcludeRule
-		    excludeRules += [getGroup: { 'org.apache.mahout' }, getModule: { 'mahout-core' } ] as ExcludeRule
-		    excludeRules += [getGroup: { 'org.apache.hadoop' }, getModule: { 'hadoop-client' } ] as ExcludeRule
-		}
-    }
-}
-
-task listJars << {
-    configurations.shadow.each { println it.name }
-}
-
-def copyDependencyJarsForHadoopCluster() {
-    copy {
-        from configurations.hadoopClusterRuntime
-        into 'build/libs'
-    }
-}
-
-build {
-    doLast {
-        copyDependencyJarsForHadoopCluster()
-    }
-}
-
-eclipse {
-    classpath {
-        // Add the dependencies and the src dirs for the integrationTest source-set to the
-        // .classpath file that will be generated by the eclipse plugin.
-        plusConfigurations += [configurations.integrationTestCompile]
-        // Comment out the following two lines if you want to generate an eclipse project quickly.
-        downloadSources = true
-        downloadJavadoc = false
-    }
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/pom.xml
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/pom.xml b/bigtop-bigpetstore/pom.xml
deleted file mode 100644
index ef328ae..0000000
--- a/bigtop-bigpetstore/pom.xml
+++ /dev/null
@@ -1,584 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-	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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<groupId>org.apache.bigtop</groupId>
-	<artifactId>BigPetStore</artifactId>
-	<version>0.9.0-SNAPSHOT</version>
-	<properties>
-		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-		<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-		<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-		<slf4j.version>1.7.5</slf4j.version>
-		<guava.version>15.0</guava.version>
-		<hadoop.version>2.2.0</hadoop.version>
-		<hive.version>0.12.0</hive.version>
-		<datanucleus.version>3.2.2</datanucleus.version>
-		<datanucleus.jpa.version>3.2.1</datanucleus.jpa.version>
-		<bonecp.version>0.9.0-SNAPSHOT.RELEASE</bonecp.version>
-		<derby.version>10.10.1.1</derby.version>
-		<plugin.surefire.version>2.17</plugin.surefire.version>
-	</properties>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.kohsuke</groupId>
-			<artifactId>graphviz-api</artifactId>
-			<version>1.0</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.crunch</groupId>
-			<artifactId>crunch-core</artifactId>
-			<version>0.9.0-hadoop2</version>
-		</dependency>
-
-		<!-- misc deps -->
-		<dependency>
-			<groupId>com.jolbox</groupId>
-			<artifactId>bonecp</artifactId>
-			<version>${bonecp.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.derby</groupId>
-			<artifactId>derby</artifactId>
-			<version>${derby.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-
-		<!-- From pig profile -->
-		<dependency>
-			<groupId>commons-lang</groupId>
-			<artifactId>commons-lang</artifactId>
-			<version>2.6</version>
-		</dependency>
-
-		<dependency>
-			<groupId>joda-time</groupId>
-			<artifactId>joda-time</artifactId>
-			<version>2.3</version>
-		</dependency>
-		<!-- end pig profile -->
-		<!-- From hive profile -->
-		<dependency>
-			<groupId>org.apache.commons</groupId>
-			<artifactId>commons-lang3</artifactId>
-			<version>3.1</version>
-		</dependency>
-		<!-- end hive profile -->
-		<!-- From Crunch profile -->
-		<dependency>
-			<groupId>com.google.protobuf</groupId>
-			<artifactId>protobuf-java</artifactId>
-			<version>2.5.0</version>
-		</dependency>
-		<!-- end crunch profile -->
-		<!-- From Mahout profile -->
-		<dependency>
-			<groupId>commons-logging</groupId>
-			<artifactId>commons-logging</artifactId>
-			<version>1.1.3</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.mahout</groupId>
-			<artifactId>mahout-math</artifactId>
-			<version>0.9</version>
-		</dependency>
-		<dependency>
-			<groupId>com.thoughtworks.xstream</groupId>
-			<artifactId>xstream</artifactId>
-			<version>LATEST</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.lucene</groupId>
-			<artifactId>lucene-core</artifactId>
-			<version>LATEST</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.lucene</groupId>
-			<artifactId>lucene-analyzers-common</artifactId>
-			<version>LATEST</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.mahout.commons</groupId>
-			<artifactId>commons-cli</artifactId>
-			<version>LATEST</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.commons</groupId>
-			<artifactId>commons-math3</artifactId>
-			<version>LATEST</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.solr</groupId>
-			<artifactId>solr-commons-csv</artifactId>
-			<version>3.5.0</version>
-		</dependency>
-		<!-- end Mahout profile -->
-
-		<!-- TODO ask question about this comment -->
-		<!-- We keep this at top level so that mvn eclipse:eclipse creates a nice 
-			tidy project, but its a little messy. later we'll create a profile for eclipse 
-			and move this (and other deps) into profiles as needed. Important: Remove 
-			this dependency when running hive integration tests... -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-client</artifactId>
-			<version>${hadoop.version}</version>
-		</dependency>
-		<!-- TODO ask question about this comment -->
-		<!-- mahout deps : may need to turn these on/off when testing mahout locally -->
-		<!-- For testing on my machine, I created a bigpetstore mahout jar which 
-			is compiled for 2.2.0 . Or substitute this with the standard apache mahout-core 
-			but not sure if it will work. -->
-		<dependency>
-			<groupId>org.apache.mahout</groupId>
-			<artifactId>mahout-core</artifactId>
-			<version>0.8</version>
-		</dependency>
-		<!-- pig deps -->
-		<dependency>
-			<groupId>org.apache.pig</groupId>
-			<artifactId>pig</artifactId>
-			<classifier>h2</classifier>
-			<version>0.12.0</version>
-		</dependency>
-
-		<!--logging -->
-
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-api</artifactId>
-			<version>${slf4j.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>log4j</groupId>
-			<artifactId>log4j</artifactId>
-			<version>1.2.12</version>
-		</dependency>
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-log4j12</artifactId>
-			<version>${slf4j.version}</version>
-		</dependency>
-		<!-- hive -->
-		<dependency>
-			<groupId>org.apache.hive</groupId>
-			<artifactId>hive-common</artifactId>
-			<version>${hive.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hive</groupId>
-			<artifactId>hive-serde</artifactId>
-			<version>${hive.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hive</groupId>
-			<artifactId>hive-jdbc</artifactId>
-			<version>${hive.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hive</groupId>
-			<artifactId>hive-contrib</artifactId>
-			<version>${hive.version}</version>
-		</dependency>
-
-		<!-- datanucleus -->
-		<dependency>
-			<groupId>org.datanucleus</groupId>
-			<artifactId>datanucleus-core</artifactId>
-			<version>${datanucleus.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.datanucleus</groupId>
-			<artifactId>datanucleus-rdbms</artifactId>
-			<version>${datanucleus.jpa.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.datanucleus</groupId>
-			<artifactId>datanucleus-api-jdo</artifactId>
-			<version>${datanucleus.jpa.version}</version>
-		</dependency>
-
-		<!-- TODO eliminate this pom dependency -->
-		<dependency>
-			<groupId>org.datanucleus</groupId>
-			<artifactId>datanucleus-accessplatform-jdo-rdbms</artifactId>
-			<version>${datanucleus.jpa.version}</version>
-			<type>pom</type>
-		</dependency>
-
-		<!-- Unit test artifacts -->
-		<dependency>
-			<groupId>junit</groupId>
-			<artifactId>junit</artifactId>
-			<version>4.11</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.hamcrest</groupId>
-			<artifactId>hamcrest-all</artifactId>
-			<version>1.3</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.mrunit</groupId>
-			<artifactId>mrunit</artifactId>
-			<version>1.0.0</version>
-			<classifier>hadoop2</classifier>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<extensions>
-			<extension>
-				<groupId>org.springframework.build.aws</groupId>
-				<artifactId>org.springframework.build.aws.maven</artifactId>
-				<version>3.0.0.RELEASE</version>
-			</extension>
-		</extensions>
-		<finalName>bigpetstore-${project.version}</finalName>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-release-plugin</artifactId>
-				<version>2.5</version>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-eclipse-plugin</artifactId>
-				<version>2.9</version>
-				<configuration>
-					<downloadSources>true</downloadSources>
-					<downloadJavadocs>true</downloadJavadocs>
-				</configuration>
-			</plugin>
-
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-compiler-plugin</artifactId>
-				<version>${maven-compiler-plugin.version}</version>
-				<configuration>
-					<source>1.8</source>
-					<target>1.8</target>
-				</configuration>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<version>2.4</version>
-				<configuration>
-					<outputDirectory>${basedir}/target</outputDirectory>
-				</configuration>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<version>${plugin.surefire.version}</version>
-				<configuration>
-					<excludes>
-						<exclude>**/*TestPig.java</exclude>
-						<exclude>**/*TestHiveEmbedded.java</exclude>
-						<exclude>**/*TestCrunch.java</exclude>
-					</excludes>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-
-	<profiles>
-		<profile>
-			<id>pig</id>
-			<build>
-				<plugins>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-surefire-plugin</artifactId>
-						<version>${plugin.surefire.version}</version>
-						<configuration>
-							<excludes>
-								<exclude>**/*TestPig.java</exclude>
-								<exclude>**/*TestHiveEmbedded.java</exclude>
-								<exclude>**/*TestCrunch.java</exclude>
-								<exclude>**/*TestPetStoreTransactionGeneratorJob.java</exclude>
-							</excludes>
-
-						</configuration>
-					</plugin>
-					<plugin>
-						<groupId>org.codehaus.mojo</groupId>
-						<artifactId>build-helper-maven-plugin</artifactId>
-						<version>1.5</version>
-						<executions>
-							<execution>
-								<id>add-test-source</id>
-								<phase>generate-test-sources</phase>
-								<goals>
-									<goal>add-test-source</goal>
-								</goals>
-								<configuration>
-									<sources>
-										<source>src/integration/java</source>
-									</sources>
-								</configuration>
-							</execution>
-						</executions>
-					</plugin>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-failsafe-plugin</artifactId>
-						<version>2.12</version>
-
-						<configuration>
-							<argLine>-Xmx1g</argLine>
-							<excludes>
-								<exclude>**/*BigPetStoreMahoutIT.java</exclude>
-								<exclude>**/*BigPetStoreHiveIT.java</exclude>
-								<exclude>**/*BigPetStoreCrunchIT.java</exclude>
-							</excludes>
-						</configuration>
-						<executions>
-							<!-- States that both integration-test and verify goals of the Failsafe 
-								Maven plugin are executed. -->
-							<execution>
-								<id>integration-tests</id>
-								<goals>
-									<goal>integration-test</goal>
-									<goal>verify</goal>
-								</goals>
-							</execution>
-						</executions>
-					</plugin>
-				</plugins>
-			</build>
-		</profile>
-
-		<profile>
-			<id>hive</id>
-			<build>
-				<plugins>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-surefire-plugin</artifactId>
-						<version>${plugin.surefire.version}</version>
-						<configuration>
-							<excludes>
-								<exclude>**/*TestPig.java</exclude>
-								<exclude>**/*TestHiveEmbedded.java</exclude>
-								<exclude>**/*TestCrunch.java</exclude>
-								<exclude>**/*TestPetStoreTransactionGeneratorJob.java</exclude>
-							</excludes>
-						</configuration>
-					</plugin>
-					<plugin>
-						<groupId>org.codehaus.mojo</groupId>
-						<artifactId>build-helper-maven-plugin</artifactId>
-						<version>1.5</version>
-						<executions>
-							<execution>
-								<id>add-test-source</id>
-								<phase>generate-test-sources</phase>
-								<goals>
-									<goal>add-test-source</goal>
-								</goals>
-								<configuration>
-									<sources>
-										<source>src/integration/java</source>
-									</sources>
-								</configuration>
-							</execution>
-						</executions>
-					</plugin>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-failsafe-plugin</artifactId>
-						<version>2.12</version>
-						<configuration>
-							<excludes>
-								<exclude>**/*BigPetStoreMahoutIT.java</exclude>
-								<exclude>**/*BigPetStorePigIT.java</exclude>
-								<exclude>**/*BigPetStoreCrunchIT.java</exclude>
-							</excludes>
-						</configuration>
-						<executions>
-							<!-- States that both integration-test and verify goals of the Failsafe 
-								Maven plugin are executed. -->
-							<execution>
-								<id>integration-tests</id>
-								<goals>
-									<goal>integration-test</goal>
-									<goal>verify</goal>
-								</goals>
-							</execution>
-						</executions>
-					</plugin>
-				</plugins>
-			</build>
-			<dependencies>
-				<!-- hadoop -->
-				<!-- TODO is this version change required? Version 2.2.0 is provided 
-					by hadoop-client dependency. Shouldn't we have the same versions for the 
-					related dependencies? -->
-				<dependency>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-mapreduce-client-app</artifactId>
-					<version>2.3.0</version>
-				</dependency>
-			</dependencies>
-		</profile>
-		<profile>
-			<id>crunch</id>
-			<build>
-				<plugins>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-surefire-plugin</artifactId>
-						<version>${plugin.surefire.version}</version>
-						<configuration>
-							<excludes>
-								<exclude>**/*TestPig.java</exclude>
-								<exclude>**/*TestHiveEmbedded.java</exclude>
-								<exclude>**/*TestCrunch.java</exclude>
-								<exclude>**/*TestPetStoreTransactionGeneratorJob.java</exclude>
-							</excludes>
-						</configuration>
-					</plugin>
-					<plugin>
-						<groupId>org.codehaus.mojo</groupId>
-						<artifactId>build-helper-maven-plugin</artifactId>
-						<version>1.5</version>
-						<executions>
-							<execution>
-								<id>add-test-source</id>
-								<phase>generate-test-sources</phase>
-								<goals>
-									<goal>add-test-source</goal>
-								</goals>
-								<configuration>
-									<sources>
-										<source>src/integration/java</source>
-									</sources>
-								</configuration>
-							</execution>
-						</executions>
-					</plugin>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-failsafe-plugin</artifactId>
-						<version>2.12</version>
-						<configuration>
-							<excludes>
-								<exclude>**/*BigPetStorePigIT.java</exclude>
-								<exclude>**/*BigPetStoreHiveIT.java</exclude>
-								<exclude>**/*BigPetStoreMahoutIT.java</exclude>
-							</excludes>
-						</configuration>
-						<executions>
-							<!-- States that both integration-test and verify goals of the Failsafe 
-								Maven plugin are executed. -->
-							<execution>
-								<id>integration-tests</id>
-								<goals>
-									<goal>integration-test</goal>
-									<goal>verify</goal>
-								</goals>
-							</execution>
-						</executions>
-					</plugin>
-				</plugins>
-			</build>
-		</profile>
-		<profile>
-			<id>mahout</id>
-			<!-- TODO this property is not being used anywhere. It's not even automatically 
-				detectable. Remove? Or do something that the name suggests? -->
-			<properties>
-				<skip.unit.tests>true</skip.unit.tests>
-			</properties>
-			<build>
-				<plugins>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-surefire-plugin</artifactId>
-						<version>${plugin.surefire.version}</version>
-						<configuration>
-							<excludes>
-								<exclude>**/*TestPig.java</exclude>
-								<exclude>**/*TestHiveEmbedded.java</exclude>
-								<exclude>**/*TestCrunch.java</exclude>
-								<exclude>**/*TestPetStoreTransactionGeneratorJob.java</exclude>
-							</excludes>
-						</configuration>
-					</plugin>
-					<plugin>
-						<groupId>org.codehaus.mojo</groupId>
-						<artifactId>build-helper-maven-plugin</artifactId>
-						<version>1.5</version>
-						<executions>
-							<execution>
-								<id>add-test-source</id>
-								<phase>generate-test-sources</phase>
-								<goals>
-									<goal>add-test-source</goal>
-								</goals>
-								<configuration>
-									<sources>
-										<source>src/integration/java</source>
-									</sources>
-								</configuration>
-							</execution>
-						</executions>
-					</plugin>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-failsafe-plugin</artifactId>
-						<version>2.12</version>
-						<configuration>
-							<excludes>
-								<exclude>**/*BigPetStorePigIT.java</exclude>
-								<exclude>**/*BigPetStoreCrunchIT.java</exclude>
-								<exclude>**/*BigPetStoreHiveIT.java</exclude>
-							</excludes>
-						</configuration>
-						<executions>
-							<!-- States that both integration-test and verify goals of the Failsafe 
-								Maven plugin are executed. -->
-							<execution>
-								<id>integration-tests</id>
-								<goals>
-									<goal>integration-test</goal>
-									<goal>verify</goal>
-								</goals>
-							</execution>
-						</executions>
-					</plugin>
-				</plugins>
-			</build>
-		</profile>
-	</profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/settings.gradle
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/settings.gradle b/bigtop-bigpetstore/settings.gradle
deleted file mode 100644
index 53d74f2..0000000
--- a/bigtop-bigpetstore/settings.gradle
+++ /dev/null
@@ -1,18 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-rootProject.name = 'BigPetStore'

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStoreMahoutIT.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStoreMahoutIT.java b/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStoreMahoutIT.java
deleted file mode 100644
index b07c5a0..0000000
--- a/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStoreMahoutIT.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore;
-
-import static org.apache.bigtop.bigpetstore.ITUtils.createTestOutputPath;
-import static org.apache.bigtop.bigpetstore.ITUtils.setup;
-
-import java.util.regex.Pattern;
-
-import org.apache.bigtop.bigpetstore.recommend.ItemRecommender;
-import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants.OUTPUTS.MahoutPaths;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.base.Predicate;
-
-public class BigPetStoreMahoutIT {
-
-  public static final Path INPUT_DIR_PATH =
-          new Path(ITUtils.BPS_TEST_PIG_CLEANED, MahoutPaths.Mahout.name());
-  public static final String INPUT_DIR_PATH_STR = INPUT_DIR_PATH.toString();
-  private static final Path MAHOUT_OUTPUT_DIR = createTestOutputPath(MahoutPaths.Mahout.name());
-  private static final Path ALS_FACTORIZATION_OUTPUT_DIR =
-          createTestOutputPath(MahoutPaths.Mahout.name(), MahoutPaths.AlsFactorization.name());
-  private static final Path ALS_RECOMMENDATIONS_DIR =
-          createTestOutputPath(MahoutPaths.Mahout.name(), MahoutPaths.AlsRecommendations.name());
-
-  private ItemRecommender itemRecommender;
-
-  @Before
-  public void setupTest() throws Throwable {
-    setup();
-    try {
-      FileSystem fs = FileSystem.get(new Configuration());
-      fs.delete(MAHOUT_OUTPUT_DIR, true);
-      itemRecommender = new ItemRecommender(INPUT_DIR_PATH_STR, ALS_FACTORIZATION_OUTPUT_DIR.toString(),
-              ALS_RECOMMENDATIONS_DIR.toString());
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  private static final Predicate<String> TEST_OUTPUT_FORMAT = new Predicate<String>() {
-    private final Pattern p = Pattern.compile("^\\d+\\s\\[\\d+:\\d+\\.\\d+\\]$");
-    @Override
-    public boolean apply(String input) {
-      return p.matcher(input).matches();
-    }
-  };
-
-  @Test
-  public void testPetStorePipeline() throws Exception {
-    itemRecommender.recommend();
-    ITUtils.assertOutput(ALS_RECOMMENDATIONS_DIR, TEST_OUTPUT_FORMAT);
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStorePigIT.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStorePigIT.java b/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStorePigIT.java
deleted file mode 100644
index 78d5c6b..0000000
--- a/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/BigPetStorePigIT.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore;
-
-import static org.apache.bigtop.bigpetstore.ITUtils.BPS_TEST_GENERATED;
-import static org.apache.bigtop.bigpetstore.ITUtils.BPS_TEST_PIG_CLEANED;
-import static org.apache.bigtop.bigpetstore.ITUtils.fs;
-
-import java.io.File;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.bigtop.bigpetstore.etl.PigCSVCleaner;
-import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.pig.ExecType;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.ImmutableMap;
-
-/**
- * This is the main integration test for pig. Like all BPS integration tests, it
- * is designed to simulate exactly what will happen on the actual cluster,
- * except with a small amount of records.
- *
- * In addition to cleaning the dataset, it also runs the BPS_analytics.pig
- * script which BigPetStore ships with.
- */
-public class BigPetStorePigIT {
-
-	final static Logger log = LoggerFactory.getLogger(BigPetStorePigIT.class);
-
-	/**
-	 * An extra unsupported code path that we have so people can do ad hoc
-	 * analytics on pig data after it is cleaned.
-	 */
-	public static final Path BPS_TEST_PIG_COUNT_PRODUCTS = fs
-			.makeQualified(new Path("bps_integration_",
-					BigPetStoreConstants.OUTPUTS.pig_ad_hoc_script.name() + "0"));
-
-	static final File PIG_SCRIPT = new File("BPS_analytics.pig");
-
-	static {
-		if (!PIG_SCRIPT.exists()) {
-			throw new RuntimeException("Couldnt find pig script at " + PIG_SCRIPT.getAbsolutePath());
-		}
-	}
-
-	@Before
-	public void setupTest() throws Throwable {
-		ITUtils.setup();
-		try {
-			FileSystem.get(new Configuration()).delete(BPS_TEST_PIG_CLEANED, true);
-			FileSystem.get(new Configuration()).delete(BPS_TEST_PIG_COUNT_PRODUCTS, true);
-		} catch (Exception e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-	static Map<Path, Predicate<String>> TESTS = ImmutableMap.of(
-		/** Test of the main output */
-		BPS_TEST_PIG_CLEANED, ITUtils.VERIFICATION_PERDICATE,
-		// Example of how to count products after doing basic pig data cleanup
-		BPS_TEST_PIG_COUNT_PRODUCTS, ITUtils.VERIFICATION_PERDICATE,
-		// Test the output that is to be used as an input for Mahout.
-		BigPetStoreMahoutIT.INPUT_DIR_PATH, ITUtils.VERIFICATION_PERDICATE
-	);
-
-	@Test
-	public void testPetStoreCorePipeline() throws Exception {
-		runPig(BPS_TEST_GENERATED, BPS_TEST_PIG_CLEANED, PIG_SCRIPT);
-		for (Entry<Path, Predicate<String>> e : TESTS.entrySet()) {
-			ITUtils.assertOutput(e.getKey(), e.getValue());
-		}
-	}
-
-	private void runPig(Path input, Path output, File pigscript)
-			throws Exception {
-		new PigCSVCleaner(input, output, ExecType.LOCAL, pigscript);
-	}
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/ITUtils.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/ITUtils.java b/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/ITUtils.java
deleted file mode 100644
index fd53dc1..0000000
--- a/bigtop-bigpetstore/src/integrationTest/java/org/apache/bigtop/bigpetstore/ITUtils.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore;
-
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
-import java.net.InetAddress;
-import java.nio.charset.Charset;
-import java.util.List;
-
-import org.apache.bigtop.bigpetstore.generator.BPSGenerator;
-import org.apache.bigtop.bigpetstore.util.BigPetStoreConstants;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Job;
-import org.junit.Assert;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Predicate;
-import com.google.common.io.Files;
-
-public class ITUtils {
-  public static final Path TEST_OUTPUT_DIR = new Path("bps_integration_");
-
-  public static Predicate<String> VERIFICATION_PERDICATE = new Predicate<String>() {
-    @Override
-    public boolean apply(String input) {
-      return true;
-    }
-  };
-
-	static final Logger log = LoggerFactory.getLogger(ITUtils.class);
-
-	static FileSystem fs;
-	static {
-		try {
-			fs = FileSystem.getLocal(new Configuration());
-		} catch (Throwable e) {
-			String cpath = (String) System.getProperties().get("java.class.path");
-			String msg = "";
-			for (String cp : cpath.split(":")) {
-				if (cp.contains("hadoop")) {
-					msg += cp.replaceAll("hadoop", "**HADOOP**") + "\n";
-				}
-			}
-			throw new RuntimeException("Major error:  Probably issue.   "
-			        + "Check hadoop version?  " + e.getMessage()
-			        + " .... check these classpath elements:" + msg);
-		}
-	}
-
-	public static final Path BPS_TEST_GENERATED =
-	        createTestOutputPath(BigPetStoreConstants.OUTPUTS.generated.name());
-	public static final Path BPS_TEST_PIG_CLEANED =
-	        createTestOutputPath (BigPetStoreConstants.OUTPUTS.cleaned.name());
-
-	public static Path createTestOutputPath(String... pathParts) {
-	  Path path = TEST_OUTPUT_DIR;
-	  for(String pathPart: pathParts) {
-	    path = new Path(path, pathPart);
-	  }
-	  return path;
-	}
-
-	/**
-	 * Some simple checks to make sure that unit tests in local FS. these arent
-	 * designed to be run against a distribtued system.
-	 */
-	public static void checkConf(Configuration conf) throws Exception {
-		if (conf.get("mapreduce.jobtracker.address") == null) {
-			log.warn("Missing mapreduce.jobtracker.address???????!!!! " + "This can be the case in hive tests which use special "
-					+ "configurations, but we should fix it sometime.");
-			return;
-		}
-		if (!conf.get("mapreduce.jobtracker.address").equals("local")) {
-			throw new RuntimeException("ERROR: bad conf : " + "mapreduce.jobtracker.address");
-		}
-		if (!conf.get("fs.AbstractFileSystem.file.impl").contains("Local")) {
-			throw new RuntimeException("ERROR: bad conf : " + "mapreduce.jobtracker.address");
-		}
-		try {
-			InetAddress addr = java.net.InetAddress.getLocalHost();
-			System.out.println("Localhost = hn=" + addr.getHostName() + " / ha=" + addr.getHostAddress());
-		} catch (Throwable e) {
-			throw new RuntimeException(" ERROR : Hadoop wont work at all  on this machine yet"
-					+ "...I can't get / resolve localhost ! Check java version/ " + "/etc/hosts / DNS or other networking related issues on your box"
-					+ e.getMessage());
-		}
-	}
-
-	/**
-	 * Creates a generated input data set in
-	 *
-	 * test_data_directory/generated. i.e.
-	 * test_data_directory/generated/part-r-00000
-	 */
-	public static void setup() throws Throwable {
-		Configuration conf = new Configuration();
-
-		// debugging for Jeff and others in local fs that won't build
-		checkConf(conf);
-
-		conf.setInt(BPSGenerator.props.bigpetstore_records.name(), BPSGenerator.DEFAULT_NUM_RECORDS);
-
-		if (FileSystem.getLocal(conf).exists(BPS_TEST_GENERATED)) {
-			return;
-		}
-
-		Job createInput = BPSGenerator.getCreateTransactionRecordsJob(BPS_TEST_GENERATED, conf);
-		createInput.waitForCompletion(true);
-
-		Path outputfile = new Path(BPS_TEST_GENERATED, "part-r-00000");
-		List<String> lines = Files.readLines(FileSystem.getLocal(conf).pathToFile(outputfile), Charset.defaultCharset());
-		log.info("output : " + FileSystem.getLocal(conf).pathToFile(outputfile));
-		for (String l : lines) {
-			System.out.println(l);
-		}
-	}
-
-
-	// A functions that logs the output file as a verification test
-	public static void assertOutput(Path base, Predicate<String> validator) throws Exception {
-	  FileSystem fs = FileSystem.getLocal(new Configuration());
-
-	  FileStatus[] files = fs.listStatus(base);
-	  // print out all the files.
-	  for (FileStatus stat : files) {
-	    System.out.println(stat.getPath() + "  " + stat.getLen());
-	  }
-
-	  /**
-	   * Support map OR reduce outputs
-	   */
-	  Path partm = new Path(base, "part-m-00000");
-	  Path partr = new Path(base, "part-r-00000");
-	  Path p = fs.exists(partm) ? partm : partr;
-
-	  /**
-	   * Now we read through the file and validate its contents.
-	   */
-	  BufferedReader r = new BufferedReader(new InputStreamReader(fs.open(p)));
-
-	  // line:{"product":"big chew toy","count":3}
-	  while (r.ready()) {
-	    String line = r.readLine();
-	    log.info("line:" + line);
-	    // System.out.println("line:"+line);
-	    Assert.assertTrue("validationg line : " + line, validator.apply(line));
-	  }
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/contract/PetStoreStatistics.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/contract/PetStoreStatistics.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/contract/PetStoreStatistics.java
deleted file mode 100755
index ed618a8..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/contract/PetStoreStatistics.java
+++ /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.
- */
-
-
-package org.apache.bigtop.bigpetstore.contract;
-
-import java.util.Map;
-
-/**
- * This is the contract for the web site. This object is created by each ETL
- * tool : Summary stats.
- */
-public abstract class PetStoreStatistics {
-
-    public abstract Map<String, ? extends Number> numberOfTransactionsByState()
-            throws Exception;
-
-    public abstract Map<String, ? extends Number> numberOfProductsByProduct()
-            throws Exception;
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/CrunchETL.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/CrunchETL.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/CrunchETL.java
deleted file mode 100755
index f6f459c..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/CrunchETL.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.etl;
-
-import java.util.Map;
-
-import org.apache.bigtop.bigpetstore.contract.PetStoreStatistics;
-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.impl.mem.MemPipeline;
-import org.apache.crunch.impl.mr.MRPipeline;
-import org.apache.crunch.io.From;
-import org.apache.crunch.types.avro.Avros;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-public class CrunchETL extends PetStoreStatistics {
-
-    public static MapFn<LineItem, String> COUNT_BY_PRODUCT = new MapFn<LineItem, String>() {
-        public String map(LineItem lineItem) {
-            try {
-                return lineItem.getDescription();
-            } catch (Throwable t) {
-                throw new RuntimeException(t);
-            }
-        }
-    };
-    public static MapFn<LineItem, String> COUNT_BY_STATE = new MapFn<LineItem, String>() {
-        public String map(LineItem lineItem) {
-            try {
-                return lineItem.getDescription();
-            } catch (Throwable t) {
-                throw new RuntimeException(t);
-            }
-        }
-    };
-
-    PCollection<LineItem> lineItems;
-
-    public CrunchETL(Path input, Path output) throws Exception {
-        Pipeline pipeline = MemPipeline.getInstance();
-        PCollection<String> lines = pipeline.read(From.textFile(new Path(input,
-                "part-r-00000")));
-        System.out.println("crunch : " + lines.getName() + "  "
-                + lines.getSize());
-        lineItems = lines.parallelDo(ETL, Avros.reflects(LineItem.class));
-
-    }
-
-    public static MapFn ETL = new MapFn<String, LineItem>() {
-        @Override
-        public LineItem map(String input) {
-            String[] fields = input.split(",");
-            LineItem li = new LineItem();
-            li.setAppName(fields[1]);
-            li.setFirstName(fields[3]);
-            // ...
-            li.setDescription(fields[fields.length - 1]);
-            return li;
-        }
-    };
-
-    @Override
-    public Map<String, ? extends Number> numberOfTransactionsByState()
-            throws Exception {
-        PTable<String, Long> counts = lineItems.parallelDo(COUNT_BY_STATE,
-                Avros.strings()).count();
-        Map m = counts.materializeToMap();
-
-        System.out.println("Crunch:::  " + m);
-        return m;
-    }
-
-    @Override
-    public Map<String, ? extends Number> numberOfProductsByProduct()
-            throws Exception {
-        PTable<String, Long> counts = lineItems.parallelDo(COUNT_BY_PRODUCT,
-                Avros.strings()).count();
-        Map m = counts.materializeToMap();
-        //CrunchETL. System.out.println("Crunch:::  " + m);
-        return m;
-    }
-
-    public static void main(String... args) throws Exception {
-        /**
-         * PCollection<String> lines = MemPipeline .collectionOf(
-         *  "BigPetStore,storeCode_AK,1  lindsay,franco,Sat Jan 10 00:11:10 EST 1970,10.5,dog-food"
-         *  "BigPetStore,storeCode_AZ,1  tom,giles,Sun Dec 28 23:08:45 EST 1969,10.5,dog-food"
-         *  "BigPetStore,storeCode_CA,1  brandon,ewing,Mon Dec 08 20:23:57 EST 1969,16.5,organic-dog-food"
-         *  "BigPetStore,storeCode_CA,2  angie,coleman,Thu Dec 11 07:00:31 EST 1969,10.5,dog-food"
-         *  "BigPetStore,storeCode_CA,3  angie,coleman,Tue Jan 20 06:24:23 EST 1970,7.5,cat-food"
-         *  "BigPetStore,storeCode_CO,1  sharon,trevino,Mon Jan 12 07:52:10 EST 1970,30.1,antelope snacks"
-         *  "BigPetStore,storeCode_CT,1  kevin,fitzpatrick,Wed Dec 10 05:24:13 EST 1969,10.5,dog-food"
-         *  "BigPetStore,storeCode_NY,1  dale,holden,Mon Jan 12 23:02:13 EST 1970,19.75,fish-food"
-         *  "BigPetStore,storeCode_NY,2  dale,holden,Tue Dec 30 12:29:52 EST 1969,10.5,dog-food"
-         *  "BigPetStore,storeCode_OK,1  donnie,tucker,Sun Jan 18 04:50:26 EST 1970,7.5,cat-food"
-         * );
-         **/
-        // FAILS
-        Pipeline pipeline = new MRPipeline(CrunchETL.class);
-
-        PCollection<String> lines = pipeline.read(From.textFile(new Path(
-                "/tmp/BigPetStore1388719888255/generated/part-r-00000")));
-
-
-        PCollection<LineItem> lineItems = lines.parallelDo(
-                new MapFn<String, LineItem>() {
-                    @Override
-                    public LineItem map(String input) {
-
-                        System.out.println("proc1 " + input);
-                        String[] fields = input.split(",");
-                        LineItem li = new LineItem();
-                        li.setAppName("" + fields[1]);
-                        li.setFirstName("" + fields[3]);
-                        li.setDescription("" + fields[fields.length - 1]);
-                        return li;
-                    }
-                }, Avros.reflects(LineItem.class));
-
-        for (LineItem i : lineItems.materialize())
-            System.out.println(i);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/6ec6cebf/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/LineItem.java
----------------------------------------------------------------------
diff --git a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/LineItem.java b/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/LineItem.java
deleted file mode 100755
index 87e5d0d..0000000
--- a/bigtop-bigpetstore/src/main/java/org/apache/bigtop/bigpetstore/etl/LineItem.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/** 
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.bigtop.bigpetstore.etl;
-
-import java.io.Serializable;
-
-public class LineItem implements Serializable{
-
-    public LineItem(String appName, String storeCode, Integer lineId, String firstName, String lastName, String timestamp, Double price, String description){
-        super();
-        this.appName=appName;
-        this.storeCode=storeCode;
-        this.lineId=lineId;
-        this.firstName=firstName;
-        this.lastName=lastName;
-        this.timestamp=timestamp;
-        this.price=price;
-        this.description=description;
-    }
-
-    String appName;
-    String storeCode;
-    Integer lineId;
-    String firstName;
-    String lastName;
-    String timestamp;
-    Double price;
-    String description;
-
-    public LineItem(){
-        super();
-    }
-
-    public String getAppName(){
-        return appName;
-    }
-
-    public void setAppName(String appName){
-        this.appName=appName;
-    }
-
-    public String getStoreCode(){
-        return storeCode;
-    }
-
-    public void setStoreCode(String storeCode){
-        this.storeCode=storeCode;
-    }
-
-    public int getLineId(){
-        return lineId;
-    }
-
-    public void setLineId(int lineId){
-        this.lineId=lineId;
-    }
-
-    public String getFirstName(){
-        return firstName;
-    }
-
-    public void setFirstName(String firstName){
-        this.firstName=firstName;
-    }
-
-    public String getLastName(){
-        return lastName;
-    }
-
-    public void setLastName(String lastName){
-        this.lastName=lastName;
-    }
-
-    public String getTimestamp(){
-        return timestamp;
-    }
-
-    public void setTimestamp(String timestamp){
-        this.timestamp=timestamp;
-    }
-
-    public double getPrice(){
-        return price;
-    }
-
-    public void setPrice(double price){
-        this.price=price;
-    }
-
-    public String getDescription(){
-        return description;
-    }
-
-    public void setDescription(String description){
-        this.description=description;
-    }
-
-    // other constructors, parsers, etc.
-}
\ No newline at end of file